From 6dee296538e131b460f8d13bc9d100d2d9f491c5 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 18 Nov 2020 10:10:12 +0800 Subject: [PATCH 001/404] removed docker-entrypoint-initdb.d --- docker-entrypoint-initdb.d/default.sh | 13 ------------ docker-entrypoint-initdb.d/docker.sh | 9 -------- docker-entrypoint-initdb.d/logstash.sh | 29 -------------------------- 3 files changed, 51 deletions(-) delete mode 100644 docker-entrypoint-initdb.d/default.sh delete mode 100644 docker-entrypoint-initdb.d/docker.sh delete mode 100644 docker-entrypoint-initdb.d/logstash.sh diff --git a/docker-entrypoint-initdb.d/default.sh b/docker-entrypoint-initdb.d/default.sh deleted file mode 100644 index 7b4a2cb6..00000000 --- a/docker-entrypoint-initdb.d/default.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/usr/bin/env bash - -set -e - -clickhouse client -n <<-EOSQL - USE default; - - CREATE TABLE IF NOT EXISTS daily ( - day Date, - level String, - total UInt64 - ) ENGINE = SummingMergeTree(day, (day, level), 8192); -EOSQL diff --git a/docker-entrypoint-initdb.d/docker.sh b/docker-entrypoint-initdb.d/docker.sh deleted file mode 100644 index e514e7ae..00000000 --- a/docker-entrypoint-initdb.d/docker.sh +++ /dev/null @@ -1,9 +0,0 @@ -#!/usr/bin/env bash - -set -e - -clickhouse client -n <<-EOSQL - CREATE DATABASE IF NOT EXISTS docker; - USE docker; - CREATE TABLE IF NOT EXISTS docker.docker (x Int32) ENGINE = Log; -EOSQL diff --git a/docker-entrypoint-initdb.d/logstash.sh b/docker-entrypoint-initdb.d/logstash.sh deleted file mode 100644 index 785220a0..00000000 --- a/docker-entrypoint-initdb.d/logstash.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/usr/bin/env bash - -set -e - -clickhouse client -n <<-EOSQL - USE default; - CREATE TABLE IF NOT EXISTS logstash ( - date Date, - date_nullable Nullable(Date), - level String, - message String, - str String, - str_nullable Nullable(String), - num Int64, - num_nullable Nullable(Int64), - fnum Float64, - fnum_nullable Nullable(Float64) - ) - ENGINE = SummingMergeTree(date, (date, level, message), 8192); - - -EOSQL - -# CREATE TABLE IF NOT EXISTS logstash ( -# date Date, -# level String, -# message String -# ) ENGINE = SummingMergeTree(); -# From d9197346df1af6dd4238fbdcd241c85b777faff0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 17 Nov 2020 09:15:19 +0800 Subject: [PATCH 002/404] updated doc --- README.md | 107 +++++++++++++++++++++++++++++++----------------------- 1 file changed, 62 insertions(+), 45 deletions(-) diff --git a/README.md b/README.md index 6339ecc0..3f4ca890 100644 --- a/README.md +++ b/README.md @@ -5,13 +5,14 @@ clickhouse_sinker is a sinker program that transfer kafka message into [ClickHouse](https://clickhouse.yandex/). +Refers to [design](./design.md) for how it works. + ## Features - Uses Native ClickHouse client-server TCP protocol, with higher performance than HTTP. - Easy to use and deploy, you don't need write any hard code, just care about the configuration file -- Support multiple parsers: csv, fastjson, gjson. +- Support multiple parsers: fastjson(recommended), gjson, csv. - Support multiple Kafka client: sarama, kafka-go. -- Custom parser support. - Support multiple sinker tasks, each runs on parallel. - Support multiply kafka and ClickHouse clusters. - Bulk insert (by config `bufferSize` and `flushInterval`). @@ -22,15 +23,29 @@ clickhouse_sinker is a sinker program that transfer kafka message into [ClickHou - At least once delivery guarantee. - Dynamic config management with Nacos. +## Supported data types + +- [x] UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 +- [x] Float32, Float64 +- [x] String +- [x] FixedString +- [x] Date, DateTime, DateTime64 (custom layout parser) +- [x] Array(UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64) +- [x] Array(Float32, Float64) +- [x] Array(String) +- [x] Array(FixedString) +- [x] Nullable +- [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) + ## Install && Run -### By binary files (suggested) +### By binary files (recommended) Download the binary files from [release](https://github.com/housepower/clickhouse_sinker/releases), choose the executable binary file according to your env, modify the `conf` files, then run `./clickhouse_sinker --local-cfg-dir conf` ### By container image -Download the binary files from [release](https://quay.io/repository/housepower/clickhouse_sinker), modify the `conf` files, then run `docker run --volume conf:/etc/clickhouse_sinker quay.io/housepower/clickhouse_sinker` +Modify the `conf` files, then run `docker run --volume conf:/etc/clickhouse_sinker quay.io/housepower/clickhouse_sinker` ### By source @@ -50,28 +65,10 @@ make build ./dist/clickhouse_sinker --local-cfg-dir conf ``` -## Examples - -- look at the [integration test](https://github.com/housepower/clickhouse_sinker/blob/master/go.test.sh). -- there is a simple [tutorial in Chinese](https://note.youdao.com/ynoteshare1/index.html?id=c4b4a84a08e2312da6c6d733a5074c7a&type=note) which created by user @taiyang-li. - -## Supported data types - -- [x] UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 -- [x] Float32, Float64 -- [x] String -- [x] FixedString -- [x] Date, DateTime, DateTime64 (Custom Layout parser) -- [x] Array(UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64) -- [x] Array(Float32, Float64) -- [x] Array(String) -- [x] Array(FixedString) -- [x] Nullable -- [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) - ## Configuration -See config [example](./conf/config.json) +Refers to how [integration test](./go.test.sh) use the [example config](./conf/config.json). +Also refers to [code](./config/config.go) for all config items. ### Authentication with Kafka @@ -79,7 +76,7 @@ clickhouse_sinker support following three authentication mechanism: * [x] No authentication -An example clickhouse_sinker config: +An example kafka config: ``` "kfk1": { @@ -90,7 +87,7 @@ An example clickhouse_sinker config: * [x] SASL/PLAIN -An example clickhouse_sinker config: +An example kafka config: ``` "kfk2": { "brokers": "127.0.0.1:9093", @@ -98,6 +95,7 @@ An example clickhouse_sinker config: "enable": true, "password": "username", "username": "password" + }, "Version": "0.10.2.1" } ``` @@ -113,7 +111,7 @@ $ sudo yum install -y krb5-libs 2. Supply a valid /etc/krb5.conf file for each client. Usually this can be the same krb5.conf file used by the Kerberos Distribution Center (KDC). -An example clickhouse_sinker config: +An example kafka config: ``` "kfk3": { @@ -141,22 +139,25 @@ sasl.mechanism:GSSAPI sasl.jaas.config:com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab=\"/home/keytab/zhangtao.keytab\" principal=\"zhangtao/localhost@ALANWANG.COM\"; ``` -### Dynamic Config Management with Nacos +## Configuration Management +### Nacos + +Sinker is able to register with Nacos, get and apply config changes. Controled by: - CLI parameters: `nacos-register-enable, nacos-addr, nacos-namespace-id, nacos-group, nacos-username, nacos-password` - env variables: `NACOS_REGISTER_ENABLE, NACOS_ADDR, NACOS_NAMESPACE_ID, NACOS_GROUP, NACOS_USERNAME, NACOS_PASSWORD` -### Dynamic Config Management with Consul +### Consul -Currently sinker is able to register with Consul, but not able to get config from Consul. +Currently sinker is able to register with Consul, but not able to get config. Controled by: - CLI parameters: `consul-register-enable, consul-addr, consul-deregister-critical-services-after` - env variables: `CONSUL_REGISTER_ENABLE, CONSUL_ADDR, CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER` -### Dynamic Config Management with Local Files +### Local Files TODO. Currently sinker is able to parse local config files at startup, but not able to detect file changes. ## Prometheus Metrics @@ -171,30 +172,46 @@ Sinker registers with Nacos if CLI `--consul-register-enable` or env `CONSUL_REG * [x] Push to promethues -If CLI `--push-gateway-addrs` or env `PUSH_GATEWAY_ADDRS` (a list of comma-separated urls) is present, metrics are pushed to the given one of given URLs regualarly. +If CLI `--push-gateway-addrs` or env `PUSH_GATEWAY_ADDRS` (a list of comma-separated urls) is present, metrics are pushed to one of given URLs regualarly. -## Custom message parser +## Extending -- You just need to implement the parser interface on your own +There are several abstract interfaces which you can implement to support more message format, message queue and config management mechanism. ``` type Parser interface { Parse(bs []byte) model.Metric } -``` -See [json parser](./parser/json.go) +type Inputer interface { + Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) error + Run(ctx context.Context) + Stop() error + CommitMessages(ctx context.Context, message *model.InputMessage) error +} -# Debugging +// RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... +type RemoteConfManager interface { + Init(properties map[string]interface{}) error + // Register this instance, and keep-alive via heartbeat. + Register(ip string, port int) error + Deregister(ip string, port int) error + // GetInstances fetchs healthy instances. + // Mature service-discovery solutions(Nacos, Consul etc.) have client side cache + // so that frequent invoking of GetInstances() and GetGlobalConfig() don't harm. + GetInstances() (instances []Instance, err error) + // GetConfig fetchs the config. The manager shall not reference the returned Config object after call. + GetConfig() (conf *Config, err error) + // PublishConfig publishs the config. The manager shall not reference the passed Config object after call. + PublishConfig(conf *Config) (err error) +} -```bash -echo '{"date": "2019-07-11T12:10:30Z", "level": "info", "message": "msg4"}' | kafkacat -b 127.0.0.1:9093 -P -t logstash +``` -clickhouse-client -q 'select * from default.logstash' -2019-12-16 info msg4 -2019-07-11 info msg4 -2015-05-11 info msg4 -2019-07-11 info msg4 +## Why not [`Kafka Engine`](https://clickhouse.tech/docs/en/engines/table-engines/integrations/kafka/) built in ClickHouse? -``` +- My experience indicates `Kafka Engine` is complicated, buggy and hard to debug. +- `Kafka Engine` runs inside the db process, lowers the database stability. On the other side, [Vertica](https://www.vertica.com/)'s official kafka importer is separated with the database server. +- `Kafka Engine` doesn't support custom sharding policy. +- Neither `Kafka Engine` nor clickhouse_sinker support exactly-once. From 09c77a67b39189bf33c976c6692bfa7300bd2b18 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 18 Nov 2020 11:05:29 +0800 Subject: [PATCH 003/404] use sarama if auth with kerberos --- README.md | 2 +- config/config.go | 22 +++++++++++++--------- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index 3f4ca890..e8a1c92c 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,7 @@ Refers to [design](./design.md) for how it works. - Uses Native ClickHouse client-server TCP protocol, with higher performance than HTTP. - Easy to use and deploy, you don't need write any hard code, just care about the configuration file - Support multiple parsers: fastjson(recommended), gjson, csv. -- Support multiple Kafka client: sarama, kafka-go. +- Support multiple Kafka client: kafka-go(recommended), sarama. - Support multiple sinker tasks, each runs on parallel. - Support multiply kafka and ClickHouse clusters. - Bulk insert (by config `bufferSize` and `flushInterval`). diff --git a/config/config.go b/config/config.go index 25ae2a03..a1a24adc 100644 --- a/config/config.go +++ b/config/config.go @@ -275,10 +275,22 @@ func (cfg *Config) Normallize() (err error) { func (cfg *Config) normallizeTasks() (err error) { for taskName, taskConfig := range cfg.Tasks { + if _, ok := cfg.Kafka[taskConfig.Kafka]; !ok { + err = errors.Errorf("task %s config is invalid, kafka %s doesn't exist.", taskConfig.Name, taskConfig.Kafka) + return + } + if _, ok := cfg.Clickhouse[taskConfig.Clickhouse]; !ok { + err = errors.Errorf("task %s config is invalid, clickhouse %s doesn't exist.", taskConfig.Name, taskConfig.Clickhouse) + return + } if taskConfig.Name != taskName { taskConfig.Name = taskName } - if taskConfig.KafkaClient == "" { + kfkCfg := cfg.Kafka[taskConfig.Kafka] + if kfkCfg.Sasl.Enable && kfkCfg.Sasl.Username == "" { + //kafka-go doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 + taskConfig.KafkaClient = "sarama" + } else if taskConfig.KafkaClient == "" { taskConfig.KafkaClient = "kafka-go" } if taskConfig.Parser == "" { @@ -317,14 +329,6 @@ func (cfg *Config) normallizeTasks() (err error) { taskConfig.Dims[i].SourceName = util.GetSourceName(taskConfig.Dims[i].Name) } } - if _, ok := cfg.Kafka[taskConfig.Kafka]; !ok { - err = errors.Errorf("task %s config is invalid, kafka %s doesn't exist.", taskConfig.Name, taskConfig.Kafka) - return - } - if _, ok := cfg.Clickhouse[taskConfig.Clickhouse]; !ok { - err = errors.Errorf("task %s config is invalid, clickhouse %s doesn't exist.", taskConfig.Name, taskConfig.Clickhouse) - return - } } return } From 8e243b8004150974a0a7599c4c62e16f90ce525c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 18 Nov 2020 14:52:48 +0800 Subject: [PATCH 004/404] minro doc change --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index e8a1c92c..26b5af0f 100644 --- a/README.md +++ b/README.md @@ -110,7 +110,6 @@ $ sudo yum install -y krb5-libs ``` 2. Supply a valid /etc/krb5.conf file for each client. Usually this can be the same krb5.conf file used by the Kerberos Distribution Center (KDC). - An example kafka config: ``` @@ -139,6 +138,8 @@ sasl.mechanism:GSSAPI sasl.jaas.config:com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab=\"/home/keytab/zhangtao.keytab\" principal=\"zhangtao/localhost@ALANWANG.COM\"; ``` +Note: Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. + ## Configuration Management ### Nacos From 8564f9fc168a5806424f7a8c690e8dc12cad188d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 18 Nov 2020 17:30:14 +0800 Subject: [PATCH 005/404] removed config.Version --- README.md | 23 +++++++---------------- conf/config.json | 3 +-- config/config.go | 1 - docker/conf/config.json | 3 +-- input/kafka_sarama.go | 12 ++++-------- 5 files changed, 13 insertions(+), 29 deletions(-) diff --git a/README.md b/README.md index 26b5af0f..056d5ea1 100644 --- a/README.md +++ b/README.md @@ -80,8 +80,7 @@ An example kafka config: ``` "kfk1": { - "brokers": "127.0.0.1:9092", - "Version": "0.10.2.1" + "brokers": "127.0.0.1:9092" } ``` @@ -95,23 +94,13 @@ An example kafka config: "enable": true, "password": "username", "username": "password" - }, - "Version": "0.10.2.1" + } } ``` * [x] SASL/GSSAPI(Kerberos) -You need to configuring a Kerberos Client on the host on which the clickhouse_sinker run. Refers to [Kafka Security](https://kafka.apache.org/documentation/#security). - -1. Install the krb5-libs package on all of the client machine. -``` -$ sudo yum install -y krb5-libs -``` -2. Supply a valid /etc/krb5.conf file for each client. Usually this can be the same krb5.conf file used by the Kerberos Distribution Center (KDC). - An example kafka config: - ``` "kfk3": { "brokers": "127.0.0.1:9094", @@ -125,8 +114,7 @@ An example kafka config: "username": "zhangtao/localhost", "realm": "ALANWANG.COM" } - }, - "Version": "0.10.2.1" + } } ``` @@ -138,7 +126,10 @@ sasl.mechanism:GSSAPI sasl.jaas.config:com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab=\"/home/keytab/zhangtao.keytab\" principal=\"zhangtao/localhost@ALANWANG.COM\"; ``` -Note: Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. +Notes: +- Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. +- Kafka-go doesn't support Kerberos authentication. Here's the [issue](https://github.com/segmentio/kafka-go/issues/539). +- I tested sarama Kerberos authentication against Kafka [2.11-2.2.1](https://archive.apache.org/dist/kafka/2.2.1/kafka_2.11-2.2.1.tgz). Not sure other Kafka versions work. ## Configuration Management diff --git a/conf/config.json b/conf/config.json index 63e1c076..f7f406f5 100644 --- a/conf/config.json +++ b/conf/config.json @@ -26,8 +26,7 @@ "realm": "", "disablepafxfast": false } - }, - "version": "2.0.0" + } } }, "common": { diff --git a/config/config.go b/config/config.go index a1a24adc..c1aba397 100644 --- a/config/config.go +++ b/config/config.go @@ -89,7 +89,6 @@ type KafkaConfig struct { DisablePAFXFAST bool } } - Version string } // ClickHouseConfig configuration parameters diff --git a/docker/conf/config.json b/docker/conf/config.json index 15f894cb..fe1389fe 100644 --- a/docker/conf/config.json +++ b/docker/conf/config.json @@ -26,8 +26,7 @@ "realm": "", "disablepafxfast": false } - }, - "version": "2.0.0" + } } }, "common": { diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index f6400cb8..4db16169 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -81,18 +81,14 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg m k.stopped = make(chan struct{}) k.putFn = putFn config := sarama.NewConfig() - if kfkCfg.Version != "" { - version, err := sarama.ParseKafkaVersion(kfkCfg.Version) - if err != nil { - err = errors.Wrap(err, "") - return err - } - config.Version = version - } + config.Version = sarama.V0_11_0_0 // sarama.Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags) // check for authentication if kfkCfg.Sasl.Enable { config.Net.SASL.Enable = true + if config.Version.IsAtLeast(sarama.V1_0_0_0) { + config.Net.SASL.Version = sarama.SASLHandshakeV1 + } if kfkCfg.Sasl.Username != "" { config.Net.SASL.Mechanism = sarama.SASLTypePlaintext config.Net.SASL.User = kfkCfg.Sasl.Username From e2469eb6e82dd392be3ca58742b1a0da05c815fc Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 20 Nov 2020 23:28:26 +0800 Subject: [PATCH 006/404] add back KafkaConfig.Version --- README.md | 45 ++++++++++++++++++++++++++++++----------- conf/config.json | 5 +++-- config/config.go | 1 + design.md | 2 -- docker/conf/config.json | 5 +++-- input/kafka_sarama.go | 7 +++++-- 6 files changed, 45 insertions(+), 20 deletions(-) diff --git a/README.md b/README.md index 056d5ea1..60a6e1fc 100644 --- a/README.md +++ b/README.md @@ -74,17 +74,18 @@ Also refers to [code](./config/config.go) for all config items. clickhouse_sinker support following three authentication mechanism: -* [x] No authentication +- No authentication An example kafka config: ``` "kfk1": { - "brokers": "127.0.0.1:9092" + "brokers": "127.0.0.1:9092", + "version": "2.2.1" } ``` -* [x] SASL/PLAIN +- SASL/PLAIN An example kafka config: ``` @@ -94,11 +95,12 @@ An example kafka config: "enable": true, "password": "username", "username": "password" - } + }, + "version": "2.2.1" } ``` -* [x] SASL/GSSAPI(Kerberos) +- SASL/GSSAPI(Kerberos) An example kafka config: ``` @@ -114,11 +116,13 @@ An example kafka config: "username": "zhangtao/localhost", "realm": "ALANWANG.COM" } - } + }, + "version": "2.2.1" } ``` FYI. The same config looks like the following in Java code: + ``` security.protocol:SASL_PLAINTEXT sasl.kerberos.service.name:kafka @@ -126,10 +130,7 @@ sasl.mechanism:GSSAPI sasl.jaas.config:com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab=\"/home/keytab/zhangtao.keytab\" principal=\"zhangtao/localhost@ALANWANG.COM\"; ``` -Notes: -- Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. -- Kafka-go doesn't support Kerberos authentication. Here's the [issue](https://github.com/segmentio/kafka-go/issues/539). -- I tested sarama Kerberos authentication against Kafka [2.11-2.2.1](https://archive.apache.org/dist/kafka/2.2.1/kafka_2.11-2.2.1.tgz). Not sure other Kafka versions work. +Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. I tested sarama Kerberos authentication against Kafka [2.2.1](https://archive.apache.org/dist/kafka/2.2.1/kafka_2.11-2.2.1.tgz). Not sure other Kafka versions work. ## Configuration Management @@ -156,13 +157,13 @@ TODO. Currently sinker is able to parse local config files at startup, but not a All metrics are defined in `statistics.go`. You can create Grafana dashboard for clickhouse_sinker by importing the template `clickhouse_sinker-dashboard.json`. -* [x] Pull with prometheus +- Pull with prometheus Metrics are exposed at `http://ip:port/metrics`. IP is the outbound IP of this machine. Port is from CLI `--http-port` or env `HTTP_PORT`. Sinker registers with Nacos if CLI `--consul-register-enable` or env `CONSUL_REGISTER_ENABLE` is present. However Prometheus is [unable](https://github.com/alibaba/nacos/issues/1032) to obtain dynamic service list from nacos server. -* [x] Push to promethues +- Push to promethues If CLI `--push-gateway-addrs` or env `PUSH_GATEWAY_ADDRS` (a list of comma-separated urls) is present, metrics are pushed to one of given URLs regualarly. @@ -207,3 +208,23 @@ type RemoteConfManager interface { - `Kafka Engine` runs inside the db process, lowers the database stability. On the other side, [Vertica](https://www.vertica.com/)'s official kafka importer is separated with the database server. - `Kafka Engine` doesn't support custom sharding policy. - Neither `Kafka Engine` nor clickhouse_sinker support exactly-once. + + +## Kafka Compatibility + +Kafka broker [exposes versions of various APIs it supports since 0.10.0.0](https://kafka.apache.org/protocol#api_versions). + +### Kafka-go + +- Kafka-go [negotiate it's protocol Version](https://github.com/segmentio/kafka-go/blob/c66d8ca149e7f1a7905b47a60962745ceb08a6a9/conn.go#L209). +- Kafka-go [doesn't support Kerberos authentication](https://github.com/segmentio/kafka-go/issues/539). + +### Sarama + +- Sarama guarantees compatibility [with Kafka 2.4 through 2.6](https://github.com/Shopify/sarama/blob/master/README.md#compatibility-and-api-stability). +- Sarama [has tied it's protocol usage to the Version field in Config](https://github.com/Shopify/sarama/issues/1732). + +### Conclusion + +- Neither Kafka-go nor sarama is mature as Java clients. You need to try both if clickhouse_sinker fails to connect with Kafka. +- Our experience is sarama can't work well with new kafka server if set its `Config.Version` to "0.11.0.0". So we suggest `KafkaConfig.Version` in clickhouse_sinker config matchs the Kafka server. diff --git a/conf/config.json b/conf/config.json index f7f406f5..be108b08 100644 --- a/conf/config.json +++ b/conf/config.json @@ -26,7 +26,8 @@ "realm": "", "disablepafxfast": false } - } + }, + "version": "2.2.1" } }, "common": { @@ -36,4 +37,4 @@ "flushInterval": 5, "logLevel": "debug" } -} \ No newline at end of file +} diff --git a/config/config.go b/config/config.go index c1aba397..a1a24adc 100644 --- a/config/config.go +++ b/config/config.go @@ -89,6 +89,7 @@ type KafkaConfig struct { DisablePAFXFAST bool } } + Version string } // ClickHouseConfig configuration parameters diff --git a/design.md b/design.md index 315fb03d..d8a7f84d 100644 --- a/design.md +++ b/design.md @@ -11,8 +11,6 @@ So if you setup ClickHouse properly(ReplacingMergeTree ORDER BY (__kafak_topic, It's hard for clickhouse_sinker to guarantee exactly-once semantic without ReplacingMergeTree. Kafka consumer group load-balance cause duplicated messages if one consumer quit suddenly. -Recently clickhouse_sinker has been reconstructed and achieved very big preformance improvement. There's no design document right now. @sundy-li and I may add some later. - The flow is: - Fetch message via kafka-go or samara, which starts internally an goroutine for each partition. diff --git a/docker/conf/config.json b/docker/conf/config.json index fe1389fe..5b27a420 100644 --- a/docker/conf/config.json +++ b/docker/conf/config.json @@ -26,7 +26,8 @@ "realm": "", "disablepafxfast": false } - } + }, + "version": "2.2.1" } }, "common": { @@ -35,4 +36,4 @@ "flushInterval": 5, "logLevel": "debug" } -} \ No newline at end of file +} diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 4db16169..9b7fc372 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -75,13 +75,16 @@ func (h MyConsumerGroupHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, c } // Init Initialise the kafka instance with configuration -func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) error { +func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) (err error) { k.taskCfg = cfg.Tasks[taskName] kfkCfg := cfg.Kafka[k.taskCfg.Kafka] k.stopped = make(chan struct{}) k.putFn = putFn config := sarama.NewConfig() - config.Version = sarama.V0_11_0_0 + if config.Version, err = sarama.ParseKafkaVersion(kfkCfg.Version); err != nil { + err = errors.Wrapf(err, "") + return + } // sarama.Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags) // check for authentication if kfkCfg.Sasl.Enable { From 4015c60d84402ba47f350bc7d9f2cfcf17aa20f4 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 21 Nov 2020 10:52:26 +0800 Subject: [PATCH 007/404] added doc for sharding policy --- README.md | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 60a6e1fc..c11d1fcc 100644 --- a/README.md +++ b/README.md @@ -132,11 +132,22 @@ sasl.jaas.config:com.sun.security.auth.module.Krb5LoginModule required useKeyT Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. I tested sarama Kerberos authentication against Kafka [2.2.1](https://archive.apache.org/dist/kafka/2.2.1/kafka_2.11-2.2.1.tgz). Not sure other Kafka versions work. +### Sharding Policy + +Every message is routed to a determined ClickHouse node. + +By default, the node number is caculated by `(kafka_offset/roundup(batch_size))%clickhouse_nodes`, where `roundup()` round upward an unsigned integer to the the nearest 2^n. + +This above expression can be customized with `shardingKey` and `shardingPolicy`. `shardingKey` value is a column name. `shardingPolicy` value could be: + +- `stripe,`. This requires `shardingKey` be a numeric-like (bool, int, float, date etc.) column. The expression is `(uint64(shardingKey)/stripe_size)%clickhouse_nodes`. +- `hash`. This requires `shardingKey` be a string-like column. The hash function used internally is [xxHash64](https://github.com/cespare/xxhash). The expression is `xxhash64(string(shardingKey))%clickhouse_nodes`. + ## Configuration Management ### Nacos -Sinker is able to register with Nacos, get and apply config changes. +Sinker is able to register with Nacos, get and apply config changes dynamically without restart the whole process. Controled by: - CLI parameters: `nacos-register-enable, nacos-addr, nacos-namespace-id, nacos-group, nacos-username, nacos-password` @@ -144,14 +155,15 @@ Controled by: ### Consul -Currently sinker is able to register with Consul, but not able to get config. +Currently sinker is able to register with Consul, but unable to get config. Controled by: - CLI parameters: `consul-register-enable, consul-addr, consul-deregister-critical-services-after` - env variables: `CONSUL_REGISTER_ENABLE, CONSUL_ADDR, CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER` ### Local Files -TODO. Currently sinker is able to parse local config files at startup, but not able to detect file changes. + +Currently sinker is able to parse local config files at startup, but unable to detect file changes. ## Prometheus Metrics From 209d7a20608ab58d0a92c52346a2a4b8056154e3 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 23 Nov 2020 17:01:27 +0800 Subject: [PATCH 008/404] kafka-go SASL/PLAIN --- input/kafka_go.go | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/input/kafka_go.go b/input/kafka_go.go index e92917a2..a58ea170 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -23,6 +23,7 @@ import ( "github.com/pkg/errors" "github.com/segmentio/kafka-go" + "github.com/segmentio/kafka-go/sasl/plain" log "github.com/sirupsen/logrus" "github.com/housepower/clickhouse_sinker/config" @@ -51,14 +52,11 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model kfkCfg := cfg.Kafka[k.taskCfg.Kafka] k.stopped = make(chan struct{}) k.putFn = putFn - if kfkCfg.Sasl.Enable && kfkCfg.Sasl.Username == "" { - return errors.Errorf("kafka-go doesn't support SASL/GSSAPI(Kerberos)") - } offset := kafka.LastOffset if k.taskCfg.Earliest { offset = kafka.FirstOffset } - k.r = kafka.NewReader(kafka.ReaderConfig{ + readerCfg := &kafka.ReaderConfig{ Brokers: strings.Split(kfkCfg.Brokers, ","), GroupID: k.taskCfg.ConsumerGroup, Topic: k.taskCfg.Topic, @@ -67,7 +65,20 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model MaxBytes: k.taskCfg.BufferSize * k.taskCfg.MsgSizeHint, MaxWait: time.Duration(k.taskCfg.FlushInterval) * time.Second, CommitInterval: time.Second, // flushes commits to Kafka every second - }) + } + if kfkCfg.Sasl.Enable { + if kfkCfg.Sasl.Username != "" { + readerCfg.Dialer = &kafka.Dialer{ + SASLMechanism: plain.Mechanism{ + Username: kfkCfg.Sasl.Username, + Password: kfkCfg.Sasl.Password, + }, + } + } else { + return errors.Errorf("kafka-go doesn't support SASL/GSSAPI(Kerberos)") + } + } + k.r = kafka.NewReader(*readerCfg) return nil } From e1e57dc35cbc269e40b6f5c92d95f029b25dc283 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 27 Nov 2020 09:51:47 +0800 Subject: [PATCH 009/404] fix ConsumeOffsets metrics tags --- model/message.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/model/message.go b/model/message.go index 67164fd2..62c60c82 100644 --- a/model/message.go +++ b/model/message.go @@ -80,7 +80,7 @@ LOOP: if err := bs.fnCommit(j, off); err != nil { return err } - statistics.ConsumeOffsets.WithLabelValues(bs.taskCfg.Name, strconv.Itoa(j), bs.taskCfg.Topic).Set(float64(off)) + statistics.ConsumeOffsets.WithLabelValues(bs.taskCfg.Name, bs.taskCfg.Topic, strconv.Itoa(j)).Set(float64(off)) } } eNext := e.Next() From b45b8107676f5e0a43cf714e06d347ee92da6073 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 26 Nov 2020 11:25:19 +0800 Subject: [PATCH 010/404] added ssl support --- README.md | 85 ++++++++++++++++++++++++++++++++++--------- config/config.go | 12 ++++++ input/kafka_go.go | 30 ++++++++++++--- input/kafka_sarama.go | 9 ++++- task/ring.go | 2 +- util/common.go | 35 ++++++++++++++++++ 6 files changed, 148 insertions(+), 25 deletions(-) diff --git a/README.md b/README.md index c11d1fcc..0e3c5fa1 100644 --- a/README.md +++ b/README.md @@ -9,16 +9,17 @@ Refers to [design](./design.md) for how it works. ## Features -- Uses Native ClickHouse client-server TCP protocol, with higher performance than HTTP. +- Uses native ClickHouse client-server TCP protocol, with higher performance than HTTP. - Easy to use and deploy, you don't need write any hard code, just care about the configuration file - Support multiple parsers: fastjson(recommended), gjson, csv. - Support multiple Kafka client: kafka-go(recommended), sarama. +- Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/GSSAPI and combinations of them. - Support multiple sinker tasks, each runs on parallel. - Support multiply kafka and ClickHouse clusters. - Bulk insert (by config `bufferSize` and `flushInterval`). - Parse messages concurrently. - Write batches concurrently. -- Every batch is sharded to a determined clickhouse node. Exit if loop write failed. +- Every batch is sharded to a determined clickhouse node. Exit if loop write fail. - Custom sharding policy (by config `shardingKey` and `shardingPolicy`). - At least once delivery guarantee. - Dynamic config management with Nacos. @@ -70,9 +71,54 @@ make build Refers to how [integration test](./go.test.sh) use the [example config](./conf/config.json). Also refers to [code](./config/config.go) for all config items. -### Authentication with Kafka +### Kafka Encryption -clickhouse_sinker support following three authentication mechanism: +clickhouse_sinker supports following encryption mechanisms: + +- No encryption + +An example kafka config: + +``` + "kfk1": { + "brokers": "192.168.31.64:9092", + "version": "2.2.1" + } +``` + +- Encryption using SSL + +An example kafka config: +``` + "kfk2": { + "brokers": "192.168.31.64:9093", + "tls": { + "enable": true, + "@caCertFiles": "Required. It's the CA certificate with which Kafka brokers certs be signed. This cert is added to kafka.client.truststore.jks which kafka-console-consumer.sh uses", + "caCertFiles": "/etc/security/ca-cert", + "@insecureSkipVerify": "Whether disable broker FQDN verification. Set it to `true` if kafka-console-consumer.sh uses `ssl.endpoint.identification.algorithm=`.", + "insecureSkipVerify": true + } + } +``` + +FYI. `kafka-console-consumer.sh` works as the following setup: + +``` +$ cat config/client_SSL.properties +security.protocol=SSL +ssl.truststore.location=/etc/security/kafka.client.truststore.jks +ssl.truststore.password=123456 +ssl.endpoint.identification.algorithm= + +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/client_SSL.properties +``` + +Please follow [`Kafka SSL setup`](https://kafka.apache.org/documentation/#security_ssl). Use `-keyalg RSA` when you create the broker keystore, otherwise there will be no cipher suites in common between the keystore and those Golang supports. See [this](https://github.com/Shopify/sarama/issues/643#issuecomment-216839760) for reference. + +### Kafka Authentication + +clickhouse_sinker support following following authentication mechanisms: - No authentication @@ -80,7 +126,8 @@ An example kafka config: ``` "kfk1": { - "brokers": "127.0.0.1:9092", + "brokers": "192.168.31.64:9092", + "@version": "Required if you use sarama. It's the the Kafka server version.", "version": "2.2.1" } ``` @@ -90,7 +137,7 @@ An example kafka config: An example kafka config: ``` "kfk2": { - "brokers": "127.0.0.1:9093", + "brokers": "192.168.31.64:9093", "sasl": { "enable": true, "password": "username", @@ -105,29 +152,33 @@ An example kafka config: An example kafka config: ``` "kfk3": { - "brokers": "127.0.0.1:9094", + "brokers": "192.168.31.64:9094", "sasl": { "enable": true, "gssapi": { + "@authtype": "1 - Username and password, 2 - Keytab", "authtype": 2, - "keytabpath": "/home/keytab/zhangtao.keytab", + "keytabpath": "/etc/security/mmmtest.keytab", "kerberosconfigpath": "/etc/krb5.conf", "servicename": "kafka", - "username": "zhangtao/localhost", + "@username": "`principal` consists of `username` `@` `realm`", + "username": "mmm", "realm": "ALANWANG.COM" } - }, - "version": "2.2.1" + } } ``` -FYI. The same config looks like the following in Java code: +FYI. `kafka-console-consumer.sh` works as the following setup: ``` -security.protocol:SASL_PLAINTEXT -sasl.kerberos.service.name:kafka -sasl.mechanism:GSSAPI -sasl.jaas.config:com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab=\"/home/keytab/zhangtao.keytab\" principal=\"zhangtao/localhost@ALANWANG.COM\"; +$ cat config/client_SASL_PLAINTEXT.properties +security.protocol=SASL_PLAINTEXT +sasl.kerberos.service.name=kafka +sasl.mechanism=GSSAPI +sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab="/etc/security/mmmtest.keytab" principal="mmm@ALANWANG.COM"; + +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/client_SASL_PLAINTEXT.properties ``` Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. I tested sarama Kerberos authentication against Kafka [2.2.1](https://archive.apache.org/dist/kafka/2.2.1/kafka_2.11-2.2.1.tgz). Not sure other Kafka versions work. @@ -229,7 +280,7 @@ Kafka broker [exposes versions of various APIs it supports since 0.10.0.0](https ### Kafka-go - Kafka-go [negotiate it's protocol Version](https://github.com/segmentio/kafka-go/blob/c66d8ca149e7f1a7905b47a60962745ceb08a6a9/conn.go#L209). -- Kafka-go [doesn't support Kerberos authentication](https://github.com/segmentio/kafka-go/issues/539). +- Kafka-go [doesn't support Kerberos authentication](https://github.com/segmentio/kafka-go/issues/237). ### Sarama diff --git a/config/config.go b/config/config.go index a1a24adc..7239a3e5 100644 --- a/config/config.go +++ b/config/config.go @@ -73,6 +73,13 @@ type Config struct { // KafkaConfig configuration parameters type KafkaConfig struct { Brokers string + TLS struct { + Enable bool + CaCertFiles string // Required. It's the CA certificate with which Kafka brokers certs be signed. + ClientCertFile string // Required if Kafka brokers require client authentication. + ClientKeyFile string // Required if and only if ClientCertFile is present. + InsecureSkipVerify bool // Whether disable broker FQDN verification. + } //simplified sarama.Config.Net.SASL to only support SASL/PLAIN and SASL/GSSAPI(Kerberos) Sasl struct { Enable bool @@ -256,6 +263,11 @@ func (cfg *Config) Normallize() (err error) { if err = cfg.normallizeTasks(); err != nil { return } + for _, kfkConfig := range cfg.Kafka { + if kfkConfig.Version == "" { + kfkConfig.Version = "2.2.1" + } + } for _, chConfig := range cfg.Clickhouse { if chConfig.RetryTimes < 0 { chConfig.RetryTimes = 0 diff --git a/input/kafka_go.go b/input/kafka_go.go index a58ea170..0a5eca48 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -17,6 +17,7 @@ package input import ( "context" + "crypto/tls" "io" "strings" "time" @@ -29,6 +30,7 @@ import ( "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/statistics" + "github.com/housepower/clickhouse_sinker/util" ) var _ Inputer = (*KafkaGo)(nil) @@ -47,7 +49,7 @@ func NewKafkaGo() *KafkaGo { } // Init Initialise the kafka instance with configuration -func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) error { +func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) (err error) { k.taskCfg = cfg.Tasks[taskName] kfkCfg := cfg.Kafka[k.taskCfg.Kafka] k.stopped = make(chan struct{}) @@ -65,19 +67,35 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model MaxBytes: k.taskCfg.BufferSize * k.taskCfg.MsgSizeHint, MaxWait: time.Duration(k.taskCfg.FlushInterval) * time.Second, CommitInterval: time.Second, // flushes commits to Kafka every second + Logger: log.StandardLogger(), + } + var dialer *kafka.Dialer + if kfkCfg.TLS.Enable { + var tlsConfig *tls.Config + if tlsConfig, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.InsecureSkipVerify); err != nil { + return + } + dialer = &kafka.Dialer{ + DualStack: true, + TLS: tlsConfig, + } } if kfkCfg.Sasl.Enable { if kfkCfg.Sasl.Username != "" { - readerCfg.Dialer = &kafka.Dialer{ - SASLMechanism: plain.Mechanism{ - Username: kfkCfg.Sasl.Username, - Password: kfkCfg.Sasl.Password, - }, + if dialer == nil { + dialer = &kafka.Dialer{DualStack: true} + } + dialer.SASLMechanism = plain.Mechanism{ + Username: kfkCfg.Sasl.Username, + Password: kfkCfg.Sasl.Password, } } else { return errors.Errorf("kafka-go doesn't support SASL/GSSAPI(Kerberos)") } } + if dialer != nil { + readerCfg.Dialer = dialer + } k.r = kafka.NewReader(*readerCfg) return nil } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 9b7fc372..aee581aa 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -27,6 +27,7 @@ import ( "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/statistics" + "github.com/housepower/clickhouse_sinker/util" ) var _ Inputer = (*KafkaSarama)(nil) @@ -85,7 +86,13 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg m err = errors.Wrapf(err, "") return } - // sarama.Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags) + sarama.Logger = log.StandardLogger() + if kfkCfg.TLS.Enable { + config.Net.TLS.Enable = true + if config.Net.TLS.Config, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.InsecureSkipVerify); err != nil { + return + } + } // check for authentication if kfkCfg.Sasl.Enable { config.Net.SASL.Enable = true diff --git a/task/ring.go b/task/ring.go index 047ad5ca..54b331db 100644 --- a/task/ring.go +++ b/task/ring.go @@ -50,7 +50,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { ring.ringCeilingOff = msgOffset + 1 } - if ring.service.sharder != nil { + if ring.service.sharder != nil && msgRow.Row != nil { if msgRow.Shard, err = ring.service.sharder.Calc(msgRow.Row); err != nil { log.Fatalf("%s: got error %+v", ring.service.taskCfg.Name, err) } diff --git a/util/common.go b/util/common.go index aeedcc4e..12c97558 100644 --- a/util/common.go +++ b/util/common.go @@ -16,7 +16,10 @@ limitations under the License. package util import ( + "crypto/tls" + "crypto/x509" "fmt" + "io/ioutil" "log" "net" "os" @@ -26,6 +29,7 @@ import ( "time" "github.com/fagongzi/goetty" + "github.com/pkg/errors" ) var ( @@ -101,6 +105,37 @@ LOOP: return } +// Refers to: +// https://medium.com/processone/using-tls-authentication-for-your-go-kafka-client-3c5841f2a625 +// https://github.com/denji/golang-tls +// https://www.baeldung.com/java-keystore-truststore-difference +func NewTLSConfig(caCertFiles, clientCertFile, clientKeyFile string, insecureSkipVerify bool) (*tls.Config, error) { + tlsConfig := tls.Config{} + // Load client cert + if clientCertFile != "" && clientKeyFile != "" { + cert, err := tls.LoadX509KeyPair(clientCertFile, clientKeyFile) + if err != nil { + err = errors.Wrapf(err, "") + return &tlsConfig, err + } + tlsConfig.Certificates = []tls.Certificate{cert} + } + + // Load CA cert + caCertPool := x509.NewCertPool() + for _, caCertFile := range strings.Split(caCertFiles, ",") { + caCert, err := ioutil.ReadFile(caCertFile) + if err != nil { + err = errors.Wrapf(err, "") + return &tlsConfig, err + } + caCertPool.AppendCertsFromPEM(caCert) + } + tlsConfig.RootCAs = caCertPool + tlsConfig.InsecureSkipVerify = insecureSkipVerify + return &tlsConfig, nil +} + func EnvStringVar(value *string, key string) { realKey := strings.ReplaceAll(strings.ToUpper(key), "-", "_") val, found := os.LookupEnv(realKey) From 406901a1b77805e6227d069458173a8667dfd813 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 27 Nov 2020 16:27:09 +0800 Subject: [PATCH 011/404] fix #85 --- go.mod | 3 +-- go.sum | 10 ++++++---- health/health.go | 2 +- pool/conn.go | 9 +++++++-- 4 files changed, 15 insertions(+), 9 deletions(-) diff --git a/go.mod b/go.mod index f6a27a53..0290206f 100644 --- a/go.mod +++ b/go.mod @@ -20,7 +20,6 @@ require ( github.com/hashicorp/golang-lru v0.5.4 // indirect github.com/hashicorp/memberlist v0.1.5 // indirect github.com/hashicorp/serf v0.8.5 // indirect - github.com/heptiolabs/healthcheck v0.0.0-20180807145615-6ff867650f40 github.com/jmoiron/jsonq v0.0.0-20150511023944-e874b168d07e // indirect github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 // indirect github.com/k0kubun/pp v3.0.1+incompatible @@ -35,9 +34,9 @@ require ( github.com/sundy-li/go_commons v0.0.0-20191120121859-84e4cb0afce4 github.com/tidwall/gjson v1.1.3 github.com/tidwall/match v1.0.0 // indirect + github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/valyala/fastjson v1.4.1 golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208 // indirect golang.org/x/sys v0.0.0-20200917061948-648f2a039071 // indirect golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e - gopkg.in/DATA-DOG/go-sqlmock.v1 v1.3.0 // indirect ) diff --git a/go.sum b/go.sum index f9f45914..6b582dec 100644 --- a/go.sum +++ b/go.sum @@ -2,6 +2,8 @@ github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/ClickHouse/clickhouse-go v1.4.3 h1:iAFMa2UrQdR5bHJ2/yaSLffZkxpcOYQMCUuKeNXGdqc= github.com/ClickHouse/clickhouse-go v1.4.3/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= +github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= +github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/DataDog/datadog-go v2.2.0+incompatible h1:V5BKkxACZLjzHjSgBbr2gvLA2Ae49yhc6CSY7MLy5k4= github.com/DataDog/datadog-go v2.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= @@ -155,8 +157,6 @@ github.com/hashicorp/memberlist v0.1.5/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2p github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= github.com/hashicorp/serf v0.8.5 h1:ZynDUIQiA8usmRgPdGPHFdPnb1wgGI9tK3mO9hcAJjc= github.com/hashicorp/serf v0.8.5/go.mod h1:UpNcs7fFbpKIyZaUuSW6EPiH+eZC7OuyFD+wc1oal+k= -github.com/heptiolabs/healthcheck v0.0.0-20180807145615-6ff867650f40 h1:GT4RsKmHh1uZyhmTkWJTDALRjSHYQp6FRKrotf0zhAs= -github.com/heptiolabs/healthcheck v0.0.0-20180807145615-6ff867650f40/go.mod h1:NtmN9h8vrTveVQRLHcX2HQ5wIPBDCsZ351TGbZWgg38= github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8= github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uczJe5YQdrYB16oTJlGSC/OyZDqUk9xX4= @@ -294,6 +294,8 @@ github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIK github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5NYJe/zRYDwOu9ku6YHy+Iw7l5DM= +github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -314,6 +316,8 @@ github.com/tidwall/match v1.0.0 h1:Ym1EcFkp+UQ4ptxfWlW+iMdq5cPH5nEuGzdf/Pb7VmI= github.com/tidwall/match v1.0.0/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 h1:kF/7m/ZU+0D4Jj5eZ41Zm3IH/J8OElK1Qtd7tVKAwLk= github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3/go.mod h1:QDlpd3qS71vYtakd2hmdpqhJ9nwv6mD6A30bQ1BPBFE= +github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= +github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926 h1:G3dpKMzFDjgEh2q1Z7zUUtKa8ViPtH+ocF0bE0g00O8= github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= github.com/valyala/fastjson v1.4.1 h1:hrltpHpIpkaxll8QltMU8c3QZ5+qIiCL8yKqPFJI/yE= @@ -400,8 +404,6 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -gopkg.in/DATA-DOG/go-sqlmock.v1 v1.3.0 h1:FVCohIoYO7IJoDDVpV2pdq7SgrMH6wHnuTyrdrxJNoY= -gopkg.in/DATA-DOG/go-sqlmock.v1 v1.3.0/go.mod h1:OdE7CF6DbADk7lN8LIKRzRJTTZXIjtWgA5THM5lhBAw= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/health/health.go b/health/health.go index 12f28c47..89770092 100644 --- a/health/health.go +++ b/health/health.go @@ -1,5 +1,5 @@ package health -import "github.com/heptiolabs/healthcheck" +import "github.com/troian/healthcheck" var Health = healthcheck.NewHandler() diff --git a/pool/conn.go b/pool/conn.go index 6caf02f0..ac1e1a80 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -25,9 +25,9 @@ import ( "sync" "time" - "github.com/heptiolabs/healthcheck" "github.com/housepower/clickhouse_sinker/health" "github.com/pkg/errors" + "github.com/troian/healthcheck" log "github.com/sirupsen/logrus" "github.com/sundy-li/go_commons/utils" @@ -107,7 +107,6 @@ func InitConn(name, hosts string, port int, db, username, password, dsnParams st err = errors.Wrapf(err, "") return } - health.Health.AddReadinessCheck(dsn, healthcheck.DatabasePingCheck(sqlDB, 10*time.Second)) cc.connections = append(cc.connections, &Connection{sqlDB, dsn}) } lock.Lock() @@ -116,6 +115,9 @@ func InitConn(name, hosts string, port int, db, username, password, dsnParams st cc2.ref++ return } + for _, conn := range cc.connections { + health.Health.AddReadinessCheck(conn.dsn, healthcheck.DatabasePingCheck(conn.DB, 30*time.Second)) + } poolMaps[name] = &cc return nil } @@ -128,6 +130,9 @@ func FreeConn(name string) { if cc.ref <= 0 { delete(poolMaps, name) } + for _, conn := range cc.connections { + health.Health.RemoveReadinessCheck(conn.dsn) + } } } From 148a2a1eccaf9439cae8c8ffd726e5f638b42afe Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 27 Nov 2020 17:39:32 +0800 Subject: [PATCH 012/404] renamed push-gateway-addrs to metric-push-gateway-addrs --- README.md | 2 +- cmd/clickhouse_sinker/main.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 0e3c5fa1..5d3124af 100644 --- a/README.md +++ b/README.md @@ -228,7 +228,7 @@ Sinker registers with Nacos if CLI `--consul-register-enable` or env `CONSUL_REG - Push to promethues -If CLI `--push-gateway-addrs` or env `PUSH_GATEWAY_ADDRS` (a list of comma-separated urls) is present, metrics are pushed to one of given URLs regualarly. +If CLI `--metric-push-gateway-addrs` or env `METRIC_PUSH_GATEWAY_ADDRS` (a list of comma-separated urls) is present, metrics are pushed to one of given URLs regualarly. ## Extending diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index de1c6997..f19474ea 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -92,7 +92,7 @@ func initCmdOptions() { // 2. Replace options with the corresponding env variable if present. util.EnvBoolVar(&cmdOps.ShowVer, "v") util.EnvIntVar(&cmdOps.HTTPPort, "http-port") - util.EnvStringVar(&cmdOps.PushGatewayAddrs, "push-gateway-addrs") + util.EnvStringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs") util.EnvIntVar(&cmdOps.PushInterval, "push-interval") util.EnvStringVar(&cmdOps.LocalCfgDir, "local-cfg-dir") @@ -110,7 +110,7 @@ func initCmdOptions() { // 3. Replace options with the corresponding CLI parameter if present. flag.BoolVar(&cmdOps.ShowVer, "v", cmdOps.ShowVer, "show build version and quit") flag.IntVar(&cmdOps.HTTPPort, "http-port", cmdOps.HTTPPort, "http listen port") - flag.StringVar(&cmdOps.PushGatewayAddrs, "push-gateway-addrs", cmdOps.PushGatewayAddrs, "prometheus push gatway address") + flag.StringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs", cmdOps.PushGatewayAddrs, "a list of comma-separated prometheus push gatway address") flag.IntVar(&cmdOps.PushInterval, "push-interval", cmdOps.PushInterval, "push interval in seconds") flag.StringVar(&cmdOps.LocalCfgDir, "local-cfg-dir", cmdOps.LocalCfgDir, "local config dir") From 8cdadf6c8d374cdb52c1d328efff4ee8b563fa28 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 28 Nov 2020 22:15:04 +0800 Subject: [PATCH 013/404] quote column names --- Dockerfile | 2 +- Dockerfile_goreleaser | 2 +- input/kafka_go.go | 2 +- output/clickhouse.go | 4 +++- util/common.go | 3 --- 5 files changed, 6 insertions(+), 7 deletions(-) diff --git a/Dockerfile b/Dockerfile index e456121a..8dabd851 100644 --- a/Dockerfile +++ b/Dockerfile @@ -15,7 +15,7 @@ COPY --from=builder /app/dist/nacos_publish_config /usr/local/bin/nacos_publish_ # Customize behavior with following env variables: # - V # - HTTP_PORT -# - PUSH_GATEWAY_ADDRS +# - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL # - LOCAL_CFG_DIR # - CONSUL_REGISTER_ENABLE diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index 75160645..83536bb5 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -8,7 +8,7 @@ ADD ./nacos_publish_config /usr/local/bin/nacos_publish_config # Customize behavior with following env variables: # - V # - HTTP_PORT -# - PUSH_GATEWAY_ADDRS +# - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL # - LOCAL_CFG_DIR # - CONSUL_REGISTER_ENABLE diff --git a/input/kafka_go.go b/input/kafka_go.go index 0a5eca48..ae003461 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -67,7 +67,7 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model MaxBytes: k.taskCfg.BufferSize * k.taskCfg.MsgSizeHint, MaxWait: time.Duration(k.taskCfg.FlushInterval) * time.Second, CommitInterval: time.Second, // flushes commits to Kafka every second - Logger: log.StandardLogger(), + ErrorLogger: log.StandardLogger(), //kafka-go INFO log is too verbose } var dialer *kafka.Dialer if kfkCfg.TLS.Enable { diff --git a/output/clickhouse.go b/output/clickhouse.go index e9e118c8..dea64835 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -213,14 +213,16 @@ func (c *ClickHouse) initSchema() (err error) { } //根据 Dms 生成prepare的sql语句 c.Dms = make([]string, 0, len(c.Dims)) + quotedDms := make([]string, 0, len(c.Dims)) for _, d := range c.Dims { c.Dms = append(c.Dms, d.Name) + quotedDms = append(quotedDms, fmt.Sprintf("`%s`", d.Name)) } var params = make([]string, len(c.Dims)) for i := range params { params[i] = "?" } - c.prepareSQL = "INSERT INTO " + c.chCfg.DB + "." + c.taskCfg.TableName + " (" + strings.Join(c.Dms, ",") + ") " + + c.prepareSQL = "INSERT INTO " + c.chCfg.DB + "." + c.taskCfg.TableName + " (" + strings.Join(quotedDms, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" log.Infof("%s: Prepare sql=> %s", c.taskCfg.Name, c.prepareSQL) diff --git a/util/common.go b/util/common.go index 12c97558..f2d39ab1 100644 --- a/util/common.go +++ b/util/common.go @@ -66,9 +66,6 @@ func StringContains(arr []string, str string) bool { // GetSourceName returns the field name in message for the given ClickHouse column func GetSourceName(name string) (sourcename string) { sourcename = strings.Replace(name, ".", "\\.", -1) - if strings.HasPrefix(sourcename, "_") && !strings.HasPrefix(sourcename, "__") { - sourcename = "@" + sourcename[1:] - } return } From 3d1dcf539ecbf8437b03aea9aa61933a797ab90b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 30 Nov 2020 23:22:15 +0800 Subject: [PATCH 014/404] reduced WritingPoolBacklog channel size --- input/kafka_sarama.go | 1 - output/clickhouse.go | 1 - util/common.go | 2 +- 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index aee581aa..1a918ee0 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -56,7 +56,6 @@ func (h MyConsumerGroupHandler) Setup(sess sarama.ConsumerGroupSession) error { } func (h MyConsumerGroupHandler) Cleanup(_ sarama.ConsumerGroupSession) error { log.Infof("%s: consumer group %s cleanup", h.k.taskCfg.Name, h.k.taskCfg.ConsumerGroup) - //TODO: Flush all rings helps to consuming duplicated messages? time.Sleep(5 * time.Second) return nil } diff --git a/output/clickhouse.go b/output/clickhouse.go index dea64835..f21fa910 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -70,7 +70,6 @@ func (c *ClickHouse) Init() (err error) { // Send a batch to clickhouse func (c *ClickHouse) Send(batch *model.Batch, callback func(batch *model.Batch) error) { - // TODO workerpool parallel statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Inc() _ = util.GlobalWritingPool.Submit(func() { c.loopWrite(batch, callback) diff --git a/util/common.go b/util/common.go index f2d39ab1..02963de4 100644 --- a/util/common.go +++ b/util/common.go @@ -50,7 +50,7 @@ func InitGlobalParsingPool(maxWorkers int) { // InitGlobalWritingPool initialize GlobalWritingPool func InitGlobalWritingPool(maxWorkers int) { - GlobalWritingPool = NewWorkerPool(maxWorkers, runtime.NumCPU()) + GlobalWritingPool = NewWorkerPool(maxWorkers, runtime.NumCPU()/4) } // StringContains check if contains string in array From 73b4bd4b7fbd84919f4a0f855dccbd7b72a887f0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 2 Dec 2020 12:03:44 +0800 Subject: [PATCH 015/404] tolerates replica single-point-failure --- README.md | 11 +++++---- conf/config.json | 8 ++++-- config/config.go | 6 ++--- design.md | 10 ++++---- docker/conf/config.json | 8 ++++-- input/kafka_go.go | 2 +- output/clickhouse.go | 2 +- pool/conn.go | 55 ++++++++++++++++++++++------------------- 8 files changed, 58 insertions(+), 44 deletions(-) diff --git a/README.md b/README.md index 5d3124af..8a4f4155 100644 --- a/README.md +++ b/README.md @@ -19,8 +19,9 @@ Refers to [design](./design.md) for how it works. - Bulk insert (by config `bufferSize` and `flushInterval`). - Parse messages concurrently. - Write batches concurrently. -- Every batch is sharded to a determined clickhouse node. Exit if loop write fail. +- Every batch is routed to a determined clickhouse shard. Exit if loop write fail. - Custom sharding policy (by config `shardingKey` and `shardingPolicy`). +- Tolerate replica single-point-failure. - At least once delivery guarantee. - Dynamic config management with Nacos. @@ -185,14 +186,14 @@ Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://d ### Sharding Policy -Every message is routed to a determined ClickHouse node. +Every message is routed to a determined ClickHouse shard. -By default, the node number is caculated by `(kafka_offset/roundup(batch_size))%clickhouse_nodes`, where `roundup()` round upward an unsigned integer to the the nearest 2^n. +By default, the node number is caculated by `(kafka_offset/roundup(batch_size))%clickhouse_shards`, where `roundup()` round upward an unsigned integer to the the nearest 2^n. This above expression can be customized with `shardingKey` and `shardingPolicy`. `shardingKey` value is a column name. `shardingPolicy` value could be: -- `stripe,`. This requires `shardingKey` be a numeric-like (bool, int, float, date etc.) column. The expression is `(uint64(shardingKey)/stripe_size)%clickhouse_nodes`. -- `hash`. This requires `shardingKey` be a string-like column. The hash function used internally is [xxHash64](https://github.com/cespare/xxhash). The expression is `xxhash64(string(shardingKey))%clickhouse_nodes`. +- `stripe,`. This requires `shardingKey` be a numeric-like (bool, int, float, date etc.) column. The expression is `(uint64(shardingKey)/stripe_size)%clickhouse_shards`. +- `hash`. This requires `shardingKey` be a string-like column. The hash function used internally is [xxHash64](https://github.com/cespare/xxhash). The expression is `xxhash64(string(shardingKey))%clickhouse_shards`. ## Configuration Management diff --git a/conf/config.json b/conf/config.json index be108b08..c2397708 100644 --- a/conf/config.json +++ b/conf/config.json @@ -2,7 +2,11 @@ "clickhouse": { "ch1": { "db": "default", - "host": "127.0.0.1", + "hosts": [ + [ + "127.0.0.1" + ] + ], "password": "", "retryTimes": 0, "port": 9000, @@ -37,4 +41,4 @@ "flushInterval": 5, "logLevel": "debug" } -} +} \ No newline at end of file diff --git a/config/config.go b/config/config.go index 7239a3e5..7b156ca4 100644 --- a/config/config.go +++ b/config/config.go @@ -101,9 +101,9 @@ type KafkaConfig struct { // ClickHouseConfig configuration parameters type ClickHouseConfig struct { - DB string - Host string - Port int + DB string + Hosts [][]string + Port int Username string Password string diff --git a/design.md b/design.md index d8a7f84d..869ae540 100644 --- a/design.md +++ b/design.md @@ -5,7 +5,7 @@ clickhouse_sinker guarantee: - at-least-once -- Duplicated messages (per topic-partition-offset) are routed to the same ClickHouse node +- Duplicated messages (per topic-partition-offset) are routed to the same ClickHouse shard. So if you setup ClickHouse properly(ReplacingMergeTree ORDER BY (__kafak_topic, __kafka_partition, __kafka_offset)), you could get exactly-once semantic. @@ -16,14 +16,14 @@ The flow is: - Fetch message via kafka-go or samara, which starts internally an goroutine for each partition. - Parse messages in a global goroutine pool(pool size is customizable), fill the result to a ring according to the message's partition and offset. - Generate a batch if messages in a ring reach a batchSize bondary, or flush timer fire. This ensures offset/batchSize be same for all messages inside a batch. -- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse instances). Batch is routed according to (offset/batchSize)%num_clickhouse_instances. +- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). Batch is routed according to `(kafka_offset/roundup(batch_size))%clickhouse_shards`. ## Sharding with custom key and policy clickhouse_sinker guarantee: - at-least-once -- Every message is routed to the determined (per `shardingKey` and `shardingPolicy`) ClickHouse node. +- Every message is routed to the determined (per `shardingKey` and `shardingPolicy`) ClickHouse shard. `shardingKey` value is a column name. `shardingPolicy` value is `stripe,` or `hash`. The hash function used internally is xxHash64. @@ -32,9 +32,9 @@ The flow is: - Fetch message via kafka-go or samara, which starts internally an goroutine for each partition. - Parse messages in a global goroutine pool(pool size is customizable), fill the result to a ring according to the message's partition and offset. -- Shard messages in a ring if reach a batchSize bondary, or flush timer fire. There's one-to-one relationship between shard slots and ClickHouse nodes. +- Shard messages in a ring if reach a batchSize bondary, or flush timer fire. There's one-to-one relationship between shard slots and ClickHouse shards. - Generate batches for all shard slots if messages in one shard slot reach batchSize, or flush timer fire. Those batches form a `BatchGroup`. The `before` relationship could be impossilbe if messages of a partition are distributed to multiple batches. So those batches need to be committed after ALL of them have been written to clickhouse. -- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse instances). +- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). # Remote Config Management diff --git a/docker/conf/config.json b/docker/conf/config.json index 5b27a420..f6f3d8ef 100644 --- a/docker/conf/config.json +++ b/docker/conf/config.json @@ -2,7 +2,11 @@ "clickhouse": { "ch1": { "db": "default", - "host": "127.0.0.1", + "hosts": [ + [ + "127.0.0.1" + ] + ], "password": "", "retryTimes": 0, "port": 9000, @@ -36,4 +40,4 @@ "flushInterval": 5, "logLevel": "debug" } -} +} \ No newline at end of file diff --git a/input/kafka_go.go b/input/kafka_go.go index ae003461..287bd111 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -66,7 +66,7 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model MinBytes: k.taskCfg.MinBufferSize * k.taskCfg.MsgSizeHint, MaxBytes: k.taskCfg.BufferSize * k.taskCfg.MsgSizeHint, MaxWait: time.Duration(k.taskCfg.FlushInterval) * time.Second, - CommitInterval: time.Second, // flushes commits to Kafka every second + CommitInterval: time.Second, // flushes commits to Kafka every second ErrorLogger: log.StandardLogger(), //kafka-go INFO log is too verbose } var dialer *kafka.Dialer diff --git a/output/clickhouse.go b/output/clickhouse.go index f21fa910..76037945 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -59,7 +59,7 @@ func NewClickHouse(cfg *config.Config, taskName string) *ClickHouse { // Init the clickhouse intance func (c *ClickHouse) Init() (err error) { - if err = pool.InitConn(c.taskCfg.Clickhouse, c.chCfg.Host, c.chCfg.Port, c.chCfg.DB, c.chCfg.Username, c.chCfg.Password, c.chCfg.DsnParams); err != nil { + if err = pool.InitConn(c.taskCfg.Clickhouse, c.chCfg.Hosts, c.chCfg.Port, c.chCfg.DB, c.chCfg.Username, c.chCfg.Password, c.chCfg.DsnParams); err != nil { return } if err = c.initSchema(); err != nil { diff --git a/pool/conn.go b/pool/conn.go index ac1e1a80..00ecf33b 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -65,8 +65,7 @@ func (c *Connection) ReConnect() error { return nil } -func InitConn(name, hosts string, port int, db, username, password, dsnParams string) (err error) { - var ips, ips2, dsnArr []string +func InitConn(name string, hosts [][]string, port int, db, username, password, dsnParams string) (err error) { var sqlDB *sql.DB lock.Lock() if poolMaps == nil { @@ -78,37 +77,37 @@ func InitConn(name, hosts string, port int, db, username, password, dsnParams st return } lock.Unlock() - // if contains ',', that means it's a ip list - if strings.Contains(hosts, ",") { - ips = strings.Split(strings.TrimSpace(hosts), ",") - } else { - ips = []string{hosts} - } - for _, ip := range ips { - if ips2, err = utils.GetIp4Byname(ip); err != nil { - // fallback to ip - err = nil - } else { - ip = ips2[0] + + var cc ClusterConnections + cc.ref = 1 + // Each shard has a *sql.DB which connects to all replicas inside the shard. + // "alt_hosts" tolerates replica single-point-failure. + for _, replicas := range hosts { + numReplicas := len(replicas) + replicaAddrs := make([]string, numReplicas) + for i, ip := range replicas { + if ips2, err := utils.GetIp4Byname(ip); err == nil { + ip = ips2[0] + } + replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) + } + dsn := fmt.Sprintf("tcp://%s?database=%s&username=%s&password=%s&block_size=%d", + replicaAddrs[0], db, username, password, BlockSize) + if numReplicas > 1 { + dsn += "&connection_open_strategy=in_order&alt_hosts=" + strings.Join(replicaAddrs[1:numReplicas], ",") } - dsn := fmt.Sprintf("tcp://%s:%d?database=%s&username=%s&password=%s&block_size=%d", - ip, port, db, username, password, BlockSize) if dsnParams != "" { dsn += "&" + dsnParams } - dsnArr = append(dsnArr, dsn) - } - - log.Infof("clickhouse dsn of %s: %+v", name, dsnArr) - var cc ClusterConnections - cc.ref = 1 - for _, dsn := range dsnArr { if sqlDB, err = sql.Open("clickhouse", dsn); err != nil { err = errors.Wrapf(err, "") return } + sqlDB.SetMaxIdleConns(1) + sqlDB.SetConnMaxIdleTime(10 * time.Second) cc.connections = append(cc.connections, &Connection{sqlDB, dsn}) } + lock.Lock() defer lock.Unlock() if cc2, ok := poolMaps[name]; ok { @@ -116,7 +115,10 @@ func InitConn(name, hosts string, port int, db, username, password, dsnParams st return } for _, conn := range cc.connections { - health.Health.AddReadinessCheck(conn.dsn, healthcheck.DatabasePingCheck(conn.DB, 30*time.Second)) + if err = health.Health.AddReadinessCheck(conn.dsn, healthcheck.DatabasePingCheck(conn.DB, 30*time.Second)); err != nil { + err = errors.Wrapf(err, "") + log.Errorf("got error: %+v", err) + } } poolMaps[name] = &cc return nil @@ -131,7 +133,10 @@ func FreeConn(name string) { delete(poolMaps, name) } for _, conn := range cc.connections { - health.Health.RemoveReadinessCheck(conn.dsn) + if err := health.Health.RemoveReadinessCheck(conn.dsn); err != nil { + err = errors.Wrapf(err, "") + log.Errorf("got error: %+v", err) + } } } } From 6e0a8e43ebc05462c4e8f92aa06e4471f62188ee Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 2 Dec 2020 14:14:53 +0800 Subject: [PATCH 016/404] added SASL/SCRAM support --- README.md | 78 +++++++++++++++++++++++++++++++++++-------- config/config.go | 25 +++++++++++--- go.mod | 1 + input/kafka_go.go | 24 +++++++++---- input/kafka_sarama.go | 50 +++++++++++++++++++++++---- 5 files changed, 148 insertions(+), 30 deletions(-) diff --git a/README.md b/README.md index 8a4f4155..0553990a 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,7 @@ Refers to [design](./design.md) for how it works. - Easy to use and deploy, you don't need write any hard code, just care about the configuration file - Support multiple parsers: fastjson(recommended), gjson, csv. - Support multiple Kafka client: kafka-go(recommended), sarama. -- Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/GSSAPI and combinations of them. +- Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/SCRAM, SASL/GSSAPI and combinations of them. - Support multiple sinker tasks, each runs on parallel. - Support multiply kafka and ClickHouse clusters. - Bulk insert (by config `bufferSize` and `flushInterval`). @@ -83,6 +83,7 @@ An example kafka config: ``` "kfk1": { "brokers": "192.168.31.64:9092", + "@version": "Required if you use sarama. It's the the Kafka server version.", "version": "2.2.1" } ``` @@ -93,6 +94,7 @@ An example kafka config: ``` "kfk2": { "brokers": "192.168.31.64:9093", + "version": "2.2.1", "tls": { "enable": true, "@caCertFiles": "Required. It's the CA certificate with which Kafka brokers certs be signed. This cert is added to kafka.client.truststore.jks which kafka-console-consumer.sh uses", @@ -106,13 +108,13 @@ An example kafka config: FYI. `kafka-console-consumer.sh` works as the following setup: ``` -$ cat config/client_SSL.properties +$ cat config/SSL_NOAUTH_client.properties security.protocol=SSL ssl.truststore.location=/etc/security/kafka.client.truststore.jks ssl.truststore.password=123456 ssl.endpoint.identification.algorithm= -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/client_SSL.properties +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/SSL_NOAUTH_client.properties ``` Please follow [`Kafka SSL setup`](https://kafka.apache.org/documentation/#security_ssl). Use `-keyalg RSA` when you create the broker keystore, otherwise there will be no cipher suites in common between the keystore and those Golang supports. See [this](https://github.com/Shopify/sarama/issues/643#issuecomment-216839760) for reference. @@ -137,25 +139,73 @@ An example kafka config: An example kafka config: ``` - "kfk2": { - "brokers": "192.168.31.64:9093", + "kfk3": { + "brokers": "192.168.31.64:9094", + "version": "2.2.1", "sasl": { "enable": true, - "password": "username", - "username": "password" - }, - "version": "2.2.1" + "mechanism": "PLAIN", + "username": "alice", + "password": "alice-secret" + } + } +``` + +FYI. Java clients work with the following setup: + +``` +$ cat config/PLAINTEXT_PLAIN_client.properties +security.protocol=SASL_PLAINTEXT +sasl.kerberos.service.name=kafka +sasl.mechanism=PLAIN +sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required username="alice" password="alice-secret"; + +$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_PLAIN_client.properties + +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_PLAIN_client.properties +``` + +- SASL/SCRAM + +An example kafka config: +``` + "kfk4": { + "brokers": "192.168.31.64:9094", + "version": "2.2.1", + "sasl": { + "enable": true, + "@mechanism": "SCRAM-SHA-256 or SCRAM-SHA-512", + "mechanism": "SCRAM-SHA-256", + "username": "alice", + "password": "alice-secret" + } } ``` +FYI. Java clients work with the following setup: + +``` +$ cat config/PLAINTEXT_SCRAM_client.properties +security.protocol=SASL_PLAINTEXT +sasl.kerberos.service.name=kafka +sasl.mechanism=SCRAM-SHA-256 +sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required username="alice" password="alice-secret"; + +$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_SCRAM_client.properties + +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_SCRAM_client.properties +``` + - SASL/GSSAPI(Kerberos) An example kafka config: ``` - "kfk3": { + "kfk5": { "brokers": "192.168.31.64:9094", + "version": "2.2.1", "sasl": { "enable": true, + "mechanism": "GSSAPI", "gssapi": { "@authtype": "1 - Username and password, 2 - Keytab", "authtype": 2, @@ -170,16 +220,18 @@ An example kafka config: } ``` -FYI. `kafka-console-consumer.sh` works as the following setup: +FYI. Java clients work with the following setup: ``` -$ cat config/client_SASL_PLAINTEXT.properties +$ cat config/PLAINTEXT_GSSAPI_client.properties security.protocol=SASL_PLAINTEXT sasl.kerberos.service.name=kafka sasl.mechanism=GSSAPI sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab="/etc/security/mmmtest.keytab" principal="mmm@ALANWANG.COM"; -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/client_SASL_PLAINTEXT.properties +$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_GSSAPI_client.properties + +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_GSSAPI_client.properties ``` Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. I tested sarama Kerberos authentication against Kafka [2.2.1](https://archive.apache.org/dist/kafka/2.2.1/kafka_2.11-2.2.1.tgz). Not sure other Kafka versions work. diff --git a/config/config.go b/config/config.go index 7b156ca4..e658a300 100644 --- a/config/config.go +++ b/config/config.go @@ -73,6 +73,7 @@ type Config struct { // KafkaConfig configuration parameters type KafkaConfig struct { Brokers string + Version string TLS struct { Enable bool CaCertFiles string // Required. It's the CA certificate with which Kafka brokers certs be signed. @@ -82,8 +83,16 @@ type KafkaConfig struct { } //simplified sarama.Config.Net.SASL to only support SASL/PLAIN and SASL/GSSAPI(Kerberos) Sasl struct { - Enable bool + // Whether or not to use SASL authentication when connecting to the broker + // (defaults to false). + Enable bool + // Mechanism is the name of the enabled SASL mechanism. + // Possible values: PLAIN, SCRAM-SHA-256, SCRAM-SHA-512, GSSAPI (defaults to PLAIN) + Mechanism string + // Username is the authentication identity (authcid) to present for + // SASL/PLAIN or SASL/SCRAM authentication Username string + // Password for SASL/PLAIN or SASL/SCRAM authentication Password string GSSAPI struct { AuthType int //1. KRB5_USER_AUTH, 2. KRB5_KEYTAB_AUTH @@ -96,7 +105,6 @@ type KafkaConfig struct { DisablePAFXFAST bool } } - Version string } // ClickHouseConfig configuration parameters @@ -263,10 +271,19 @@ func (cfg *Config) Normallize() (err error) { if err = cfg.normallizeTasks(); err != nil { return } - for _, kfkConfig := range cfg.Kafka { + for kfkName, kfkConfig := range cfg.Kafka { if kfkConfig.Version == "" { kfkConfig.Version = "2.2.1" } + if kfkConfig.Sasl.Enable { + kfkConfig.Sasl.Mechanism = strings.ToUpper(kfkConfig.Sasl.Mechanism) + switch kfkConfig.Sasl.Mechanism { + case "PLAIN", "SCRAM-SHA-256", "SCRAM-SHA-512", "GSSAPI": + default: + err = errors.Errorf("kafka %s mechanism %s is unsupported", kfkName, kfkConfig.Sasl.Mechanism) + return + } + } } for _, chConfig := range cfg.Clickhouse { if chConfig.RetryTimes < 0 { @@ -277,7 +294,7 @@ func (cfg *Config) Normallize() (err error) { sort.Strings(taskNames) for _, taskName := range taskNames { if _, ok := cfg.Tasks[taskName]; !ok { - err = errors.Errorf("Instance %s assignment is Invalid, task %s doesn't exit", instAddr, taskName) + err = errors.Errorf("instance %s assignment is invalid, task %s doesn't exit", instAddr, taskName) return } } diff --git a/go.mod b/go.mod index 0290206f..2b416e21 100644 --- a/go.mod +++ b/go.mod @@ -36,6 +36,7 @@ require ( github.com/tidwall/match v1.0.0 // indirect github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/valyala/fastjson v1.4.1 + github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208 // indirect golang.org/x/sys v0.0.0-20200917061948-648f2a039071 // indirect golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e diff --git a/input/kafka_go.go b/input/kafka_go.go index 287bd111..1275485c 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -25,6 +25,7 @@ import ( "github.com/pkg/errors" "github.com/segmentio/kafka-go" "github.com/segmentio/kafka-go/sasl/plain" + "github.com/segmentio/kafka-go/sasl/scram" log "github.com/sirupsen/logrus" "github.com/housepower/clickhouse_sinker/config" @@ -81,16 +82,27 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model } } if kfkCfg.Sasl.Enable { - if kfkCfg.Sasl.Username != "" { - if dialer == nil { - dialer = &kafka.Dialer{DualStack: true} - } + if dialer == nil { + dialer = &kafka.Dialer{DualStack: true} + } + switch kfkCfg.Sasl.Mechanism { + case "PLAIN": dialer.SASLMechanism = plain.Mechanism{ Username: kfkCfg.Sasl.Username, Password: kfkCfg.Sasl.Password, } - } else { - return errors.Errorf("kafka-go doesn't support SASL/GSSAPI(Kerberos)") + case "SCRAM-SHA-256": + if dialer.SASLMechanism, err = scram.Mechanism(scram.SHA256, kfkCfg.Sasl.Username, kfkCfg.Sasl.Password); err != nil { + err = errors.Wrapf(err, "") + return + } + case "SCRAM-SHA-512": + if dialer.SASLMechanism, err = scram.Mechanism(scram.SHA512, kfkCfg.Sasl.Username, kfkCfg.Sasl.Password); err != nil { + err = errors.Wrapf(err, "") + return + } + default: + return errors.Errorf("kafka-go doesn't support SASL/%s authentication", kfkCfg.Sasl.Mechanism) } } if dialer != nil { diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 1a918ee0..9947da44 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -17,12 +17,16 @@ package input import ( "context" + "crypto/sha256" + "crypto/sha512" + "hash" "strings" "time" "github.com/Shopify/sarama" "github.com/pkg/errors" log "github.com/sirupsen/logrus" + "github.com/xdg/scram" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" @@ -98,14 +102,17 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg m if config.Version.IsAtLeast(sarama.V1_0_0_0) { config.Net.SASL.Version = sarama.SASLHandshakeV1 } - if kfkCfg.Sasl.Username != "" { - config.Net.SASL.Mechanism = sarama.SASLTypePlaintext - config.Net.SASL.User = kfkCfg.Sasl.Username - config.Net.SASL.Password = kfkCfg.Sasl.Password - } else { - config.Net.SASL.Mechanism = sarama.SASLTypeGSSAPI - config.Net.SASL.GSSAPI = kfkCfg.Sasl.GSSAPI + config.Net.SASL.Mechanism = (sarama.SASLMechanism)(kfkCfg.Sasl.Mechanism) + switch config.Net.SASL.Mechanism { + case "SCRAM-SHA-256": + config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &XDGSCRAMClient{HashGeneratorFcn: SHA256} } + case "SCRAM-SHA-512": + config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &XDGSCRAMClient{HashGeneratorFcn: SHA256} } + default: } + config.Net.SASL.User = kfkCfg.Sasl.Username + config.Net.SASL.Password = kfkCfg.Sasl.Password + config.Net.SASL.GSSAPI = kfkCfg.Sasl.GSSAPI } if k.taskCfg.Earliest { config.Consumer.Offsets.Initial = sarama.OffsetOldest @@ -159,3 +166,32 @@ func (k *KafkaSarama) Stop() error { func (k *KafkaSarama) Description() string { return "kafka consumer of topic " + k.taskCfg.Topic } + +// Predefined SCRAMClientGeneratorFunc, copied from https://github.com/Shopify/sarama/blob/master/examples/sasl_scram_client/scram_client.go + +var SHA256 scram.HashGeneratorFcn = func() hash.Hash { return sha256.New() } +var SHA512 scram.HashGeneratorFcn = func() hash.Hash { return sha512.New() } + +type XDGSCRAMClient struct { + *scram.Client + *scram.ClientConversation + scram.HashGeneratorFcn +} + +func (x *XDGSCRAMClient) Begin(userName, password, authzID string) (err error) { + x.Client, err = x.HashGeneratorFcn.NewClient(userName, password, authzID) + if err != nil { + return err + } + x.ClientConversation = x.Client.NewConversation() + return nil +} + +func (x *XDGSCRAMClient) Step(challenge string) (response string, err error) { + response, err = x.ClientConversation.Step(challenge) + return +} + +func (x *XDGSCRAMClient) Done() bool { + return x.ClientConversation.Done() +} From bd9d7f76a975b0e79e9b3563a0e1a89baaea6459 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 3 Dec 2020 10:06:39 +0800 Subject: [PATCH 017/404] fix possible zero queueSize --- README.md | 2 +- cmd/clickhouse_sinker/main.go | 6 +++--- util/common.go | 6 +++++- 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 0553990a..b5b34b00 100644 --- a/README.md +++ b/README.md @@ -15,7 +15,7 @@ Refers to [design](./design.md) for how it works. - Support multiple Kafka client: kafka-go(recommended), sarama. - Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/SCRAM, SASL/GSSAPI and combinations of them. - Support multiple sinker tasks, each runs on parallel. -- Support multiply kafka and ClickHouse clusters. +- Support multiple kafka and ClickHouse clusters. - Bulk insert (by config `bufferSize` and `flushInterval`). - Parse messages concurrently. - Write batches concurrently. diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index f19474ea..073d13cd 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -413,7 +413,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { } } } - // 2. Stop all tasks in parallel found at the step 2. + // 2. Stop all tasks in parallel found at previous step. for _, taskName := range tasksToStop { if task, ok := s.tasks[taskName]; ok { task.NotifyStop() @@ -427,7 +427,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { log.Warnf("Failed to stop task %s. It's disappeared.", taskName) } } - // 3. Initailize all tasks which is new or its config differ. + // 3. Initailize all tasks which are new or their config differ. var newTasks []*task.Service if taskNames, ok := newCfg.Assignment[selfAddr]; ok { for _, taskName := range taskNames { @@ -450,7 +450,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { totalConn := pool.GetTotalConn() util.GlobalWritingPool.Resize(totalConn) - // 5. Start new tasks. We don't do it at step 4 in order to avoid goroutine leak due to errors raised by later steps. + // 5. Start new tasks. We don't do it at step 3 in order to avoid goroutine leak due to errors raised by later steps. for _, t := range newTasks { go t.Run(s.ctx) } diff --git a/util/common.go b/util/common.go index 02963de4..16be5669 100644 --- a/util/common.go +++ b/util/common.go @@ -50,7 +50,11 @@ func InitGlobalParsingPool(maxWorkers int) { // InitGlobalWritingPool initialize GlobalWritingPool func InitGlobalWritingPool(maxWorkers int) { - GlobalWritingPool = NewWorkerPool(maxWorkers, runtime.NumCPU()/4) + queueSize := runtime.NumCPU() / 4 + if queueSize < 3 { + queueSize = 3 + } + GlobalWritingPool = NewWorkerPool(maxWorkers, queueSize) } // StringContains check if contains string in array From 7403b75351d6c77e50b12a4cb4bb66a4d561de99 Mon Sep 17 00:00:00 2001 From: sundyli <543950155@qq.com> Date: Fri, 4 Dec 2020 09:05:54 +0800 Subject: [PATCH 018/404] Update conn.go --- pool/conn.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/pool/conn.go b/pool/conn.go index 00ecf33b..f07901cf 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -60,6 +60,7 @@ func (c *Connection) ReConnect() error { log.Info("reconnect to ", c.dsn, err.Error()) return err } + setDbParams(sqlDB) log.Info("reconnect success to ", c.dsn) c.DB = sqlDB return nil @@ -103,8 +104,7 @@ func InitConn(name string, hosts [][]string, port int, db, username, password, d err = errors.Wrapf(err, "") return } - sqlDB.SetMaxIdleConns(1) - sqlDB.SetConnMaxIdleTime(10 * time.Second) + setDbParams(sqlDB) cc.connections = append(cc.connections, &Connection{sqlDB, dsn}) } @@ -124,6 +124,11 @@ func InitConn(name string, hosts [][]string, port int, db, username, password, d return nil } +func setDbParams(sqlDB *sql.DB) { + sqlDB.SetMaxIdleConns(1) + sqlDB.SetConnMaxIdleTime(10 * time.Second) +} + func FreeConn(name string) { lock.Lock() defer lock.Unlock() From 23f3cab2edca6ffb313e7ba1c1cc41939df03c53 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 3 Dec 2020 10:06:39 +0800 Subject: [PATCH 019/404] fix possible zero queueSize --- README.md | 4 ++-- cmd/clickhouse_sinker/main.go | 6 +++--- util/common.go | 6 +++++- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 0553990a..e22b01a0 100644 --- a/README.md +++ b/README.md @@ -15,7 +15,7 @@ Refers to [design](./design.md) for how it works. - Support multiple Kafka client: kafka-go(recommended), sarama. - Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/SCRAM, SASL/GSSAPI and combinations of them. - Support multiple sinker tasks, each runs on parallel. -- Support multiply kafka and ClickHouse clusters. +- Support multiple kafka and ClickHouse clusters. - Bulk insert (by config `bufferSize` and `flushInterval`). - Parse messages concurrently. - Write batches concurrently. @@ -240,7 +240,7 @@ Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://d Every message is routed to a determined ClickHouse shard. -By default, the node number is caculated by `(kafka_offset/roundup(batch_size))%clickhouse_shards`, where `roundup()` round upward an unsigned integer to the the nearest 2^n. +By default, the shard number is caculated by `(kafka_offset/roundup(batch_size))%clickhouse_shards`, where `roundup()` round upward an unsigned integer to the the nearest 2^n. This above expression can be customized with `shardingKey` and `shardingPolicy`. `shardingKey` value is a column name. `shardingPolicy` value could be: diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index f19474ea..073d13cd 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -413,7 +413,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { } } } - // 2. Stop all tasks in parallel found at the step 2. + // 2. Stop all tasks in parallel found at previous step. for _, taskName := range tasksToStop { if task, ok := s.tasks[taskName]; ok { task.NotifyStop() @@ -427,7 +427,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { log.Warnf("Failed to stop task %s. It's disappeared.", taskName) } } - // 3. Initailize all tasks which is new or its config differ. + // 3. Initailize all tasks which are new or their config differ. var newTasks []*task.Service if taskNames, ok := newCfg.Assignment[selfAddr]; ok { for _, taskName := range taskNames { @@ -450,7 +450,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { totalConn := pool.GetTotalConn() util.GlobalWritingPool.Resize(totalConn) - // 5. Start new tasks. We don't do it at step 4 in order to avoid goroutine leak due to errors raised by later steps. + // 5. Start new tasks. We don't do it at step 3 in order to avoid goroutine leak due to errors raised by later steps. for _, t := range newTasks { go t.Run(s.ctx) } diff --git a/util/common.go b/util/common.go index 02963de4..16be5669 100644 --- a/util/common.go +++ b/util/common.go @@ -50,7 +50,11 @@ func InitGlobalParsingPool(maxWorkers int) { // InitGlobalWritingPool initialize GlobalWritingPool func InitGlobalWritingPool(maxWorkers int) { - GlobalWritingPool = NewWorkerPool(maxWorkers, runtime.NumCPU()/4) + queueSize := runtime.NumCPU() / 4 + if queueSize < 3 { + queueSize = 3 + } + GlobalWritingPool = NewWorkerPool(maxWorkers, queueSize) } // StringContains check if contains string in array From f2c1fea4d68a3eaf457d4f0cccae3be3cfa29f22 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 8 Dec 2020 09:37:58 +0800 Subject: [PATCH 020/404] minor improvement of systest config --- .github/workflows/lint.yml | 4 +-- Makefile | 2 +- docker/conf/config.json | 1 + docker/conf/tasks/test1.json | 42 ++++++++++++------------- docker/conf/tasks/test_auto_schema.json | 27 +++++++--------- go.test.sh | 22 ++++++------- output/clickhouse.go | 22 ++++++------- pool/conn.go | 18 +++++------ 8 files changed, 65 insertions(+), 73 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 6f057d3a..e1ca2ff0 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -16,13 +16,13 @@ jobs: uses: golangci/golangci-lint-action@v2 with: # Required: the version of golangci-lint is required and must be specified without patch version: we always use the latest patch version. - version: v1.32.2 + version: v1.33 # Optional: working directory, useful for monorepos # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true \ No newline at end of file diff --git a/Makefile b/Makefile index 33ed3a6c..fb903bc5 100644 --- a/Makefile +++ b/Makefile @@ -25,7 +25,7 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-dir conf/ diff --git a/docker/conf/config.json b/docker/conf/config.json index f6f3d8ef..bd7340c2 100644 --- a/docker/conf/config.json +++ b/docker/conf/config.json @@ -38,6 +38,7 @@ "bufferSize": 90000, "minBufferSize": 2000, "flushInterval": 5, + "layoutDateTime": "2006-01-02 15:04:05.999999999Z07:00", "logLevel": "debug" } } \ No newline at end of file diff --git a/docker/conf/tasks/test1.json b/docker/conf/tasks/test1.json index 8a78f9b9..0c72cd60 100644 --- a/docker/conf/tasks/test1.json +++ b/docker/conf/tasks/test1.json @@ -1,25 +1,25 @@ { - - "name" : "test1", - + "name": "test1", "kafka": "kfk1", "topic": "topic1", - "consumerGroup" : "test_sinker", - "earliest" : true, - "parser" : "json", - "clickhouse" : "ch1", - - "tableName" : "test1", - - "dims" : [ - {"name" : "timestamp" , "type" : "UInt64"}, - {"name" : "name" , "type" : "String"} + "consumerGroup": "test_sinker", + "earliest": true, + "parser": "json", + "clickhouse": "ch1", + "tableName": "test1", + "dims": [ + { + "name": "time", + "type": "DateTime" + }, + { + "name": "name", + "type": "String" + }, + { + "name": "value", + "type": "Float32" + } ], - - "metrics" : [ - {"name" : "value" , "type" : "Float32"} - ], - - "bufferSize" : 50000 -} - + "bufferSize": 50000 +} \ No newline at end of file diff --git a/docker/conf/tasks/test_auto_schema.json b/docker/conf/tasks/test_auto_schema.json index e6b6d381..4c0161b5 100644 --- a/docker/conf/tasks/test_auto_schema.json +++ b/docker/conf/tasks/test_auto_schema.json @@ -1,18 +1,15 @@ { - - "name" : "test_auto_schema", - + "name": "test_auto_schema", "kafka": "kfk1", "topic": "topic1", - "consumerGroup" : "test_auto_schema", - "earliest" : true, - "parser" : "json", - "clickhouse" : "ch1", - - "autoSchema" : true, - "tableName" : "test_auto_schema", - "excludeColumns" : ["day", "time"], - - "bufferSize" : 50000 -} - + "consumerGroup": "test_auto_schema", + "earliest": true, + "parser": "json", + "clickhouse": "ch1", + "autoSchema": true, + "tableName": "test_auto_schema", + "excludeColumns": [ + "day" + ], + "bufferSize": 50000 +} \ No newline at end of file diff --git a/go.test.sh b/go.test.sh index 78f33538..44ef3c2a 100755 --- a/go.test.sh +++ b/go.test.sh @@ -5,36 +5,33 @@ curl "localhost:8123" -d 'DROP TABLE IF EXISTS test1' curl "localhost:8123" -d 'CREATE TABLE IF NOT EXISTS test1 ( `day` Date DEFAULT toDate(time), - `time` DateTime DEFAULT toDateTime(timestamp / 1000), - `timestamp` UInt64, + `time` DateTime, `name` String, `value` Float64 ) ENGINE = MergeTree PARTITION BY day -ORDER BY time' +ORDER BY (time, name)' curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_auto_schema' curl "localhost:8123" -d 'CREATE TABLE IF NOT EXISTS test_auto_schema ( `day` Date DEFAULT toDate(time), - `time` DateTime DEFAULT toDateTime(timestamp / 1000), - `timestamp` UInt64, + `time` DateTime, `name` String, `value` Float64 ) ENGINE = MergeTree PARTITION BY day -ORDER BY time' - +ORDER BY (time, name)' ## send the messages to kafka -current_timestamp=`date +%s`000 +now=`date --rfc-3339=ns` for i in `seq 1 100000`;do - echo "{\"timestamp\" : \"${current_timestamp}\", \"name\" : \"sundy-li\", \"value\" : \"$i\" }" + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : \"$i\" }" done > a.json +echo "generated a.json" echo "cat /tmp/a.json | kafka-console-producer --topic topic1 --broker-list localhost:9092" > send.sh - sudo docker cp a.json kafka:/tmp/ sudo docker cp send.sh kafka:/tmp/ sudo docker exec kafka sh /tmp/send.sh @@ -53,9 +50,8 @@ echo "Got test_auto_schema count => $count" ## reset kafka consumer-group offsets -echo "kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_sinker --all-topics --to-earliest; kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_auto_schema --all-topics --to-earliest" > reset-offsets.sh -sudo docker cp reset-offsets.sh kafka:/tmp/ -sudo docker exec kafka sh /tmp/reset-offsets.sh +sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_sinker --all-topics --to-earliest +sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_auto_schema --all-topics --to-earliest ## truncate tables curl "localhost:8123" -d 'TRUNCATE TABLE test1' diff --git a/output/clickhouse.go b/output/clickhouse.go index 76037945..89b04735 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -139,7 +139,7 @@ func shouldReconnect(err error) bool { // LoopWrite will dead loop to write the records func (c *ClickHouse) loopWrite(batch *model.Batch, callback func(batch *model.Batch) error) { var err error - times := c.chCfg.RetryTimes + var times int for { if err = c.write(batch); err == nil { for { @@ -150,13 +150,12 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, callback func(batch *model.Ba log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.taskCfg.Name) return } - log.Errorf("%s: committing offset(try #%d) failed with error %+v", c.taskCfg.Name, c.chCfg.RetryTimes-times, err) - if c.chCfg.RetryTimes > 0 { - times-- - if times <= 0 { - os.Exit(-1) - } + log.Errorf("%s: committing offset(try #%d) failed with error %+v", c.taskCfg.Name, times, err) + times++ + if c.chCfg.RetryTimes <= 0 || times < c.chCfg.RetryTimes { time.Sleep(10 * time.Second) + } else { + os.Exit(-1) } } } @@ -166,12 +165,11 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, callback func(batch *model.Ba } log.Errorf("%s: flush batch(try #%d) failed with error %+v", c.taskCfg.Name, c.chCfg.RetryTimes-times, err) statistics.FlushMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) - if c.chCfg.RetryTimes > 0 { - times-- - if times <= 0 { - os.Exit(-1) - } + times++ + if c.chCfg.RetryTimes <= 0 || times < c.chCfg.RetryTimes { time.Sleep(10 * time.Second) + } else { + os.Exit(-1) } } } diff --git a/pool/conn.go b/pool/conn.go index f07901cf..93ee5c4b 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -60,7 +60,7 @@ func (c *Connection) ReConnect() error { log.Info("reconnect to ", c.dsn, err.Error()) return err } - setDbParams(sqlDB) + setDBParams(sqlDB) log.Info("reconnect success to ", c.dsn) c.DB = sqlDB return nil @@ -104,7 +104,7 @@ func InitConn(name string, hosts [][]string, port int, db, username, password, d err = errors.Wrapf(err, "") return } - setDbParams(sqlDB) + setDBParams(sqlDB) cc.connections = append(cc.connections, &Connection{sqlDB, dsn}) } @@ -124,7 +124,7 @@ func InitConn(name string, hosts [][]string, port int, db, username, password, d return nil } -func setDbParams(sqlDB *sql.DB) { +func setDBParams(sqlDB *sql.DB) { sqlDB.SetMaxIdleConns(1) sqlDB.SetConnMaxIdleTime(10 * time.Second) } @@ -134,13 +134,13 @@ func FreeConn(name string) { defer lock.Unlock() if cc, ok := poolMaps[name]; ok { cc.ref-- - if cc.ref <= 0 { + if cc.ref == 0 { delete(poolMaps, name) - } - for _, conn := range cc.connections { - if err := health.Health.RemoveReadinessCheck(conn.dsn); err != nil { - err = errors.Wrapf(err, "") - log.Errorf("got error: %+v", err) + for _, conn := range cc.connections { + if err := health.Health.RemoveReadinessCheck(conn.dsn); err != nil { + err = errors.Wrapf(err, conn.dsn) + log.Errorf("got error: %+v", err) + } } } } From 0429fc238bc5c6c61482de7c9089eea98f297b30 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 11 Dec 2020 11:28:25 +0800 Subject: [PATCH 021/404] added --local-cfg-file --- Dockerfile | 1 + Dockerfile_goreleaser | 1 + README.md | 13 +++- cmd/clickhouse_sinker/main.go | 19 +++++- cmd/nacos_publish_config/main.go | 18 +++++- config/config.go | 17 ++++- docker/conf/config.json | 44 ------------- docker/conf/tasks/test1.json | 25 -------- docker/conf/tasks/test_auto_schema.json | 15 ----- docker/config.json | 84 +++++++++++++++++++++++++ go.test.sh | 4 +- 11 files changed, 147 insertions(+), 94 deletions(-) delete mode 100644 docker/conf/config.json delete mode 100644 docker/conf/tasks/test1.json delete mode 100644 docker/conf/tasks/test_auto_schema.json create mode 100644 docker/config.json diff --git a/Dockerfile b/Dockerfile index 8dabd851..1b466e57 100644 --- a/Dockerfile +++ b/Dockerfile @@ -18,6 +18,7 @@ COPY --from=builder /app/dist/nacos_publish_config /usr/local/bin/nacos_publish_ # - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL # - LOCAL_CFG_DIR +# - LOCAL_CFG_FILE # - CONSUL_REGISTER_ENABLE # - CONSUL_ADDR # - CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index 83536bb5..e1eb2c0b 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -11,6 +11,7 @@ ADD ./nacos_publish_config /usr/local/bin/nacos_publish_config # - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL # - LOCAL_CFG_DIR +# - LOCAL_CFG_FILE # - CONSUL_REGISTER_ENABLE # - CONSUL_ADDR # - CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER diff --git a/README.md b/README.md index e22b01a0..9da2d61b 100644 --- a/README.md +++ b/README.md @@ -43,7 +43,7 @@ Refers to [design](./design.md) for how it works. ### By binary files (recommended) -Download the binary files from [release](https://github.com/housepower/clickhouse_sinker/releases), choose the executable binary file according to your env, modify the `conf` files, then run `./clickhouse_sinker --local-cfg-dir conf` +Download the binary files from [release](https://github.com/housepower/clickhouse_sinker/releases), choose the executable binary file according to your env, modify the `conf` files, then run `./clickhouse_sinker --local-cfg-dir conf`. If you prefer a single config file, you can merge the `conf` files into one, and pass its path to `--local-cfg-file`. ### By container image @@ -69,7 +69,7 @@ make build ## Configuration -Refers to how [integration test](./go.test.sh) use the [example config](./conf/config.json). +Refers to how [integration test](./go.test.sh) use the [example config](./docker/config.json). Also refers to [code](./config/config.go) for all config items. ### Kafka Encryption @@ -249,6 +249,11 @@ This above expression can be customized with `shardingKey` and `shardingPolicy`. ## Configuration Management +The precedence of config items: + +- CLI parameters > env variables +- Nacos > Consul > Local Config File > Local Config Dir + ### Nacos Sinker is able to register with Nacos, get and apply config changes dynamically without restart the whole process. @@ -268,6 +273,10 @@ Controled by: ### Local Files Currently sinker is able to parse local config files at startup, but unable to detect file changes. +Controled by: + +- CLI parameters: `local-cfg-file, local-cfg-dir` +- env variables: `LOCAL_CFG_FILE, LOCAL_CFG_DIR` ## Prometheus Metrics diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 073d13cd..23d1e48f 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -51,6 +51,7 @@ type CmdOptions struct { PushGatewayAddrs string PushInterval int LocalCfgDir string + LocalCfgFile string ConsulRegister bool ConsulAddr string ConsulDeregisterCriticalServiceAfter string @@ -78,6 +79,7 @@ func initCmdOptions() { PushGatewayAddrs: "", PushInterval: 10, LocalCfgDir: "/etc/clickhouse_sinker", + LocalCfgFile: "/etc/clickhouse_sinker.json", ConsulRegister: false, ConsulAddr: "http://127.0.0.1:8500", ConsulDeregisterCriticalServiceAfter: "30m", @@ -112,7 +114,8 @@ func initCmdOptions() { flag.IntVar(&cmdOps.HTTPPort, "http-port", cmdOps.HTTPPort, "http listen port") flag.StringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs", cmdOps.PushGatewayAddrs, "a list of comma-separated prometheus push gatway address") flag.IntVar(&cmdOps.PushInterval, "push-interval", cmdOps.PushInterval, "push interval in seconds") - flag.StringVar(&cmdOps.LocalCfgDir, "local-cfg-dir", cmdOps.LocalCfgDir, "local config dir") + flag.StringVar(&cmdOps.LocalCfgDir, "local-cfg-dir", cmdOps.LocalCfgDir, "local config dir. requires a file named `config.json`, and some task json files under `tasks` folder") + flag.StringVar(&cmdOps.LocalCfgFile, "local-cfg-file", cmdOps.LocalCfgFile, "local config file") flag.BoolVar(&cmdOps.ConsulRegister, "consul-register-enable", cmdOps.ConsulRegister, "register current instance in consul") flag.StringVar(&cmdOps.ConsulAddr, "consul-addr", cmdOps.ConsulAddr, "consul api interface address") @@ -262,8 +265,18 @@ func (s *Sinker) Run() { go s.pusher.Run(s.ctx) } if s.rcm == nil { - if newCfg, err = config.ParseLocalConfig(cmdOps.LocalCfgDir); err != nil { - log.Fatalf("%+v", err) + if _, err = os.Stat(cmdOps.LocalCfgFile); err == nil { + if newCfg, err = config.ParseLocalCfgFile(cmdOps.LocalCfgFile); err != nil { + log.Fatalf("%+v", err) + return + } + } else if _, err = os.Stat(cmdOps.LocalCfgFile); err == nil { + if newCfg, err = config.ParseLocalCfgDir(cmdOps.LocalCfgDir); err != nil { + log.Fatalf("%+v", err) + return + } + } else { + log.Fatalf("expect --local-cfg-file or --local-cfg-dir") return } if err = newCfg.Normallize(); err != nil { diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 7f1074db..562d62b7 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -18,6 +18,7 @@ package main import ( "flag" "fmt" + "os" "reflect" "runtime" "strconv" @@ -44,6 +45,7 @@ var ( `nacos group name. Empty string doesn't work!`) localCfgDir = flag.String("local-cfg-dir", "/etc/clickhouse_sinker", "local config dir") + localCfgFile = flag.String("local-cfg-file", "/etc/clickhouse_sinker.json", "local config file") instances = flag.String("instances", "", "a list of comma-separated ip:port[@weight]") testRegister = flag.Bool("test-register", false, "whether run TestRegister") ) @@ -62,9 +64,21 @@ func getProperties() map[string]interface{} { func PublishSinkerConfig() { var err error var cfg *config.Config - if cfg, err = config.ParseLocalConfig(*localCfgDir); err != nil { - log.Fatalf("%+v", err) + if _, err = os.Stat(*localCfgFile); err == nil { + if cfg, err = config.ParseLocalCfgFile(*localCfgFile); err != nil { + log.Fatalf("%+v", err) + return + } + } else if _, err = os.Stat(*localCfgDir); err == nil { + if cfg, err = config.ParseLocalCfgDir(*localCfgDir); err != nil { + log.Fatalf("%+v", err) + return + } + } else { + log.Fatalf("expect --local-cfg-file or --local-cfg-dir") + return } + if err = cfg.Normallize(); err != nil { log.Fatalf("%+v", err) return diff --git a/config/config.go b/config/config.go index e658a300..412de61b 100644 --- a/config/config.go +++ b/config/config.go @@ -173,7 +173,7 @@ const ( defaultTaskReplicas = 1 ) -func ParseLocalConfig(cfgPath string) (cfg *Config, err error) { +func ParseLocalCfgDir(cfgPath string) (cfg *Config, err error) { var f = "config.json" f = filepath.Join(cfgPath, f) var s string @@ -210,6 +210,21 @@ func ParseLocalConfig(cfgPath string) (cfg *Config, err error) { return } +func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { + cfg = &Config{} + var b []byte + b, err = ioutil.ReadFile(cfgPath) + if err != nil { + err = errors.Wrapf(err, "") + return + } + if err = json.Unmarshal(b, cfg); err != nil { + err = errors.Wrapf(err, "") + return + } + return +} + func (cfg *Config) AssignTasks(instances []Instance) { cfg.Assignment = make(map[string][]string) assignment := make(map[string]map[string]int) diff --git a/docker/conf/config.json b/docker/conf/config.json deleted file mode 100644 index bd7340c2..00000000 --- a/docker/conf/config.json +++ /dev/null @@ -1,44 +0,0 @@ -{ - "clickhouse": { - "ch1": { - "db": "default", - "hosts": [ - [ - "127.0.0.1" - ] - ], - "password": "", - "retryTimes": 0, - "port": 9000, - "username": "" - } - }, - "kafka": { - "kfk1": { - "brokers": "127.0.0.1:9093", - "sasl": { - "enable": false, - "password": "", - "username": "", - "gssapi": { - "authtype": 0, - "keytabpath": "", - "kerberosconfigpath": "", - "servicename": "", - "username": "", - "password": "", - "realm": "", - "disablepafxfast": false - } - }, - "version": "2.2.1" - } - }, - "common": { - "bufferSize": 90000, - "minBufferSize": 2000, - "flushInterval": 5, - "layoutDateTime": "2006-01-02 15:04:05.999999999Z07:00", - "logLevel": "debug" - } -} \ No newline at end of file diff --git a/docker/conf/tasks/test1.json b/docker/conf/tasks/test1.json deleted file mode 100644 index 0c72cd60..00000000 --- a/docker/conf/tasks/test1.json +++ /dev/null @@ -1,25 +0,0 @@ -{ - "name": "test1", - "kafka": "kfk1", - "topic": "topic1", - "consumerGroup": "test_sinker", - "earliest": true, - "parser": "json", - "clickhouse": "ch1", - "tableName": "test1", - "dims": [ - { - "name": "time", - "type": "DateTime" - }, - { - "name": "name", - "type": "String" - }, - { - "name": "value", - "type": "Float32" - } - ], - "bufferSize": 50000 -} \ No newline at end of file diff --git a/docker/conf/tasks/test_auto_schema.json b/docker/conf/tasks/test_auto_schema.json deleted file mode 100644 index 4c0161b5..00000000 --- a/docker/conf/tasks/test_auto_schema.json +++ /dev/null @@ -1,15 +0,0 @@ -{ - "name": "test_auto_schema", - "kafka": "kfk1", - "topic": "topic1", - "consumerGroup": "test_auto_schema", - "earliest": true, - "parser": "json", - "clickhouse": "ch1", - "autoSchema": true, - "tableName": "test_auto_schema", - "excludeColumns": [ - "day" - ], - "bufferSize": 50000 -} \ No newline at end of file diff --git a/docker/config.json b/docker/config.json new file mode 100644 index 00000000..c37e3b97 --- /dev/null +++ b/docker/config.json @@ -0,0 +1,84 @@ +{ + "clickhouse": { + "ch1": { + "db": "default", + "hosts": [ + [ + "127.0.0.1" + ] + ], + "password": "", + "retryTimes": 0, + "port": 9000, + "username": "" + } + }, + "kafka": { + "kfk1": { + "brokers": "127.0.0.1:9093", + "version": "2.2.1", + "sasl": { + "enable": false, + "password": "", + "username": "", + "gssapi": { + "authtype": 0, + "keytabpath": "", + "kerberosconfigpath": "", + "servicename": "", + "username": "", + "password": "", + "realm": "", + "disablepafxfast": false + } + } + } + }, + "common": { + "bufferSize": 90000, + "minBufferSize": 2000, + "flushInterval": 5, + "layoutDateTime": "2006-01-02 15:04:05.999999999Z07:00", + "logLevel": "debug" + }, + "tasks": { + "test1": { + "kafka": "kfk1", + "topic": "topic1", + "consumerGroup": "test_sinker", + "earliest": true, + "parser": "json", + "clickhouse": "ch1", + "tableName": "test1", + "dims": [ + { + "name": "time", + "type": "DateTime" + }, + { + "name": "name", + "type": "String" + }, + { + "name": "value", + "type": "Float32" + } + ], + "bufferSize": 50000 + }, + "test_auto_schema": { + "kafka": "kfk1", + "topic": "topic1", + "consumerGroup": "test_auto_schema", + "earliest": true, + "parser": "json", + "clickhouse": "ch1", + "autoSchema": true, + "tableName": "test_auto_schema", + "excludeColumns": [ + "day" + ], + "bufferSize": 50000 + } + } +} \ No newline at end of file diff --git a/go.test.sh b/go.test.sh index 44ef3c2a..f834ceec 100755 --- a/go.test.sh +++ b/go.test.sh @@ -37,7 +37,7 @@ sudo docker cp send.sh kafka:/tmp/ sudo docker exec kafka sh /tmp/send.sh ## start clickhouse_sinker to consume -timeout 30 ./dist/clickhouse_sinker --local-cfg-dir docker/conf +timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/config.json ## check result count=`curl "localhost:8123" -d 'select count() from test1'` @@ -58,7 +58,7 @@ curl "localhost:8123" -d 'TRUNCATE TABLE test1' curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' ## publish clickhouse_sinker config -./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-dir docker/conf +./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/config.json ## start clickhouse_sinker to consume timeout 30 ./dist/clickhouse_sinker --nacos-register-enable --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos From 9d242515dcf9f5ad6a9627010cd5b1b45b69792c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 11 Dec 2020 13:30:21 +0800 Subject: [PATCH 022/404] better permanent error handle --- .github/workflows/lint.yml | 2 +- Makefile | 2 +- output/clickhouse.go | 63 +++++++++++++++++++------------------- 3 files changed, 33 insertions(+), 34 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index e1ca2ff0..6b90fa74 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -22,7 +22,7 @@ jobs: # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true \ No newline at end of file diff --git a/Makefile b/Makefile index fb903bc5..79ee0beb 100644 --- a/Makefile +++ b/Makefile @@ -25,7 +25,7 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-dir conf/ diff --git a/output/clickhouse.go b/output/clickhouse.go index 89b04735..e6b1e0f5 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -17,6 +17,7 @@ package output import ( "context" + "database/sql" std_errors "errors" "fmt" "os" @@ -79,60 +80,54 @@ func (c *ClickHouse) Send(batch *model.Batch, callback func(batch *model.Batch) // Write kvs to clickhouse func (c *ClickHouse) write(batch *model.Batch) error { + var numErr int + var err, tmpErr error + var stmt *sql.Stmt + var tx *sql.Tx if len(*batch.Rows) == 0 { return nil } conn := pool.GetConn(c.taskCfg.Clickhouse, batch.BatchIdx) - tx, err := conn.Begin() - if err != nil { - if shouldReconnect(err) { - _ = conn.ReConnect() - statistics.ClickhouseReconnectTotal.WithLabelValues(c.taskCfg.Name).Inc() - } - return err + if tx, err = conn.Begin(); err != nil { + goto ERR } - - stmt, err := tx.Prepare(c.prepareSQL) - if err != nil { - log.Errorf("%s: tx.Prepare failed with error %+v", c.taskCfg.Name, err.Error()) - if shouldReconnect(err) { - _ = conn.ReConnect() - statistics.ClickhouseReconnectTotal.WithLabelValues(c.taskCfg.Name).Inc() - } - return err + if stmt, err = tx.Prepare(c.prepareSQL); err != nil { + goto ERR } - defer stmt.Close() - var numErr int for _, row := range *batch.Rows { - if _, err = stmt.Exec(*row...); err != nil { - err = errors.Wrap(err, "") + if _, tmpErr = stmt.Exec(*row...); tmpErr != nil { numErr++ + err = tmpErr } } if err != nil { log.Errorf("%s: stmt.Exec failed %d times with errors %+v", c.taskCfg.Name, numErr, err) - return err + goto ERR } if err = tx.Commit(); err != nil { - err = errors.Wrap(err, "") - if shouldReconnect(err) { - _ = conn.ReConnect() - statistics.ClickhouseReconnectTotal.WithLabelValues(c.taskCfg.Name).Inc() - } - return err + goto ERR } statistics.FlushMsgsTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) return err +ERR: + if shouldReconnect(err) { + _ = conn.ReConnect() + statistics.ClickhouseReconnectTotal.WithLabelValues(c.taskCfg.Name).Inc() + } + return err } func shouldReconnect(err error) bool { + if err == nil { + return false + } if strings.Contains(err.Error(), "connection refused") || strings.Contains(err.Error(), "bad connection") { return true } - log.Infof("not match reconnect rules: %v", err.Error()) + log.Infof("permanent error: %v", err.Error()) return false } @@ -166,7 +161,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, callback func(batch *model.Ba log.Errorf("%s: flush batch(try #%d) failed with error %+v", c.taskCfg.Name, c.chCfg.RetryTimes-times, err) statistics.FlushMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) times++ - if c.chCfg.RetryTimes <= 0 || times < c.chCfg.RetryTimes { + if shouldReconnect(err) && (c.chCfg.RetryTimes <= 0 || times < c.chCfg.RetryTimes) { time.Sleep(10 * time.Second) } else { os.Exit(-1) @@ -183,8 +178,9 @@ func (c *ClickHouse) Stop() error { func (c *ClickHouse) initSchema() (err error) { if c.taskCfg.AutoSchema { conn := pool.GetConn(c.taskCfg.Clickhouse, 0) - rs, err := conn.Query(fmt.Sprintf(selectSQLTemplate, c.chCfg.DB, c.taskCfg.TableName)) - if err != nil { + var rs *sql.Rows + if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, c.chCfg.DB, c.taskCfg.TableName)); err != nil { + err = errors.Wrapf(err, "") return err } defer rs.Close() @@ -192,7 +188,10 @@ func (c *ClickHouse) initSchema() (err error) { c.Dims = make([]*model.ColumnWithType, 0, 10) var name, typ, defaultKind string for rs.Next() { - _ = rs.Scan(&name, &typ, &defaultKind) + if err = rs.Scan(&name, &typ, &defaultKind); err != nil { + err = errors.Wrapf(err, "") + return err + } typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") if !util.StringContains(c.taskCfg.ExcludeColumns, name) && defaultKind != "MATERIALIZED" { c.Dims = append(c.Dims, &model.ColumnWithType{Name: name, Type: typ, SourceName: util.GetSourceName(name)}) From c490c9115d3b2372c520f39b1a48832921a68862 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 18 Dec 2020 14:01:35 +0000 Subject: [PATCH 023/404] workerpool.go: Panic when use zero workers in WorkerPool --- pool/conn.go | 3 ++- util/workerpool.go | 4 ++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/pool/conn.go b/pool/conn.go index 93ee5c4b..578958a7 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -124,9 +124,10 @@ func InitConn(name string, hosts [][]string, port int, db, username, password, d return nil } +// TODO, pool this func setDBParams(sqlDB *sql.DB) { sqlDB.SetMaxIdleConns(1) - sqlDB.SetConnMaxIdleTime(10 * time.Second) + sqlDB.SetConnMaxLifetime(120 * time.Second) } func FreeConn(name string) { diff --git a/util/workerpool.go b/util/workerpool.go index fcc02c57..ce386072 100644 --- a/util/workerpool.go +++ b/util/workerpool.go @@ -27,6 +27,10 @@ type WorkerPool struct { // New creates and starts a pool of worker goroutines. func NewWorkerPool(maxWorkers int, queueSize int) *WorkerPool { + if maxWorkers <= 0 { + panic("WorkerNum must be greater than zero") + } + w := &WorkerPool{ maxWorkers: maxWorkers, workChan: make(chan func(), queueSize), From 2cb790cc18a53402cdf08d92137c6d947247aff8 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 14 Dec 2020 21:25:16 +0800 Subject: [PATCH 024/404] add docs --- .github/workflows/gh-pages.yml | 19 +++++ .gitignore | 3 + dev/deploy_gh_pages.sh | 11 +++ dev/license.header | 12 +++ docs/README.md | 10 +++ docs/config.js | 135 +++++++++++++++++++++++++++++++++ design.md => docs/design.md | 0 docs/guide/README.md | 1 + docs/public/favicon.ico | Bin 0 -> 8992 bytes docs/public/logo_320px.svg | 24 ++++++ package.json | 19 +++++ 11 files changed, 234 insertions(+) create mode 100644 .github/workflows/gh-pages.yml create mode 100755 dev/deploy_gh_pages.sh create mode 100644 dev/license.header create mode 100644 docs/README.md create mode 100644 docs/config.js rename design.md => docs/design.md (100%) create mode 100644 docs/guide/README.md create mode 100644 docs/public/favicon.ico create mode 100644 docs/public/logo_320px.svg create mode 100644 package.json diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml new file mode 100644 index 00000000..ec055ad5 --- /dev/null +++ b/.github/workflows/gh-pages.yml @@ -0,0 +1,19 @@ +name: gh-pages + +on: + push: + branches: [master] + +jobs: + build: + runs-on: ubuntu-20.04 + steps: + - uses: actions/checkout@v2 + - uses: actions/setup-node@v1 + with: + node-version: "14.x" + - run: "[[ -z $(git show HEAD -- docs) ]] || npm install" + - run: "[[ -z $(git show HEAD -- docs) ]] || npm run docs:build" + - run: "[[ -z $(git show HEAD -- docs) ]] || dev/deploy_gh_pages.sh" + env: + PAGES_DEPLOY_TOKEN: ${{ secrets.PAGES_DEPLOY_TOKEN }} diff --git a/.gitignore b/.gitignore index df6965a8..dcd08b24 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,9 @@ tmp sh .vscode +node_modules/ +package-lock.json + .DS_Store *.log .idea diff --git a/dev/deploy_gh_pages.sh b/dev/deploy_gh_pages.sh new file mode 100755 index 00000000..ad68cf92 --- /dev/null +++ b/dev/deploy_gh_pages.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +cd docs/.vuepress/dist + +git config --global user.name 'GitHub Workflow' +git config --global user.email 'dummy@dummy.dummy' + +git init +git add -A +git commit -m 'Deploy GitHub Pages' +git push -f https://sundy-li:${PAGES_DEPLOY_TOKEN}@github.com/housepower/clickhouse_sinker.git master:gh-pages diff --git a/dev/license.header b/dev/license.header new file mode 100644 index 00000000..99c2228f --- /dev/null +++ b/dev/license.header @@ -0,0 +1,12 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. \ No newline at end of file diff --git a/docs/README.md b/docs/README.md new file mode 100644 index 00000000..bd5799ed --- /dev/null +++ b/docs/README.md @@ -0,0 +1,10 @@ +--- +home: true +heroImage: /logo_320px.svg +actionText: Get Started → +actionLink: /guide/introduction +features: + - title: clickhouse_sinker + details: A tool to sinker the data into ClickHouse +footer: Apache License 2.0 +--- diff --git a/docs/config.js b/docs/config.js new file mode 100644 index 00000000..691125b3 --- /dev/null +++ b/docs/config.js @@ -0,0 +1,135 @@ +module.exports = { + base: "/ClickHouse-Native-JDBC/", + title: "ClickHouse Native JDBC", + evergreen: true, + plugins: ["mermaidjs"], + locales: { + "/": { + lang: "en-US", + description: + "clickhouse_sinker a tool to sinker the data into ClickHouse", + }, + "/zh/": { + lang: "zh-CN", + description: "clickhouse_sinker 一个将数据摄入到ClickHouse的工具", + }, + }, + themeConfig: { + locales: { + "/": { + selectText: "Languages", + label: "English", + ariaLabel: "Languages", + editLinkText: "Edit this page on GitHub", + serviceWorker: { + updatePopup: { + message: "New content is available.", + buttonText: "Refresh", + }, + }, + algolia: {}, + nav: [ + { text: "Guide", link: "/guide/introduction" }, + { text: "Dev", link: "/dev/contribute" }, + { text: "Deep Dive", link: "/deep-dive/native_protocol" }, + { text: "Release Notes", link: "/release-notes/" }, + { + text: "GitHub", + link: "https://github.com/housepower/ClickHouse-Native-JDBC", + }, + ], + sidebar: { + "/guide/": [ + ["introduction", "Introduction"], + { + title: "Usage", + children: [ + ["jdbc_driver", "JDBC Driver"], + ["datasource", "Datasource"], + ["connection_pool", "Connection Pool"], + ["spark_integration", "Spark Integration"], + ], + }, + ["troubleshooting", "Troubleshooting"], + ], + "/dev/": [ + ["contribute", "Contribute Guide"], + { + title: "Release", + children: [ + ["internal_release", "Internal Release"], + ["public_release", "Public Release"], + ], + }, + ], + "/deep-dive/": [ + { + title: "ClickHouse", + children: [ + ["data_type", "DataType"], + ["native_protocol", "Native Protocol"], + ], + }, + ], + "/release-notes/": [], + }, + }, + "/zh/": { + selectText: "选择语言", + label: "简体中文", + editLinkText: "在 GitHub 上编辑此页", + serviceWorker: { + updatePopup: { + message: "发现新内容可用.", + buttonText: "刷新", + }, + }, + algolia: {}, + nav: [ + { text: "指南", link: "/zh/guide/introduction" }, + { text: "开发", link: "/zh/dev/contribute" }, + { text: "深入", link: "/zh/deep-dive/native_protocol" }, + { text: "发行注记", link: "/zh/release-notes/" }, + { + text: "源码仓库", + link: "https://github.com/housepower/ClickHouse-Native-JDBC", + }, + ], + sidebar: { + "/zh/guide/": [ + ["introduction", "简介"], + { + title: "使用", + children: [ + ["jdbc_driver", "JDBC 驱动"], + ["connection_pool", "连接池"], + ["spark_integration", "Spark 集成"], + ], + }, + ["troubleshooting", "常见问题"], + ], + "/zh/dev/": [ + ["contribute", "贡献指南"], + { + title: "发布", + children: [ + ["internal_release", "内部发布"], + ["public_release", "公开发布"], + ], + }, + ], + "/zh/deep-dive/": [ + { + title: "ClickHouse", + children: [ + ["data_type", "数据类型"], + ["native_protocol", "原生协议"], + ], + }, + ], + "/zh/release-notes/": [], + }, + }, + }, + }, +}; diff --git a/design.md b/docs/design.md similarity index 100% rename from design.md rename to docs/design.md diff --git a/docs/guide/README.md b/docs/guide/README.md new file mode 100644 index 00000000..cd3d4522 --- /dev/null +++ b/docs/guide/README.md @@ -0,0 +1 @@ +# User Guide diff --git a/docs/public/favicon.ico b/docs/public/favicon.ico new file mode 100644 index 0000000000000000000000000000000000000000..1278351a1774d32a55f7f56633135562c786ab8e GIT binary patch literal 8992 zcmch7Ra6vS)b9)ookN3km%z|n4k95vlqdtzDDg*-8khm3OOcWWsi8%>25A8YR6wP> zVGxEEF5kLq-S_+QJ?wp6&RJ)zbN1f9-;OvFBV8(THgW&}K&AHxYIe6`{zpiO@3h(6 z3lac;cT*3lWfAyiKQAbY1D1^o_nzuPsC1R4rPlL12!SJr_o!>dWeOLO5xYxLm@cg& z85w1X`S|3k5*aVrC@mfM!ymm*HNhqt;5}kNx<^hlq*3>;ruBk*45qIB`R)jI&6T3B zT#imJ)Qr_qyIYK=)chCBH=oj2i!&t*I5zz6E@s56Hz^s-MgZmtR^2D=7<&ZXhQ7Np zZa&6IvrB(2rt~t7@q9K1CFw{ExBiY{#5`y0Y*7H(#|Lt$;}{DYi@TgEZOO?nhD(XO z1&q&W-D+`NNt^1baLYG`eP#oSB$vecg?CDP3fn6VU>a2)B1bx#Jtx9QSH=~?Eg8Aa zF(kt-jR14x17sg62xE$&L6D4w6gP>+v6dvn8&}Rda*9 z2&P;Ndr}^Px0yc*Rkhi5rNIYa-soYxP#%CkO%KYSKki~o@QvUb&}h)Q;e+tCrA1yvO?Mqoyoz&-9X!Z9aPGNw#bVI?{lJ?!5m`d^q zf>-bb`E?Ndo(~W1Me7rDH^Bb7xip@~_F3gYHu^J8uQ4-kgcDFL1Vdmic21&P z@=hAQer(AfqPe+03r*y*jUL-JJY1X4(9Zx&lFt$kuPS!f3#lFfe~3-I;`5RBQO1QV z?T$G8Cs0TPTBcok#LZbeZ#{fSRn95n)|V>aBhmcXKf;F}ZBKkd##gxRBSqh9+6%-< zz%?h!jXV!8>3#eRo@|=;xo%{f8X>nU!YI#sq6$LE!6ZhM$!C>2k|lPuGj+aF32$eY6}#biI? zdUhG}QBcnP0lOt-*uP8yZIhOiIu1sikHB-bjr*4Fr`^2)D9p0(VxM?daa1#O8HP#P z@q3%VSMv}nY%oQRMDy|r5iXJ`yM|AIu44|bG8L2g`l6QLY}fGjUnx*$d!q@TKfgvj zs`-#r>vz_7_<_i7y)wKuYJO8#uS3^m6UT!eCCH5yitPzTih=4w!IK_Wd_ow;l`td~69?m%;G>4K z`a;oHiHUhSls+x(jfymdZE1@q>McQgK*y0}>8+NwE7hysZ z8O11bAd!J`yw%Tw1#9;6xUZ@0*%;KQ_-GF?S+UN)-Yi5xSYw~Z+34|3yi z;t%lF6OwQcB$3ni?PiDqVM^wuSt*E{I+UQbzO<&1ZucpdV!kgGdlSy`UQK_fi(8;I+o-QP^6LMSl*t>X#Um|5o$L{{AF9B5%u~q|18imgt;nS zkx@(l02!n7DV&UIX|x$&#dc8wTY$Isd5r-bC^ncHd7uieZm%gM#(6TizseSt8&&bT z@JcWRe)7Jaov`zV?|hE!Y{&H^p$KiXo#)j*W4^yi_^tw^k$-=7?N%}A%*C77Il-eo z3hL=8s{)-nUyxYRK+QZS4pds$X>IVXVzMuT{ib_#{bw%uHzp!h=oDzr_;%OA-^V)9 zSUz39Mrr-M`LycaVZJlMMM9&v39rWZ5Vkj~PpBEtz6st1w^S!S|?vNm&S3_e>E`U&>%A?l8Ix?1fMRnt9VLywkl3fTW6nWG3iy5s? zW(oE5n=g0F5MMTqJd7WfLn|x(Ntm6S>@7)AC&#%iJqPiCmftG0hm~Duo{dX|MhH9x zl>zOw@ywW@?gE4N<`R$wSQF1|xHXiMQKT#QS<{JO=;ARd1Y%i9dsZWqd8rNS0)o<8 z7sJVyRXCT(mO9WMMA%24M{%d?Ltg&*eAV93k?GKxU1VZSdrXo)()yU4KNtz{lxjTq zNxC@zK}D8V?s#k7R5f)SOCqiKiCw9e6WCPumnhG7hDMd@K}U#5T@wTmy{<)a2`qhD z?BMxS%V6{)i}X7Nu%m0UNN{Zkn`!fM=V&P@iFA@C6a2XHFDkj~t#WUb$^?fDC3B#n z;y2czi^mv<}Sj!q%d@n}}eIMkI;36n>fOo zG;l4LxieCajMxXdX0)XIrKU6?fNXFHhnOhE6d@#E3T2M>_DAfB>NTRLnUB9(4|#!(^|x`{(VzuYW}D~YZ9qH?7N zrC&js@7-i<`M`50l@{@af`o$I8vGwl<8&Y@%6oOad?#_knaF^|Rd2_dUKP{Y#5OPb z^)x?fa#0b0`&goFC7&H;>XF4;VGfN1Qcq)8X<3~1vqL>Q1cbtBD=$5^MrdM!2>p331K*SKgJ zbM^iGr&TDL!ppNPz~>&<(5agh%sNYtp7r;vrFJ=soBbJ@J>gS_QV@oOlZdkd?mK4# zwI}l&d}$j;W<%;p-Q}|}mOTo|vwzRstWk?T9I{9vot2|;&ArUzy^aP6Qt#ZR@A0k1 zqOGPR37B>E${`?M^#^xw-NRU9p01PSHr7JFu#&zBP4sn$_3>^5O_Xe5Ieg;>_xC8! z`XsbI(T*}VHWQdcVO?*c2=6oS_mqHyyl7Y_&zpJ_%zX1t>A!6AD*nhs#X z8aqBmda2n@FGN}LKBk+aQQc^7qW`M;(*C5~vS;0JC zSkx{h&{*gEM%zMzhga@Ck96DD$v@LNo*`B*@1)w|*TZzBFR8gQ&cmj+0mgF)q;-aa zNE-Bx_Kbh#MRyNxwxJiWtYW7MLcS6bCw~sSdG#a33(?)QUvez9J6C^$T!mETDx9Ai zw|w^ZqJPU{kB@!jwZ$dnxfhfuIVDXUdVe=0T>IQ~SLkr|Bigo_*tDMg=*b;319=U; ze@ym(;G9sxPbxn>q8K5Bs(x$U|Fxw!wq|g@4ZzM@d}H z)NuA$KYJ<~nzlhRTn_rjnxUTPu&g{bE50{d$5awpknl-+vgyu>cac_7X^*%b zQOiajt+FpyX>j2;;pC}#m}inF8&Vv0{wGsoI?%8>^kQF@>sd>!AD2%$dLuv}nP`zq zb1h)3`q?%)IH<c@hB+9#vEXx|E8(hcXv z?I~$mDOw-FxO~`Mq1z{Smao|lnW8crNztz-SYDL!T#ejTIwe@d{rB5-z}NcXV&;P? zaNQXqK$&UAFWdH|6A=11HPZmG4c}=+Xmf*9q*Uymk-vg;E0$;rV|Va zTy@s$hJ0(8-X54WK&yJ{Ng>bizNrR~cD?t&C5Mv)XAW~$uJjTuujp~yf~*_Qmh`~Cfo@qZ zq<_%|KJ*OPWP>+|x)N+Ud?8LA=$E9ro#%YmivYFKEC=XXmlsxL=BKA*nk5HMJ0XpKk{Pd9Xegi?49Kt<2TJBjR!>nDH7XZZl~_u zLSJ?q5%mh;t#x-9l*AHEJSWPxz6)&L=%zlnX==Zu{t08WB#;=nYctk8fmNbbfyFTx zyfPt2R>T;{zfFsx*P#aa@BB9_VdX196Quw1XBykshmgos#d#>pbsrhA)?%*YVXLR> zzmtyS+4gqG2F*7nyo>Z-l&jWs!m!}P-WEyKO2mjq2g8(YujGsk0VV1M>``6!W%ba9 z?sP=;<~YQt9r3t$aQ7wC_&Dun)+y_5HG9YILRpt-j%S(kf6;7po2|BWR}-x>_g>3| z&~3%V_tm8spA!8$c9K@QwOLXc@|CYZGyO3V&7%F3rGOdr>-oGp{Yrd!Z10JR+yNd- zhcA=)no!LfnS43RJ|3U~DYs`toPWh#0U35u)j%>|Lr?{Dh>|@&RINTVQsT0o%U2HJ zK3%yxJWJd6kHy)@Tgd!}=Om(~{TNbe>P2#+3(Yry)lK{r>Jg(Fd<$EPW-vA87(cx- zdWrp%yfRu**u`cji{56&S006^nQD-?R6S= zJS;iJft2kZ4l1!;?+*GqSJlcl=u(f77u^2jUU4HLx0b=+Rx4;2wQ4nb+i|oXzKbq_ z;Smw-h218?(ng^3l9X!O?&}S;dxf#X7O=dON{T;;HW~lq6y*tt3MREKh)KiT)?`l2 z*&Vt+FX3&SffH8DU&M7+kB&t@V*Mbq_z-nfjO_1ylu z7*}6Nh9HzL!t_-hM@$N-Q!-G2#vZAWhLhMOdF$eBPm-QC_oK2agD3s&XX^+|{xy@YYMC@jpLoR|`jC(03^-dJyFY!-+_++GV8ml7h`Wauam!O){GVTn5H z)`A3XeQ=7;z);>+h`W@N>j8QO_m?MvnLU$x%vpDD>W& z1pp1eJK;PMkw{;`%l_MsM$`6j$6M(rx`kT0Q7)NQRUxhEN>^_6$XC^1d&fYtNJ$82 zU{bucpm$2VsV&cNSMWZXWd;07jWNO!f5qrGhAYUdnR3K&W$Cwt;^T{liV2+rjGLY~fy+i~VA6tdKrMA<$NcADRd>M^S7Iu@;21MXP)am{rO$+8htF!Dy6957G zN|Ak|QUza-THJ4^IbzlFvf;C1M494w>kX~>Clz=7x^hS(*^IF{Jj|N(TA|(=#>IZ) zqvMo>R!2gv82ftr=Wd}e-M|3zN`~JrGOWc<<@HGhIZ{}cG}sYg+?uZD-O?xAgVn83 zU#V|Yh2R-hhsTJX7xlJa{^2oCzEE{2bG_HAi-E}RG=(8d!jmc5HS4*y%~J*aqvPH# z27R6^u~>|sVGpDCh^`3(&*sm`w5;*LGM!Fo{P+p~2C;WG=08&Vv}3T5!OU=C!4+bVD|O-^m@&7du&X5kNPobZh=n75?EbdxK~~RV z7W@%6Vjk2W@KBWa+HLn{NV_h}}IDt_lk?G4^rIb55=?g0^M%R^xN++a1o=5*y+X#X;#M;G@dAEWaos^l6iA5PB z`7ov9y35b91o=Ugkh&OR&(xPA}1hKg5o4 z!MWB{6)VmpBUJb_jy)cQpDC=&Wyhca=1dO%V_YsM2Cru8rg!3%*tl1At$B(FuZo@1 zxyZVtfUXfIHVPHIv)R46Q6X*N&nE8lJD6(|BmBWDCnxvLs2p1oT7-PbGOE{xX6VAW zA{Kl4{O6aaWM#7rYlqgXvAr)UEfSwg<)mxmC^$z$V=ccyNaoP!t~X1 zz7CHRN6&u*7K3Aaa#@#nKa2F_^>l;-vUhO<@i zOmTkgNKFJ3 z(__U#5ZApW=9tUuArl3@U!(UehQz#z^9TDjfvE?q?%c-%r5y(#Ht8p^7kqjn76EMSOE z_8}g=;^Sd4b>Bi?kY(b6_e<$mvo3lk4!ZEl34{gM)4o-SAzM$88m$i}4m00+EZ13r zCbzc>nBq)g{&f;#uCnj5Qm$)Jsu?!_A3Rv)`#%34>qERwkGkKdww57#Yt%KO(`GhzJ3#F>B_qxphZ z$+bG@Q?omh*EoY91LULr<5fz>d3?zNffPf|!*6=DXP2yUkmmT8drA<^2tXqZA=4_;FJLgM^hm#2woKzm>JzKiVam{II+Xs}lmEBeJjM4><`# z6ISx=2lfhYeB$Ic999DLmw_!69_kRca0*PIhs1pDBmNty;0-Psm;iwLd`0>%tEg9J zhIO&}$L~*0>00HQo=1&jvaq%+SrU#@9$OQST-u!0wk0e(#(!tJmzsmhks`aLcc_Vd zMSNQzX!ku>W6-r85uvLv7(t)zzx_)2h1}sgiz&KcQq4s0kH&~r-~P%uV}D1vhk%Eq z?yu2+^>OdBN7hctvW9WYZ83zkbtUe8{ zYGY%gv&1VBU0ZUkK6)&5Z@3Kc9@JM8hNSov(8S5O(Rpw|MTv` z(3DSy4An8^Z8Gip6*&&1wUXfNNb1Eynio6l0luKvYnVR+7{@3cLYE#SuePox?HZSw zeKZpxyHTl?7uA{~Lne|z89K*kp9L54>PsOr4P-9&4=Z%%#H`nftf0~HfxHEPo7e0u zK4SHJCKcef>6m}3?m}E#T+o`}ET1GP=jR#c-31_CGA+?`$D@^jWOVmKm_`M<=_Q`EY|UPQiO8oPM-!#hGM5@n zih-PoRg{%|*kdq+mnE_Lo!q?PECG@mnX#{cV%oWjly)`fMT@C7l@{pJ;sgoDk#_>m zyIClM^#s#oV0gpDv$}o;x+~Yyo+)<_tdvmLBy&D`xBMu#Kh`)pxY=wrG|Ln|lt7w& z^K>ox@?qy=2EM?CFLBZ|7u^Llb&~C@>$i^D#Lw>il~C4M%f8kKJMZJ#$fSLWz9hx9 z*YCrl;8-s9q29B999g)HH}78;$~}K)CIOp*9R*&eqfjVA>Fl}W1_qIdkLVvTxt_Kq zvc36izx{Krht3tEKtR#puB*P6P?uZ`O$a?{Tah~PYp;)C8l87Tf^@r#)Za3bB3TJC=%I#3cQPMbMNS5wZ)WJ_#gst_8N=gW^$9H{~@(!rVz2p6; z);b`#jXG`l^ts@(DI17)dk&IDX-*c0O<8Sj8jvHH-LeLVR_l_b<2 zHq-9as`-uk@3CI|=IYIYZ*CKhVKmM2P!Kc2#Wg25NIh{I!jcsxogGTsNOR%|Nhe&4 z@{OjF)P_-VnRg%6Wkuz5K-cuR6~4U_W-Qew0%*D!dTXCJ$Sa6DxfupznR>(oymWCp zv+Rd7ssqr2lcnvOFQu+kdtE((Ibu%LOkZsK2~F_u7VRv=lHQekXM=}%V&>wHz9a{@ zI3Z_47tgfEG9F|lDH^%#9pj~%Zw=quD4ZXdNN3xj^m1PGEA}zL&4g11g};>TAbl%d ze))MlhXUs}v8;Ev1%8U%GnwiT;ouR~>R>F2C&Emm{vMl~C)}&Z(KX(;`_C+`U~`E|y*#=g>KkD^ z#>M+~S73Qx@a?XHezQs}Uc^Jqmg4$Jc=+u+6%|zg?lwtE4}>YyYU^vAF>aipamV@M z@)Xq@)K+bO{v`32ZPE4q<@xws!#JDwK{>Oyb-m;G2f4drRFFy#T(S?*??JwtJo0$N zlU%?mPDLucsE$DXckgY>5I-YQJIXSKA#MDGWa%N;rU>0e=swz<$R!3?A8}x}=#bjz z)ObFY>xulFcXZpFO-YbCz@MUFAd7t+rZTyoRgZd!p+GCIwR^c8=dUlhNRw((({oNZ zXNqJ@S~!cR4$v`)0W$QG7y+Xn2mjwpLIVLj`=>Rq-q52OU@a(T2@p?X`~Tgx0cS!+ zetJ<-+OC+DwxZx5Z$@Ed->K+s2oy_jrX(o^fYKwlET+P%P0|>v1Yi6w4}ARJ0TbtU cN4LOxxV%Qk^>1}|lP>^09V2M1Haz100F~1&tN;K2 literal 0 HcmV?d00001 diff --git a/docs/public/logo_320px.svg b/docs/public/logo_320px.svg new file mode 100644 index 00000000..f6c95098 --- /dev/null +++ b/docs/public/logo_320px.svg @@ -0,0 +1,24 @@ + + + 画板备份 4 + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/package.json b/package.json new file mode 100644 index 00000000..3888a237 --- /dev/null +++ b/package.json @@ -0,0 +1,19 @@ +{ + "name": "clickhouse_sinker", + "version": "1.0.0", + "dependencies": {}, + "devDependencies": { + "vuepress": "^1.7.1", + "vuepress-plugin-mermaidjs": "^1.8.1" + }, + "scripts": { + "docs:dev": "vuepress dev docs", + "docs:build": "vuepress build docs" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/housepower/clickhouse_sinker.git" + }, + "license": "Apache-2.0", + "homepage": "https://github.com/housepower/clickhouse_sinker#readme" +} From c81ccce070ef91cacb120820847859712ed33e17 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sat, 19 Dec 2020 12:51:23 +0000 Subject: [PATCH 025/404] Docs: add guide --- README.md | 349 +-------------------- conf/config.json | 2 +- config.json | 43 +++ docs/.vuepress/config.js | 72 +++++ docs/{ => .vuepress}/public/favicon.ico | Bin docs/{ => .vuepress}/public/logo_320px.svg | 0 docs/README.md | 2 +- docs/config.js | 135 -------- docs/configuration/config.md | 72 +++++ docs/configuration/configuration.md | 1 + docs/configuration/flag.md | 38 +++ docs/configuration/task.md | 41 +++ docs/{ => dev}/design.md | 2 +- docs/dev/introduction.md | 329 +++++++++++++++++++ docs/guide/README.md | 2 +- docs/guide/install.md | 25 ++ docs/guide/install_run.md | 1 + docs/guide/run.md | 139 ++++++++ 18 files changed, 766 insertions(+), 487 deletions(-) create mode 100644 config.json create mode 100644 docs/.vuepress/config.js rename docs/{ => .vuepress}/public/favicon.ico (100%) rename docs/{ => .vuepress}/public/logo_320px.svg (100%) delete mode 100644 docs/config.js create mode 100644 docs/configuration/config.md create mode 100644 docs/configuration/configuration.md create mode 100644 docs/configuration/flag.md create mode 100644 docs/configuration/task.md rename docs/{ => dev}/design.md (99%) create mode 100644 docs/dev/introduction.md create mode 100644 docs/guide/install.md create mode 100644 docs/guide/install_run.md create mode 100644 docs/guide/run.md diff --git a/README.md b/README.md index 9da2d61b..b0fad81d 100644 --- a/README.md +++ b/README.md @@ -5,351 +5,4 @@ clickhouse_sinker is a sinker program that transfer kafka message into [ClickHouse](https://clickhouse.yandex/). -Refers to [design](./design.md) for how it works. - -## Features - -- Uses native ClickHouse client-server TCP protocol, with higher performance than HTTP. -- Easy to use and deploy, you don't need write any hard code, just care about the configuration file -- Support multiple parsers: fastjson(recommended), gjson, csv. -- Support multiple Kafka client: kafka-go(recommended), sarama. -- Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/SCRAM, SASL/GSSAPI and combinations of them. -- Support multiple sinker tasks, each runs on parallel. -- Support multiple kafka and ClickHouse clusters. -- Bulk insert (by config `bufferSize` and `flushInterval`). -- Parse messages concurrently. -- Write batches concurrently. -- Every batch is routed to a determined clickhouse shard. Exit if loop write fail. -- Custom sharding policy (by config `shardingKey` and `shardingPolicy`). -- Tolerate replica single-point-failure. -- At least once delivery guarantee. -- Dynamic config management with Nacos. - -## Supported data types - -- [x] UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 -- [x] Float32, Float64 -- [x] String -- [x] FixedString -- [x] Date, DateTime, DateTime64 (custom layout parser) -- [x] Array(UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64) -- [x] Array(Float32, Float64) -- [x] Array(String) -- [x] Array(FixedString) -- [x] Nullable -- [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) - -## Install && Run - -### By binary files (recommended) - -Download the binary files from [release](https://github.com/housepower/clickhouse_sinker/releases), choose the executable binary file according to your env, modify the `conf` files, then run `./clickhouse_sinker --local-cfg-dir conf`. If you prefer a single config file, you can merge the `conf` files into one, and pass its path to `--local-cfg-file`. - -### By container image - -Modify the `conf` files, then run `docker run --volume conf:/etc/clickhouse_sinker quay.io/housepower/clickhouse_sinker` - -### By source - -- Install Golang - -- Go Get - -``` -go get -u github.com/housepower/clickhouse_sinker/... -``` - -- Build && Run - -``` -make build -## modify the config files, set the configuration directory, then run it -./dist/clickhouse_sinker --local-cfg-dir conf -``` - -## Configuration - -Refers to how [integration test](./go.test.sh) use the [example config](./docker/config.json). -Also refers to [code](./config/config.go) for all config items. - -### Kafka Encryption - -clickhouse_sinker supports following encryption mechanisms: - -- No encryption - -An example kafka config: - -``` - "kfk1": { - "brokers": "192.168.31.64:9092", - "@version": "Required if you use sarama. It's the the Kafka server version.", - "version": "2.2.1" - } -``` - -- Encryption using SSL - -An example kafka config: -``` - "kfk2": { - "brokers": "192.168.31.64:9093", - "version": "2.2.1", - "tls": { - "enable": true, - "@caCertFiles": "Required. It's the CA certificate with which Kafka brokers certs be signed. This cert is added to kafka.client.truststore.jks which kafka-console-consumer.sh uses", - "caCertFiles": "/etc/security/ca-cert", - "@insecureSkipVerify": "Whether disable broker FQDN verification. Set it to `true` if kafka-console-consumer.sh uses `ssl.endpoint.identification.algorithm=`.", - "insecureSkipVerify": true - } - } -``` - -FYI. `kafka-console-consumer.sh` works as the following setup: - -``` -$ cat config/SSL_NOAUTH_client.properties -security.protocol=SSL -ssl.truststore.location=/etc/security/kafka.client.truststore.jks -ssl.truststore.password=123456 -ssl.endpoint.identification.algorithm= - -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/SSL_NOAUTH_client.properties -``` - -Please follow [`Kafka SSL setup`](https://kafka.apache.org/documentation/#security_ssl). Use `-keyalg RSA` when you create the broker keystore, otherwise there will be no cipher suites in common between the keystore and those Golang supports. See [this](https://github.com/Shopify/sarama/issues/643#issuecomment-216839760) for reference. - -### Kafka Authentication - -clickhouse_sinker support following following authentication mechanisms: - -- No authentication - -An example kafka config: - -``` - "kfk1": { - "brokers": "192.168.31.64:9092", - "@version": "Required if you use sarama. It's the the Kafka server version.", - "version": "2.2.1" - } -``` - -- SASL/PLAIN - -An example kafka config: -``` - "kfk3": { - "brokers": "192.168.31.64:9094", - "version": "2.2.1", - "sasl": { - "enable": true, - "mechanism": "PLAIN", - "username": "alice", - "password": "alice-secret" - } - } -``` - -FYI. Java clients work with the following setup: - -``` -$ cat config/PLAINTEXT_PLAIN_client.properties -security.protocol=SASL_PLAINTEXT -sasl.kerberos.service.name=kafka -sasl.mechanism=PLAIN -sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required username="alice" password="alice-secret"; - -$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_PLAIN_client.properties - -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_PLAIN_client.properties -``` - -- SASL/SCRAM - -An example kafka config: -``` - "kfk4": { - "brokers": "192.168.31.64:9094", - "version": "2.2.1", - "sasl": { - "enable": true, - "@mechanism": "SCRAM-SHA-256 or SCRAM-SHA-512", - "mechanism": "SCRAM-SHA-256", - "username": "alice", - "password": "alice-secret" - } - } -``` - -FYI. Java clients work with the following setup: - -``` -$ cat config/PLAINTEXT_SCRAM_client.properties -security.protocol=SASL_PLAINTEXT -sasl.kerberos.service.name=kafka -sasl.mechanism=SCRAM-SHA-256 -sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required username="alice" password="alice-secret"; - -$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_SCRAM_client.properties - -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_SCRAM_client.properties -``` - -- SASL/GSSAPI(Kerberos) - -An example kafka config: -``` - "kfk5": { - "brokers": "192.168.31.64:9094", - "version": "2.2.1", - "sasl": { - "enable": true, - "mechanism": "GSSAPI", - "gssapi": { - "@authtype": "1 - Username and password, 2 - Keytab", - "authtype": 2, - "keytabpath": "/etc/security/mmmtest.keytab", - "kerberosconfigpath": "/etc/krb5.conf", - "servicename": "kafka", - "@username": "`principal` consists of `username` `@` `realm`", - "username": "mmm", - "realm": "ALANWANG.COM" - } - } - } -``` - -FYI. Java clients work with the following setup: - -``` -$ cat config/PLAINTEXT_GSSAPI_client.properties -security.protocol=SASL_PLAINTEXT -sasl.kerberos.service.name=kafka -sasl.mechanism=GSSAPI -sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab="/etc/security/mmmtest.keytab" principal="mmm@ALANWANG.COM"; - -$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_GSSAPI_client.properties - -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_GSSAPI_client.properties -``` - -Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. I tested sarama Kerberos authentication against Kafka [2.2.1](https://archive.apache.org/dist/kafka/2.2.1/kafka_2.11-2.2.1.tgz). Not sure other Kafka versions work. - -### Sharding Policy - -Every message is routed to a determined ClickHouse shard. - -By default, the shard number is caculated by `(kafka_offset/roundup(batch_size))%clickhouse_shards`, where `roundup()` round upward an unsigned integer to the the nearest 2^n. - -This above expression can be customized with `shardingKey` and `shardingPolicy`. `shardingKey` value is a column name. `shardingPolicy` value could be: - -- `stripe,`. This requires `shardingKey` be a numeric-like (bool, int, float, date etc.) column. The expression is `(uint64(shardingKey)/stripe_size)%clickhouse_shards`. -- `hash`. This requires `shardingKey` be a string-like column. The hash function used internally is [xxHash64](https://github.com/cespare/xxhash). The expression is `xxhash64(string(shardingKey))%clickhouse_shards`. - -## Configuration Management - -The precedence of config items: - -- CLI parameters > env variables -- Nacos > Consul > Local Config File > Local Config Dir - -### Nacos - -Sinker is able to register with Nacos, get and apply config changes dynamically without restart the whole process. -Controled by: - -- CLI parameters: `nacos-register-enable, nacos-addr, nacos-namespace-id, nacos-group, nacos-username, nacos-password` -- env variables: `NACOS_REGISTER_ENABLE, NACOS_ADDR, NACOS_NAMESPACE_ID, NACOS_GROUP, NACOS_USERNAME, NACOS_PASSWORD` - -### Consul - -Currently sinker is able to register with Consul, but unable to get config. -Controled by: - -- CLI parameters: `consul-register-enable, consul-addr, consul-deregister-critical-services-after` -- env variables: `CONSUL_REGISTER_ENABLE, CONSUL_ADDR, CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER` - -### Local Files - -Currently sinker is able to parse local config files at startup, but unable to detect file changes. -Controled by: - -- CLI parameters: `local-cfg-file, local-cfg-dir` -- env variables: `LOCAL_CFG_FILE, LOCAL_CFG_DIR` - -## Prometheus Metrics - -All metrics are defined in `statistics.go`. You can create Grafana dashboard for clickhouse_sinker by importing the template `clickhouse_sinker-dashboard.json`. - -- Pull with prometheus - -Metrics are exposed at `http://ip:port/metrics`. IP is the outbound IP of this machine. Port is from CLI `--http-port` or env `HTTP_PORT`. - -Sinker registers with Nacos if CLI `--consul-register-enable` or env `CONSUL_REGISTER_ENABLE` is present. However Prometheus is [unable](https://github.com/alibaba/nacos/issues/1032) to obtain dynamic service list from nacos server. - -- Push to promethues - -If CLI `--metric-push-gateway-addrs` or env `METRIC_PUSH_GATEWAY_ADDRS` (a list of comma-separated urls) is present, metrics are pushed to one of given URLs regualarly. - - -## Extending - -There are several abstract interfaces which you can implement to support more message format, message queue and config management mechanism. - -``` -type Parser interface { - Parse(bs []byte) model.Metric -} - -type Inputer interface { - Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) error - Run(ctx context.Context) - Stop() error - CommitMessages(ctx context.Context, message *model.InputMessage) error -} - -// RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... -type RemoteConfManager interface { - Init(properties map[string]interface{}) error - // Register this instance, and keep-alive via heartbeat. - Register(ip string, port int) error - Deregister(ip string, port int) error - // GetInstances fetchs healthy instances. - // Mature service-discovery solutions(Nacos, Consul etc.) have client side cache - // so that frequent invoking of GetInstances() and GetGlobalConfig() don't harm. - GetInstances() (instances []Instance, err error) - // GetConfig fetchs the config. The manager shall not reference the returned Config object after call. - GetConfig() (conf *Config, err error) - // PublishConfig publishs the config. The manager shall not reference the passed Config object after call. - PublishConfig(conf *Config) (err error) -} - -``` - -## Why not [`Kafka Engine`](https://clickhouse.tech/docs/en/engines/table-engines/integrations/kafka/) built in ClickHouse? - -- My experience indicates `Kafka Engine` is complicated, buggy and hard to debug. -- `Kafka Engine` runs inside the db process, lowers the database stability. On the other side, [Vertica](https://www.vertica.com/)'s official kafka importer is separated with the database server. -- `Kafka Engine` doesn't support custom sharding policy. -- Neither `Kafka Engine` nor clickhouse_sinker support exactly-once. - - -## Kafka Compatibility - -Kafka broker [exposes versions of various APIs it supports since 0.10.0.0](https://kafka.apache.org/protocol#api_versions). - -### Kafka-go - -- Kafka-go [negotiate it's protocol Version](https://github.com/segmentio/kafka-go/blob/c66d8ca149e7f1a7905b47a60962745ceb08a6a9/conn.go#L209). -- Kafka-go [doesn't support Kerberos authentication](https://github.com/segmentio/kafka-go/issues/237). - -### Sarama - -- Sarama guarantees compatibility [with Kafka 2.4 through 2.6](https://github.com/Shopify/sarama/blob/master/README.md#compatibility-and-api-stability). -- Sarama [has tied it's protocol usage to the Version field in Config](https://github.com/Shopify/sarama/issues/1732). - -### Conclusion - -- Neither Kafka-go nor sarama is mature as Java clients. You need to try both if clickhouse_sinker fails to connect with Kafka. -- Our experience is sarama can't work well with new kafka server if set its `Config.Version` to "0.11.0.0". So we suggest `KafkaConfig.Version` in clickhouse_sinker config matchs the Kafka server. +Refers to [docs](http://github.com/housepower/clickhouse_sinker/dev/introduction.html#features) to see how it works. \ No newline at end of file diff --git a/conf/config.json b/conf/config.json index c2397708..261b634e 100644 --- a/conf/config.json +++ b/conf/config.json @@ -10,7 +10,7 @@ "password": "", "retryTimes": 0, "port": 9000, - "username": "" + "username": "default" } }, "kafka": { diff --git a/config.json b/config.json new file mode 100644 index 00000000..cf32567e --- /dev/null +++ b/config.json @@ -0,0 +1,43 @@ +{ + "clickhouse": { + "default": { + "db": "default", + "hosts": [ + [ + "127.0.0.1" + ] + ], + "port": 9000 + } + }, + "kafka": { + "default": { + "brokers": "146.196.79.232:9094", + "version": "2.2.1" + } + }, + "common": { + "bufferSize": 90000, + "minBufferSize": 1, + "msgSizeHint": 1000, + "flushInterval": 5, + "logLevel": "debug" + }, + "tasks": { + "logstash": { + "name": "logstash", + "kafkaClient": "kafka-go", + "kafka": "default", + "topic": "logstash", + "consumerGroup": "logstash_sinker", + "parser": "json", + "clickhouse": "default", + "tableName": "logstash", + "autoSchema": true, + "@desc_of_exclude_columns": "this columns will be excluded by insert SQL ", + "excludeColumns": [ + "day" + ] + } + } +} diff --git a/docs/.vuepress/config.js b/docs/.vuepress/config.js new file mode 100644 index 00000000..d34f4678 --- /dev/null +++ b/docs/.vuepress/config.js @@ -0,0 +1,72 @@ +module.exports = { + base: "/clickhouse_sinker/", + title: "clickhouse_sinker", + evergreen: true, + plugins: ["mermaidjs"], + locales: { + "/": { + lang: "en-US", + title: "clickhouse_sinker", + description: "clickhouse_sinker a tool to sink the data into ClickHouse", + }, + "/zh/": { + lang: "zh-CN", + title: "clickhouse_sinker", + description: "clickhouse_sinker 一个将数据摄入到ClickHouse的工具", + }, + }, + themeConfig: { + locales: { + "/": { + selectText: "Languages", + label: "English", + ariaLabel: "Languages", + editLinkText: "Edit this page on GitHub", + serviceWorker: { + updatePopup: { + message: "New content is available.", + buttonText: "Refresh", + }, + }, + algolia: {}, + nav: [ + { text: "Get Started", link: "/guide/install" }, + { text: "Introduction", link: "/dev/introduction" }, + { text: "Configuration", link: "/configuration/flag" }, + { + text: "GitHub", + link: "https://github.com/housepower/clickhouse_sinker", + }, + ], + sidebar: { + "/guide/": [ + { + title: "Install and Run", + children: [ + ["install", "Install"], + ["run", "Run"], + ], + }, + ], + + "/configuration/": [ + { + title: "Configuration", + children: [ + ["flag", "Flag"], + ["config", "Config"], + ["task", "Task"], + ] + } + ], + + "/dev/": [ + ["introduction", "Introduction"], + ["design", "Design"], + ], + }, + } + + }, + }, +}; diff --git a/docs/public/favicon.ico b/docs/.vuepress/public/favicon.ico similarity index 100% rename from docs/public/favicon.ico rename to docs/.vuepress/public/favicon.ico diff --git a/docs/public/logo_320px.svg b/docs/.vuepress/public/logo_320px.svg similarity index 100% rename from docs/public/logo_320px.svg rename to docs/.vuepress/public/logo_320px.svg diff --git a/docs/README.md b/docs/README.md index bd5799ed..582d7b01 100644 --- a/docs/README.md +++ b/docs/README.md @@ -5,6 +5,6 @@ actionText: Get Started → actionLink: /guide/introduction features: - title: clickhouse_sinker - details: A tool to sinker the data into ClickHouse + details: A tool to sink the data into ClickHouse footer: Apache License 2.0 --- diff --git a/docs/config.js b/docs/config.js deleted file mode 100644 index 691125b3..00000000 --- a/docs/config.js +++ /dev/null @@ -1,135 +0,0 @@ -module.exports = { - base: "/ClickHouse-Native-JDBC/", - title: "ClickHouse Native JDBC", - evergreen: true, - plugins: ["mermaidjs"], - locales: { - "/": { - lang: "en-US", - description: - "clickhouse_sinker a tool to sinker the data into ClickHouse", - }, - "/zh/": { - lang: "zh-CN", - description: "clickhouse_sinker 一个将数据摄入到ClickHouse的工具", - }, - }, - themeConfig: { - locales: { - "/": { - selectText: "Languages", - label: "English", - ariaLabel: "Languages", - editLinkText: "Edit this page on GitHub", - serviceWorker: { - updatePopup: { - message: "New content is available.", - buttonText: "Refresh", - }, - }, - algolia: {}, - nav: [ - { text: "Guide", link: "/guide/introduction" }, - { text: "Dev", link: "/dev/contribute" }, - { text: "Deep Dive", link: "/deep-dive/native_protocol" }, - { text: "Release Notes", link: "/release-notes/" }, - { - text: "GitHub", - link: "https://github.com/housepower/ClickHouse-Native-JDBC", - }, - ], - sidebar: { - "/guide/": [ - ["introduction", "Introduction"], - { - title: "Usage", - children: [ - ["jdbc_driver", "JDBC Driver"], - ["datasource", "Datasource"], - ["connection_pool", "Connection Pool"], - ["spark_integration", "Spark Integration"], - ], - }, - ["troubleshooting", "Troubleshooting"], - ], - "/dev/": [ - ["contribute", "Contribute Guide"], - { - title: "Release", - children: [ - ["internal_release", "Internal Release"], - ["public_release", "Public Release"], - ], - }, - ], - "/deep-dive/": [ - { - title: "ClickHouse", - children: [ - ["data_type", "DataType"], - ["native_protocol", "Native Protocol"], - ], - }, - ], - "/release-notes/": [], - }, - }, - "/zh/": { - selectText: "选择语言", - label: "简体中文", - editLinkText: "在 GitHub 上编辑此页", - serviceWorker: { - updatePopup: { - message: "发现新内容可用.", - buttonText: "刷新", - }, - }, - algolia: {}, - nav: [ - { text: "指南", link: "/zh/guide/introduction" }, - { text: "开发", link: "/zh/dev/contribute" }, - { text: "深入", link: "/zh/deep-dive/native_protocol" }, - { text: "发行注记", link: "/zh/release-notes/" }, - { - text: "源码仓库", - link: "https://github.com/housepower/ClickHouse-Native-JDBC", - }, - ], - sidebar: { - "/zh/guide/": [ - ["introduction", "简介"], - { - title: "使用", - children: [ - ["jdbc_driver", "JDBC 驱动"], - ["connection_pool", "连接池"], - ["spark_integration", "Spark 集成"], - ], - }, - ["troubleshooting", "常见问题"], - ], - "/zh/dev/": [ - ["contribute", "贡献指南"], - { - title: "发布", - children: [ - ["internal_release", "内部发布"], - ["public_release", "公开发布"], - ], - }, - ], - "/zh/deep-dive/": [ - { - title: "ClickHouse", - children: [ - ["data_type", "数据类型"], - ["native_protocol", "原生协议"], - ], - }, - ], - "/zh/release-notes/": [], - }, - }, - }, - }, -}; diff --git a/docs/configuration/config.md b/docs/configuration/config.md new file mode 100644 index 00000000..9da1e987 --- /dev/null +++ b/docs/configuration/config.md @@ -0,0 +1,72 @@ +# basic configs +> Here we use json with comments for documentation + + +``` +{ + // clickhouse configs, it's map[string]ClickHouse for multiple clickhouse + "clickhouse": { + // key for clickhouse config + "ch1": { + "db": "default", // database name + + // hosts for connection, it's Array(Array(String)) + // we can put hosts with same shard into the inner array + // it helps data deduplication for ReplicateMergeTree when driver error occurs + "hosts": [ + [ + "127.0.0.1" + ] + ], + "password": "", + // retryTimes when error occurs in inserting datas + "retryTimes": 0, + "port": 9000, + "username": "default" + } + }, + + // kafka configs + "kafka": { + "kfk1": { + "brokers": "127.0.0.1:9093", + + // somethings about sasl + "sasl": { + "enable": false, + "password": "", + "username": "", + "gssapi": { + "authtype": 0, + "keytabpath": "", + "kerberosconfigpath": "", + "servicename": "", + "username": "", + "password": "", + "realm": "", + "disablepafxfast": false + } + }, + + // kafka version, if you use sarama, the version must be specified + "version": "2.2.1" + } + }, + + "common": { + // batch size to insert into clickhouse + "bufferSize": 90000, + // min batch size to insert into clickhouse + "minBufferSize": 1, + + // msg bytes per message + "msgSizeHint": 1000, + + // interval flush the batch + "flushInterval": 5, + + // log level + "logLevel": "debug" + } +} +``` \ No newline at end of file diff --git a/docs/configuration/configuration.md b/docs/configuration/configuration.md new file mode 100644 index 00000000..af4abbfe --- /dev/null +++ b/docs/configuration/configuration.md @@ -0,0 +1 @@ +# Configuration \ No newline at end of file diff --git a/docs/configuration/flag.md b/docs/configuration/flag.md new file mode 100644 index 00000000..4a3429c0 --- /dev/null +++ b/docs/configuration/flag.md @@ -0,0 +1,38 @@ +# run args + +``` +./clickhouse_sinker -h + +Usage of ./dist/clickhouse_sinker: + -consul-addr string + consul api interface address (default "http://127.0.0.1:8500") + -consul-deregister-critical-services-after string + configure service check DeregisterCriticalServiceAfter (default "30m") + -consul-register-enable + register current instance in consul + -cpunum int + Indicate the number of CPU (default 1) + -http-port int + http listen port (default 2112) + -local-cfg-dir config.json + local config dir. requires a file named config.json, and some task json files under `tasks` folder (default "/etc/clickhouse_sinker") + -local-cfg-file string + local config file (default "/etc/clickhouse_sinker.json") + -metric-push-gateway-addrs string + a list of comma-separated prometheus push gatway address + -nacos-addr string + a list of comma-separated nacos server addresses (default "127.0.0.1:8848") + -nacos-group string + nacos group name. Empty string doesn't work! (default "DEFAULT_GROUP") + -nacos-namespace-id string + nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work! + -nacos-password string + nacos password (default "nacos") + -nacos-register-enable + register current instance in nacos + -nacos-username string + nacos username (default "nacos") + -push-interval int + push interval in seconds (default 10) + -v show build version and quit +``` \ No newline at end of file diff --git a/docs/configuration/task.md b/docs/configuration/task.md new file mode 100644 index 00000000..53adc249 --- /dev/null +++ b/docs/configuration/task.md @@ -0,0 +1,41 @@ +# task config + +``` +{ + "name": "daily_request", + + // kafka cluster + "kafka": "kfk1", + "topic": "topic", + + // kafka consume from earliest or latest + "earliest": true, + // kafka consumer group + "consumerGroup": "group", + + // message parser + "parser": "json", + + // clickhouse cluster + "clickhouse": "ch1", + + // table name + "tableName": "daily", + + // columns of the table + "dims": [ + { + "name": "day", + "type": "Date", + "sourceName": "day" + }, + ... + ], + + // if it's specified, the schema will be auto mapped from clickhouse, + "autoSchema" : true, + // "this columns will be excluded by insert SQL " + "excludeColumns": [] +} + +``` \ No newline at end of file diff --git a/docs/design.md b/docs/dev/design.md similarity index 99% rename from docs/design.md rename to docs/dev/design.md index 869ae540..60f84fdc 100644 --- a/docs/design.md +++ b/docs/dev/design.md @@ -64,7 +64,7 @@ Each task can be assigned to multiple instances. Each task declares how many ins ## The coordinator(outside this project) -- The coordinator provides API and/or webui to add/delete/modify tasks. +- The coordinator provides API and/or webui to add/delete/modify tasks. - The coordinator watches (do `service discovery`) instance startup/disappear events, and assign tasks to instances (do `publish config`). Refers to `cmd/nacos_publish_config/main.go` to assign tasks (from local config) via consistent-hash to instances(from CLI). ## The schedule platform(outside this project) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md new file mode 100644 index 00000000..825e34c9 --- /dev/null +++ b/docs/dev/introduction.md @@ -0,0 +1,329 @@ +# clickhouse_sinker + +[![Build Status](https://travis-ci.com/housepower/clickhouse_sinker.svg?branch=master)](https://travis-ci.com/housepower/clickhouse_sinker) +[![Go Report Card](https://goreportcard.com/badge/github.com/housepower/clickhouse_sinker)](https://goreportcard.com/report/github.com/housepower/clickhouse_sinker) + +clickhouse_sinker is a sinker program that transfer kafka message into [ClickHouse](https://clickhouse.yandex/). + +Refers to [design](./design.md) for how it works. + +## Features + +- Uses native ClickHouse client-server TCP protocol, with higher performance than HTTP. +- Easy to use and deploy, you don't need write any hard code, just care about the configuration file +- Support multiple parsers: fastjson(recommended), gjson, csv. +- Support multiple Kafka client: kafka-go(recommended), sarama. +- Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/SCRAM, SASL/GSSAPI and combinations of them. +- Support multiple sinker tasks, each runs on parallel. +- Support multiple kafka and ClickHouse clusters. +- Bulk insert (by config `bufferSize` and `flushInterval`). +- Parse messages concurrently. +- Write batches concurrently. +- Every batch is routed to a determined clickhouse shard. Exit if loop write fail. +- Custom sharding policy (by config `shardingKey` and `shardingPolicy`). +- Tolerate replica single-point-failure. +- At least once delivery guarantee. +- Dynamic config management with Nacos. + +## Supported data types + +- [x] UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 +- [x] Float32, Float64 +- [x] String +- [x] FixedString +- [x] Date, DateTime, DateTime64 (custom layout parser) +- [x] Array(UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64) +- [x] Array(Float32, Float64) +- [x] Array(String) +- [x] Array(FixedString) +- [x] Nullable +- [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) + + + +## Configuration + +Refers to how [integration test](./go.test.sh) use the [example config](./docker/config.json). +Also refers to [code](./config/config.go) for all config items. + +### Kafka Encryption + +clickhouse_sinker supports following encryption mechanisms: + +- No encryption + +An example kafka config: + +``` + "kfk1": { + "brokers": "192.168.31.64:9092", + "@version": "Required if you use sarama. It's the the Kafka server version.", + "version": "2.2.1" + } +``` + +- Encryption using SSL + +An example kafka config: +``` + "kfk2": { + "brokers": "192.168.31.64:9093", + "version": "2.2.1", + "tls": { + "enable": true, + "@caCertFiles": "Required. It's the CA certificate with which Kafka brokers certs be signed. This cert is added to kafka.client.truststore.jks which kafka-console-consumer.sh uses", + "caCertFiles": "/etc/security/ca-cert", + "@insecureSkipVerify": "Whether disable broker FQDN verification. Set it to `true` if kafka-console-consumer.sh uses `ssl.endpoint.identification.algorithm=`.", + "insecureSkipVerify": true + } + } +``` + +FYI. `kafka-console-consumer.sh` works as the following setup: + +``` +$ cat config/SSL_NOAUTH_client.properties +security.protocol=SSL +ssl.truststore.location=/etc/security/kafka.client.truststore.jks +ssl.truststore.password=123456 +ssl.endpoint.identification.algorithm= + +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/SSL_NOAUTH_client.properties +``` + +Please follow [`Kafka SSL setup`](https://kafka.apache.org/documentation/#security_ssl). Use `-keyalg RSA` when you create the broker keystore, otherwise there will be no cipher suites in common between the keystore and those Golang supports. See [this](https://github.com/Shopify/sarama/issues/643#issuecomment-216839760) for reference. + +### Kafka Authentication + +clickhouse_sinker support following following authentication mechanisms: + +- No authentication + +An example kafka config: + +``` + "kfk1": { + "brokers": "192.168.31.64:9092", + "@version": "Required if you use sarama. It's the the Kafka server version.", + "version": "2.2.1" + } +``` + +- SASL/PLAIN + +An example kafka config: +``` + "kfk3": { + "brokers": "192.168.31.64:9094", + "version": "2.2.1", + "sasl": { + "enable": true, + "mechanism": "PLAIN", + "username": "alice", + "password": "alice-secret" + } + } +``` + +FYI. Java clients work with the following setup: + +``` +$ cat config/PLAINTEXT_PLAIN_client.properties +security.protocol=SASL_PLAINTEXT +sasl.kerberos.service.name=kafka +sasl.mechanism=PLAIN +sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required username="alice" password="alice-secret"; + +$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_PLAIN_client.properties + +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_PLAIN_client.properties +``` + +- SASL/SCRAM + +An example kafka config: +``` + "kfk4": { + "brokers": "192.168.31.64:9094", + "version": "2.2.1", + "sasl": { + "enable": true, + "@mechanism": "SCRAM-SHA-256 or SCRAM-SHA-512", + "mechanism": "SCRAM-SHA-256", + "username": "alice", + "password": "alice-secret" + } + } +``` + +FYI. Java clients work with the following setup: + +``` +$ cat config/PLAINTEXT_SCRAM_client.properties +security.protocol=SASL_PLAINTEXT +sasl.kerberos.service.name=kafka +sasl.mechanism=SCRAM-SHA-256 +sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required username="alice" password="alice-secret"; + +$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_SCRAM_client.properties + +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_SCRAM_client.properties +``` + +- SASL/GSSAPI(Kerberos) + +An example kafka config: +``` + "kfk5": { + "brokers": "192.168.31.64:9094", + "version": "2.2.1", + "sasl": { + "enable": true, + "mechanism": "GSSAPI", + "gssapi": { + "@authtype": "1 - Username and password, 2 - Keytab", + "authtype": 2, + "keytabpath": "/etc/security/mmmtest.keytab", + "kerberosconfigpath": "/etc/krb5.conf", + "servicename": "kafka", + "@username": "`principal` consists of `username` `@` `realm`", + "username": "mmm", + "realm": "ALANWANG.COM" + } + } + } +``` + +FYI. Java clients work with the following setup: + +``` +$ cat config/PLAINTEXT_GSSAPI_client.properties +security.protocol=SASL_PLAINTEXT +sasl.kerberos.service.name=kafka +sasl.mechanism=GSSAPI +sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab="/etc/security/mmmtest.keytab" principal="mmm@ALANWANG.COM"; + +$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_GSSAPI_client.properties + +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_GSSAPI_client.properties +``` + +Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. I tested sarama Kerberos authentication against Kafka [2.2.1](https://archive.apache.org/dist/kafka/2.2.1/kafka_2.11-2.2.1.tgz). Not sure other Kafka versions work. + +### Sharding Policy + +Every message is routed to a determined ClickHouse shard. + +By default, the shard number is caculated by `(kafka_offset/roundup(batch_size))%clickhouse_shards`, where `roundup()` round upward an unsigned integer to the the nearest 2^n. + +This above expression can be customized with `shardingKey` and `shardingPolicy`. `shardingKey` value is a column name. `shardingPolicy` value could be: + +- `stripe,`. This requires `shardingKey` be a numeric-like (bool, int, float, date etc.) column. The expression is `(uint64(shardingKey)/stripe_size)%clickhouse_shards`. +- `hash`. This requires `shardingKey` be a string-like column. The hash function used internally is [xxHash64](https://github.com/cespare/xxhash). The expression is `xxhash64(string(shardingKey))%clickhouse_shards`. + +## Configuration Management + +The precedence of config items: + +- CLI parameters > env variables +- Nacos > Consul > Local Config File > Local Config Dir + +### Nacos + +Sinker is able to register with Nacos, get and apply config changes dynamically without restart the whole process. +Controled by: + +- CLI parameters: `nacos-register-enable, nacos-addr, nacos-namespace-id, nacos-group, nacos-username, nacos-password` +- env variables: `NACOS_REGISTER_ENABLE, NACOS_ADDR, NACOS_NAMESPACE_ID, NACOS_GROUP, NACOS_USERNAME, NACOS_PASSWORD` + +### Consul + +Currently sinker is able to register with Consul, but unable to get config. +Controled by: + +- CLI parameters: `consul-register-enable, consul-addr, consul-deregister-critical-services-after` +- env variables: `CONSUL_REGISTER_ENABLE, CONSUL_ADDR, CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER` + +### Local Files + +Currently sinker is able to parse local config files at startup, but unable to detect file changes. +Controled by: + +- CLI parameters: `local-cfg-file, local-cfg-dir` +- env variables: `LOCAL_CFG_FILE, LOCAL_CFG_DIR` + +## Prometheus Metrics + +All metrics are defined in `statistics.go`. You can create Grafana dashboard for clickhouse_sinker by importing the template `clickhouse_sinker-dashboard.json`. + +- Pull with prometheus + +Metrics are exposed at `http://ip:port/metrics`. IP is the outbound IP of this machine. Port is from CLI `--http-port` or env `HTTP_PORT`. + +Sinker registers with Nacos if CLI `--consul-register-enable` or env `CONSUL_REGISTER_ENABLE` is present. However Prometheus is [unable](https://github.com/alibaba/nacos/issues/1032) to obtain dynamic service list from nacos server. + +- Push to promethues + +If CLI `--metric-push-gateway-addrs` or env `METRIC_PUSH_GATEWAY_ADDRS` (a list of comma-separated urls) is present, metrics are pushed to one of given URLs regualarly. + + +## Extending + +There are several abstract interfaces which you can implement to support more message format, message queue and config management mechanism. + +``` +type Parser interface { + Parse(bs []byte) model.Metric +} + +type Inputer interface { + Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) error + Run(ctx context.Context) + Stop() error + CommitMessages(ctx context.Context, message *model.InputMessage) error +} + +// RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... +type RemoteConfManager interface { + Init(properties map[string]interface{}) error + // Register this instance, and keep-alive via heartbeat. + Register(ip string, port int) error + Deregister(ip string, port int) error + // GetInstances fetchs healthy instances. + // Mature service-discovery solutions(Nacos, Consul etc.) have client side cache + // so that frequent invoking of GetInstances() and GetGlobalConfig() don't harm. + GetInstances() (instances []Instance, err error) + // GetConfig fetchs the config. The manager shall not reference the returned Config object after call. + GetConfig() (conf *Config, err error) + // PublishConfig publishs the config. The manager shall not reference the passed Config object after call. + PublishConfig(conf *Config) (err error) +} + +``` + +## Why not [`Kafka Engine`](https://clickhouse.tech/docs/en/engines/table-engines/integrations/kafka/) built in ClickHouse? + +- My experience indicates `Kafka Engine` is complicated, buggy and hard to debug. +- `Kafka Engine` runs inside the db process, lowers the database stability. On the other side, [Vertica](https://www.vertica.com/)'s official kafka importer is separated with the database server. +- `Kafka Engine` doesn't support custom sharding policy. +- Neither `Kafka Engine` nor clickhouse_sinker support exactly-once. + + +## Kafka Compatibility + +Kafka broker [exposes versions of various APIs it supports since 0.10.0.0](https://kafka.apache.org/protocol#api_versions). + +### Kafka-go + +- Kafka-go [negotiate it's protocol Version](https://github.com/segmentio/kafka-go/blob/c66d8ca149e7f1a7905b47a60962745ceb08a6a9/conn.go#L209). +- Kafka-go [doesn't support Kerberos authentication](https://github.com/segmentio/kafka-go/issues/237). + +### Sarama + +- Sarama guarantees compatibility [with Kafka 2.4 through 2.6](https://github.com/Shopify/sarama/blob/master/README.md#compatibility-and-api-stability). +- Sarama [has tied it's protocol usage to the Version field in Config](https://github.com/Shopify/sarama/issues/1732). + +### Conclusion + +- Neither Kafka-go nor sarama is mature as Java clients. You need to try both if clickhouse_sinker fails to connect with Kafka. +- Our experience is sarama can't work well with new kafka server if set its `Config.Version` to "0.11.0.0". So we suggest `KafkaConfig.Version` in clickhouse_sinker config matchs the Kafka server. diff --git a/docs/guide/README.md b/docs/guide/README.md index cd3d4522..7e0bdcaa 100644 --- a/docs/guide/README.md +++ b/docs/guide/README.md @@ -1 +1 @@ -# User Guide +# Get Started diff --git a/docs/guide/install.md b/docs/guide/install.md new file mode 100644 index 00000000..506f0a23 --- /dev/null +++ b/docs/guide/install.md @@ -0,0 +1,25 @@ +# Install + +## By binary files (recommended) + +Download the binary files from [release](https://github.com/housepower/clickhouse_sinker/releases), choose the executable binary file according to your env. + +## By container image + +`docker run --volume conf:/etc/clickhouse_sinker quay.io/housepower/clickhouse_sinker` + +## By source + +- Install Golang + +- Go Get + +``` +go get -u github.com/housepower/clickhouse_sinker/... +``` + +- Build && Run + +``` +make build +``` diff --git a/docs/guide/install_run.md b/docs/guide/install_run.md new file mode 100644 index 00000000..648db68c --- /dev/null +++ b/docs/guide/install_run.md @@ -0,0 +1 @@ +# Install and Run \ No newline at end of file diff --git a/docs/guide/run.md b/docs/guide/run.md new file mode 100644 index 00000000..5314593d --- /dev/null +++ b/docs/guide/run.md @@ -0,0 +1,139 @@ +# Run + +## Requirements + +Note: we shall enable we have `clickhouse-server` and `kafka` envs, before running clickhouse_sinker. + +## Configs + +> There are two ways to pass the local config, multiple files and single file. + +- For multiple files: + + `./clickhouse_sinker --local-cfg-dir conf` + + `conf` is the configuration directorys, and it'll read `conf/config.json` as main config and all tasks files in `conf/tasks/*.json` as task configs + +- For single file: + + `./clickhouse_sinker --local-cfg-file config_single.json`. + +> Read more detail descriptions of config in [here](docs/configuration.md) + +## Example + +* Let's use single file to test `clickhouse_sinker` + + - let's touch config.json + `touch config.json` + + - put sample config into the config file + + ``` + { + "clickhouse": { + "default": { + "db": "default", + "hosts": [ + [ + "127.0.0.1" + ] + ], + "port": 9000 + } + }, + "kafka": { + "default": { + "brokers": "127.0.0.1:9092", + "version": "2.2.1" + } + }, + "common": { + "bufferSize": 90000, + "minBufferSize": 1, + "msgSizeHint": 1000, + "flushInterval": 5, + "logLevel": "debug" + }, + "tasks": { + "logstash": { + "name" : "logstash", + "kafkaClient": "kafka-go", + "kafka": "default", + "topic": "logstash", + "consumerGroup" : "logstash_sinker", + "parser" : "json", + "clickhouse" : "default", + "tableName" : "logstash", + + "autoSchema" : true, + "@desc_of_exclude_columns" : "this columns will be excluded by insert SQL ", + "excludeColumns" : ["day"] + } + } + } + ``` + +* Create a simple table in Clickhouse + + > It's not the duty for clickhouse_sinker to auto create table, so we should maually do that. + + ``` + CREATE TABLE logstash + ( + `time` DateTime, + `day` Date DEFAULT toDate(time), + `request_uri` String, + `age` UInt8 + ) + ENGINE = Memory + + Ok. + + 0 rows in set. Elapsed: 0.014 sec. + + ``` + +* Enable topic is created in kafka + + > I use [kaf](https://github.com/birdayz/kaf) tool to create topics. + + ``` + kaf topic create logstash -p 1 -r 1 + ✅ Created topic! + Topic Name: logstash + Partitions: 1 + Replication Factor: 1 + Cleanup Policy: delete + ``` + + +* Run clickhouse_sinker + + ``` + ./clickhouse_sinker --local-cfg-file config.json + ``` + + +* Send messages to the topic + + ``` + echo '{"time" : "2020-12-18T03:38:39.000Z", "age" : 33 }' | kaf -b '127.0.0.1:9092' produce logstash + echo '{"time" : "2020-12-18T03:38:39.000Z", "age" : 33 }' | kaf -b '127.0.0.1:9092' produce logstash + echo '{"time" : "2020-12-18T03:38:39.000Z", "age" : 33 }' | kaf -b '127.0.0.1:9092' produce logstash + ``` + + - Check the data in clickhouse + ``` + SELECT * + FROM logstash + + ┌────────────────time─┬────────day─┬─request_uri─┬─age─┐ + │ 2020-12-18 11:38:39 │ 2020-12-18 │ │ 33 │ + │ 2020-12-18 11:38:39 │ 2020-12-18 │ │ 33 │ + │ 2020-12-18 11:38:39 │ 2020-12-18 │ │ 33 │ + └─────────────────────┴────────────┴─────────────┴─────┘ + + 3 rows in set. Elapsed: 0.016 sec. + + ``` From f0921d5dc94949d88ea02b5da5f9a27e481d201b Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sat, 19 Dec 2020 13:02:23 +0000 Subject: [PATCH 026/404] Update Token --- dev/deploy_gh_pages.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/deploy_gh_pages.sh b/dev/deploy_gh_pages.sh index ad68cf92..5fd22310 100755 --- a/dev/deploy_gh_pages.sh +++ b/dev/deploy_gh_pages.sh @@ -8,4 +8,4 @@ git config --global user.email 'dummy@dummy.dummy' git init git add -A git commit -m 'Deploy GitHub Pages' -git push -f https://sundy-li:${PAGES_DEPLOY_TOKEN}@github.com/housepower/clickhouse_sinker.git master:gh-pages +git push -f https://pan3793:${PAGES_DEPLOY_TOKEN}@github.com/housepower/clickhouse_sinker.git master:gh-pages From 78155c03f0fcc5ca39efa2582fc8b7570e17d32b Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sat, 19 Dec 2020 13:14:08 +0000 Subject: [PATCH 027/404] Update Token --- dev/deploy_gh_pages.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/deploy_gh_pages.sh b/dev/deploy_gh_pages.sh index 5fd22310..09f8e5cc 100755 --- a/dev/deploy_gh_pages.sh +++ b/dev/deploy_gh_pages.sh @@ -8,4 +8,4 @@ git config --global user.email 'dummy@dummy.dummy' git init git add -A git commit -m 'Deploy GitHub Pages' -git push -f https://pan3793:${PAGES_DEPLOY_TOKEN}@github.com/housepower/clickhouse_sinker.git master:gh-pages +git push -f https://sundy-li:${SINKER_DEPLOY_KEY}@github.com/housepower/clickhouse_sinker.git master:gh-pages From 5c83e91f93753aa1136d0a5496ebd79caa45d361 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sat, 19 Dec 2020 13:17:39 +0000 Subject: [PATCH 028/404] Update Token --- dev/deploy_gh_pages.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/deploy_gh_pages.sh b/dev/deploy_gh_pages.sh index 09f8e5cc..12523b24 100755 --- a/dev/deploy_gh_pages.sh +++ b/dev/deploy_gh_pages.sh @@ -2,10 +2,10 @@ cd docs/.vuepress/dist -git config --global user.name 'GitHub Workflow' +git config --global user.name 'sundy' git config --global user.email 'dummy@dummy.dummy' git init git add -A git commit -m 'Deploy GitHub Pages' -git push -f https://sundy-li:${SINKER_DEPLOY_KEY}@github.com/housepower/clickhouse_sinker.git master:gh-pages +git push -f https://sundy:${SINKER_DEPLOY_KEY}@github.com/housepower/clickhouse_sinker.git master:gh-pages From d4e0074f91a2e9d3aed66e55a5db9a2c08ca4569 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sat, 19 Dec 2020 13:26:51 +0000 Subject: [PATCH 029/404] Update users --- dev/deploy_gh_pages.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/deploy_gh_pages.sh b/dev/deploy_gh_pages.sh index 12523b24..c6a19658 100755 --- a/dev/deploy_gh_pages.sh +++ b/dev/deploy_gh_pages.sh @@ -8,4 +8,4 @@ git config --global user.email 'dummy@dummy.dummy' git init git add -A git commit -m 'Deploy GitHub Pages' -git push -f https://sundy:${SINKER_DEPLOY_KEY}@github.com/housepower/clickhouse_sinker.git master:gh-pages +git push -f https://sundy-li:${SINKER_DEPLOY_KEY}@github.com/housepower/clickhouse_sinker.git master:gh-pages From 9b4ece20e2b9d4b33dca47f8a0e49c5a6e88e5f9 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sat, 19 Dec 2020 13:32:45 +0000 Subject: [PATCH 030/404] Update tokens --- .github/workflows/gh-pages.yml | 2 +- dev/deploy_gh_pages.sh | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index ec055ad5..37b4d8b0 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -16,4 +16,4 @@ jobs: - run: "[[ -z $(git show HEAD -- docs) ]] || npm run docs:build" - run: "[[ -z $(git show HEAD -- docs) ]] || dev/deploy_gh_pages.sh" env: - PAGES_DEPLOY_TOKEN: ${{ secrets.PAGES_DEPLOY_TOKEN }} + PAGES_DEPLOY_TOKEN: ${{ secrets.SINKER_DEPLOY_KEY }} diff --git a/dev/deploy_gh_pages.sh b/dev/deploy_gh_pages.sh index c6a19658..5252202f 100755 --- a/dev/deploy_gh_pages.sh +++ b/dev/deploy_gh_pages.sh @@ -2,10 +2,10 @@ cd docs/.vuepress/dist -git config --global user.name 'sundy' +git config --global user.name 'workflow' git config --global user.email 'dummy@dummy.dummy' git init git add -A git commit -m 'Deploy GitHub Pages' -git push -f https://sundy-li:${SINKER_DEPLOY_KEY}@github.com/housepower/clickhouse_sinker.git master:gh-pages +git push -f https://sundy-li:${PAGES_DEPLOY_TOKEN}@github.com/housepower/clickhouse_sinker.git master:gh-pages From e09973325c399cd2e20662087feb3d6a4c0e4b64 Mon Sep 17 00:00:00 2001 From: GitHub Workflow Date: Sun, 20 Dec 2020 03:02:56 +0000 Subject: [PATCH 031/404] Doc: update --- .github/workflows/gh-pages.yml | 2 +- README.md | 4 +++- dev/deploy_gh_pages.sh | 4 ++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index 37b4d8b0..25ee5331 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -16,4 +16,4 @@ jobs: - run: "[[ -z $(git show HEAD -- docs) ]] || npm run docs:build" - run: "[[ -z $(git show HEAD -- docs) ]] || dev/deploy_gh_pages.sh" env: - PAGES_DEPLOY_TOKEN: ${{ secrets.SINKER_DEPLOY_KEY }} + TOKEN_GITHUB: ${{ secrets.TOKEN_GITHUB }} diff --git a/README.md b/README.md index b0fad81d..db9c2254 100644 --- a/README.md +++ b/README.md @@ -5,4 +5,6 @@ clickhouse_sinker is a sinker program that transfer kafka message into [ClickHouse](https://clickhouse.yandex/). -Refers to [docs](http://github.com/housepower/clickhouse_sinker/dev/introduction.html#features) to see how it works. \ No newline at end of file +[Get Started](https://housepower.github.io/clickhouse_sinker/) + +Refers to [docs](https://housepower.github.io/clickhouse_sinker/dev/introduction.html#features) to see how it works. \ No newline at end of file diff --git a/dev/deploy_gh_pages.sh b/dev/deploy_gh_pages.sh index 5252202f..9e366bee 100755 --- a/dev/deploy_gh_pages.sh +++ b/dev/deploy_gh_pages.sh @@ -2,10 +2,10 @@ cd docs/.vuepress/dist -git config --global user.name 'workflow' +git config --global user.name 'GitHub Workflow' git config --global user.email 'dummy@dummy.dummy' git init git add -A git commit -m 'Deploy GitHub Pages' -git push -f https://sundy-li:${PAGES_DEPLOY_TOKEN}@github.com/housepower/clickhouse_sinker.git master:gh-pages +git push -f https://sundy-li:${TOKEN_GITHUB}@github.com/housepower/clickhouse_sinker.git master:gh-pages From 29f86c38219474b710d8b2ba7f75ef6e149d333c Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sun, 20 Dec 2020 03:52:07 +0000 Subject: [PATCH 032/404] Doc: update --- docs/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/README.md b/docs/README.md index 582d7b01..ee61de8d 100644 --- a/docs/README.md +++ b/docs/README.md @@ -2,7 +2,7 @@ home: true heroImage: /logo_320px.svg actionText: Get Started → -actionLink: /guide/introduction +actionLink: /guide/install features: - title: clickhouse_sinker details: A tool to sink the data into ClickHouse From a08632c25b3c662576d605439c855902efeb6ff1 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sun, 20 Dec 2020 04:46:45 +0000 Subject: [PATCH 033/404] remove go_commons dep --- cmd/clickhouse_sinker/main.go | 8 ++--- config/config.go | 7 ++--- docs/configuration/flag.md | 2 -- docs/guide/run.md | 2 +- go.mod | 4 --- go.sum | 8 ----- pool/conn.go | 4 +-- util/app.go | 50 ++++++++++++++++++++++++++++++ util/conf.go | 58 +++++++++++++++++++++++++++++++++++ util/net.go | 17 ++++++++++ util/signal.go | 14 +++++++++ 11 files changed, 149 insertions(+), 25 deletions(-) create mode 100644 util/app.go create mode 100644 util/conf.go create mode 100644 util/net.go create mode 100644 util/signal.go diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 23d1e48f..de954d85 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -28,8 +28,6 @@ import ( "strings" "time" - _ "github.com/ClickHouse/clickhouse-go" - "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/health" "github.com/housepower/clickhouse_sinker/input" @@ -39,10 +37,12 @@ import ( "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/task" "github.com/housepower/clickhouse_sinker/util" + + _ "github.com/ClickHouse/clickhouse-go" + "github.com/google/gops/agent" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus/promhttp" log "github.com/sirupsen/logrus" - "github.com/sundy-li/go_commons/app" ) type CmdOptions struct { @@ -164,7 +164,7 @@ func main() { log.Fatalf("%+v", err) } - app.Run("clickhouse_sinker", func() error { + util.Run("clickhouse_sinker", func() error { var rcm config.RemoteConfManager var properties map[string]interface{} if cmdOps.ConsulRegister { diff --git a/config/config.go b/config/config.go index 412de61b..dbe08a34 100644 --- a/config/config.go +++ b/config/config.go @@ -25,9 +25,8 @@ import ( "time" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" - "github.com/sundy-li/go_commons/utils" + "github.com/pkg/errors" ) // RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... @@ -177,7 +176,7 @@ func ParseLocalCfgDir(cfgPath string) (cfg *Config, err error) { var f = "config.json" f = filepath.Join(cfgPath, f) var s string - if s, err = utils.ExtendFile(f); err != nil { + if s, err = util.ExtendFile(f); err != nil { err = errors.Wrapf(err, "") return } @@ -195,7 +194,7 @@ func ParseLocalCfgDir(cfgPath string) (cfg *Config, err error) { cfg.Tasks = make(map[string]*TaskConfig) for _, f := range files { if strings.HasSuffix(f.Name(), ".json") { - if s, err = utils.ExtendFile(filepath.Join(cfgPath, "tasks", f.Name())); err != nil { + if s, err = util.ExtendFile(filepath.Join(cfgPath, "tasks", f.Name())); err != nil { err = errors.Wrapf(err, "") return } diff --git a/docs/configuration/flag.md b/docs/configuration/flag.md index 4a3429c0..e59dc139 100644 --- a/docs/configuration/flag.md +++ b/docs/configuration/flag.md @@ -10,8 +10,6 @@ Usage of ./dist/clickhouse_sinker: configure service check DeregisterCriticalServiceAfter (default "30m") -consul-register-enable register current instance in consul - -cpunum int - Indicate the number of CPU (default 1) -http-port int http listen port (default 2112) -local-cfg-dir config.json diff --git a/docs/guide/run.md b/docs/guide/run.md index 5314593d..56d1c401 100644 --- a/docs/guide/run.md +++ b/docs/guide/run.md @@ -18,7 +18,7 @@ Note: we shall enable we have `clickhouse-server` and `kafka` envs, before runni `./clickhouse_sinker --local-cfg-file config_single.json`. -> Read more detail descriptions of config in [here](docs/configuration.md) +> Read more detail descriptions of config in [here](docs/config.md) ## Example diff --git a/go.mod b/go.mod index 2b416e21..a4b44807 100644 --- a/go.mod +++ b/go.mod @@ -4,9 +4,7 @@ go 1.14 require ( github.com/ClickHouse/clickhouse-go v1.4.3 - github.com/PuerkitoBio/goquery v0.0.0-20170623162102-8806ada2a4ab // indirect github.com/Shopify/sarama v1.27.2 - github.com/andybalholm/cascadia v0.0.0-20161224141413-349dd0209470 // indirect github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878 // indirect github.com/cespare/xxhash v1.1.0 github.com/fagongzi/goetty v1.6.0 @@ -20,7 +18,6 @@ require ( github.com/hashicorp/golang-lru v0.5.4 // indirect github.com/hashicorp/memberlist v0.1.5 // indirect github.com/hashicorp/serf v0.8.5 // indirect - github.com/jmoiron/jsonq v0.0.0-20150511023944-e874b168d07e // indirect github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 // indirect github.com/k0kubun/pp v3.0.1+incompatible github.com/nacos-group/nacos-sdk-go v1.0.1 @@ -31,7 +28,6 @@ require ( github.com/sirupsen/logrus v1.4.2 github.com/smartystreets/goconvey v1.6.4 // indirect github.com/stretchr/testify v1.6.1 - github.com/sundy-li/go_commons v0.0.0-20191120121859-84e4cb0afce4 github.com/tidwall/gjson v1.1.3 github.com/tidwall/match v1.0.0 // indirect github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 diff --git a/go.sum b/go.sum index 6b582dec..d94e4513 100644 --- a/go.sum +++ b/go.sum @@ -8,8 +8,6 @@ github.com/DataDog/datadog-go v2.2.0+incompatible h1:V5BKkxACZLjzHjSgBbr2gvLA2Ae github.com/DataDog/datadog-go v2.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= -github.com/PuerkitoBio/goquery v0.0.0-20170623162102-8806ada2a4ab h1:mmbwAhEIMdWI1cvbhA1P8gKiMZHU3zncKd1F1v63M1k= -github.com/PuerkitoBio/goquery v0.0.0-20170623162102-8806ada2a4ab/go.mod h1:T9ezsOHcCrDCgA8aF1Cqr3sSYbO/xgdy8/R/XiIMAhA= github.com/Shopify/sarama v1.27.2 h1:1EyY1dsxNDUQEv0O/4TsjosHI2CgB1uo9H/v56xzTxc= github.com/Shopify/sarama v1.27.2/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt/Mc93II= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= @@ -22,8 +20,6 @@ github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRF github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 h1:zOVTBdCKFd9JbCKz9/nt+FovbjPFmb7mUnp8nH9fQBA= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/andybalholm/cascadia v0.0.0-20161224141413-349dd0209470 h1:4jHLmof+Hba81591gfH5xYA8QXzuvgksxwPNrmjR2BA= -github.com/andybalholm/cascadia v0.0.0-20161224141413-349dd0209470/go.mod h1:3I+3V7B6gTBYfdpYgIG2ymALS9H+5VDKUl3lHH7ToM4= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e h1:QEF07wC0T1rKkctt1RINW/+RMTVmiwxETico2l3gxJA= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= @@ -163,8 +159,6 @@ github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uc github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869/go.mod h1:cJ6Cj7dQo+O6GJNiMx+Pa94qKj+TG8ONdKHgMNIyyag= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= -github.com/jmoiron/jsonq v0.0.0-20150511023944-e874b168d07e h1:ZZCvgaRDZg1gC9/1xrsgaJzQUCQgniKtw0xjWywWAOE= -github.com/jmoiron/jsonq v0.0.0-20150511023944-e874b168d07e/go.mod h1:+rHyWac2R9oAZwFe1wGY2HBzFJJy++RHBg1cU23NkD8= github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= @@ -306,8 +300,6 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/sundy-li/go_commons v0.0.0-20191120121859-84e4cb0afce4 h1:dZjgiZTbc833VU2TDNWKuUgBg1C2EYureZFVDfgpWc8= -github.com/sundy-li/go_commons v0.0.0-20191120121859-84e4cb0afce4/go.mod h1:l0XGxRCIFFqUsfVrkg7mEUejTN7Rps6bAQmejFsNhLY= github.com/tebeka/strftime v0.1.3 h1:5HQXOqWKYRFfNyBMNVc9z5+QzuBtIXy03psIhtdJYto= github.com/tebeka/strftime v0.1.3/go.mod h1:7wJm3dZlpr4l/oVK0t1HYIc4rMzQ2XJlOMIUJUJH6XQ= github.com/tidwall/gjson v1.1.3 h1:u4mspaByxY+Qk4U1QYYVzGFI8qxN/3jtEV0ZDb2vRic= diff --git a/pool/conn.go b/pool/conn.go index 578958a7..7ec32f4b 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -26,11 +26,11 @@ import ( "time" "github.com/housepower/clickhouse_sinker/health" + "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" "github.com/troian/healthcheck" log "github.com/sirupsen/logrus" - "github.com/sundy-li/go_commons/utils" ) const ( @@ -87,7 +87,7 @@ func InitConn(name string, hosts [][]string, port int, db, username, password, d numReplicas := len(replicas) replicaAddrs := make([]string, numReplicas) for i, ip := range replicas { - if ips2, err := utils.GetIp4Byname(ip); err == nil { + if ips2, err := util.GetIp4Byname(ip); err == nil { ip = ips2[0] } replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) diff --git a/util/app.go b/util/app.go new file mode 100644 index 00000000..7c6401cd --- /dev/null +++ b/util/app.go @@ -0,0 +1,50 @@ +package util + +import ( + "log" +) + +func Run(appName string, initFunc, jobFunc, cleanupFunc func() error) { + log.Printf("Initial [%s]", appName) + if err := initFunc(); err != nil { + log.Printf("Initial [%s] failure: [%s]", appName, err) + panic(err) + } + log.Printf("Initial [%s] complete", appName) + go func() { + if err := jobFunc(); err != nil { + log.Printf("[%s] run error: [%v]", appName, err) + panic(err) + } + }() + + WaitForExitSign() + log.Printf("[%s] watched the exit signal, start to clean", appName) + if err := cleanupFunc(); err != nil { + log.Printf("[%s] clean failed: [%v]", appName, err) + panic(err) + } + log.Printf("[%s] clean complete, exited", appName) +} + +func Funcs(funcs ...func() error) func() error { + return func() error { + for _, fun := range funcs { + if err := fun(); err != nil { + return err + } + } + return nil + } +} +func LogWrapper(msg string, fun func() error) func() error { + return func() error { + log.Println(msg + " start") + if err := fun(); err != nil { + log.Printf("%s failed: %v", msg, err) + return err + } + log.Println(msg + " success") + return nil + } +} diff --git a/util/conf.go b/util/conf.go new file mode 100644 index 00000000..5d532d2e --- /dev/null +++ b/util/conf.go @@ -0,0 +1,58 @@ +package util + +import ( + "encoding/json" + "errors" + "fmt" + "io/ioutil" + "os" + "path/filepath" + "regexp" + "strings" +) + +const extendTag = "@extend:" + +func ExtendFile(filePath string) (string, error) { + fi, err := os.Stat(filePath) + if err != nil { + return "", err + } else if fi.IsDir() { + return "", errors.New(filePath + " is not a file.") + } + var b []byte + b, err = ioutil.ReadFile(filePath) + if err != nil { + return "", err + } + dir, err := filepath.Abs(filepath.Dir(filePath)) + if err != nil { + return "", err + } + return ExtendFileContent(dir, b) +} + +func ExtendFileContent(dir string, content []byte) (string, error) { + //检查是不是规范的json + test := new(interface{}) + err := json.Unmarshal(content, &test) + if err != nil { + return "", err + } + + //替换子json文件 + reg := regexp.MustCompile(`"` + extendTag + `.*?"`) + ret := reg.ReplaceAllStringFunc(string(content), func(match string) string { + match = match[len(extendTag)+1 : len(match)-1] + var p = match + if !strings.HasPrefix(match, "/") { + p = dir + "/" + match + } + sb, err2 := ExtendFile(p) + if err2 != nil { + err = fmt.Errorf("替换json配置[%s]失败:%s\n", match, err2.Error()) + } + return string(sb) + }) + return ret, err +} diff --git a/util/net.go b/util/net.go new file mode 100644 index 00000000..9ede6397 --- /dev/null +++ b/util/net.go @@ -0,0 +1,17 @@ +package util + +import "net" + +func GetIp4Byname(host string) (ips []string, err error) { + addrs, err := net.LookupIP(host) + if err != nil { + return + } + ips = make([]string, len(addrs)) + for i, addr := range addrs { + if ipv4 := addr.To4(); ipv4 != nil { + ips[i] = ipv4.String() + } + } + return +} diff --git a/util/signal.go b/util/signal.go new file mode 100644 index 00000000..4ccae701 --- /dev/null +++ b/util/signal.go @@ -0,0 +1,14 @@ +package util + +import ( + "os" + "os/signal" + "syscall" +) + +func WaitForExitSign() { + c := make(chan os.Signal, 1) + //结束,收到ctrl+c 信号 + signal.Notify(c, os.Interrupt, os.Kill, syscall.SIGTERM, syscall.SIGHUP) + <-c +} From 694d36320e59cad0961674625bb4deccda600858 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sun, 20 Dec 2020 04:59:34 +0000 Subject: [PATCH 034/404] make lint happy --- pool/conn.go | 2 +- util/app.go | 15 +++++++++++++++ util/conf.go | 25 ++++++++++++++++++++----- util/consistenthash.go | 15 +++++++++++++++ util/net.go | 17 ++++++++++++++++- util/signal.go | 18 ++++++++++++++++-- 6 files changed, 83 insertions(+), 9 deletions(-) diff --git a/pool/conn.go b/pool/conn.go index 7ec32f4b..34429346 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -87,7 +87,7 @@ func InitConn(name string, hosts [][]string, port int, db, username, password, d numReplicas := len(replicas) replicaAddrs := make([]string, numReplicas) for i, ip := range replicas { - if ips2, err := util.GetIp4Byname(ip); err == nil { + if ips2, err := util.GetIP4Byname(ip); err == nil { ip = ips2[0] } replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) diff --git a/util/app.go b/util/app.go index 7c6401cd..fcba7f6a 100644 --- a/util/app.go +++ b/util/app.go @@ -1,3 +1,18 @@ +/*Copyright [2019] housepower + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/util/conf.go b/util/conf.go index 5d532d2e..924dabf0 100644 --- a/util/conf.go +++ b/util/conf.go @@ -1,14 +1,29 @@ +/*Copyright [2019] housepower + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( "encoding/json" - "errors" - "fmt" "io/ioutil" "os" "path/filepath" "regexp" "strings" + + "github.com/pkg/errors" ) const extendTag = "@extend:" @@ -18,7 +33,7 @@ func ExtendFile(filePath string) (string, error) { if err != nil { return "", err } else if fi.IsDir() { - return "", errors.New(filePath + " is not a file.") + return "", errors.New("error in ExtendFile, " + filePath + " is not a file") } var b []byte b, err = ioutil.ReadFile(filePath) @@ -50,9 +65,9 @@ func ExtendFileContent(dir string, content []byte) (string, error) { } sb, err2 := ExtendFile(p) if err2 != nil { - err = fmt.Errorf("替换json配置[%s]失败:%s\n", match, err2.Error()) + err = errors.Wrapf(err2, "replace json config [%s]failed", match) } - return string(sb) + return sb }) return ret, err } diff --git a/util/consistenthash.go b/util/consistenthash.go index 86973b2e..88d1d1f9 100644 --- a/util/consistenthash.go +++ b/util/consistenthash.go @@ -1,3 +1,18 @@ +/*Copyright [2019] housepower + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + // an implementation of a ring hash. // Refers to https://github.com/golang/groupcache/blob/master/consistenthash/consistenthash.go package util diff --git a/util/net.go b/util/net.go index 9ede6397..884bd1af 100644 --- a/util/net.go +++ b/util/net.go @@ -1,8 +1,23 @@ +/*Copyright [2019] housepower + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import "net" -func GetIp4Byname(host string) (ips []string, err error) { +func GetIP4Byname(host string) (ips []string, err error) { addrs, err := net.LookupIP(host) if err != nil { return diff --git a/util/signal.go b/util/signal.go index 4ccae701..58c433cb 100644 --- a/util/signal.go +++ b/util/signal.go @@ -1,3 +1,18 @@ +/*Copyright [2019] housepower + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( @@ -8,7 +23,6 @@ import ( func WaitForExitSign() { c := make(chan os.Signal, 1) - //结束,收到ctrl+c 信号 - signal.Notify(c, os.Interrupt, os.Kill, syscall.SIGTERM, syscall.SIGHUP) + signal.Notify(c, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) <-c } From 7ef1d6a4a5570515f1a864e4db32f31c35094869 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 22 Dec 2020 17:39:48 +0800 Subject: [PATCH 035/404] fixed --local-cfg-dir --- cmd/clickhouse_sinker/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index de954d85..8bec45ba 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -270,7 +270,7 @@ func (s *Sinker) Run() { log.Fatalf("%+v", err) return } - } else if _, err = os.Stat(cmdOps.LocalCfgFile); err == nil { + } else if _, err = os.Stat(cmdOps.LocalCfgDir); err == nil { if newCfg, err = config.ParseLocalCfgDir(cmdOps.LocalCfgDir); err != nil { log.Fatalf("%+v", err) return From f13d12091c8cce976279d9fc113fc03a6572d364 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 30 Dec 2020 14:35:24 +0800 Subject: [PATCH 036/404] fix concurrentParsers --- cmd/clickhouse_sinker/main.go | 12 ++++++++++-- util/workerpool.go | 3 +++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 8bec45ba..6bba1dda 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -368,9 +368,13 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { s.tasks[taskName] = t } concurrentParsers = len(taskNames) * 10 + } + if runtime.NumCPU() >= 2 { if concurrentParsers > runtime.NumCPU()/2 { concurrentParsers = runtime.NumCPU() / 2 } + } else { + concurrentParsers = 1 } util.InitGlobalParsingPool(concurrentParsers) totalConn := pool.GetTotalConn() @@ -456,8 +460,12 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { } // 4. Resize goroutine pools. concurrentParsers := len(s.tasks) * 10 - if concurrentParsers > runtime.NumCPU()/2 { - concurrentParsers = runtime.NumCPU() / 2 + if runtime.NumCPU() >= 2 { + if concurrentParsers > runtime.NumCPU()/2 { + concurrentParsers = runtime.NumCPU() / 2 + } + } else { + concurrentParsers = 1 } util.GlobalParsingPool.Resize(concurrentParsers) totalConn := pool.GetTotalConn() diff --git a/util/workerpool.go b/util/workerpool.go index ce386072..58b060a6 100644 --- a/util/workerpool.go +++ b/util/workerpool.go @@ -30,6 +30,9 @@ func NewWorkerPool(maxWorkers int, queueSize int) *WorkerPool { if maxWorkers <= 0 { panic("WorkerNum must be greater than zero") } + if queueSize <= 0 { + panic("queueSize must be greater than zero") + } w := &WorkerPool{ maxWorkers: maxWorkers, From 318f21c19d32c851909b847b6c279cb7c017e282 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 5 Jan 2021 10:56:43 +0800 Subject: [PATCH 037/404] fix parser test --- parser/fastjson.go | 25 +++++++---- parser/fastjson_test.go | 38 ++++++++++++++++ parser/gjson_extend.go | 1 - parser/gjson_extend_test.go | 56 ++++++++++------------- parser/gjson_test.go | 90 +++++++++++++++++-------------------- parser/parser_test.go | 55 +++-------------------- 6 files changed, 126 insertions(+), 139 deletions(-) create mode 100644 parser/fastjson_test.go diff --git a/parser/fastjson.go b/parser/fastjson.go index 7359ab14..497eebb1 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -69,26 +69,35 @@ func (c *FastjsonMetric) GetInt(key string, nullable bool) interface{} { if nullable && !c.value.Exists(key) { return nil } - return int64(c.value.GetInt(key)) + return c.value.GetInt64(key) } func (c *FastjsonMetric) GetArray(key string, t string) interface{} { array := c.value.GetArray(key) if array == nil { - return nil + switch t { + case "int": + return []int64{} + case "float": + return []float64{} + case "string": + return []string{} + default: + return nil + } } switch t { - case "float": - results := make([]float64, 0, len(array)) + case "int": + results := make([]int64, 0, len(array)) for _, e := range array { - v, _ := e.Float64() + v, _ := e.Int64() results = append(results, v) } return results - case "int": - results := make([]int, 0, len(array)) + case "float": + results := make([]float64, 0, len(array)) for _, e := range array { - v, _ := e.Int() + v, _ := e.Float64() results = append(results, v) } return results diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go new file mode 100644 index 00000000..b60c1d95 --- /dev/null +++ b/parser/fastjson_test.go @@ -0,0 +1,38 @@ +package parser + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestFastjsonArray(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + actI := metric.GetArray("array_int", "int").([]int64) + expI := []int64{1, 2, 3} + require.Equal(t, expI, actI) + + actF := metric.GetArray("array_float", "float").([]float64) + expF := []float64{1.1, 2.2, 3.3} + require.Equal(t, expF, actF) + + actS := metric.GetArray("array_string", "string").([]string) + expS := []string{"aa", "bb", "cc"} + require.Equal(t, expS, actS) + + actIE := metric.GetArray("array_empty", "int").([]int64) + expIE := []int64{} + require.Equal(t, expIE, actIE) + + actFE := metric.GetArray("array_empty", "float").([]float64) + expFE := []float64{} + require.Equal(t, expFE, actFE) + + actSE := metric.GetArray("array_empty", "string").([]string) + expSE := []string{} + require.Equal(t, expSE, actSE) +} diff --git a/parser/gjson_extend.go b/parser/gjson_extend.go index ca550a4d..46a5b206 100644 --- a/parser/gjson_extend.go +++ b/parser/gjson_extend.go @@ -90,7 +90,6 @@ func (c *GjsonExtendMetric) Get(key string) interface{} { } func (c *GjsonExtendMetric) GetString(key string, nullable bool) interface{} { - //判断object val := c.mp[key] if nullable && val == nil { diff --git a/parser/gjson_extend_test.go b/parser/gjson_extend_test.go index 428d957f..3d7c0097 100644 --- a/parser/gjson_extend_test.go +++ b/parser/gjson_extend_test.go @@ -38,19 +38,6 @@ func TestGjsonExtendIntNullableTrue(t *testing.T) { require.Nil(t, result, "err should be nothing") } -func TestGjsonExtendArrayInt(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - arr := metric.GetArray("mp_a", "int").([]int64) - expected := []int64{1, 2, 3} - for i := range arr { - require.Equal(t, arr[i], expected[i]) - } -} - func TestGjsonExtendStr(t *testing.T) { pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) parser := pp.Get() @@ -83,19 +70,6 @@ func TestGjsonExtendStrNullableTrue(t *testing.T) { require.Nil(t, result, "err should be nothing") } -func TestGjsonExtendArrayString(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - arr := metric.GetArray("mps_a", "string").([]string) - expected := []string{"aa", "bb", "cc"} - for i := range arr { - require.Equal(t, arr[i], expected[i]) - } -} - func TestGjsonExtendFloat(t *testing.T) { pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) parser := pp.Get() @@ -128,17 +102,35 @@ func TestGjsonExtendFloatNullableTrue(t *testing.T) { require.Nil(t, result, "err should be nothing") } -func TestGjsonExtendArrayFloat(t *testing.T) { +func TestGjsonExtendArray(t *testing.T) { pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - arr := metric.GetArray("mp_f", "float").([]float64) - expected := []float64{1.11, 2.22, 3.33} - for i := range arr { - require.Equal(t, arr[i], expected[i]) - } + actI := metric.GetArray("mp.i", "int").([]int64) + expI := []int64{1, 2, 3} + require.Equal(t, actI, expI) + + actF := metric.GetArray("mp.f", "float").([]float64) + expF := []float64{1.1, 2.2, 3.3} + require.Equal(t, expF, actF) + + actS := metric.GetArray("mp.s", "string").([]string) + expS := []string{"aa", "bb", "cc"} + require.Equal(t, expS, actS) + + actIE := metric.GetArray("mp.e", "int").([]int64) + expIE := []int64{} + require.Equal(t, expIE, actIE) + + actFE := metric.GetArray("mp.e", "float").([]float64) + expFE := []float64{} + require.Equal(t, expFE, actFE) + + actSE := metric.GetArray("mp.e", "string").([]string) + expSE := []string{} + require.Equal(t, expSE, actSE) } func TestGjsonExtendElasticDateTime(t *testing.T) { diff --git a/parser/gjson_test.go b/parser/gjson_test.go index 8385a36e..eacfbd64 100644 --- a/parser/gjson_test.go +++ b/parser/gjson_test.go @@ -7,7 +7,7 @@ import ( ) func TestGjsonInt(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -17,18 +17,18 @@ func TestGjsonInt(t *testing.T) { require.Equal(t, result, expected) } func TestGjsonIntNullableFalse(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - var expected int = 0 - result := metric.GetInt("its_not_exist", false).(int) + var expected int64 = 0 + result := metric.GetInt("its_not_exist", false).(int64) require.Equal(t, expected, result) } func TestGjsonIntNullableTrue(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -37,21 +37,8 @@ func TestGjsonIntNullableTrue(t *testing.T) { require.Nil(t, result, "err should be nothing") } -func TestGjsonArrayInt(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - arr := metric.GetArray("mp.a", "int").([]int64) - expected := []int64{1, 2, 3} - for i := range arr { - require.Equal(t, arr[i], expected[i]) - } -} - func TestGjsonStr(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -61,7 +48,7 @@ func TestGjsonStr(t *testing.T) { require.Equal(t, result, expected) } func TestGjsonStrNullableFalse(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -71,7 +58,7 @@ func TestGjsonStrNullableFalse(t *testing.T) { require.Equal(t, result, expected) } func TestGjsonStrNullableTrue(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -80,21 +67,8 @@ func TestGjsonStrNullableTrue(t *testing.T) { require.Nil(t, result, "err should be nothing") } -func TestGjsonArrayString(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - arr := metric.GetArray("mps.a", "string").([]string) - expected := []string{"aa", "bb", "cc"} - for i := range arr { - require.Equal(t, arr[i], expected[i]) - } -} - func TestGjsonFloat(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -105,17 +79,17 @@ func TestGjsonFloat(t *testing.T) { } func TestGjsonFloatNullableFalse(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - var expected int = 0 - result := metric.GetFloat("percent_not_exist", false).(int) + var expected float64 = 0 + result := metric.GetFloat("percent_not_exist", false).(float64) require.Equal(t, result, expected) } func TestGjsonFloatNullableTrue(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -124,21 +98,39 @@ func TestGjsonFloatNullableTrue(t *testing.T) { require.Nil(t, result, "err should be nothing") } -func TestGjsonArrayFloat(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) +func TestGjsonArray(t *testing.T) { + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - arr := metric.GetArray("mp.f", "float").([]float64) - expected := []float64{1.11, 2.22, 3.33} - for i := range arr { - require.Equal(t, arr[i], expected[i]) - } + actI := metric.GetArray("mp.i", "int").([]int64) + expI := []int64{1, 2, 3} + require.Equal(t, actI, expI) + + actF := metric.GetArray("mp.f", "float").([]float64) + expF := []float64{1.1, 2.2, 3.3} + require.Equal(t, expF, actF) + + actS := metric.GetArray("mp.s", "string").([]string) + expS := []string{"aa", "bb", "cc"} + require.Equal(t, expS, actS) + + actIE := metric.GetArray("mp.e", "int").([]int64) + expIE := []int64{} + require.Equal(t, expIE, actIE) + + actFE := metric.GetArray("mp.e", "float").([]float64) + expFE := []float64{} + require.Equal(t, expFE, actFE) + + actSE := metric.GetArray("mp.e", "string").([]string) + expSE := []string{} + require.Equal(t, expSE, actSE) } func TestGjsonElasticDateTime(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -151,7 +143,7 @@ func TestGjsonElasticDateTime(t *testing.T) { } func TestGjsonElasticDateTimeNullableFalse(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -162,7 +154,7 @@ func TestGjsonElasticDateTimeNullableFalse(t *testing.T) { } func TestGjsonElasticDateTimeNullableTrue(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) diff --git a/parser/parser_test.go b/parser/parser_test.go index 9ccbf9b9..5dc25127 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -18,9 +18,7 @@ import ( "encoding/json" "log" "testing" - "time" - "github.com/stretchr/testify/require" "github.com/tidwall/gjson" "github.com/valyala/fastjson" ) @@ -36,14 +34,14 @@ var jsonSample = []byte(`{ "version":"5.8.3", "success":0, "percent":0.11, - "mp": {"a" : [1,2,3]}, - "mpf": {"a" : [1.1,2.2,3.3]}, - "mps": {"a" : ["aa","bb","cc"]}, - "date": "2019-12-16T12:10:30Z" + "mp": {"i" : [1,2,3], "f": [1.1,2.2,3.3], "s": ["aa","bb","cc"], "e": []}, + "date": "2019-12-16T12:10:30Z", + "array_int": [1,2,3], + "array_float": [1.1,2.2,3.3], + "array_string": ["aa","bb","cc"], + "array_empty": [] }`) -var jsonSample2 = []byte(`{"time":"2006-01-02 15:04:05","timestamp":"2006-01-02T15:04:05.123+08:00","item_guid":"bus070_ins062","metric_name":"CPU繁忙率","alg_name":"Ripple","value":60,"upper":100,"lower":60,"yhat_upper":100,"yhat_lower":60,"yhat_flag":23655,"total_anomaly":61357,"anomaly":0.3,"abnormal_type":22,"abnormality":913,"container_id":39929,"hard_upper":100,"hard_lower":60,"hard_anomaly":39371,"shift_tag":38292,"season_tag":56340,"spike_tag":13231,"is_missing":0,"str_array":["tag3","tag5"],"int_array":[123,456]}`) - func BenchmarkUnmarshalljson(b *testing.B) { mp := map[string]interface{}{} for i := 0; i < b.N; i++ { @@ -111,44 +109,3 @@ func BenchmarkUnmarshalGabon2(b *testing.B) { _ = result["trycount"].Int() } } - -func TestGjsonExtend(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - arr := metric.GetArray("mp.a", "int").([]int64) - expected := []int64{1, 2, 3} - for i := range arr { - require.Equal(t, arr[i], expected[i]) - } - - metric, _ = parser.Parse(jsonSample2) - arr2 := metric.GetArray("str_array", "string").([]string) - exp2 := []string{"tag3", "tag5"} - require.Equal(t, exp2, arr2) -} - -func TestFastJson(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", []string{DefaultTSLayout[0], "2006-01-02 15:04:05", time.RFC3339}) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample2) - - ts1 := metric.GetDateTime("time", false) - exp1, _ := time.Parse("2006-01-02 15:04:05", "2006-01-02 15:04:05") - require.Equal(t, exp1, ts1) - - ts2 := metric.GetDateTime64("timestamp", false) - exp2, _ := time.Parse(time.RFC3339, "2006-01-02T15:04:05.123+08:00") - require.Equal(t, exp2, ts2) - - arr := metric.GetArray("str_array", "string").([]string) - exp3 := []string{"tag3", "tag5"} - require.Equal(t, exp3, arr) - - arr2 := metric.GetArray("int_array", "int").([]int) - exp4 := []int{123, 456} - require.Equal(t, exp4, arr2) -} From 17378553d66d6614bfee1ef6f8e531add80937b6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 5 Jan 2021 14:33:40 +0800 Subject: [PATCH 038/404] remove gjson_extend and cleanup parser tests --- parser/fastjson_test.go | 136 ++++++++++++++++++++ parser/gjson_extend.go | 241 ------------------------------------ parser/gjson_extend_test.go | 168 ------------------------- parser/gjson_test.go | 162 ++++++++++++------------ parser/parser.go | 3 - parser/parser_test.go | 4 +- 6 files changed, 225 insertions(+), 489 deletions(-) delete mode 100644 parser/gjson_extend.go delete mode 100644 parser/gjson_extend_test.go diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go index b60c1d95..9256b801 100644 --- a/parser/fastjson_test.go +++ b/parser/fastjson_test.go @@ -2,10 +2,146 @@ package parser import ( "testing" + "time" "github.com/stretchr/testify/require" ) +func TestFastjsonInt(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act int64 + exp = 1536813227 + act = metric.GetInt("its", false).(int64) + require.Equal(t, exp, act) + + exp = 0 + act = metric.GetInt("its_not_exist", false).(int64) + require.Equal(t, exp, act) + + actual := metric.GetInt("its_not_exist", true) + require.Nil(t, actual, "err should be nothing") +} + +func TestFastjsonFloat(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act float64 + exp = 0.11 + act = metric.GetFloat("percent", false).(float64) + require.Equal(t, exp, act) + + exp = 0.0 + act = metric.GetFloat("percent_not_exist", false).(float64) + require.Equal(t, exp, act) + + actual := metric.GetFloat("percent_not_exist", true) + require.Nil(t, actual, "err should be nothing") +} + +func TestFastjsonString(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act string + exp = "ws" + act = metric.GetString("channel", false).(string) + require.Equal(t, exp, act) + + exp = "" + act = metric.GetString("channel_not_exist", false).(string) + require.Equal(t, exp, act) + + actual := metric.GetString("channel_not_exist", true) + require.Nil(t, actual, "err should be nothing") +} + +func TestFastjsonDate(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.UTC) + act = metric.GetDate("time1", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDate("time1_not_exist", false).(time.Time) + require.Equal(t, exp, act) + + actual := metric.GetDate("time1_not_exist", true) + require.Nil(t, actual, "err should be nothing") +} + +func TestFastjsonDateTime(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) + act = metric.GetDateTime("time2", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDateTime("time2_not_exist", false).(time.Time) + require.Equal(t, exp, act) + + actual := metric.GetDateTime("time2_not_exist", true) + require.Nil(t, actual, "err should be nothing") +} + +func TestFastjsonDateTime64(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) + act = metric.GetDateTime64("time3", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDateTime64("time3_not_exist", false).(time.Time) + require.Equal(t, exp, act) + + actual := metric.GetDateTime64("time3_not_exist", true) + require.Nil(t, actual, "err should be nothing") +} + +func TestFastjsonElasticDateTime(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act int64 + // {"date": "2019-12-16T12:10:30Z"} + // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC + exp = 1576498230 + act = metric.GetElasticDateTime("time2", false).(int64) + require.Equal(t, exp, act) + + exp = -62135596800 + act = metric.GetElasticDateTime("time2_not_exist", false).(int64) + require.Equal(t, exp, act) + + actual := metric.GetElasticDateTime("time2_not_exist", true) + require.Nil(t, actual, "err should be nothing") +} + func TestFastjsonArray(t *testing.T) { pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) parser := pp.Get() diff --git a/parser/gjson_extend.go b/parser/gjson_extend.go deleted file mode 100644 index 46a5b206..00000000 --- a/parser/gjson_extend.go +++ /dev/null @@ -1,241 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package parser - -import ( - "fmt" - "time" - - "github.com/tidwall/gjson" - - "github.com/housepower/clickhouse_sinker/model" -) - -var _ Parser = (*GjsonExtendParser)(nil) - -type GjsonExtendParser struct { - tsLayout []string -} - -type GjsonExtendMetric struct { - mp map[string]interface{} - tsLayout []string -} - -// { -// "aa" : "1", -// "bb" : { -// "cc" : 3, -// "dd" : [ "33", "44"] -// } -// } - -// will be - -// { -// "aa" : "1", -// "bb_cc" : 3, -// "bb_dd" : ["33", "44"] -// } -func (p *GjsonExtendParser) Parse(bs []byte) (metric model.Metric, err error) { - var mp = make(map[string]interface{}) - - jsonResults := gjson.ParseBytes(bs) - jsonResults.ForEach(func(key gjson.Result, value gjson.Result) bool { - if key.String() != "" { - if value.Type != gjson.JSON { - mp[key.String()] = value.Value() - } else { - injectObject(key.String(), mp, value) - } - } - return true - }) - - metric = &GjsonExtendMetric{mp, p.tsLayout} - return -} - -func injectObject(prefix string, result map[string]interface{}, t gjson.Result) { - if t.IsArray() { - result[prefix] = t - return - } - t.ForEach(func(key gjson.Result, value gjson.Result) bool { - switch value.Type { - case gjson.JSON: - injectObject(prefix+"."+key.String(), result, value) - default: - result[prefix+"."+key.String()] = value.Value() - } - return true - }) -} - -func (c *GjsonExtendMetric) Get(key string) interface{} { - return c.mp[key] -} - -func (c *GjsonExtendMetric) GetString(key string, nullable bool) interface{} { - val := c.mp[key] - - if nullable && val == nil { - return nil - } - - if val == nil { - return "" - } - switch v := val.(type) { - case map[string]interface{}: - return GetJSONShortStr(v) - - case string: - return v - - default: - return "" - } -} - -func (c *GjsonExtendMetric) GetArray(key string, t string) interface{} { - v, ok := c.mp[key].(gjson.Result) - - slice := v.Array() - switch t { - case "string": - results := make([]string, 0, len(slice)) - if !ok { - return results - } - for _, s := range slice { - results = append(results, s.String()) - } - return results - - case "float": - results := make([]float64, 0, len(slice)) - - if !ok { - return results - } - - for _, s := range slice { - results = append(results, s.Float()) - } - return results - - case "int": - results := make([]int64, 0, len(slice)) - if !ok { - return results - } - for _, s := range slice { - results = append(results, s.Int()) - } - return results - - default: - panic("not supported array type " + t) - } -} - -func (c *GjsonExtendMetric) GetFloat(key string, nullable bool) interface{} { - val := c.mp[key] - if nullable && val == nil { - return nil - } - if val == nil { - return 0 - } - switch v := val.(type) { - case float64: - return v - default: - return 0 - } -} - -func (c *GjsonExtendMetric) GetInt(key string, nullable bool) interface{} { - val := c.mp[key] - if nullable && val == nil { - return nil - } - - if val == nil { - return 0 - } - switch v := val.(type) { - case float64: - return int64(v) - default: - return 0 - } -} - -func (c *GjsonExtendMetric) GetDate(key string, nullable bool) interface{} { - probe := c.mp[key] - - if nullable && probe == nil { - return nil - } - - val := fmt.Sprintf("%v", c.GetString(key, false)) - - t, _ := time.Parse(c.tsLayout[0], val) - return t -} - -func (c *GjsonExtendMetric) GetDateTime(key string, nullable bool) interface{} { - probe := c.mp[key] - - if nullable && probe == nil { - return nil - } - - if v := c.GetFloat(key, false).(float64); v != 0 { - return time.Unix(int64(v), int64(v*1e9)%1e9) - } - - val := c.GetString(key, false).(string) - t, _ := time.Parse(c.tsLayout[1], val) - return t -} - -func (c *GjsonExtendMetric) GetDateTime64(key string, nullable bool) interface{} { - probe := c.mp[key] - - if nullable && probe == nil { - return nil - } - - if v := c.GetFloat(key, false).(float64); v != 0 { - return time.Unix(int64(v), int64(v*1e9)%1e9) - } - - val := c.GetString(key, false).(string) - t, _ := time.Parse(c.tsLayout[2], val) - return t -} - -func (c *GjsonExtendMetric) GetElasticDateTime(key string, nullable bool) interface{} { - val := c.GetString(key, nullable) - if val == nil { - return nil - } - t, _ := time.Parse(time.RFC3339, val.(string)) - return t.Unix() -} diff --git a/parser/gjson_extend_test.go b/parser/gjson_extend_test.go deleted file mode 100644 index 3d7c0097..00000000 --- a/parser/gjson_extend_test.go +++ /dev/null @@ -1,168 +0,0 @@ -package parser - -import ( - "testing" - - "github.com/stretchr/testify/require" -) - -func TestGjsonExtendInt(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var expected int64 = 1536813227 - result := metric.GetInt("its", false).(int64) - require.Equal(t, result, expected) -} - -func TestGjsonExtendIntNullableFalse(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var expected int = 0 - result := metric.GetInt("its_not_exist", false).(int) - require.Equal(t, result, expected) -} - -func TestGjsonExtendIntNullableTrue(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - result := metric.GetInt("its_not_exist", true) - require.Nil(t, result, "err should be nothing") -} - -func TestGjsonExtendStr(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var expected string = "ws" - result := metric.GetString("channel", false).(string) - require.Equal(t, result, expected) -} - -func TestGjsonExtendStrNullableFalse(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var expected string = "" - result := metric.GetString("channel_not_exist", false).(string) - require.Equal(t, result, expected) -} - -func TestGjsonExtendStrNullableTrue(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - result := metric.GetString("channel_not_exist", true) - require.Nil(t, result, "err should be nothing") -} - -func TestGjsonExtendFloat(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var expected float64 = 0.11 - result := metric.GetFloat("percent", false).(float64) - require.Equal(t, result, expected) -} - -func TestGjsonExtendFloatNullableFalse(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var expected int = 0 - result := metric.GetFloat("percent_not_exist", false).(int) - require.Equal(t, result, expected) -} - -func TestGjsonExtendFloatNullableTrue(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - result := metric.GetFloat("percent_not_exist", true) - require.Nil(t, result, "err should be nothing") -} - -func TestGjsonExtendArray(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - actI := metric.GetArray("mp.i", "int").([]int64) - expI := []int64{1, 2, 3} - require.Equal(t, actI, expI) - - actF := metric.GetArray("mp.f", "float").([]float64) - expF := []float64{1.1, 2.2, 3.3} - require.Equal(t, expF, actF) - - actS := metric.GetArray("mp.s", "string").([]string) - expS := []string{"aa", "bb", "cc"} - require.Equal(t, expS, actS) - - actIE := metric.GetArray("mp.e", "int").([]int64) - expIE := []int64{} - require.Equal(t, expIE, actIE) - - actFE := metric.GetArray("mp.e", "float").([]float64) - expFE := []float64{} - require.Equal(t, expFE, actFE) - - actSE := metric.GetArray("mp.e", "string").([]string) - expSE := []string{} - require.Equal(t, expSE, actSE) -} - -func TestGjsonExtendElasticDateTime(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - // {"date": "2019-12-16T12:10:30Z"} - // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC - var expected int64 = 1576498230 - result := metric.GetElasticDateTime("date", false).(int64) - require.Equal(t, result, expected) -} - -func TestGjsonExtendElasticDateTimeNullableFalse(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var expected int64 = -62135596800 - result := metric.GetElasticDateTime("date_not_exist", false).(int64) - require.Equal(t, result, expected) -} - -func TestGjsonExtendElasticDateTimeNullableTrue(t *testing.T) { - pp := NewParserPool("gjson_extend", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - result := metric.GetElasticDateTime("date_not_exist", true) - require.Nil(t, result, "err should be nothing") -} diff --git a/parser/gjson_test.go b/parser/gjson_test.go index eacfbd64..c2a2b647 100644 --- a/parser/gjson_test.go +++ b/parser/gjson_test.go @@ -2,6 +2,7 @@ package parser import ( "testing" + "time" "github.com/stretchr/testify/require" ) @@ -12,90 +13,133 @@ func TestGjsonInt(t *testing.T) { defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - var expected int64 = 1536813227 - result := metric.GetInt("its", false).(int64) - require.Equal(t, result, expected) -} -func TestGjsonIntNullableFalse(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) + var exp, act int64 + exp = 1536813227 + act = metric.GetInt("its", false).(int64) + require.Equal(t, exp, act) + + exp = 0 + act = metric.GetInt("its_not_exist", false).(int64) + require.Equal(t, exp, act) - var expected int64 = 0 - result := metric.GetInt("its_not_exist", false).(int64) - require.Equal(t, expected, result) + actual := metric.GetInt("its_not_exist", true) + require.Nil(t, actual, "err should be nothing") } -func TestGjsonIntNullableTrue(t *testing.T) { +func TestGjsonFloat(t *testing.T) { pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - result := metric.GetInt("its_not_exist", true) - require.Nil(t, result, "err should be nothing") -} + var exp, act float64 + exp = 0.11 + act = metric.GetFloat("percent", false).(float64) + require.Equal(t, exp, act) -func TestGjsonStr(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) + exp = 0.0 + act = metric.GetFloat("percent_not_exist", false).(float64) + require.Equal(t, exp, act) - var expected string = "ws" - result := metric.GetString("channel", false).(string) - require.Equal(t, result, expected) + actual := metric.GetFloat("percent_not_exist", true) + require.Nil(t, actual, "err should be nothing") } -func TestGjsonStrNullableFalse(t *testing.T) { + +func TestGjsonString(t *testing.T) { pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - var expected string = "" - result := metric.GetString("channel_not_exist", false).(string) - require.Equal(t, result, expected) + var exp, act string + exp = "ws" + act = metric.GetString("channel", false).(string) + require.Equal(t, exp, act) + + exp = "" + act = metric.GetString("channel_not_exist", false).(string) + require.Equal(t, exp, act) + + actual := metric.GetString("channel_not_exist", true) + require.Nil(t, actual, "err should be nothing") } -func TestGjsonStrNullableTrue(t *testing.T) { + +func TestGjsonDate(t *testing.T) { pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - result := metric.GetString("channel_not_exist", true) - require.Nil(t, result, "err should be nothing") + var exp, act time.Time + exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.UTC) + act = metric.GetDate("time1", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDate("time1_not_exist", false).(time.Time) + require.Equal(t, exp, act) + + actual := metric.GetDate("time1_not_exist", true) + require.Nil(t, actual, "err should be nothing") } -func TestGjsonFloat(t *testing.T) { +func TestGjsonDateTime(t *testing.T) { pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - var expected float64 = 0.11 - result := metric.GetFloat("percent", false).(float64) - require.Equal(t, result, expected) + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) + act = metric.GetDateTime("time2", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDateTime("time2_not_exist", false).(time.Time) + require.Equal(t, exp, act) + + actual := metric.GetDateTime("time2_not_exist", true) + require.Nil(t, actual, "err should be nothing") } -func TestGjsonFloatNullableFalse(t *testing.T) { +func TestGjsonDateTime64(t *testing.T) { pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - var expected float64 = 0 - result := metric.GetFloat("percent_not_exist", false).(float64) - require.Equal(t, result, expected) + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) + act = metric.GetDateTime64("time3", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDateTime64("time3_not_exist", false).(time.Time) + require.Equal(t, exp, act) + + actual := metric.GetDateTime64("time3_not_exist", true) + require.Nil(t, actual, "err should be nothing") } -func TestGjsonFloatNullableTrue(t *testing.T) { + +func TestGjsonElasticDateTime(t *testing.T) { pp := NewParserPool("gjson", nil, "", DefaultTSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - result := metric.GetFloat("percent_not_exist", true) - require.Nil(t, result, "err should be nothing") + var exp, act int64 + // {"date": "2019-12-16T12:10:30Z"} + // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC + exp = 1576498230 + act = metric.GetElasticDateTime("time2", false).(int64) + require.Equal(t, exp, act) + + exp = -62135596800 + act = metric.GetElasticDateTime("time2_not_exist", false).(int64) + require.Equal(t, exp, act) + + actual := metric.GetElasticDateTime("time2_not_exist", true) + require.Nil(t, actual, "err should be nothing") } func TestGjsonArray(t *testing.T) { @@ -128,37 +172,3 @@ func TestGjsonArray(t *testing.T) { expSE := []string{} require.Equal(t, expSE, actSE) } - -func TestGjsonElasticDateTime(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - // {"date": "2019-12-16T12:10:30Z"} - // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC - var expected int64 = 1576498230 - result := metric.GetElasticDateTime("date", false).(int64) - require.Equal(t, result, expected) -} - -func TestGjsonElasticDateTimeNullableFalse(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var expected int64 = -62135596800 - result := metric.GetElasticDateTime("date_not_exist", false).(int64) - require.Equal(t, result, expected) -} - -func TestGjsonElasticDateTimeNullableTrue(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - result := metric.GetElasticDateTime("date_not_exist", true) - require.Nil(t, result, "err should be nothing") -} diff --git a/parser/parser.go b/parser/parser.go index e5df044c..177aacc4 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -64,9 +64,6 @@ func (pp *Pool) Get() Parser { return &FastjsonParser{tsLayout: pp.tsLayout} case "csv": return &CsvParser{pp.csvFormat, pp.delimiter, pp.tsLayout} - //extend gjson that could extract the map - case "gjson_extend": - return &GjsonExtendParser{pp.tsLayout} default: return &FastjsonParser{tsLayout: pp.tsLayout} } diff --git a/parser/parser_test.go b/parser/parser_test.go index 5dc25127..f5bea01c 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -35,7 +35,9 @@ var jsonSample = []byte(`{ "success":0, "percent":0.11, "mp": {"i" : [1,2,3], "f": [1.1,2.2,3.3], "s": ["aa","bb","cc"], "e": []}, - "date": "2019-12-16T12:10:30Z", + "time1": "2019-12-16", + "time2": "2019-12-16T12:10:30Z", + "time3": "2019-12-16T12:10:30.123Z", "array_int": [1,2,3], "array_float": [1.1,2.2,3.3], "array_string": ["aa","bb","cc"], From b7e3d47a0572951e103bcb6e63d7a6eaa5a12f8c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 5 Jan 2021 15:03:25 +0800 Subject: [PATCH 039/404] removed some dead code --- column/column.go | 23 ---------- column/impls/number.go | 100 ----------------------------------------- column/impls/string.go | 44 ------------------ column/register.go | 90 ------------------------------------- model/logkv.go | 34 -------------- 5 files changed, 291 deletions(-) delete mode 100644 column/column.go delete mode 100644 column/impls/number.go delete mode 100644 column/impls/string.go delete mode 100644 column/register.go delete mode 100644 model/logkv.go diff --git a/column/column.go b/column/column.go deleted file mode 100644 index f0513c32..00000000 --- a/column/column.go +++ /dev/null @@ -1,23 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package column - -// IColumn is the Column interface for clickhouse -type IColumn interface { - Name() string - DefaultValue() interface{} - GetValue(val interface{}) interface{} -} diff --git a/column/impls/number.go b/column/impls/number.go deleted file mode 100644 index 0341269f..00000000 --- a/column/impls/number.go +++ /dev/null @@ -1,100 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ -package impls - -import ( - "fmt" -) - -// IntColumn -type IntColumn struct { - name string -} - -// NewIntColumn get an instance of Int column -func NewIntColumn(bits int, isUint bool) *IntColumn { - name := fmt.Sprintf("Int%d", bits) - if isUint { - name = "U" + name - } - return &IntColumn{name: name} -} - -// Name this column name -func (c *IntColumn) Name() string { - return c.name -} - -// DefaultValue for int column 0 -func (c *IntColumn) DefaultValue() interface{} { - return int64(0) -} - -// only judge int and float64 -func (c *IntColumn) GetValue(val interface{}) interface{} { - switch v := val.(type) { - case int: - return int64(v) - case float64: - return int64(v) - default: - return int64(0) - } -} - -// FloatColumn -type FloatColumn struct { - name string - bits int -} - -// NewFloatColumn new instance of Float column -func NewFloatColumn(bits int) *FloatColumn { - name := fmt.Sprintf("Float%d", bits) - return &FloatColumn{name: name, bits: bits} -} - -// Name return the column name -func (c *FloatColumn) Name() string { - return c.name -} - -// DefaultValue of float column 0 -func (c *FloatColumn) DefaultValue() interface{} { - if c.bits == 32 { - return float32(0) - } - return float64(0) -} - -// only judge int and float64 -func (c *FloatColumn) GetValue(val interface{}) interface{} { - switch v := val.(type) { - case int: - if c.bits == 32 { - return float32(v) - } - return float64(v) - case float64: - if c.bits == 32 { - return float32(v) - } - return v - } - if c.bits == 32 { - return float32(0) - } - return float64(0) -} diff --git a/column/impls/string.go b/column/impls/string.go deleted file mode 100644 index 0b057370..00000000 --- a/column/impls/string.go +++ /dev/null @@ -1,44 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ -package impls - -// StringColumn type -type StringColumn struct { -} - -// Name of this column -func (c *StringColumn) Name() string { - return "String" -} - -// DefaultValue of string column is empty string -func (c *StringColumn) DefaultValue() interface{} { - return "" -} - -// NewStringColumn returns instance of string column -func NewStringColumn() *StringColumn { - return &StringColumn{} -} - -// only judge string column -func (c *StringColumn) GetValue(val interface{}) interface{} { - switch v := val.(type) { - case string: - return v - default: - return "" - } -} diff --git a/column/register.go b/column/register.go deleted file mode 100644 index 925c42a0..00000000 --- a/column/register.go +++ /dev/null @@ -1,90 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ -package column - -import ( - "github.com/housepower/clickhouse_sinker/column/impls" -) - -var ( - columns = map[string]IColumn{} -) - -type creator func() IColumn - -func regist(name string, creator creator) { - columns[name] = creator() -} - -// GetColumnByName get the IColumn by the name of type -func GetColumnByName(name string) IColumn { - return columns[name] -} - -// init register column types for different data types -func init() { - regist("UInt8", func() IColumn { - return impls.NewIntColumn(8, false) - }) - regist("UInt16", func() IColumn { - return impls.NewIntColumn(16, false) - }) - regist("UInt32", func() IColumn { - return impls.NewIntColumn(32, false) - }) - regist("UInt64", func() IColumn { - return impls.NewIntColumn(64, false) - }) - - regist("Int8", func() IColumn { - return impls.NewIntColumn(8, false) - }) - regist("Int16", func() IColumn { - return impls.NewIntColumn(16, false) - }) - regist("Int32", func() IColumn { - return impls.NewIntColumn(32, false) - }) - regist("Int64", func() IColumn { - return impls.NewIntColumn(64, false) - }) - - regist("Date", func() IColumn { - return impls.NewIntColumn(16, true) - }) - - regist("DateTime", func() IColumn { - return impls.NewIntColumn(32, true) - }) - - regist("DateTime64", func() IColumn { - return impls.NewIntColumn(64, true) - }) - - regist("Float32", func() IColumn { - return impls.NewFloatColumn(32) - }) - regist("Float64", func() IColumn { - return impls.NewFloatColumn(64) - }) - - regist("String", func() IColumn { - return impls.NewStringColumn() - }) - - regist("FixedString", func() IColumn { - return impls.NewStringColumn() - }) -} diff --git a/model/logkv.go b/model/logkv.go deleted file mode 100644 index ce52fec8..00000000 --- a/model/logkv.go +++ /dev/null @@ -1,34 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package model - -import ( - "github.com/housepower/clickhouse_sinker/column" -) - -// LogKV -type LogKV map[string]interface{} - -// GetValueByType returns the value of the field based on column type -// uses the GetValue function defined in each column definitions -func (logkv LogKV) GetValueByType(key string, typ string) interface{} { - val := logkv[key] - col := column.GetColumnByName(typ) - if val == nil { - return col.DefaultValue() - } - return col.GetValue(val) -} From 090a1b02ab7019177c4c47247cac6e23ca65803c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 6 Jan 2021 15:08:19 +0800 Subject: [PATCH 040/404] array support in CSV parser --- parser/csv.go | 55 +++++++++-- parser/csv_test.go | 207 +++++++++++++++++++++++++++++++++++------- parser/parser_test.go | 3 + 3 files changed, 224 insertions(+), 41 deletions(-) diff --git a/parser/csv.go b/parser/csv.go index aebbc8ba..815b758c 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -18,6 +18,7 @@ import ( "bytes" "encoding/csv" "strconv" + "strings" "time" "github.com/housepower/clickhouse_sinker/model" @@ -26,7 +27,7 @@ import ( var _ Parser = (*CsvParser)(nil) -// CsvParser implementation to parse input from a CSV format +// CsvParser implementation to parse input from a CSV format per RFC 4180 type CsvParser struct { title []string delimiter string @@ -36,7 +37,7 @@ type CsvParser struct { // Parse extract a list of comma-separated values from the data func (p *CsvParser) Parse(bs []byte) (metric model.Metric, err error) { r := csv.NewReader(bytes.NewReader(bs)) - r.Comma = ',' + r.FieldsPerRecord = len(p.title) if len(p.delimiter) > 0 { r.Comma = rune(p.delimiter[0]) } @@ -87,7 +88,7 @@ func (c *CsvMetric) GetFloat(key string, nullable bool) interface{} { return n } } - return 0 + return float64(0) } // GetInt returns int @@ -99,12 +100,54 @@ func (c *CsvMetric) GetInt(key string, nullable bool) interface{} { return n } } - return 0 + return int64(0) } -// GetArray is Empty implemented for CsvMetric +// GetArray parse an CSV encoded array func (c *CsvMetric) GetArray(key string, t string) interface{} { - return []interface{}{} + var err error + var array []string + var r *csv.Reader + val := c.GetString(key, false).(string) + valLen := len(val) + if val == "" || val[0] != '[' || val[valLen-1] != ']' { + goto QUIT + } + r = csv.NewReader(strings.NewReader(val[1 : valLen-1])) + if array, err = r.Read(); err != nil { + goto QUIT + } + switch t { + case "int": + results := make([]int64, 0, len(array)) + for _, e := range array { + v, _ := strconv.ParseInt(e, 10, 64) + results = append(results, v) + } + return results + case "float": + results := make([]float64, 0, len(array)) + for _, e := range array { + v, _ := strconv.ParseFloat(e, 64) + results = append(results, v) + } + return results + case "string": + return array + default: + panic("not supported array type " + t) + } +QUIT: + switch t { + case "int": + return []int64{} + case "float": + return []float64{} + case "string": + return []string{} + default: + return nil + } } func (c *CsvMetric) GetDate(key string, nullable bool) interface{} { diff --git a/parser/csv_test.go b/parser/csv_test.go index 0a4eed00..ce25393b 100644 --- a/parser/csv_test.go +++ b/parser/csv_test.go @@ -21,39 +21,176 @@ import ( "github.com/stretchr/testify/require" ) -func TestParseCsv(t *testing.T) { - testCases := []struct { - msg string - values []string - }{ - { - `1,"DO,NOT,SPLIT",42`, - []string{"1", "DO,NOT,SPLIT", "42"}, - }, - - { - `2,Daniel,26`, - []string{"2", "Daniel", "26"}, - }, - - { - `2,Daniel,`, - []string{"2", "Daniel", ""}, - }, - - { - `2,,Daniel`, - []string{"2", "", "Daniel"}, - }, - } - - pp := NewParserPool("csv", nil, ",", []string{"2006-01-02", time.RFC3339, time.RFC3339}) - csvParser := pp.Get() - defer pp.Put(csvParser) - for _, c := range testCases { - metric, _ := csvParser.Parse([]byte(c.msg)) - csvMetric, ok := metric.(*CsvMetric) - require.Equal(t, ok, true) - require.Equal(t, c.values, csvMetric.values) - } +func TestCsvInt(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + var exp, act int64 + exp = 1536813227 + act = metric.GetInt("its", false).(int64) + require.Equal(t, exp, act) + + exp = 0 + act = metric.GetInt("its_not_exist", false).(int64) + require.Equal(t, exp, act) + + act = metric.GetInt("its_not_exist", true).(int64) + require.Equal(t, exp, act) +} + +func TestCsvFloat(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + var exp, act float64 + exp = 0.11 + act = metric.GetFloat("percent", false).(float64) + require.Equal(t, exp, act) + + exp = 0.0 + act = metric.GetFloat("percent_not_exist", false).(float64) + require.Equal(t, exp, act) + + act = metric.GetFloat("percent_not_exist", true).(float64) + require.Equal(t, exp, act) +} + +func TestCsvString(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + var exp, act string + exp = `escaped_"ws` + act = metric.GetString("channel", false).(string) + require.Equal(t, exp, act) + + exp = "" + act = metric.GetString("channel_not_exist", false).(string) + require.Equal(t, exp, act) + + act = metric.GetString("channel_not_exist", true).(string) + require.Equal(t, exp, act) +} + +func TestCsvDate(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.UTC) + act = metric.GetDate("time1", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDate("time1_not_exist", false).(time.Time) + require.Equal(t, exp, act) + + act = metric.GetDate("time1_not_exist", true).(time.Time) + require.Equal(t, exp, act) +} + +func TestCsvDateTime(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) + act = metric.GetDateTime("time2", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDateTime("time2_not_exist", false).(time.Time) + require.Equal(t, exp, act) + + act = metric.GetDateTime("time2_not_exist", true).(time.Time) + require.Equal(t, exp, act) +} + +func TestCsvDateTime64(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) + act = metric.GetDateTime64("time3", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDateTime64("time3_not_exist", false).(time.Time) + require.Equal(t, exp, act) + + act = metric.GetDateTime64("time3_not_exist", true).(time.Time) + require.Equal(t, exp, act) +} + +func TestCsvElasticDateTime(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + var exp, act int64 + // {"date": "2019-12-16T12:10:30Z"} + // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC + exp = 1576498230 + act = metric.GetElasticDateTime("time2", false).(int64) + require.Equal(t, exp, act) + + exp = -62135596800 + act = metric.GetElasticDateTime("time2_not_exist", false).(int64) + require.Equal(t, exp, act) + + act = metric.GetElasticDateTime("time2_not_exist", true).(int64) + require.Equal(t, exp, act) +} + +func TestCsvArray(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + actI := metric.GetArray("array_int", "int").([]int64) + expI := []int64{1, 2, 3} + require.Equal(t, expI, actI) + + actF := metric.GetArray("array_float", "float").([]float64) + expF := []float64{1.1, 2.2, 3.3} + require.Equal(t, expF, actF) + + actS := metric.GetArray("array_string", "string").([]string) + expS := []string{"aa", "bb", "cc"} + require.Equal(t, expS, actS) + + actIE := metric.GetArray("array_empty", "int").([]int64) + expIE := []int64{} + require.Equal(t, expIE, actIE) + + actFE := metric.GetArray("array_empty", "float").([]float64) + expFE := []float64{} + require.Equal(t, expFE, actFE) + + actSE := metric.GetArray("array_empty", "string").([]string) + expSE := []string{} + require.Equal(t, expSE, actSE) } diff --git a/parser/parser_test.go b/parser/parser_test.go index f5bea01c..e77248e5 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -44,6 +44,9 @@ var jsonSample = []byte(`{ "array_empty": [] }`) +var csvSampleSchema = []string{"its", "percent", "channel", "time1", "time2", "time3", "array_int", "array_float", "array_string", "array_empty"} +var csvSample = []byte(`1536813227,"0.11","escaped_""ws",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30.123Z,"[1,2,3]","[1.1,2.2,3.3]","[aa,bb,cc]","[]"`) + func BenchmarkUnmarshalljson(b *testing.B) { mp := map[string]interface{}{} for i := 0; i < b.N; i++ { From c9abc2c5d8a4b749aa8fcee147cb9880c417954c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 11 Jan 2021 14:05:53 +0800 Subject: [PATCH 041/404] single task --- cmd/clickhouse_sinker/main.go | 244 ++++++++--------------------- cmd/nacos_publish_config/main.go | 117 +------------- config/config.go | 259 +++++++------------------------ config/consul.go | 75 --------- config/nacos.go | 82 ++-------- docs/configuration/flag.md | 4 +- docs/dev/introduction.md | 6 +- go.mod | 12 -- go.sum | 105 ------------- go.test.sh | 2 +- input/kafka_go.go | 28 ++-- input/kafka_sarama.go | 28 ++-- output/clickhouse.go | 54 ++++--- pool/conn.go | 90 +++-------- task/ring.go | 35 +++-- task/sharding.go | 23 +-- task/task.go | 56 ++++--- util/conf.go | 73 --------- util/consistenthash.go | 77 --------- 19 files changed, 276 insertions(+), 1094 deletions(-) delete mode 100644 config/consul.go delete mode 100644 util/conf.go delete mode 100644 util/consistenthash.go diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 6bba1dda..af110e64 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -46,21 +46,17 @@ import ( ) type CmdOptions struct { - ShowVer bool - HTTPPort int - PushGatewayAddrs string - PushInterval int - LocalCfgDir string - LocalCfgFile string - ConsulRegister bool - ConsulAddr string - ConsulDeregisterCriticalServiceAfter string - NacosRegister bool - NacosAddr string - NacosNamespaceID string - NacosGroup string - NacosUsername string - NacosPassword string + ShowVer bool + HTTPPort int + PushGatewayAddrs string + PushInterval int + LocalCfgFile string + NacosAddr string + NacosNamespaceID string + NacosGroup string + NacosUsername string + NacosPassword string + NacosDataID string } var ( @@ -74,21 +70,17 @@ var ( func initCmdOptions() { // 1. Set options to default value. cmdOps = CmdOptions{ - ShowVer: false, - HTTPPort: 2112, - PushGatewayAddrs: "", - PushInterval: 10, - LocalCfgDir: "/etc/clickhouse_sinker", - LocalCfgFile: "/etc/clickhouse_sinker.json", - ConsulRegister: false, - ConsulAddr: "http://127.0.0.1:8500", - ConsulDeregisterCriticalServiceAfter: "30m", - NacosRegister: false, - NacosAddr: "127.0.0.1:8848", - NacosNamespaceID: "", - NacosGroup: "DEFAULT_GROUP", - NacosUsername: "nacos", - NacosPassword: "nacos", + ShowVer: false, + HTTPPort: 2112, + PushGatewayAddrs: "", + PushInterval: 10, + LocalCfgFile: "/etc/clickhouse_sinker.json", + NacosAddr: "127.0.0.1:8848", + NacosNamespaceID: "", + NacosGroup: "DEFAULT_GROUP", + NacosUsername: "nacos", + NacosPassword: "nacos", + NacosDataID: "", } // 2. Replace options with the corresponding env variable if present. @@ -96,39 +88,28 @@ func initCmdOptions() { util.EnvIntVar(&cmdOps.HTTPPort, "http-port") util.EnvStringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs") util.EnvIntVar(&cmdOps.PushInterval, "push-interval") - util.EnvStringVar(&cmdOps.LocalCfgDir, "local-cfg-dir") - util.EnvBoolVar(&cmdOps.ConsulRegister, "consul-register-enable") - util.EnvStringVar(&cmdOps.ConsulAddr, "consul-addr") - util.EnvStringVar(&cmdOps.ConsulDeregisterCriticalServiceAfter, "consul-deregister-critical-services-after") - - util.EnvBoolVar(&cmdOps.NacosRegister, "nacos-register-enable") util.EnvStringVar(&cmdOps.NacosAddr, "nacos-addr") - util.EnvStringVar(&cmdOps.NacosNamespaceID, "nacos-namespace-id") - util.EnvStringVar(&cmdOps.NacosGroup, "nacos-group") util.EnvStringVar(&cmdOps.NacosUsername, "nacos-username") util.EnvStringVar(&cmdOps.NacosPassword, "nacos-password") + util.EnvStringVar(&cmdOps.NacosNamespaceID, "nacos-namespace-id") + util.EnvStringVar(&cmdOps.NacosGroup, "nacos-group") + util.EnvStringVar(&cmdOps.NacosDataID, "nacos-dataid") // 3. Replace options with the corresponding CLI parameter if present. flag.BoolVar(&cmdOps.ShowVer, "v", cmdOps.ShowVer, "show build version and quit") flag.IntVar(&cmdOps.HTTPPort, "http-port", cmdOps.HTTPPort, "http listen port") flag.StringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs", cmdOps.PushGatewayAddrs, "a list of comma-separated prometheus push gatway address") flag.IntVar(&cmdOps.PushInterval, "push-interval", cmdOps.PushInterval, "push interval in seconds") - flag.StringVar(&cmdOps.LocalCfgDir, "local-cfg-dir", cmdOps.LocalCfgDir, "local config dir. requires a file named `config.json`, and some task json files under `tasks` folder") flag.StringVar(&cmdOps.LocalCfgFile, "local-cfg-file", cmdOps.LocalCfgFile, "local config file") - flag.BoolVar(&cmdOps.ConsulRegister, "consul-register-enable", cmdOps.ConsulRegister, "register current instance in consul") - flag.StringVar(&cmdOps.ConsulAddr, "consul-addr", cmdOps.ConsulAddr, "consul api interface address") - flag.StringVar(&cmdOps.ConsulDeregisterCriticalServiceAfter, "consul-deregister-critical-services-after", cmdOps.ConsulDeregisterCriticalServiceAfter, - "configure service check DeregisterCriticalServiceAfter") - - flag.BoolVar(&cmdOps.NacosRegister, "nacos-register-enable", cmdOps.NacosRegister, "register current instance in nacos") flag.StringVar(&cmdOps.NacosAddr, "nacos-addr", cmdOps.NacosAddr, "a list of comma-separated nacos server addresses") + flag.StringVar(&cmdOps.NacosUsername, "nacos-username", cmdOps.NacosUsername, "nacos username") + flag.StringVar(&cmdOps.NacosPassword, "nacos-password", cmdOps.NacosPassword, "nacos password") flag.StringVar(&cmdOps.NacosNamespaceID, "nacos-namespace-id", cmdOps.NacosNamespaceID, `nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work!`) flag.StringVar(&cmdOps.NacosGroup, "nacos-group", cmdOps.NacosGroup, `nacos group name. Empty string doesn't work!`) - flag.StringVar(&cmdOps.NacosUsername, "nacos-username", cmdOps.NacosUsername, "nacos username") - flag.StringVar(&cmdOps.NacosPassword, "nacos-password", cmdOps.NacosPassword, "nacos password") + flag.StringVar(&cmdOps.NacosDataID, "nacos-dataid", cmdOps.NacosDataID, "nacos dataid") flag.Parse() } @@ -144,15 +125,12 @@ func init() { } // GenTask generate a task via config -func GenTask(cfg *config.Config, taskName string) (taskImpl *task.Service) { - taskCfg := cfg.Tasks[taskName] - ck := output.NewClickHouse(cfg, taskName) +func GenTask(cfg *config.Config) (taskImpl *task.Service) { + taskCfg := &cfg.Task + ck := output.NewClickHouse(cfg) pp := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, []string{taskCfg.LayoutDate, taskCfg.LayoutDateTime, taskCfg.LayoutDateTime64}) - var inputer input.Inputer - if taskCfg.Kafka != "" { - inputer = input.NewInputer(taskCfg.KafkaClient) - } - taskImpl = task.NewTaskService(inputer, ck, pp, cfg, taskName) + inputer := input.NewInputer(taskCfg.KafkaClient) + taskImpl = task.NewTaskService(inputer, ck, pp, cfg) return } @@ -167,12 +145,7 @@ func main() { util.Run("clickhouse_sinker", func() error { var rcm config.RemoteConfManager var properties map[string]interface{} - if cmdOps.ConsulRegister { - rcm = &config.ConsulConfManager{} - properties = make(map[string]interface{}) - properties["consulAddr"] = cmdOps.ConsulAddr - properties["deregisterCriticalServiceAfter"] = cmdOps.ConsulDeregisterCriticalServiceAfter - } else if cmdOps.NacosRegister { + if cmdOps.NacosDataID != "" { rcm = &config.NacosConfManager{} properties = make(map[string]interface{}) properties["serverAddrs"] = cmdOps.NacosAddr @@ -180,14 +153,12 @@ func main() { properties["password"] = cmdOps.NacosPassword properties["namespaceId"] = cmdOps.NacosNamespaceID properties["group"] = cmdOps.NacosGroup + properties["dataId"] = cmdOps.NacosDataID } if rcm != nil { if err := rcm.Init(properties); err != nil { log.Fatalf("%+v", err) } - if err := rcm.Register(selfIP, cmdOps.HTTPPort); err != nil { - log.Fatalf("%+v", err) - } } runner = NewSinker(rcm) return runner.Init() @@ -233,7 +204,7 @@ func main() { type Sinker struct { curCfg *config.Config pusher *statistics.Pusher - tasks map[string]*task.Service + task *task.Service rcm config.RemoteConfManager ctx context.Context cancel context.CancelFunc @@ -247,12 +218,11 @@ func NewSinker(rcm config.RemoteConfManager) *Sinker { return s } -// Init initializes the list of tasks func (s *Sinker) Init() (err error) { return } -// Run rull all tasks in different go routines +// Run rull task in different go routines func (s *Sinker) Run() { var err error var newCfg *config.Config @@ -270,11 +240,6 @@ func (s *Sinker) Run() { log.Fatalf("%+v", err) return } - } else if _, err = os.Stat(cmdOps.LocalCfgDir); err == nil { - if newCfg, err = config.ParseLocalCfgDir(cmdOps.LocalCfgDir); err != nil { - log.Fatalf("%+v", err) - return - } } else { log.Fatalf("expect --local-cfg-file or --local-cfg-dir") return @@ -283,8 +248,6 @@ func (s *Sinker) Run() { log.Fatalf("%+v", err) return } - // Assign all tasks to myself. - newCfg.AssignTasks([]config.Instance{{Addr: selfAddr, Weight: 1}}) if err = s.applyConfig(newCfg); err != nil { log.Fatalf("%+v", err) return @@ -313,12 +276,10 @@ func (s *Sinker) Run() { } } -// Close shutdown tasks +// Close shutdown task func (s *Sinker) Close() { s.cancel() - for _, task := range s.tasks { - task.Stop() - } + s.task.Stop() util.GlobalParsingPool.StopWait() util.GlobalWritingPool.StopWait() @@ -327,14 +288,11 @@ func (s *Sinker) Close() { if s.pusher != nil { s.pusher.Stop() } - if s.rcm != nil { - _ = s.rcm.Deregister(selfIP, cmdOps.HTTPPort) - } } func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { var lvl log.Level - if lvl, err = log.ParseLevel(newCfg.Common.LogLevel); err != nil { + if lvl, err = log.ParseLevel(newCfg.LogLevel); err != nil { err = errors.Wrapf(err, "") return } @@ -357,18 +315,12 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { log.Infof("going to apply the first config: %+v", string(bsNewCfg)) util.InitGlobalTimerWheel() - concurrentParsers := 10 - s.tasks = make(map[string]*task.Service) - if taskNames, ok := newCfg.Assignment[selfAddr]; ok { - for _, taskName := range taskNames { - t := GenTask(newCfg, taskName) - if err = t.Init(); err != nil { - return - } - s.tasks[taskName] = t - } - concurrentParsers = len(taskNames) * 10 + t := GenTask(newCfg) + if err = t.Init(); err != nil { + return } + s.task = t + concurrentParsers := 10 if runtime.NumCPU() >= 2 { if concurrentParsers > runtime.NumCPU()/2 { concurrentParsers = runtime.NumCPU() / 2 @@ -380,9 +332,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { totalConn := pool.GetTotalConn() util.InitGlobalWritingPool(totalConn) - for _, t := range s.tasks { - go t.Run(s.ctx) - } + go s.task.Run(s.ctx) s.curCfg = newCfg return } @@ -394,89 +344,33 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { return } log.Infof("going to apply a different config: %+v", string(bsNewCfg)) - //1. Found all tasks need to stop. - // Each such task matches at least one of the following conditions: - // - task not in new assignment - // - task config differ - // - task clickhouse config differ - // - task kafka config differ - var tasksToStop []string - if taskNames, ok := s.curCfg.Assignment[selfAddr]; ok { - for _, taskName := range taskNames { - var needStop bool - curTaskCfg := s.curCfg.Tasks[taskName] - if newTaskCfg, ok2 := newCfg.Tasks[taskName]; ok2 { - if !reflect.DeepEqual(newTaskCfg, curTaskCfg) { - needStop = true - } else { - chName := curTaskCfg.Clickhouse - curChCfg := s.curCfg.Clickhouse[chName] - newChCfg := newCfg.Clickhouse[chName] - if !reflect.DeepEqual(newChCfg, curChCfg) { - needStop = true - } - kfkName := curTaskCfg.Kafka - curKfkCfg := s.curCfg.Kafka[kfkName] - newKfkCfg := newCfg.Kafka[kfkName] - if !reflect.DeepEqual(newKfkCfg, curKfkCfg) { - needStop = true - } - } - } else { - needStop = true - } - if needStop { - tasksToStop = append(tasksToStop, taskName) - } - } - } - // 2. Stop all tasks in parallel found at previous step. - for _, taskName := range tasksToStop { - if task, ok := s.tasks[taskName]; ok { - task.NotifyStop() - } - } - for _, taskName := range tasksToStop { - if task, ok := s.tasks[taskName]; ok { - task.Stop() - delete(s.tasks, taskName) - } else { - log.Warnf("Failed to stop task %s. It's disappeared.", taskName) + + if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { + // 1. Stop task + s.task.Stop() + + // 2. Generate, initialize and start task + t := GenTask(newCfg) + if err = t.Init(); err != nil { + return } - } - // 3. Initailize all tasks which are new or their config differ. - var newTasks []*task.Service - if taskNames, ok := newCfg.Assignment[selfAddr]; ok { - for _, taskName := range taskNames { - if _, ok2 := s.tasks[taskName]; !ok2 { - t := GenTask(newCfg, taskName) - if err = t.Init(); err != nil { - return - } - s.tasks[taskName] = t - newTasks = append(newTasks, t) + go t.Run(s.ctx) + s.task = t + + // 3. Resize goroutine pools. + concurrentParsers := 10 + if runtime.NumCPU() >= 2 { + if concurrentParsers > runtime.NumCPU()/2 { + concurrentParsers = runtime.NumCPU() / 2 } + } else { + concurrentParsers = 1 } + util.GlobalParsingPool.Resize(concurrentParsers) + totalConn := pool.GetTotalConn() + util.GlobalWritingPool.Resize(totalConn) } - // 4. Resize goroutine pools. - concurrentParsers := len(s.tasks) * 10 - if runtime.NumCPU() >= 2 { - if concurrentParsers > runtime.NumCPU()/2 { - concurrentParsers = runtime.NumCPU() / 2 - } - } else { - concurrentParsers = 1 - } - util.GlobalParsingPool.Resize(concurrentParsers) - totalConn := pool.GetTotalConn() - util.GlobalWritingPool.Resize(totalConn) - - // 5. Start new tasks. We don't do it at step 3 in order to avoid goroutine leak due to errors raised by later steps. - for _, t := range newTasks { - go t.Run(s.ctx) - } - - // 6. Record the new config. + // Record the new config s.curCfg = newCfg return } diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 562d62b7..f443c308 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -17,18 +17,12 @@ package main import ( "flag" - "fmt" "os" "reflect" - "runtime" - "strconv" - "strings" "time" _ "github.com/ClickHouse/clickhouse-go" "github.com/housepower/clickhouse_sinker/config" - "github.com/housepower/clickhouse_sinker/util" - "github.com/k0kubun/pp" log "github.com/sirupsen/logrus" ) @@ -43,11 +37,10 @@ var ( `nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work!`) nacosGroup = flag.String("nacos-group", "DEFAULT_GROUP", `nacos group name. Empty string doesn't work!`) + nacosDataID = flag.String("nacos-dataid", "", + `nacos data id, the task name`) - localCfgDir = flag.String("local-cfg-dir", "/etc/clickhouse_sinker", "local config dir") localCfgFile = flag.String("local-cfg-file", "/etc/clickhouse_sinker.json", "local config file") - instances = flag.String("instances", "", "a list of comma-separated ip:port[@weight]") - testRegister = flag.Bool("test-register", false, "whether run TestRegister") ) // Empty is not valid namespaceID @@ -58,6 +51,7 @@ func getProperties() map[string]interface{} { properties["password"] = *nacosPassword properties["namespaceId"] = *nacosNamespaceID properties["group"] = *nacosGroup + properties["dataId"] = *nacosDataID return properties } @@ -69,13 +63,8 @@ func PublishSinkerConfig() { log.Fatalf("%+v", err) return } - } else if _, err = os.Stat(*localCfgDir); err == nil { - if cfg, err = config.ParseLocalCfgDir(*localCfgDir); err != nil { - log.Fatalf("%+v", err) - return - } } else { - log.Fatalf("expect --local-cfg-file or --local-cfg-dir") + log.Fatalf("expect --local-cfg-file") return } @@ -84,26 +73,6 @@ func PublishSinkerConfig() { return } - var insts []config.Instance - if *instances == "" { - sinkerAddr := fmt.Sprintf("%s:%d", util.GetOutboundIP().String(), 2112) - insts = []config.Instance{{Addr: sinkerAddr, Weight: 1}} - } else { - for _, instInfo := range strings.Split(*instances, ",") { - var inst config.Instance - fields := strings.Split(instInfo, "@") - inst.Addr = fields[0] - if len(fields) >= 2 { - if inst.Weight, err = strconv.Atoi(fields[1]); err != nil { - log.Fatalf("%+v", err) - } - } - insts = append(insts, inst) - } - } - cfg.AssignTasks(insts) - _, _ = pp.Println("going to publish following config: ", cfg) - ncm := config.NacosConfManager{} properties := getProperties() if err = ncm.Init(properties); err != nil { @@ -125,88 +94,10 @@ func PublishSinkerConfig() { } } -func TestRegister() { - var err error - ncm := config.NacosConfManager{} - properties := getProperties() - if err = ncm.Init(properties); err != nil { - log.Fatalf("%+v", err) - } - - var insts []config.Instance - log.Infof("nacos try to deregister all existing instances") - if insts, err = ncm.GetInstances(); err == nil { - for _, inst := range insts { - fields := strings.Split(inst.Addr, ":") - ip := fields[0] - var port int - if port, err = strconv.Atoi(fields[1]); err != nil { - log.Fatalf("failed to parse address %+v", inst.Addr) - } - log.Infof("nacos try to deregister %s:%d", ip, port) - if err = ncm.Deregister(ip, port); err != nil { - log.Warnf("ncm.Deregister(%s, %d) failed. %+v", ip, port, err) - } - } - } else { - log.Warnf("ncm.GetInstances failed. %+v", err) - } - - ip := "127.0.0.1" - port := 22 - //naming_client.NamingClient.SelectInstances() throws errors if "do not have useful host, ignore it", "instance list is empty!" - //So there shall be at leas one alive instance during the test. - log.Infof("nacos register %s:%d", ip, port) - if err = ncm.Register(ip, port); err != nil { - log.Fatalf("%+v", err) - } - log.Infof("nacos register %s:%d", ip, port+1) - if err = ncm.Register(ip, port+1); err != nil { - log.Fatalf("%+v", err) - } - - expInsts := []config.Instance{ - {Addr: fmt.Sprintf("%s:%d", ip, port), Weight: runtime.NumCPU()}, - {Addr: fmt.Sprintf("%s:%d", ip, port+1), Weight: runtime.NumCPU()}, - } - //naming_client.HostReactor.asyncUpdateService() updates cache every 10s. - //So we need sleep a while to ensure at leas one update occurred. - time.Sleep(10 * time.Second) - if insts, err = ncm.GetInstances(); err != nil { - log.Fatalf("%+v", err) - } - if !reflect.DeepEqual(expInsts, insts) { - log.Fatalf("got different instances: %+v", insts) - } - - log.Infof("nacos deregister %s:%d", ip, port) - if err = ncm.Deregister(ip, port); err != nil { - log.Fatalf("%+v", err) - } - expInsts = []config.Instance{ - {Addr: fmt.Sprintf("%s:%d", ip, port+1), Weight: runtime.NumCPU()}, - } - time.Sleep(10 * time.Second) - if insts, err = ncm.GetInstances(); err != nil { - log.Fatalf("%+v", err) - } - if !reflect.DeepEqual(expInsts, insts) { - log.Fatalf("got different instances: %+v", insts) - } - - if err = ncm.Deregister(ip, port); err != nil { - log.Fatalf("%+v", err) - } - //naming_client.NamingClient.SelectInstances() throws errors "instance list is empty!" -} - func main() { log.SetFormatter(&log.TextFormatter{ FullTimestamp: true, }) flag.Parse() - if *testRegister { - TestRegister() - } PublishSinkerConfig() } diff --git a/config/config.go b/config/config.go index dbe08a34..fedfa26e 100644 --- a/config/config.go +++ b/config/config.go @@ -18,9 +18,6 @@ package config import ( "encoding/json" "io/ioutil" - "os" - "path/filepath" - "sort" "strings" "time" @@ -32,41 +29,18 @@ import ( // RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... type RemoteConfManager interface { Init(properties map[string]interface{}) error - // Register this instance, and keep-alive via heartbeat. - Register(ip string, port int) error - Deregister(ip string, port int) error - // GetInstances fetchs healthy instances. - // Mature service-discovery solutions(Nacos, Consul etc.) have client side cache - // so that frequent invoking of GetInstances() and GetGlobalConfig() don't harm. - GetInstances() (instances []Instance, err error) // GetConfig fetchs the config. The manager shall not reference the returned Config object after call. GetConfig() (conf *Config, err error) - // PublishConfig publishs the config. The manager shall not reference the passed Config object after call. + // PublishConfig publishs the config. PublishConfig(conf *Config) (err error) } -type Instance struct { - Addr string - Weight int -} - // Config struct used for different configurations use type Config struct { - Kafka map[string]*KafkaConfig - Clickhouse map[string]*ClickHouseConfig - Tasks map[string]*TaskConfig - Common struct { - FlushInterval int - BufferSize int - MinBufferSize int - MsgSizeHint int - LayoutDate string - LayoutDateTime string - LayoutDateTime64 string - LogLevel string - Replicas int //on how many sinker instances a task runs - } - Assignment map[string][]string //map instance_name to a list of task_name + Kafka KafkaConfig + Clickhouse ClickHouseConfig + Task TaskConfig + LogLevel string } // KafkaConfig configuration parameters @@ -123,7 +97,6 @@ type TaskConfig struct { Name string KafkaClient string - Kafka string Topic string ConsumerGroup string @@ -134,8 +107,7 @@ type TaskConfig struct { CsvFormat []string Delimiter string - Clickhouse string - TableName string + TableName string // AutoSchema will auto fetch the schema from clickhouse AutoSchema bool @@ -158,7 +130,6 @@ type TaskConfig struct { LayoutDate string `json:"layoutDate,omitempty"` LayoutDateTime string `json:"layoutDateTime,omitempty"` LayoutDateTime64 string `json:"layoutDateTime64,omitempty"` - Replicas int //on how many sinker instances this task runs } const ( @@ -169,46 +140,9 @@ const ( defaultLayoutDate = "2006-01-02" defaultLayoutDateTime = time.RFC3339 defaultLayoutDateTime64 = time.RFC3339 - defaultTaskReplicas = 1 + defaultLogLevel = "info" ) -func ParseLocalCfgDir(cfgPath string) (cfg *Config, err error) { - var f = "config.json" - f = filepath.Join(cfgPath, f) - var s string - if s, err = util.ExtendFile(f); err != nil { - err = errors.Wrapf(err, "") - return - } - cfg = &Config{} - if err = json.Unmarshal([]byte(s), cfg); err != nil { - err = errors.Wrapf(err, "") - return - } - - var files []os.FileInfo - if files, err = ioutil.ReadDir(filepath.Join(cfgPath, "tasks")); err != nil { - err = errors.Wrapf(err, "") - return - } - cfg.Tasks = make(map[string]*TaskConfig) - for _, f := range files { - if strings.HasSuffix(f.Name(), ".json") { - if s, err = util.ExtendFile(filepath.Join(cfgPath, "tasks", f.Name())); err != nil { - err = errors.Wrapf(err, "") - return - } - taskConfig := &TaskConfig{} - if err = json.Unmarshal([]byte(s), taskConfig); err != nil { - err = errors.Wrapf(err, "") - return - } - cfg.Tasks[taskConfig.Name] = taskConfig - } - } - return -} - func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { cfg = &Config{} var b []byte @@ -224,154 +158,69 @@ func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { return } -func (cfg *Config) AssignTasks(instances []Instance) { - cfg.Assignment = make(map[string][]string) - assignment := make(map[string]map[string]int) - hr := util.NewHashRing(nil) - for _, inst := range instances { - hr.Add(inst.Addr, inst.Weight) - cfg.Assignment[inst.Addr] = make([]string, 0) - assignment[inst.Addr] = make(map[string]int) - } - for _, taskConfig := range cfg.Tasks { - for i := 0; i < taskConfig.Replicas; i++ { - instAddr := hr.Get(taskConfig.Name) - assignment[instAddr][taskConfig.Name] = 1 +// normallize and validate configuration +func (cfg *Config) Normallize() (err error) { + if cfg.Kafka.Version == "" { + cfg.Kafka.Version = "2.2.1" + } + if cfg.Kafka.Sasl.Enable { + cfg.Kafka.Sasl.Mechanism = strings.ToUpper(cfg.Kafka.Sasl.Mechanism) + switch cfg.Kafka.Sasl.Mechanism { + case "PLAIN", "SCRAM-SHA-256", "SCRAM-SHA-512", "GSSAPI": + default: + err = errors.Errorf("kafka SASL mechanism %s is unsupported", cfg.Kafka.Sasl.Mechanism) + return } } - for _, inst := range instances { - for taskName := range assignment[inst.Addr] { - cfg.Assignment[inst.Addr] = append(cfg.Assignment[inst.Addr], taskName) + if cfg.Clickhouse.RetryTimes < 0 { + cfg.Clickhouse.RetryTimes = 0 + } + + if cfg.Kafka.Sasl.Enable && cfg.Kafka.Sasl.Username == "" { + //kafka-go doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 + cfg.Task.KafkaClient = "sarama" + } else if cfg.Task.KafkaClient == "" { + cfg.Task.KafkaClient = "kafka-go" + } + if cfg.Task.Parser == "" { + cfg.Task.Parser = "fastjson" + } + + for i := range cfg.Task.Dims { + if cfg.Task.Dims[i].SourceName == "" { + cfg.Task.Dims[i].SourceName = util.GetSourceName(cfg.Task.Dims[i].Name) } - sort.Strings(cfg.Assignment[inst.Addr]) } -} -// normallize and validate configuration -func (cfg *Config) Normallize() (err error) { - if cfg.Common.FlushInterval <= 0 { - cfg.Common.FlushInterval = defaultFlushInterval + if cfg.Task.FlushInterval <= 0 { + cfg.Task.FlushInterval = defaultFlushInterval } - if cfg.Common.BufferSize <= 0 { - cfg.Common.BufferSize = defaultBufferSize + if cfg.Task.BufferSize <= 0 { + cfg.Task.BufferSize = defaultBufferSize } else { - cfg.Common.BufferSize = 1 << util.GetShift(cfg.Common.BufferSize) + cfg.Task.BufferSize = 1 << util.GetShift(cfg.Task.BufferSize) } - if cfg.Common.MinBufferSize <= 0 { - cfg.Common.MinBufferSize = defaultMinBufferSize + if cfg.Task.MinBufferSize <= 0 { + cfg.Task.MinBufferSize = defaultMinBufferSize } else { - cfg.Common.MinBufferSize = 1 << util.GetShift(cfg.Common.MinBufferSize) + cfg.Task.MinBufferSize = 1 << util.GetShift(cfg.Task.MinBufferSize) } - if cfg.Common.MsgSizeHint <= 0 { - cfg.Common.MsgSizeHint = defaultMsgSizeHint + if cfg.Task.MsgSizeHint <= 0 { + cfg.Task.MsgSizeHint = defaultMsgSizeHint } - if cfg.Common.LayoutDate == "" { - cfg.Common.LayoutDate = defaultLayoutDate + if cfg.Task.LayoutDate == "" { + cfg.Task.LayoutDate = defaultLayoutDate } - if cfg.Common.LayoutDateTime == "" { - cfg.Common.LayoutDateTime = defaultLayoutDateTime + if cfg.Task.LayoutDateTime == "" { + cfg.Task.LayoutDateTime = defaultLayoutDateTime } - if cfg.Common.LayoutDateTime64 == "" { - cfg.Common.LayoutDateTime64 = defaultLayoutDateTime64 + if cfg.Task.LayoutDateTime64 == "" { + cfg.Task.LayoutDateTime64 = defaultLayoutDateTime64 } - switch strings.ToLower(cfg.Common.LogLevel) { + switch strings.ToLower(cfg.LogLevel) { case "panic", "fatal", "error", "warn", "warning", "info", "debug", "trace": default: - cfg.Common.LogLevel = "info" - } - if cfg.Common.Replicas <= 0 { - cfg.Common.Replicas = defaultTaskReplicas - } - if err = cfg.normallizeTasks(); err != nil { - return - } - for kfkName, kfkConfig := range cfg.Kafka { - if kfkConfig.Version == "" { - kfkConfig.Version = "2.2.1" - } - if kfkConfig.Sasl.Enable { - kfkConfig.Sasl.Mechanism = strings.ToUpper(kfkConfig.Sasl.Mechanism) - switch kfkConfig.Sasl.Mechanism { - case "PLAIN", "SCRAM-SHA-256", "SCRAM-SHA-512", "GSSAPI": - default: - err = errors.Errorf("kafka %s mechanism %s is unsupported", kfkName, kfkConfig.Sasl.Mechanism) - return - } - } - } - for _, chConfig := range cfg.Clickhouse { - if chConfig.RetryTimes < 0 { - chConfig.RetryTimes = 0 - } - } - for instAddr, taskNames := range cfg.Assignment { - sort.Strings(taskNames) - for _, taskName := range taskNames { - if _, ok := cfg.Tasks[taskName]; !ok { - err = errors.Errorf("instance %s assignment is invalid, task %s doesn't exit", instAddr, taskName) - return - } - } - } - return -} - -func (cfg *Config) normallizeTasks() (err error) { - for taskName, taskConfig := range cfg.Tasks { - if _, ok := cfg.Kafka[taskConfig.Kafka]; !ok { - err = errors.Errorf("task %s config is invalid, kafka %s doesn't exist.", taskConfig.Name, taskConfig.Kafka) - return - } - if _, ok := cfg.Clickhouse[taskConfig.Clickhouse]; !ok { - err = errors.Errorf("task %s config is invalid, clickhouse %s doesn't exist.", taskConfig.Name, taskConfig.Clickhouse) - return - } - if taskConfig.Name != taskName { - taskConfig.Name = taskName - } - kfkCfg := cfg.Kafka[taskConfig.Kafka] - if kfkCfg.Sasl.Enable && kfkCfg.Sasl.Username == "" { - //kafka-go doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 - taskConfig.KafkaClient = "sarama" - } else if taskConfig.KafkaClient == "" { - taskConfig.KafkaClient = "kafka-go" - } - if taskConfig.Parser == "" { - taskConfig.Parser = "fastjson" - } - if taskConfig.FlushInterval <= 0 { - taskConfig.FlushInterval = cfg.Common.FlushInterval - } - if taskConfig.BufferSize <= 0 { - taskConfig.BufferSize = cfg.Common.BufferSize - } else { - taskConfig.BufferSize = 1 << util.GetShift(taskConfig.BufferSize) - } - if taskConfig.MinBufferSize <= 0 { - taskConfig.MinBufferSize = cfg.Common.MinBufferSize - } else { - taskConfig.MinBufferSize = 1 << util.GetShift(taskConfig.BufferSize) - } - if taskConfig.MsgSizeHint <= 0 { - taskConfig.MsgSizeHint = cfg.Common.MsgSizeHint - } - if taskConfig.LayoutDate == "" { - taskConfig.LayoutDate = cfg.Common.LayoutDate - } - if taskConfig.LayoutDateTime == "" { - taskConfig.LayoutDateTime = cfg.Common.LayoutDateTime - } - if taskConfig.LayoutDateTime64 == "" { - taskConfig.LayoutDateTime64 = cfg.Common.LayoutDateTime64 - } - if taskConfig.Replicas <= 0 { - taskConfig.Replicas = cfg.Common.Replicas - } - for i := range taskConfig.Dims { - if taskConfig.Dims[i].SourceName == "" { - taskConfig.Dims[i].SourceName = util.GetSourceName(taskConfig.Dims[i].Name) - } - } + cfg.LogLevel = defaultLogLevel } return } diff --git a/config/consul.go b/config/consul.go deleted file mode 100644 index 6961941b..00000000 --- a/config/consul.go +++ /dev/null @@ -1,75 +0,0 @@ -package config - -import ( - "fmt" - "runtime" - - "github.com/hashicorp/consul/api" - "github.com/pkg/errors" - log "github.com/sirupsen/logrus" -) - -var _ RemoteConfManager = (*ConsulConfManager)(nil) - -type ConsulConfManager struct { - consulAgent *api.Agent - deregisterCriticalServiceAfter string -} - -func (ccm *ConsulConfManager) Init(properties map[string]interface{}) (err error) { - consulConfig := api.DefaultConfig() - consulConfig.Address = properties["consulAddr"].(string) - consulClient, _ := api.NewClient(consulConfig) - ccm.consulAgent = consulClient.Agent() - ccm.deregisterCriticalServiceAfter = properties["deregisterCriticalServiceAfter"].(string) - return -} - -func (ccm *ConsulConfManager) Register(ip string, port int) (err error) { - log.Infof("Consul: register service") - appID := fmt.Sprintf("clickhouse_sinker-%s-%d", ip, port) - err = ccm.consulAgent.ServiceRegister(&api.AgentServiceRegistration{ - Name: "clickhouse_sinker", - ID: appID, - Port: port, - Address: ip, - Weights: &api.AgentWeights{Passing: runtime.NumCPU(), Warning: 0}, - Check: &api.AgentServiceCheck{ - CheckID: appID + "-http-heath", - Name: "/ready", - Interval: "15s", - Timeout: "15s", - HTTP: fmt.Sprintf("http://%s:%d/ready?full=1", ip, port), - DeregisterCriticalServiceAfter: ccm.deregisterCriticalServiceAfter, - }, - }) - if err != nil { - err = errors.Wrapf(err, "") - } - return -} - -func (ccm *ConsulConfManager) Deregister(ip string, port int) (err error) { - log.Debug("Consul: deregister service") - appID := fmt.Sprintf("clickhouse_sinker-%s-%d", ip, port) - err = ccm.consulAgent.ServiceDeregister(appID) - if err != nil { - err = errors.Wrapf(err, "") - } - return -} - -func (ccm *ConsulConfManager) GetInstances() (instances []Instance, err error) { - //FIXME: implement it! - return -} - -func (ccm *ConsulConfManager) GetConfig() (conf *Config, err error) { - //FIXME: implement it! - return -} - -func (ccm *ConsulConfManager) PublishConfig(conf *Config) (err error) { - //FIXME: implement it! - return -} diff --git a/config/nacos.go b/config/nacos.go index f0af00d0..0d606fce 100644 --- a/config/nacos.go +++ b/config/nacos.go @@ -2,33 +2,23 @@ package config import ( "encoding/json" - "fmt" "path/filepath" - "runtime" - "sort" "strconv" "strings" "github.com/nacos-group/nacos-sdk-go/clients" "github.com/nacos-group/nacos-sdk-go/clients/config_client" - "github.com/nacos-group/nacos-sdk-go/clients/naming_client" "github.com/nacos-group/nacos-sdk-go/common/constant" - "github.com/nacos-group/nacos-sdk-go/model" "github.com/nacos-group/nacos-sdk-go/vo" "github.com/pkg/errors" ) var _ RemoteConfManager = (*NacosConfManager)(nil) -const ( - ServiceName = "clickhouse_sinker" - DataID = "clickhouse_sinker.json" -) - type NacosConfManager struct { configClient config_client.IConfigClient - namingClient naming_client.INamingClient group string + dataID string } func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) { @@ -85,74 +75,17 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) return } - ncm.namingClient, err = clients.CreateNamingClient(map[string]interface{}{ - "serverConfigs": sc, - "clientConfig": cc, - }) - if err != nil { - err = errors.Wrapf(err, "") - return - } - ncm.group = group - return -} - -func (ncm *NacosConfManager) Register(ip string, port int) (err error) { - _, err = ncm.namingClient.RegisterInstance(vo.RegisterInstanceParam{ - Ip: ip, - Port: uint64(port), - ServiceName: ServiceName, - Weight: float64(runtime.NumCPU()), - GroupName: ncm.group, - Enable: true, - Healthy: true, - Ephemeral: true, - }) - if err != nil { - err = errors.Wrapf(err, "") - } - return -} - -func (ncm *NacosConfManager) Deregister(ip string, port int) (err error) { - _, err = ncm.namingClient.DeregisterInstance( - vo.DeregisterInstanceParam{ - Ip: ip, - Port: uint64(port), - ServiceName: ServiceName, - GroupName: ncm.group, - Ephemeral: true, - }) - if err != nil { - err = errors.Wrapf(err, "") + if _, ok = properties["dataId"]; ok { + ncm.dataID = properties["dataId"].(string) } return } -func (ncm *NacosConfManager) GetInstances() (instances []Instance, err error) { - var insts []model.Instance - insts, err = ncm.namingClient.SelectInstances(vo.SelectInstancesParam{ - ServiceName: ServiceName, - GroupName: ncm.group, - HealthyOnly: true, - }) - //SelectInstances throws errors if "do not have useful host, ignore it", "instance list is empty!" - if err != nil { - err = errors.Wrapf(err, "") - return - } - for _, inst := range insts { - instances = append(instances, Instance{Addr: fmt.Sprintf("%s:%d", inst.Ip, inst.Port), Weight: int(inst.Weight)}) - } - sort.Slice(instances, func(i, j int) bool { return (instances[i].Addr < instances[j].Addr) }) - return -} - func (ncm *NacosConfManager) GetConfig() (conf *Config, err error) { var content string content, err = ncm.configClient.GetConfig(vo.ConfigParam{ - DataId: DataID, + DataId: ncm.dataID, Group: ncm.group, }) if err != nil { @@ -164,6 +97,10 @@ func (ncm *NacosConfManager) GetConfig() (conf *Config, err error) { err = errors.Wrapf(err, "") return } + if ncm.dataID != conf.Task.Name { + err = errors.Errorf("DataId %s doesn't match with config: %s", ncm.dataID, content) + return + } return } @@ -175,7 +112,7 @@ func (ncm *NacosConfManager) PublishConfig(conf *Config) (err error) { } content := string(bs) _, err = ncm.configClient.PublishConfig(vo.ConfigParam{ - DataId: DataID, + DataId: conf.Task.Name, Group: ncm.group, Content: content, }) @@ -183,5 +120,6 @@ func (ncm *NacosConfManager) PublishConfig(conf *Config) (err error) { err = errors.Wrapf(err, "") return } + ncm.dataID = conf.Task.Name return } diff --git a/docs/configuration/flag.md b/docs/configuration/flag.md index e59dc139..204d3ef8 100644 --- a/docs/configuration/flag.md +++ b/docs/configuration/flag.md @@ -8,7 +8,7 @@ Usage of ./dist/clickhouse_sinker: consul api interface address (default "http://127.0.0.1:8500") -consul-deregister-critical-services-after string configure service check DeregisterCriticalServiceAfter (default "30m") - -consul-register-enable + -consul-cfg-enable register current instance in consul -http-port int http listen port (default 2112) @@ -26,7 +26,7 @@ Usage of ./dist/clickhouse_sinker: nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work! -nacos-password string nacos password (default "nacos") - -nacos-register-enable + -nacos-cfg-enable register current instance in nacos -nacos-username string nacos username (default "nacos") diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 825e34c9..cfbd8977 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -233,7 +233,7 @@ The precedence of config items: Sinker is able to register with Nacos, get and apply config changes dynamically without restart the whole process. Controled by: -- CLI parameters: `nacos-register-enable, nacos-addr, nacos-namespace-id, nacos-group, nacos-username, nacos-password` +- CLI parameters: `nacos-cfg-enable, nacos-addr, nacos-namespace-id, nacos-group, nacos-username, nacos-password` - env variables: `NACOS_REGISTER_ENABLE, NACOS_ADDR, NACOS_NAMESPACE_ID, NACOS_GROUP, NACOS_USERNAME, NACOS_PASSWORD` ### Consul @@ -241,7 +241,7 @@ Controled by: Currently sinker is able to register with Consul, but unable to get config. Controled by: -- CLI parameters: `consul-register-enable, consul-addr, consul-deregister-critical-services-after` +- CLI parameters: `consul-cfg-enable, consul-addr, consul-deregister-critical-services-after` - env variables: `CONSUL_REGISTER_ENABLE, CONSUL_ADDR, CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER` ### Local Files @@ -260,7 +260,7 @@ All metrics are defined in `statistics.go`. You can create Grafana dashboard for Metrics are exposed at `http://ip:port/metrics`. IP is the outbound IP of this machine. Port is from CLI `--http-port` or env `HTTP_PORT`. -Sinker registers with Nacos if CLI `--consul-register-enable` or env `CONSUL_REGISTER_ENABLE` is present. However Prometheus is [unable](https://github.com/alibaba/nacos/issues/1032) to obtain dynamic service list from nacos server. +Sinker registers with Nacos if CLI `--consul-cfg-enable` or env `CONSUL_REGISTER_ENABLE` is present. However Prometheus is [unable](https://github.com/alibaba/nacos/issues/1032) to obtain dynamic service list from nacos server. - Push to promethues diff --git a/go.mod b/go.mod index a4b44807..6c5f0556 100644 --- a/go.mod +++ b/go.mod @@ -5,21 +5,10 @@ go 1.14 require ( github.com/ClickHouse/clickhouse-go v1.4.3 github.com/Shopify/sarama v1.27.2 - github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878 // indirect github.com/cespare/xxhash v1.1.0 github.com/fagongzi/goetty v1.6.0 github.com/golang/protobuf v1.4.2 // indirect - github.com/google/btree v1.0.0 // indirect github.com/google/gops v0.3.12 - github.com/hashicorp/consul/api v1.3.0 - github.com/hashicorp/go-msgpack v0.5.5 // indirect - github.com/hashicorp/go-rootcerts v1.0.1 // indirect - github.com/hashicorp/go-sockaddr v1.0.2 // indirect - github.com/hashicorp/golang-lru v0.5.4 // indirect - github.com/hashicorp/memberlist v0.1.5 // indirect - github.com/hashicorp/serf v0.8.5 // indirect - github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 // indirect - github.com/k0kubun/pp v3.0.1+incompatible github.com/nacos-group/nacos-sdk-go v1.0.1 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.3.0 @@ -33,7 +22,6 @@ require ( github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/valyala/fastjson v1.4.1 github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c - golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208 // indirect golang.org/x/sys v0.0.0-20200917061948-648f2a039071 // indirect golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e ) diff --git a/go.sum b/go.sum index d94e4513..2ffddb90 100644 --- a/go.sum +++ b/go.sum @@ -4,8 +4,6 @@ github.com/ClickHouse/clickhouse-go v1.4.3 h1:iAFMa2UrQdR5bHJ2/yaSLffZkxpcOYQMCU github.com/ClickHouse/clickhouse-go v1.4.3/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/DataDog/datadog-go v2.2.0+incompatible h1:V5BKkxACZLjzHjSgBbr2gvLA2Ae49yhc6CSY7MLy5k4= -github.com/DataDog/datadog-go v2.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/Shopify/sarama v1.27.2 h1:1EyY1dsxNDUQEv0O/4TsjosHI2CgB1uo9H/v56xzTxc= @@ -20,18 +18,10 @@ github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRF github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 h1:zOVTBdCKFd9JbCKz9/nt+FovbjPFmb7mUnp8nH9fQBA= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e h1:QEF07wC0T1rKkctt1RINW/+RMTVmiwxETico2l3gxJA= -github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= -github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= -github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878 h1:EFSB7Zo9Eg91v7MJPVsifUysc/wPdN+NOnVe6bWbdBM= -github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878/go.mod h1:3AMJUQhVx52RsWOnlkpikZr01T/yAVN2gn0861vByNg= -github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bgentry/speakeasy v0.1.0 h1:ByYyxL9InA1OWqxJqqp2A5pYHUrCiAL6K3J+LKSsQkY= -github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAKk= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= @@ -40,10 +30,6 @@ github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible h1:C29Ae4G5GtYyYMm1aztcyj/J5ckgJm2zwdDajFbx1NY= -github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag= -github.com/circonus-labs/circonusllhist v0.1.3 h1:TJH+oke8D16535+jHExHj4nQvzlZrj7ug5D7I/orNUA= -github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= @@ -60,8 +46,6 @@ github.com/fagongzi/goetty v1.6.0 h1:LBj+N14o9g3A1w5Mp+TJYEBXWfp07mFdfUH2nvSPFkk github.com/fagongzi/goetty v1.6.0/go.mod h1:lLUyHhtjlOqatxVXgyLocwoI2o359JzLE7EWRGZiGw4= github.com/fastly/go-utils v0.0.0-20180712184237-d95a45783239 h1:Ghm4eQYC0nEPnSJdVkTrXpu9KtoVCSo1hg7mtI7G9KU= github.com/fastly/go-utils v0.0.0-20180712184237-d95a45783239/go.mod h1:Gdwt2ce0yfBxPvZrHkprdPPTTS3N5rwmLE8T22KBXlw= -github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys= -github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= github.com/frankban/quicktest v1.10.2 h1:19ARM85nVi4xH7xPXuc5eM/udya5ieh7b/Sv+d844Tk= @@ -93,10 +77,6 @@ github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0 github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c h1:964Od4U6p2jUkFxvCydnIczKteheJEzHRToSGK3Bnlw= -github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= -github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -110,49 +90,8 @@ github.com/google/gops v0.3.12/go.mod h1:38bMPVKFh+1X106CPpbLAWtZIR1+xwgzT9gew0k github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/hashicorp/consul/api v1.3.0 h1:HXNYlRkkM/t+Y/Yhxtwcy02dlYwIaoxzvxPnS+cqy78= -github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE= -github.com/hashicorp/consul/sdk v0.3.0 h1:UOxjlb4xVNF93jak1mzzoBatyFju9nrkxpVwIp/QqxQ= -github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= -github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= -github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= -github.com/hashicorp/go-cleanhttp v0.5.1 h1:dH3aiDG9Jvb5r5+bYHsikaOUIpcM0xvgMXVoDkXMzJM= -github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= -github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= -github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= -github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= -github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= -github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= -github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= -github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= -github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs= -github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= -github.com/hashicorp/go-rootcerts v1.0.1 h1:DMo4fmknnz0E0evoNYnV48RjWndOsmd6OW+09R3cEP8= -github.com/hashicorp/go-rootcerts v1.0.1/go.mod h1:pqUvnprVnM5bf7AOirdbb01K4ccR319Vf4pU3K5EGc8= -github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= -github.com/hashicorp/go-sockaddr v1.0.2 h1:ztczhD1jLxIRjVejw8gFomI1BQZOe2WoVOu0SyteCQc= -github.com/hashicorp/go-sockaddr v1.0.2/go.mod h1:rB4wwRAUzs07qva3c5SdrY/NEtAUjGlgmH/UkBUC97A= -github.com/hashicorp/go-syslog v1.0.0 h1:KaodqZuhUoZereWVIYmpUgZysurB1kBLX2j0MwMrUAE= -github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= -github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE= -github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= -github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= -github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= -github.com/hashicorp/logutils v1.0.0 h1:dLEQVugN8vlakKOUE3ihGLTZJRB4j+M2cdTm/ORI65Y= -github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= -github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= -github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= -github.com/hashicorp/memberlist v0.1.5 h1:AYBsgJOW9gab/toO5tEB8lWetVgDKZycqkebJ8xxpqM= -github.com/hashicorp/memberlist v0.1.5/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= -github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= -github.com/hashicorp/serf v0.8.5 h1:ZynDUIQiA8usmRgPdGPHFdPnb1wgGI9tK3mO9hcAJjc= -github.com/hashicorp/serf v0.8.5/go.mod h1:UpNcs7fFbpKIyZaUuSW6EPiH+eZC7OuyFD+wc1oal+k= github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8= github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uczJe5YQdrYB16oTJlGSC/OyZDqUk9xX4= @@ -169,10 +108,6 @@ github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/u github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 h1:uC1QfSlInpQF+M0ao65imhwqKnz3Q2z/d8PWZRMQvDM= -github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= -github.com/k0kubun/pp v3.0.1+incompatible h1:3tqvf7QgUnZ5tXO6pNAZlrvHgl6DvifjDrd9g2S9Z40= -github.com/k0kubun/pp v3.0.1+incompatible/go.mod h1:GWse8YhT0p8pT4ir3ZgBbfZild3tgzSScAn6HmfYukg= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19 h1:WjT3fLi9n8YWh/Ih8Q1LHAPsTqGddPcHqscN+PJ3i68= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= @@ -198,28 +133,9 @@ github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f/go.mod github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 h1:Bvq8AziQ5jFF4BHGAEDSqwPW1NJS3XshxbRCxtjFAZc= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042/go.mod h1:TPpsiPUEh0zFL1Snz4crhMlBe60PYxRHr5oFF3rRYg0= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= -github.com/mattn/go-colorable v0.0.9 h1:UVL0vNpWh04HeJXV0KLcaT7r06gOH2l4OW6ddYRUIY4= -github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= -github.com/mattn/go-isatty v0.0.3 h1:ns/ykhmWi7G9O+8a448SecJU3nSMBXJfqQkl0upE1jI= -github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/miekg/dns v1.0.14 h1:9jZdLNd/P4+SfEJ0TNyxYpsK8N4GtfylBLqtbYN1sbA= -github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/mitchellh/cli v1.0.0 h1:iGBIsUe3+HZ/AD/Vd7DErOt5sU9fa8Uj7A2s1aggv1Y= -github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= -github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= -github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/go-testing-interface v1.0.0 h1:fzU/JVNcaqHQEcVFAKeR41fkiLdIPrefOvVG1VZ96U0= -github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= -github.com/mitchellh/go-wordwrap v1.0.0/go.mod h1:ZXFpozHsX6DPmq2I0TCekCxypsnAUbP2oI0UX1GXzOo= -github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= -github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= -github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -231,9 +147,6 @@ github.com/nacos-group/nacos-sdk-go v1.0.1 h1:VNmXGlSS28xOmkO5Nxk5WRp6f1HMosAmG9 github.com/nacos-group/nacos-sdk-go v1.0.1/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= -github.com/pascaldekloe/goe v0.1.0 h1:cBOtyMzM9HTpWjXfbbunk26uA6nG3a8n06Wieeh0MwY= -github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.5.2+incompatible h1:WCjObylUIOlKy/+7Abdn34TLIkXiA4UWUMhxq9m9ZXI= @@ -245,10 +158,7 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/posener/complete v1.1.1 h1:ccV59UEOTzVDnDUEFdT95ZzHVZ+5+158q8+SJb2QV5w= -github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0 h1:miYCvYqFXtl/J9FIy8eNpBfYthAEFg+Ys0XyUVEcDsc= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= @@ -256,24 +166,18 @@ github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1: github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0 h1:ElTg5tNp4DqfV7UQjDqv2+RJlNzsDtvNAWccbItceIE= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0 h1:L+1lyG48J1zAQXA3RBX/nG/B3gjlHq0zTt2tlbJLyCY= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.8.0 h1:bLkjvFe2ZRX1DpcgZcdf7j/+MnusEps5hktST/FHA34= github.com/prometheus/common v0.8.0/go.mod h1:PC/OgXc+UN7B4ALwvn1yzVZmVwvhXp5JsbBv6wSv6i0= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLkt8= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 h1:MkV+77GLUNo5oJ0jf870itWm3D0Sjh7+Za9gazKc5LQ= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 h1:nn5Wsu0esKSJiIVhscUtVbo7ada43DJhG55ua/hjS5I= -github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= github.com/shirou/gopsutil v2.20.4+incompatible h1:cMT4rxS55zx9NVUnCkrmXCsEB/RNfG9SwHY9evtX8Ng= @@ -310,8 +214,6 @@ github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 h1:kF/7m/ZU+0D github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3/go.mod h1:QDlpd3qS71vYtakd2hmdpqhJ9nwv6mD6A30bQ1BPBFE= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= -github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926 h1:G3dpKMzFDjgEh2q1Z7zUUtKa8ViPtH+ocF0bE0g00O8= -github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= github.com/valyala/fastjson v1.4.1 h1:hrltpHpIpkaxll8QltMU8c3QZ5+qIiCL8yKqPFJI/yE= github.com/valyala/fastjson v1.4.1/go.mod h1:nV6MsjxL2IMJQUoHDIrjEI7oLyeqK6aBD7EFWPsvP8o= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c h1:u40Z8hqBAAQyv+vATcGgV0YCnDjqSL7/q/JyPhhJSPk= @@ -329,7 +231,6 @@ go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9E go.uber.org/zap v1.15.0 h1:ZZCA22JRF2gQE5FoNmhmrf7jeJJ2uhqDUNRYKm8dvmM= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190506204251-e1dfcc566284/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -340,9 +241,7 @@ golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= -golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980 h1:dfGZHvZk057jK2MCeWus/TowKpJ8y4AmooUzdBSR9GU= @@ -355,11 +254,7 @@ golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208 h1:qwRHBd0NqMbJxfbotnDhm2ByMI1Shq4Y6oRJo21SGJA= -golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= diff --git a/go.test.sh b/go.test.sh index f834ceec..ebce5b8d 100755 --- a/go.test.sh +++ b/go.test.sh @@ -61,7 +61,7 @@ curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' ./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/config.json ## start clickhouse_sinker to consume -timeout 30 ./dist/clickhouse_sinker --nacos-register-enable --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos +timeout 30 ./dist/clickhouse_sinker --nacos-cfg-enable --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos ## check result count=`curl "localhost:8123" -d 'select count() from test1'` diff --git a/input/kafka_go.go b/input/kafka_go.go index 1275485c..66ded4c7 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -38,7 +38,7 @@ var _ Inputer = (*KafkaGo)(nil) // KafkaGo implements input.Inputer type KafkaGo struct { - taskCfg *config.TaskConfig + cfg *config.Config r *kafka.Reader stopped chan struct{} putFn func(msg model.InputMessage) @@ -51,22 +51,22 @@ func NewKafkaGo() *KafkaGo { // Init Initialise the kafka instance with configuration func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) (err error) { - k.taskCfg = cfg.Tasks[taskName] - kfkCfg := cfg.Kafka[k.taskCfg.Kafka] + k.cfg = cfg + kfkCfg := &cfg.Kafka k.stopped = make(chan struct{}) k.putFn = putFn offset := kafka.LastOffset - if k.taskCfg.Earliest { + if k.cfg.Task.Earliest { offset = kafka.FirstOffset } readerCfg := &kafka.ReaderConfig{ Brokers: strings.Split(kfkCfg.Brokers, ","), - GroupID: k.taskCfg.ConsumerGroup, - Topic: k.taskCfg.Topic, + GroupID: k.cfg.Task.ConsumerGroup, + Topic: k.cfg.Task.Topic, StartOffset: offset, - MinBytes: k.taskCfg.MinBufferSize * k.taskCfg.MsgSizeHint, - MaxBytes: k.taskCfg.BufferSize * k.taskCfg.MsgSizeHint, - MaxWait: time.Duration(k.taskCfg.FlushInterval) * time.Second, + MinBytes: k.cfg.Task.MinBufferSize * k.cfg.Task.MsgSizeHint, + MaxBytes: k.cfg.Task.BufferSize * k.cfg.Task.MsgSizeHint, + MaxWait: time.Duration(k.cfg.Task.FlushInterval) * time.Second, CommitInterval: time.Second, // flushes commits to Kafka every second ErrorLogger: log.StandardLogger(), //kafka-go INFO log is too verbose } @@ -120,15 +120,15 @@ LOOP_KAFKA_GO: var msg kafka.Message if msg, err = k.r.FetchMessage(ctx); err != nil { if errors.Is(err, context.Canceled) { - log.Infof("%s: Kafka.Run quit due to context has been canceled", k.taskCfg.Name) + log.Infof("%s: Kafka.Run quit due to context has been canceled", k.cfg.Task.Name) break LOOP_KAFKA_GO } else if errors.Is(err, io.EOF) { - log.Infof("%s: Kafka.Run quit due to reader has been closed", k.taskCfg.Name) + log.Infof("%s: Kafka.Run quit due to reader has been closed", k.cfg.Task.Name) break LOOP_KAFKA_GO } else { - statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.taskCfg.Name).Inc() + statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.cfg.Task.Name).Inc() err = errors.Wrap(err, "") - log.Errorf("%s: Kafka.Run got error %+v", k.taskCfg.Name, err) + log.Errorf("%s: Kafka.Run got error %+v", k.cfg.Task.Name, err) continue } } @@ -165,5 +165,5 @@ func (k *KafkaGo) Stop() error { // Description of this kafka consumer, which topic it reads from func (k *KafkaGo) Description() string { - return "kafka consumer of topic " + k.taskCfg.Topic + return "kafka consumer of topic " + k.cfg.Task.Topic } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 9947da44..295bb2d9 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -38,7 +38,7 @@ var _ Inputer = (*KafkaSarama)(nil) // KafkaSarama implements input.Inputer type KafkaSarama struct { - taskCfg *config.TaskConfig + cfg *config.Config cg sarama.ConsumerGroup sess sarama.ConsumerGroupSession stopped chan struct{} @@ -59,7 +59,7 @@ func (h MyConsumerGroupHandler) Setup(sess sarama.ConsumerGroupSession) error { return nil } func (h MyConsumerGroupHandler) Cleanup(_ sarama.ConsumerGroupSession) error { - log.Infof("%s: consumer group %s cleanup", h.k.taskCfg.Name, h.k.taskCfg.ConsumerGroup) + log.Infof("%s: consumer group %s cleanup", h.k.cfg.Task.Name, h.k.cfg.Task.ConsumerGroup) time.Sleep(5 * time.Second) return nil } @@ -80,8 +80,9 @@ func (h MyConsumerGroupHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, c // Init Initialise the kafka instance with configuration func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) (err error) { - k.taskCfg = cfg.Tasks[taskName] - kfkCfg := cfg.Kafka[k.taskCfg.Kafka] + k.cfg = cfg + kfkCfg := &cfg.Kafka + taskCfg := &cfg.Task k.stopped = make(chan struct{}) k.putFn = putFn config := sarama.NewConfig() @@ -114,11 +115,11 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg m config.Net.SASL.Password = kfkCfg.Sasl.Password config.Net.SASL.GSSAPI = kfkCfg.Sasl.GSSAPI } - if k.taskCfg.Earliest { + if taskCfg.Earliest { config.Consumer.Offsets.Initial = sarama.OffsetOldest } - config.ChannelBufferSize = k.taskCfg.MinBufferSize - cg, err := sarama.NewConsumerGroup(strings.Split(kfkCfg.Brokers, ","), k.taskCfg.ConsumerGroup, config) + config.ChannelBufferSize = taskCfg.MinBufferSize + cg, err := sarama.NewConsumerGroup(strings.Split(kfkCfg.Brokers, ","), taskCfg.ConsumerGroup, config) if err != nil { return err } @@ -128,23 +129,24 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg m // kafka main loop func (k *KafkaSarama) Run(ctx context.Context) { + taskCfg := &k.cfg.Task LOOP_SARAMA: for { handler := MyConsumerGroupHandler{k} // `Consume` should be called inside an infinite loop, when a // server-side rebalance happens, the consumer session will need to be // recreated to get the new claims - if err := k.cg.Consume(ctx, []string{k.taskCfg.Topic}, handler); err != nil { + if err := k.cg.Consume(ctx, []string{taskCfg.Topic}, handler); err != nil { if errors.Is(err, context.Canceled) { - log.Infof("%s: Kafka.Run quit due to context has been canceled", k.taskCfg.Name) + log.Infof("%s: Kafka.Run quit due to context has been canceled", taskCfg.Name) break LOOP_SARAMA } else if errors.Is(err, sarama.ErrClosedConsumerGroup) { - log.Infof("%s: Kafka.Run quit due to consumer group has been closed", k.taskCfg.Name) + log.Infof("%s: Kafka.Run quit due to consumer group has been closed", taskCfg.Name) break LOOP_SARAMA } else { - statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.taskCfg.Name).Inc() + statistics.ConsumeMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() err = errors.Wrap(err, "") - log.Errorf("%s: Kafka.Run got error %+v", k.taskCfg.Name, err) + log.Errorf("%s: Kafka.Run got error %+v", taskCfg.Name, err) continue } } @@ -164,7 +166,7 @@ func (k *KafkaSarama) Stop() error { // Description of this kafka consumer, which topic it reads from func (k *KafkaSarama) Description() string { - return "kafka consumer of topic " + k.taskCfg.Topic + return "kafka consumer of topic " + k.cfg.Task.Topic } // Predefined SCRAMClientGeneratorFunc, copied from https://github.com/Shopify/sarama/blob/master/examples/sasl_scram_client/scram_client.go diff --git a/output/clickhouse.go b/output/clickhouse.go index e6b1e0f5..09420d87 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -45,22 +45,20 @@ type ClickHouse struct { Dims []*model.ColumnWithType Dms []string // Table Configs - taskCfg *config.TaskConfig - chCfg *config.ClickHouseConfig + cfg *config.Config prepareSQL string } // NewClickHouse new a clickhouse instance -func NewClickHouse(cfg *config.Config, taskName string) *ClickHouse { - taskCfg := cfg.Tasks[taskName] - chCfg := cfg.Clickhouse[taskCfg.Clickhouse] - return &ClickHouse{taskCfg: taskCfg, chCfg: chCfg} +func NewClickHouse(cfg *config.Config) *ClickHouse { + return &ClickHouse{cfg: cfg} } // Init the clickhouse intance func (c *ClickHouse) Init() (err error) { - if err = pool.InitConn(c.taskCfg.Clickhouse, c.chCfg.Hosts, c.chCfg.Port, c.chCfg.DB, c.chCfg.Username, c.chCfg.Password, c.chCfg.DsnParams); err != nil { + chCfg := &c.cfg.Clickhouse + if err = pool.InitConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams); err != nil { return } if err = c.initSchema(); err != nil { @@ -71,10 +69,10 @@ func (c *ClickHouse) Init() (err error) { // Send a batch to clickhouse func (c *ClickHouse) Send(batch *model.Batch, callback func(batch *model.Batch) error) { - statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Inc() + statistics.WritingPoolBacklog.WithLabelValues(c.cfg.Task.Name).Inc() _ = util.GlobalWritingPool.Submit(func() { c.loopWrite(batch, callback) - statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Dec() + statistics.WritingPoolBacklog.WithLabelValues(c.cfg.Task.Name).Dec() }) } @@ -88,7 +86,7 @@ func (c *ClickHouse) write(batch *model.Batch) error { return nil } - conn := pool.GetConn(c.taskCfg.Clickhouse, batch.BatchIdx) + conn := pool.GetConn(batch.BatchIdx) if tx, err = conn.Begin(); err != nil { goto ERR } @@ -103,19 +101,19 @@ func (c *ClickHouse) write(batch *model.Batch) error { } } if err != nil { - log.Errorf("%s: stmt.Exec failed %d times with errors %+v", c.taskCfg.Name, numErr, err) + log.Errorf("%s: stmt.Exec failed %d times with errors %+v", c.cfg.Task.Name, numErr, err) goto ERR } if err = tx.Commit(); err != nil { goto ERR } - statistics.FlushMsgsTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) + statistics.FlushMsgsTotal.WithLabelValues(c.cfg.Task.Name).Add(float64(batch.RealSize)) return err ERR: if shouldReconnect(err) { _ = conn.ReConnect() - statistics.ClickhouseReconnectTotal.WithLabelValues(c.taskCfg.Name).Inc() + statistics.ClickhouseReconnectTotal.WithLabelValues(c.cfg.Task.Name).Inc() } return err } @@ -142,12 +140,12 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, callback func(batch *model.Ba return } if std_errors.Is(err, context.Canceled) { - log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.taskCfg.Name) + log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) return } - log.Errorf("%s: committing offset(try #%d) failed with error %+v", c.taskCfg.Name, times, err) + log.Errorf("%s: committing offset(try #%d) failed with error %+v", c.cfg.Task.Name, times, err) times++ - if c.chCfg.RetryTimes <= 0 || times < c.chCfg.RetryTimes { + if c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes { time.Sleep(10 * time.Second) } else { os.Exit(-1) @@ -155,13 +153,13 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, callback func(batch *model.Ba } } if std_errors.Is(err, context.Canceled) { - log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.taskCfg.Name) + log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) return } - log.Errorf("%s: flush batch(try #%d) failed with error %+v", c.taskCfg.Name, c.chCfg.RetryTimes-times, err) - statistics.FlushMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) + log.Errorf("%s: flush batch(try #%d) failed with error %+v", c.cfg.Task.Name, c.cfg.Clickhouse.RetryTimes-times, err) + statistics.FlushMsgsErrorTotal.WithLabelValues(c.cfg.Task.Name).Add(float64(batch.RealSize)) times++ - if shouldReconnect(err) && (c.chCfg.RetryTimes <= 0 || times < c.chCfg.RetryTimes) { + if shouldReconnect(err) && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { time.Sleep(10 * time.Second) } else { os.Exit(-1) @@ -171,15 +169,15 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, callback func(batch *model.Ba // Stop free clickhouse connections func (c *ClickHouse) Stop() error { - pool.FreeConn(c.taskCfg.Clickhouse) + pool.FreeConn() return nil } func (c *ClickHouse) initSchema() (err error) { - if c.taskCfg.AutoSchema { - conn := pool.GetConn(c.taskCfg.Clickhouse, 0) + if c.cfg.Task.AutoSchema { + conn := pool.GetConn(0) var rs *sql.Rows - if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, c.chCfg.DB, c.taskCfg.TableName)); err != nil { + if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, c.cfg.Clickhouse.DB, c.cfg.Task.TableName)); err != nil { err = errors.Wrapf(err, "") return err } @@ -193,13 +191,13 @@ func (c *ClickHouse) initSchema() (err error) { return err } typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") - if !util.StringContains(c.taskCfg.ExcludeColumns, name) && defaultKind != "MATERIALIZED" { + if !util.StringContains(c.cfg.Task.ExcludeColumns, name) && defaultKind != "MATERIALIZED" { c.Dims = append(c.Dims, &model.ColumnWithType{Name: name, Type: typ, SourceName: util.GetSourceName(name)}) } } } else { c.Dims = make([]*model.ColumnWithType, 0) - for _, dim := range c.taskCfg.Dims { + for _, dim := range c.cfg.Task.Dims { c.Dims = append(c.Dims, &model.ColumnWithType{ Name: dim.Name, Type: dim.Type, @@ -218,9 +216,9 @@ func (c *ClickHouse) initSchema() (err error) { for i := range params { params[i] = "?" } - c.prepareSQL = "INSERT INTO " + c.chCfg.DB + "." + c.taskCfg.TableName + " (" + strings.Join(quotedDms, ",") + ") " + + c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.cfg.Task.TableName + " (" + strings.Join(quotedDms, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" - log.Infof("%s: Prepare sql=> %s", c.taskCfg.Name, c.prepareSQL) + log.Infof("%s: Prepare sql=> %s", c.cfg.Task.Name, c.prepareSQL) return nil } diff --git a/pool/conn.go b/pool/conn.go index 34429346..58bf9b69 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -38,8 +38,8 @@ const ( ) var ( - lock sync.Mutex - poolMaps map[string]*ClusterConnections + lock sync.Mutex + connections []*Connection ) // Connection a datastructure for storing the clickhouse connection @@ -48,11 +48,6 @@ type Connection struct { dsn string } -type ClusterConnections struct { - connections []*Connection - ref int -} - // ReConnect used for restablishing connection with server func (c *Connection) ReConnect() error { sqlDB, err := sql.Open("clickhouse", c.dsn) @@ -66,21 +61,10 @@ func (c *Connection) ReConnect() error { return nil } -func InitConn(name string, hosts [][]string, port int, db, username, password, dsnParams string) (err error) { +func InitConn(hosts [][]string, port int, db, username, password, dsnParams string) (err error) { var sqlDB *sql.DB lock.Lock() - if poolMaps == nil { - poolMaps = make(map[string]*ClusterConnections) - } - if cc, ok := poolMaps[name]; ok { - cc.ref++ - lock.Unlock() - return - } - lock.Unlock() - - var cc ClusterConnections - cc.ref = 1 + defer lock.Unlock() // Each shard has a *sql.DB which connects to all replicas inside the shard. // "alt_hosts" tolerates replica single-point-failure. for _, replicas := range hosts { @@ -105,23 +89,13 @@ func InitConn(name string, hosts [][]string, port int, db, username, password, d return } setDBParams(sqlDB) - cc.connections = append(cc.connections, &Connection{sqlDB, dsn}) - } - - lock.Lock() - defer lock.Unlock() - if cc2, ok := poolMaps[name]; ok { - cc2.ref++ - return - } - for _, conn := range cc.connections { - if err = health.Health.AddReadinessCheck(conn.dsn, healthcheck.DatabasePingCheck(conn.DB, 30*time.Second)); err != nil { + if err = health.Health.AddReadinessCheck(dsn, healthcheck.DatabasePingCheck(sqlDB, 30*time.Second)); err != nil { err = errors.Wrapf(err, "") - log.Errorf("got error: %+v", err) + return } + connections = append(connections, &Connection{sqlDB, dsn}) } - poolMaps[name] = &cc - return nil + return } // TODO, pool this @@ -130,60 +104,34 @@ func setDBParams(sqlDB *sql.DB) { sqlDB.SetConnMaxLifetime(120 * time.Second) } -func FreeConn(name string) { +func FreeConn() { lock.Lock() defer lock.Unlock() - if cc, ok := poolMaps[name]; ok { - cc.ref-- - if cc.ref == 0 { - delete(poolMaps, name) - for _, conn := range cc.connections { - if err := health.Health.RemoveReadinessCheck(conn.dsn); err != nil { - err = errors.Wrapf(err, conn.dsn) - log.Errorf("got error: %+v", err) - } - } + for _, conn := range connections { + if err := health.Health.RemoveReadinessCheck(conn.dsn); err != nil { + err = errors.Wrapf(err, conn.dsn) + log.Errorf("got error: %+v", err) } + conn.DB.Close() } + connections = []*Connection{} } func GetTotalConn() (cnt int) { lock.Lock() defer lock.Unlock() - for _, cc := range poolMaps { - cnt += cc.ref * len(cc.connections) - } - return -} - -// GetNumConn get number of connections for the given name -func GetNumConn(name string) (numConn int) { - lock.Lock() - defer lock.Unlock() - if ps, ok := poolMaps[name]; ok { - numConn = len(ps.connections) - } - return + return len(connections) } // GetConn select a clickhouse node from the cluster based on batchNum -func GetConn(name string, batchNum int64) (con *Connection) { +func GetConn(batchNum int64) (con *Connection) { lock.Lock() defer lock.Unlock() - - cc, ok := poolMaps[name] - if !ok { - return - } - con = cc.connections[batchNum%int64(len(cc.connections))] + con = connections[batchNum%int64(len(connections))] return } // CloseAll closed all connection and destroys the pool func CloseAll() { - for _, cc := range poolMaps { - for _, c := range cc.connections { - _ = c.Close() - } - } + FreeConn() } diff --git a/task/ring.go b/task/ring.go index 54b331db..a663a563 100644 --- a/task/ring.go +++ b/task/ring.go @@ -32,6 +32,7 @@ type Ring struct { func (ring *Ring) PutElem(msgRow model.MsgRow) { var err error + taskCfg := &ring.service.cfg.Task msgOffset := msgRow.Msg.Offset ring.mux.Lock() defer ring.mux.Unlock() @@ -43,7 +44,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { ring.idleCnt = 0 ring.isIdle = false ring.ringBuf = make([]model.MsgRow, ring.ringCap) - log.Infof("%s: topic %s partition %d quit idle", ring.service.taskCfg.Name, ring.service.taskCfg.Topic, ring.partition) + log.Infof("%s: topic %s partition %d quit idle", taskCfg.Name, taskCfg.Topic, ring.partition) } // assert(msgOffset < ring.ringGroundOff + ring.ringCap) if msgOffset >= ring.ringCeilingOff { @@ -52,10 +53,10 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { if ring.service.sharder != nil && msgRow.Row != nil { if msgRow.Shard, err = ring.service.sharder.Calc(msgRow.Row); err != nil { - log.Fatalf("%s: got error %+v", ring.service.taskCfg.Name, err) + log.Fatalf("%s: got error %+v", taskCfg.Name, err) } } - statistics.RingMsgs.WithLabelValues(ring.service.taskCfg.Name).Inc() + statistics.RingMsgs.WithLabelValues(taskCfg.Name).Inc() ring.ringBuf[msgOffset&(ring.ringCap-1)] = msgRow for ; ring.ringFilledOffset < ring.ringCeilingOff && ring.ringBuf[ring.ringFilledOffset&(ring.ringCap-1)].Msg != nil; ring.ringFilledOffset++ { } @@ -63,9 +64,9 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { ring.genBatchOrShard(ring.ringFilledOffset) // reschedule the delayed ForceBatchOrShard ring.tid.Stop() - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(ring.service.taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { + if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", ring.service.taskCfg.Name, err) + log.Fatalf("%s: got error %+v", taskCfg.Name, err) } } } @@ -77,9 +78,10 @@ type OffsetRange struct { func (ring *Ring) ForceBatchOrShard(arg interface{}) { var newMsg *model.InputMessage + taskCfg := &ring.service.cfg.Task select { case <-ring.service.ctx.Done(): - log.Errorf("%s: Ring.ForceBatchOrShard quit due to the context has been canceled", ring.service.taskCfg.Name) + log.Errorf("%s: Ring.ForceBatchOrShard quit due to the context has been canceled", taskCfg.Name) return default: } @@ -88,7 +90,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { defer ring.mux.Unlock() if arg != nil { newMsg = arg.(*model.InputMessage) - log.Warnf("%s: Ring.ForceBatchOrShard partition %d message range [%d, %d)", ring.service.taskCfg.Name, newMsg.Partition, ring.ringGroundOff, newMsg.Offset) + log.Warnf("%s: Ring.ForceBatchOrShard partition %d message range [%d, %d)", taskCfg.Name, newMsg.Partition, ring.ringGroundOff, newMsg.Offset) } if !ring.isIdle { if newMsg == nil { @@ -101,11 +103,11 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { ring.idleCnt = 0 ring.isIdle = true ring.ringBuf = nil - log.Infof("%s: topic %s partition %d enter idle", ring.service.taskCfg.Name, ring.service.taskCfg.Topic, ring.partition) + log.Infof("%s: topic %s partition %d enter idle", taskCfg.Name, taskCfg.Topic, ring.partition) } } } else { - statistics.RingForceBatchAllTotal.WithLabelValues(ring.service.taskCfg.Name).Inc() + statistics.RingForceBatchAllTotal.WithLabelValues(taskCfg.Name).Inc() LOOP: for { ring.genBatchOrShard(ring.ringCeilingOff) @@ -123,9 +125,9 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { // reschedule the delayed ForceBatchOrShard ring.tid.Stop() var err error - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(ring.service.taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { + if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", ring.service.taskCfg.Name, err) + log.Fatalf("%s: got error %+v", taskCfg.Name, err) } } @@ -134,6 +136,7 @@ func (ring *Ring) genBatchOrShard(expNewGroundOff int64) { if expNewGroundOff <= ring.ringGroundOff { return } + taskCfg := &ring.service.cfg.Task var gaps []OffsetRange var msgCnt, parseErrs int endOff := (ring.ringGroundOff | int64(1< 0 { log.Debugf("%s: going to flush a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", - ring.service.taskCfg.Name, ring.service.taskCfg.Topic, ring.partition, endOff-1, + taskCfg.Name, taskCfg.Topic, ring.partition, endOff-1, batch.RealSize, gaps, parseErrs) batch.BatchIdx = (endOff - 1) >> ring.batchSizeShift ring.batchSys.CreateBatchGroupSingle(batch, ring.partition, endOff-1) ring.service.batchChan <- batch if gaps == nil { - statistics.RingNormalBatchsTotal.WithLabelValues(ring.service.taskCfg.Name).Inc() + statistics.RingNormalBatchsTotal.WithLabelValues(taskCfg.Name).Inc() } else { - statistics.RingForceBatchAllGapTotal.WithLabelValues(ring.service.taskCfg.Name).Inc() + statistics.RingForceBatchAllGapTotal.WithLabelValues(taskCfg.Name).Inc() } } - statistics.RingMsgs.WithLabelValues(ring.service.taskCfg.Name).Sub(float64(batch.RealSize)) + statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(batch.RealSize)) } ring.ringGroundOff = endOff diff --git a/task/sharding.go b/task/sharding.go index 20eb5e51..a9751ed1 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -112,14 +112,15 @@ type Sharder struct { func NewSharder(service *Service) (sh *Sharder, err error) { var policy *ShardingPolicy - ckNum := pool.GetNumConn(service.taskCfg.Clickhouse) - if policy, err = NewShardingPolicy(service.taskCfg.ShardingKey, service.taskCfg.ShardingPolicy, service.clickhouse.Dms, ckNum); err != nil { + ckNum := pool.GetTotalConn() + taskCfg := &service.cfg.Task + if policy, err = NewShardingPolicy(taskCfg.ShardingKey, taskCfg.ShardingPolicy, service.clickhouse.Dms, ckNum); err != nil { return } sh = &Sharder{ service: service, policy: policy, - batchSys: model.NewBatchSys(service.taskCfg, service.fnCommit), + batchSys: model.NewBatchSys(taskCfg, service.fnCommit), ckNum: ckNum, msgBuf: make([]*model.Rows, ckNum), offsets: make([]int64, 0), @@ -139,6 +140,7 @@ func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOf defer sh.mux.Unlock() var gaps []OffsetRange var parseErrs int + taskCfg := &sh.service.cfg.Task gapBegOff := int64(-1) for i := begOff; i < endOff; i++ { msgRow := &ringBuf[i&(ringCap-1)] @@ -172,7 +174,7 @@ func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOf } if msgCnt > 0 { sh.offsets[partition] = endOff - 1 - statistics.ShardMsgs.WithLabelValues(sh.service.taskCfg.Name).Add(float64(msgCnt)) + statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Add(float64(msgCnt)) } var maxBatchSize int for i := 0; i < sh.ckNum; i++ { @@ -182,9 +184,9 @@ func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOf } } log.Debugf("%s: sharded a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", - sh.service.taskCfg.Name, sh.service.taskCfg.Topic, partition, endOff-1, + taskCfg.Name, taskCfg.Topic, partition, endOff-1, msgCnt, gaps, parseErrs) - if maxBatchSize >= sh.service.taskCfg.BufferSize { + if maxBatchSize >= taskCfg.BufferSize { sh.doFlush(nil) } return @@ -201,6 +203,7 @@ func (sh *Sharder) doFlush(_ interface{}) { var err error var msgCnt int var batches []*model.Batch + taskCfg := &sh.service.cfg.Task for i, rows := range sh.msgBuf { realSize := len(*rows) if realSize > 0 { @@ -215,20 +218,20 @@ func (sh *Sharder) doFlush(_ interface{}) { } } if msgCnt > 0 { - log.Debugf("%s: going to flush batch group for topic %v, offsets %+v, messages %d", sh.service.taskCfg.Name, sh.service.taskCfg.Name, sh.offsets, msgCnt) + log.Debugf("%s: going to flush batch group for topic %v, offsets %+v, messages %d", taskCfg.Name, taskCfg.Name, sh.offsets, msgCnt) sh.batchSys.CreateBatchGroupMulti(batches, sh.offsets) sh.offsets = sh.offsets[:0] // ALL batches in a group shall be populated before sending any one to next stage. for _, batch := range batches { sh.service.batchChan <- batch } - statistics.ShardMsgs.WithLabelValues(sh.service.taskCfg.Name).Sub(float64(msgCnt)) + statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } // reschedule the delayed ForceFlush sh.tid.Stop() - if sh.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(sh.service.taskCfg.FlushInterval)*time.Second, sh.ForceFlush, nil); err != nil { + if sh.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, sh.ForceFlush, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", sh.service.taskCfg.Name, err) + log.Fatalf("%s: got error %+v", taskCfg.Name, err) } } diff --git a/task/task.go b/task/task.go index ee2d4e0e..bb7d85ab 100644 --- a/task/task.go +++ b/task/task.go @@ -49,7 +49,6 @@ type Service struct { clickhouse *output.ClickHouse pp *parser.Pool cfg *config.Config - taskCfg *config.TaskConfig dims []*model.ColumnWithType rings []*Ring @@ -61,7 +60,7 @@ type Service struct { } // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances -func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *parser.Pool, cfg *config.Config, taskName string) *Service { +func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *parser.Pool, cfg *config.Config) *Service { return &Service{ stopped: make(chan struct{}), inputer: inputer, @@ -69,7 +68,6 @@ func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *pa started: false, pp: pp, cfg: cfg, - taskCfg: cfg.Tasks[taskName], } } @@ -86,13 +84,13 @@ func (service *Service) Init() (err error) { service.limiter2 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter3 = rate.NewLimiter(rate.Every(10*time.Second), 1) - if service.taskCfg.ShardingKey != "" { + if service.cfg.Task.ShardingKey != "" { if service.sharder, err = NewSharder(service); err != nil { return } } - err = service.inputer.Init(service.cfg, service.taskCfg.Name, service.put) + err = service.inputer.Init(service.cfg, service.cfg.Task.Name, service.put) return } @@ -101,13 +99,13 @@ func (service *Service) Run(ctx context.Context) { var err error service.started = true service.ctx, service.cancel = context.WithCancel(ctx) - log.Infof("%s: task started", service.taskCfg.Name) + log.Infof("%s: task started", service.cfg.Task.Name) go service.inputer.Run(service.ctx) if service.sharder != nil { // schedule a delayed ForceFlush - if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(service.taskCfg.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { + if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(service.cfg.Task.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", service.taskCfg.Name, err) + log.Fatalf("%s: got error %+v", service.cfg.Task.Name, err) } } @@ -118,7 +116,7 @@ LOOP: break LOOP case batch := <-service.batchChan: if err := service.flush(batch); err != nil { - log.Errorf("%s: got error %+v", service.taskCfg.Name, err) + log.Errorf("%s: got error %+v", service.cfg.Task.Name, err) } } } @@ -126,12 +124,12 @@ LOOP: } func (service *Service) fnCommit(partition int, offset int64) error { - msg := model.InputMessage{Topic: service.taskCfg.Topic, Partition: partition, Offset: offset} + msg := model.InputMessage{Topic: service.cfg.Task.Topic, Partition: partition, Offset: offset} return service.inputer.CommitMessages(service.ctx, &msg) } func (service *Service) put(msg model.InputMessage) { - statistics.ConsumeMsgsTotal.WithLabelValues(service.taskCfg.Name).Inc() + statistics.ConsumeMsgsTotal.WithLabelValues(service.cfg.Task.Name).Inc() // ensure ring for this message exist service.Lock() var ring *Ring @@ -145,7 +143,7 @@ func (service *Service) put(msg model.InputMessage) { var err error if ring == nil { - batchSizeShift := util.GetShift(service.taskCfg.BufferSize) + batchSizeShift := util.GetShift(service.cfg.Task.BufferSize) ringCap := int64(1 << (batchSizeShift + 1)) ring := &Ring{ ringBuf: make([]model.MsgRow, ringCap), @@ -157,13 +155,13 @@ func (service *Service) put(msg model.InputMessage) { idleCnt: 0, isIdle: false, partition: msg.Partition, - batchSys: model.NewBatchSys(service.taskCfg, service.fnCommit), + batchSys: model.NewBatchSys(&service.cfg.Task, service.fnCommit), service: service, } // schedule a delayed ForceBatchOrShard - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(service.taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { + if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(service.cfg.Task.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", service.taskCfg.Name, err) + log.Fatalf("%s: got error %+v", service.cfg.Task.Name, err) } service.rings[msg.Partition] = ring service.Unlock() @@ -174,18 +172,18 @@ func (service *Service) put(msg model.InputMessage) { ringGroundOff, ringFilledOffset = ring.ringGroundOff, ring.ringFilledOffset ring.mux.Unlock() if msg.Offset < ringFilledOffset { - statistics.RingMsgsOffTooSmallErrorTotal.WithLabelValues(service.taskCfg.Name).Inc() + statistics.RingMsgsOffTooSmallErrorTotal.WithLabelValues(service.cfg.Task.Name).Inc() if service.limiter2.Allow() { log.Warnf("%s: got a message(topic %v, partition %d, offset %v) left to %v", - service.taskCfg.Name, msg.Topic, msg.Partition, msg.Offset, ringFilledOffset) + service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, ringFilledOffset) } return } if msg.Offset >= ringGroundOff+ring.ringCap { - statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(service.taskCfg.Name).Inc() + statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(service.cfg.Task.Name).Inc() if service.limiter3.Allow() { log.Warnf("%s: got a message(topic %v, partition %d, offset %v) right to the range [%v, %v)", - service.taskCfg.Name, msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap) + service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap) } time.Sleep(1 * time.Second) ring.ForceBatchOrShard(&msg) @@ -193,16 +191,16 @@ func (service *Service) put(msg model.InputMessage) { } // submit message to a goroutine pool - statistics.ParsingPoolBacklog.WithLabelValues(service.taskCfg.Name).Inc() + statistics.ParsingPoolBacklog.WithLabelValues(service.cfg.Task.Name).Inc() _ = util.GlobalParsingPool.Submit(func() { var row *model.Row p := service.pp.Get() metric, err := p.Parse(msg.Value) if err != nil { - statistics.ParseMsgsErrorTotal.WithLabelValues(service.taskCfg.Name).Inc() + statistics.ParseMsgsErrorTotal.WithLabelValues(service.cfg.Task.Name).Inc() if service.limiter1.Allow() { log.Errorf("%s: failed to parse message(topic %v, partition %d, offset %v) %+v, string(value) <<<%+v>>>, got error %+v", - service.taskCfg.Name, msg.Topic, msg.Partition, msg.Offset, msg, string(msg.Value), err) + service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, msg, string(msg.Value), err) } } else { row = model.MetricToRow(metric, msg, service.dims) @@ -214,7 +212,7 @@ func (service *Service) put(msg model.InputMessage) { ring = service.rings[msg.Partition] service.Unlock() ring.PutElem(model.MsgRow{Msg: &msg, Row: row}) - statistics.ParsingPoolBacklog.WithLabelValues(service.taskCfg.Name).Dec() + statistics.ParsingPoolBacklog.WithLabelValues(service.cfg.Task.Name).Dec() }) } @@ -230,21 +228,21 @@ func (service *Service) flush(batch *model.Batch) (err error) { // NotifyStop notify task to stop, This is non-blocking. func (service *Service) NotifyStop() { - log.Infof("%s: notified to stop", service.taskCfg.Name) + log.Infof("%s: notified to stop", service.cfg.Task.Name) service.cancel() } // Stop stop kafka and clickhouse client. This is blocking. func (service *Service) Stop() { - log.Infof("%s: stopping task service...", service.taskCfg.Name) + log.Infof("%s: stopping task service...", service.cfg.Task.Name) service.cancel() if err := service.inputer.Stop(); err != nil { panic(err) } - log.Infof("%s: stopped input", service.taskCfg.Name) + log.Infof("%s: stopped input", service.cfg.Task.Name) _ = service.clickhouse.Stop() - log.Infof("%s: stopped output", service.taskCfg.Name) + log.Infof("%s: stopped output", service.cfg.Task.Name) if service.sharder != nil { service.sharder.tid.Stop() @@ -254,12 +252,12 @@ func (service *Service) Stop() { ring.tid.Stop() } } - log.Infof("%s: stopped internal timers", service.taskCfg.Name) + log.Infof("%s: stopped internal timers", service.cfg.Task.Name) if service.started { <-service.stopped } - log.Infof("%s: stopped", service.taskCfg.Name) + log.Infof("%s: stopped", service.cfg.Task.Name) } // GoID returns goroutine id diff --git a/util/conf.go b/util/conf.go deleted file mode 100644 index 924dabf0..00000000 --- a/util/conf.go +++ /dev/null @@ -1,73 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package util - -import ( - "encoding/json" - "io/ioutil" - "os" - "path/filepath" - "regexp" - "strings" - - "github.com/pkg/errors" -) - -const extendTag = "@extend:" - -func ExtendFile(filePath string) (string, error) { - fi, err := os.Stat(filePath) - if err != nil { - return "", err - } else if fi.IsDir() { - return "", errors.New("error in ExtendFile, " + filePath + " is not a file") - } - var b []byte - b, err = ioutil.ReadFile(filePath) - if err != nil { - return "", err - } - dir, err := filepath.Abs(filepath.Dir(filePath)) - if err != nil { - return "", err - } - return ExtendFileContent(dir, b) -} - -func ExtendFileContent(dir string, content []byte) (string, error) { - //检查是不是规范的json - test := new(interface{}) - err := json.Unmarshal(content, &test) - if err != nil { - return "", err - } - - //替换子json文件 - reg := regexp.MustCompile(`"` + extendTag + `.*?"`) - ret := reg.ReplaceAllStringFunc(string(content), func(match string) string { - match = match[len(extendTag)+1 : len(match)-1] - var p = match - if !strings.HasPrefix(match, "/") { - p = dir + "/" + match - } - sb, err2 := ExtendFile(p) - if err2 != nil { - err = errors.Wrapf(err2, "replace json config [%s]failed", match) - } - return sb - }) - return ret, err -} diff --git a/util/consistenthash.go b/util/consistenthash.go deleted file mode 100644 index 88d1d1f9..00000000 --- a/util/consistenthash.go +++ /dev/null @@ -1,77 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -// an implementation of a ring hash. -// Refers to https://github.com/golang/groupcache/blob/master/consistenthash/consistenthash.go -package util - -import ( - "hash/crc32" - "sort" - "strconv" -) - -type Hash func(data []byte) uint32 - -type HashRing struct { - hash Hash - keys []int // Sorted - hashMap map[int]string -} - -func NewHashRing(fn Hash) *HashRing { - m := &HashRing{ - hash: fn, - hashMap: make(map[int]string), - } - if m.hash == nil { - m.hash = crc32.ChecksumIEEE - } - return m -} - -// IsEmpty returns true if there are no items available. -func (m *HashRing) IsEmpty() bool { - return len(m.keys) == 0 -} - -// Add adds a key to the hash. -func (m *HashRing) Add(key string, replicas int) { - for i := 0; i < replicas; i++ { - hash := int(m.hash([]byte(strconv.Itoa(i) + key))) - m.keys = append(m.keys, hash) - m.hashMap[hash] = key - } - sort.Ints(m.keys) -} - -// Get gets the closest item in the hash to the provided key. -func (m *HashRing) Get(key string) string { - if m.IsEmpty() { - return "" - } - - hash := int(m.hash([]byte(key))) - - // Binary search for appropriate replica. - idx := sort.Search(len(m.keys), func(i int) bool { return m.keys[i] >= hash }) - - // Means we have cycled back to the first replica. - if idx == len(m.keys) { - idx = 0 - } - - return m.hashMap[m.keys[idx]] -} From 4f54804326a4e409bfb8c8edb4c491f1f3025dbc Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 11 Jan 2021 15:28:03 +0800 Subject: [PATCH 042/404] systest passed --- docker/config.json | 84 ----------------------------------- docker/test_auto_schema.json | 47 ++++++++++++++++++++ docker/test_fixed_schema.json | 57 ++++++++++++++++++++++++ go.test.sh | 25 ++++++----- 4 files changed, 118 insertions(+), 95 deletions(-) delete mode 100644 docker/config.json create mode 100644 docker/test_auto_schema.json create mode 100644 docker/test_fixed_schema.json diff --git a/docker/config.json b/docker/config.json deleted file mode 100644 index c37e3b97..00000000 --- a/docker/config.json +++ /dev/null @@ -1,84 +0,0 @@ -{ - "clickhouse": { - "ch1": { - "db": "default", - "hosts": [ - [ - "127.0.0.1" - ] - ], - "password": "", - "retryTimes": 0, - "port": 9000, - "username": "" - } - }, - "kafka": { - "kfk1": { - "brokers": "127.0.0.1:9093", - "version": "2.2.1", - "sasl": { - "enable": false, - "password": "", - "username": "", - "gssapi": { - "authtype": 0, - "keytabpath": "", - "kerberosconfigpath": "", - "servicename": "", - "username": "", - "password": "", - "realm": "", - "disablepafxfast": false - } - } - } - }, - "common": { - "bufferSize": 90000, - "minBufferSize": 2000, - "flushInterval": 5, - "layoutDateTime": "2006-01-02 15:04:05.999999999Z07:00", - "logLevel": "debug" - }, - "tasks": { - "test1": { - "kafka": "kfk1", - "topic": "topic1", - "consumerGroup": "test_sinker", - "earliest": true, - "parser": "json", - "clickhouse": "ch1", - "tableName": "test1", - "dims": [ - { - "name": "time", - "type": "DateTime" - }, - { - "name": "name", - "type": "String" - }, - { - "name": "value", - "type": "Float32" - } - ], - "bufferSize": 50000 - }, - "test_auto_schema": { - "kafka": "kfk1", - "topic": "topic1", - "consumerGroup": "test_auto_schema", - "earliest": true, - "parser": "json", - "clickhouse": "ch1", - "autoSchema": true, - "tableName": "test_auto_schema", - "excludeColumns": [ - "day" - ], - "bufferSize": 50000 - } - } -} \ No newline at end of file diff --git a/docker/test_auto_schema.json b/docker/test_auto_schema.json new file mode 100644 index 00000000..9e2c31ad --- /dev/null +++ b/docker/test_auto_schema.json @@ -0,0 +1,47 @@ +{ + "clickhouse": { + "db": "default", + "hosts": [ + [ + "127.0.0.1" + ] + ], + "password": "", + "retryTimes": 0, + "port": 9000, + "username": "" + }, + "kafka": { + "brokers": "127.0.0.1:9093", + "version": "2.2.1", + "sasl": { + "enable": false, + "password": "", + "username": "", + "gssapi": { + "authtype": 0, + "keytabpath": "", + "kerberosconfigpath": "", + "servicename": "", + "username": "", + "password": "", + "realm": "", + "disablepafxfast": false + } + } + }, + "task": { + "name": "test_auto_schema", + "topic": "topic1", + "consumerGroup": "test_auto_schema", + "earliest": true, + "parser": "json", + "autoSchema": true, + "tableName": "test_auto_schema", + "excludeColumns": [ + "day" + ], + "bufferSize": 50000 + }, + "logLevel": "debug" +} \ No newline at end of file diff --git a/docker/test_fixed_schema.json b/docker/test_fixed_schema.json new file mode 100644 index 00000000..dd5de888 --- /dev/null +++ b/docker/test_fixed_schema.json @@ -0,0 +1,57 @@ +{ + "clickhouse": { + "db": "default", + "hosts": [ + [ + "127.0.0.1" + ] + ], + "password": "", + "retryTimes": 0, + "port": 9000, + "username": "" + }, + "kafka": { + "brokers": "127.0.0.1:9093", + "version": "2.2.1", + "sasl": { + "enable": false, + "password": "", + "username": "", + "gssapi": { + "authtype": 0, + "keytabpath": "", + "kerberosconfigpath": "", + "servicename": "", + "username": "", + "password": "", + "realm": "", + "disablepafxfast": false + } + } + }, + "task": { + "name": "test_fixed_schema", + "topic": "topic1", + "consumerGroup": "test_fixed_schema", + "earliest": true, + "parser": "json", + "tableName": "test_fixed_schema", + "dims": [ + { + "name": "time", + "type": "DateTime" + }, + { + "name": "name", + "type": "String" + }, + { + "name": "value", + "type": "Float32" + } + ], + "bufferSize": 50000 + }, + "logLevel": "debug" +} \ No newline at end of file diff --git a/go.test.sh b/go.test.sh index ebce5b8d..00c2c9b4 100755 --- a/go.test.sh +++ b/go.test.sh @@ -1,8 +1,8 @@ #!/usr/bin/env bash ## create table -curl "localhost:8123" -d 'DROP TABLE IF EXISTS test1' -curl "localhost:8123" -d 'CREATE TABLE IF NOT EXISTS test1 +curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_fixed_schema' +curl "localhost:8123" -d 'CREATE TABLE IF NOT EXISTS test_fixed_schema ( `day` Date DEFAULT toDate(time), `time` DateTime, @@ -37,11 +37,12 @@ sudo docker cp send.sh kafka:/tmp/ sudo docker exec kafka sh /tmp/send.sh ## start clickhouse_sinker to consume -timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/config.json +timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/test_fixed_schema.json +timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/test_auto_schema.json ## check result -count=`curl "localhost:8123" -d 'select count() from test1'` -echo "Got test1 count => $count" +count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` +echo "Got test_fixed_schema count => $count" [ $count -eq 100000 ] || exit 1 count=`curl "localhost:8123" -d 'select count() from test_auto_schema'` @@ -50,22 +51,24 @@ echo "Got test_auto_schema count => $count" ## reset kafka consumer-group offsets -sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_sinker --all-topics --to-earliest +sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_fixed_schema --all-topics --to-earliest sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_auto_schema --all-topics --to-earliest ## truncate tables -curl "localhost:8123" -d 'TRUNCATE TABLE test1' +curl "localhost:8123" -d 'TRUNCATE TABLE test_fixed_schema' curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' ## publish clickhouse_sinker config -./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/config.json +./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_fixed_schema.json +./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_auto_schema.json ## start clickhouse_sinker to consume -timeout 30 ./dist/clickhouse_sinker --nacos-cfg-enable --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos +timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema +timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema ## check result -count=`curl "localhost:8123" -d 'select count() from test1'` -echo "Got test1 count => $count" +count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` +echo "Got test_fixed_schema count => $count" [ $count -eq 100000 ] || exit 1 count=`curl "localhost:8123" -d 'select count() from test_auto_schema'` From 30332a0c7fa69390185e28ae79b8b349de884f88 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 11 Jan 2021 17:12:46 +0800 Subject: [PATCH 043/404] changed doc --- Dockerfile | 10 +- Dockerfile_goreleaser | 10 +- conf/config.json | 44 -------- conf/tasks/api_request.json | 21 ---- conf/tasks/csv_request.json | 23 ----- conf/tasks/daily_request.json | 29 ------ conf/tasks/falcon_sample.json | 31 ------ conf/tasks/logstash_sample.json | 23 ----- docker/test_auto_schema.json | 25 +---- docker/test_fixed_schema.json | 25 +---- docs/configuration/config.md | 106 +++++++++++-------- docs/configuration/flag.md | 13 +-- docs/configuration/task.md | 41 -------- docs/dev/design.md | 39 ------- docs/dev/introduction.md | 173 ++++++++++++++------------------ docs/guide/install.md | 2 +- docs/guide/run.md | 109 ++++++-------------- 17 files changed, 190 insertions(+), 534 deletions(-) delete mode 100644 conf/config.json delete mode 100644 conf/tasks/api_request.json delete mode 100644 conf/tasks/csv_request.json delete mode 100644 conf/tasks/daily_request.json delete mode 100644 conf/tasks/falcon_sample.json delete mode 100644 conf/tasks/logstash_sample.json delete mode 100644 docs/configuration/task.md diff --git a/Dockerfile b/Dockerfile index 1b466e57..4309edbe 100644 --- a/Dockerfile +++ b/Dockerfile @@ -17,17 +17,13 @@ COPY --from=builder /app/dist/nacos_publish_config /usr/local/bin/nacos_publish_ # - HTTP_PORT # - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL -# - LOCAL_CFG_DIR # - LOCAL_CFG_FILE -# - CONSUL_REGISTER_ENABLE -# - CONSUL_ADDR -# - CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER -# - NACOS_REGISTER_ENABLE # - NACOS_ADDR -# - NACOS_NAMESPACE_ID -# - NACOS_GROUP # - NACOS_USERNAME # - NACOS_PASSWORD +# - NACOS_NAMESPACE_ID +# - NACOS_GROUP +# - NACOS_DATAID # See cmd/clickhouse_sinker/main.go for details. ENTRYPOINT ["/usr/local/bin/clickhouse_sinker"] diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index e1eb2c0b..d566960a 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -10,17 +10,13 @@ ADD ./nacos_publish_config /usr/local/bin/nacos_publish_config # - HTTP_PORT # - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL -# - LOCAL_CFG_DIR # - LOCAL_CFG_FILE -# - CONSUL_REGISTER_ENABLE -# - CONSUL_ADDR -# - CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER -# - NACOS_REGISTER_ENABLE # - NACOS_ADDR -# - NACOS_NAMESPACE_ID -# - NACOS_GROUP # - NACOS_USERNAME # - NACOS_PASSWORD +# - NACOS_NAMESPACE_ID +# - NACOS_GROUP +# - NACOS_DATAID # See cmd/clickhouse_sinker/main.go for details. ENTRYPOINT ["/usr/local/bin/clickhouse_sinker"] diff --git a/conf/config.json b/conf/config.json deleted file mode 100644 index 261b634e..00000000 --- a/conf/config.json +++ /dev/null @@ -1,44 +0,0 @@ -{ - "clickhouse": { - "ch1": { - "db": "default", - "hosts": [ - [ - "127.0.0.1" - ] - ], - "password": "", - "retryTimes": 0, - "port": 9000, - "username": "default" - } - }, - "kafka": { - "kfk1": { - "brokers": "127.0.0.1:9093", - "sasl": { - "enable": false, - "password": "", - "username": "", - "gssapi": { - "authtype": 0, - "keytabpath": "", - "kerberosconfigpath": "", - "servicename": "", - "username": "", - "password": "", - "realm": "", - "disablepafxfast": false - } - }, - "version": "2.2.1" - } - }, - "common": { - "bufferSize": 90000, - "minBufferSize": 1, - "msgSizeHint": 1000, - "flushInterval": 5, - "logLevel": "debug" - } -} \ No newline at end of file diff --git a/conf/tasks/api_request.json b/conf/tasks/api_request.json deleted file mode 100644 index d3386dda..00000000 --- a/conf/tasks/api_request.json +++ /dev/null @@ -1,21 +0,0 @@ -{ - - "name" : "api_request", - - "kafkaClient": "sarama", - "kafka": "kfk1", - "topic": "real_time", - "earliest" : true, - "consumerGroup" : "testgroup", - - "parser" : "json", - "clickhouse" : "ch1", - "tableName" : "api_request", - - "@desc_of_autoSchema" : "auto schema will auto fetch the schema from clickhouse", - "autoSchema" : true, - "@desc_of_exclude_columns" : "this columns will be excluded by insert SQL ", - "excludeColumns" : ["date"], - - "bufferSize" : 90000 -} diff --git a/conf/tasks/csv_request.json b/conf/tasks/csv_request.json deleted file mode 100644 index ff96156b..00000000 --- a/conf/tasks/csv_request.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - - "name" : "csv_request", - - "kafkaClient": "kafka-go", - "kafka": "kfk1", - "topic": "csv-topic", - "earliest" : true, - "consumerGroup" : "testgroup", - - "parser" : "csv", - "csvFormat": ["timestamp","level","message"], - "delimiter": ",", - "clickhouse" : "ch1", - "tableName" : "kafkaqueue", - - "@desc_of_autoSchema" : "auto schema will auto fetch the schema from clickhouse", - "autoSchema" : true, - "@desc_of_exclude_columns" : "this columns will be excluded by insert SQL ", - "excludeColumns" : ["date"], - - "bufferSize" : 90000 -} diff --git a/conf/tasks/daily_request.json b/conf/tasks/daily_request.json deleted file mode 100644 index 068b0ac2..00000000 --- a/conf/tasks/daily_request.json +++ /dev/null @@ -1,29 +0,0 @@ -{ - "name": "daily_request", - "kafka": "kfk1", - "topic": "topic", - "earliest": true, - "consumerGroup": "group", - "parser": "gjson_extend", - "clickhouse": "ch1", - "tableName": "daily", - "dims": [ - { - "name": "day", - "type": "Date", - "sourceName": "day" - }, - { - "name": "level", - "type": "String", - "sourceName": "data_level" - }, - { - "name": "total", - "type": "UInt64", - "sourceName": "data_total" - } - ], - "@desc_of_exclude_columns": "this columns will be excluded by insert SQL ", - "excludeColumns": [] -} diff --git a/conf/tasks/falcon_sample.json b/conf/tasks/falcon_sample.json deleted file mode 100644 index 0f6847fe..00000000 --- a/conf/tasks/falcon_sample.json +++ /dev/null @@ -1,31 +0,0 @@ -{ - - "name" : "faclon_sample", - - "kafkaClient": "sarama", - "kafka": "kfk1", - "topic": "falcon", - "consumerGroup" : "falcon_sample_ck", - - "parser" : "json", - "clickhouse" : "ch1", - - "tableName" : "dis_falcon_metric_sample", - - "dims" : [ - {"name" : "timestamp" , "type" : "UInt32"}, - {"name" : "ckIndex" , "type" : "UInt32"}, - {"name" : "view" , "type" : "String"}, - {"name" : "service" , "type" : "String"}, - {"name" : "topic" , "type" : "String"}, - {"name" : "uri" , "type" : "String"}, - {"name" : "endpoint" , "type" : "String"}, - {"name" : "step" , "type" : "Int16"}, - {"name" : "tagsmap" , "type" : "String"}, - {"name" : "value" , "type" : "Float32"} - ], - - "bufferSize" : 50000, - "num" : 1 -} - diff --git a/conf/tasks/logstash_sample.json b/conf/tasks/logstash_sample.json deleted file mode 100644 index 4801b73f..00000000 --- a/conf/tasks/logstash_sample.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - - "name" : "logstash_sample", - "kafkaClient": "sarama", - "kafka": "kfk1", - "topic": "logstash", - "consumerGroup" : "logstash_sample_ck", - "parser" : "json", - "clickhouse" : "ch1", - "tableName" : "logstash", - "dims" : [ - {"name" : "date" , "type" : "ElasticDateTime"}, - {"name" : "level" , "type" : "String"}, - {"name" : "message" , "type" : "String"}, - {"name" : "str_nullable" , "type" : "Nullable(String)"}, - {"name" : "num" , "type" : "Int64"}, - {"name" : "num_nullable" , "type" : "Nullable(Int64)"}, - {"name" : "fnum" , "type" : "Float64"}, - {"name" : "fnum_nullable" , "type" : "Nullable(Float64)"}, - {"name" : "date_nullable" , "type" : "Nullable(ElasticDateTime)"} - ] -} - diff --git a/docker/test_auto_schema.json b/docker/test_auto_schema.json index 9e2c31ad..ed4e330f 100644 --- a/docker/test_auto_schema.json +++ b/docker/test_auto_schema.json @@ -1,34 +1,19 @@ { "clickhouse": { - "db": "default", "hosts": [ [ "127.0.0.1" ] ], - "password": "", - "retryTimes": 0, "port": 9000, - "username": "" + "db": "default", + "username": "", + "password": "", + "retryTimes": 0 }, "kafka": { "brokers": "127.0.0.1:9093", - "version": "2.2.1", - "sasl": { - "enable": false, - "password": "", - "username": "", - "gssapi": { - "authtype": 0, - "keytabpath": "", - "kerberosconfigpath": "", - "servicename": "", - "username": "", - "password": "", - "realm": "", - "disablepafxfast": false - } - } + "version": "2.2.1" }, "task": { "name": "test_auto_schema", diff --git a/docker/test_fixed_schema.json b/docker/test_fixed_schema.json index dd5de888..52a6d479 100644 --- a/docker/test_fixed_schema.json +++ b/docker/test_fixed_schema.json @@ -1,34 +1,19 @@ { "clickhouse": { - "db": "default", "hosts": [ [ "127.0.0.1" ] ], - "password": "", - "retryTimes": 0, "port": 9000, - "username": "" + "db": "default", + "username": "", + "password": "", + "retryTimes": 0 }, "kafka": { "brokers": "127.0.0.1:9093", - "version": "2.2.1", - "sasl": { - "enable": false, - "password": "", - "username": "", - "gssapi": { - "authtype": 0, - "keytabpath": "", - "kerberosconfigpath": "", - "servicename": "", - "username": "", - "password": "", - "realm": "", - "disablepafxfast": false - } - } + "version": "2.2.1" }, "task": { "name": "test_fixed_schema", diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 9da1e987..983e5d9e 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -6,54 +6,77 @@ { // clickhouse configs, it's map[string]ClickHouse for multiple clickhouse "clickhouse": { - // key for clickhouse config - "ch1": { - "db": "default", // database name - - // hosts for connection, it's Array(Array(String)) - // we can put hosts with same shard into the inner array - // it helps data deduplication for ReplicateMergeTree when driver error occurs - "hosts": [ - [ - "127.0.0.1" - ] - ], - "password": "", - // retryTimes when error occurs in inserting datas - "retryTimes": 0, - "port": 9000, - "username": "default" - } + // hosts for connection, it's Array(Array(String)) + // we can put hosts with same shard into the inner array + // it helps data deduplication for ReplicateMergeTree when driver error occurs + "hosts": [ + [ + "127.0.0.1" + ] + ], + "port": 9000, + "username": "default" + "password": "", + "db": "default", // database name + // retryTimes when error occurs in inserting datas + "retryTimes": 0, }, // kafka configs "kafka": { - "kfk1": { - "brokers": "127.0.0.1:9093", + "brokers": "127.0.0.1:9093", - // somethings about sasl - "sasl": { - "enable": false, - "password": "", + // somethings about sasl + "sasl": { + "enable": false, + "password": "", + "username": "", + "gssapi": { + "authtype": 0, + "keytabpath": "", + "kerberosconfigpath": "", + "servicename": "", "username": "", - "gssapi": { - "authtype": 0, - "keytabpath": "", - "kerberosconfigpath": "", - "servicename": "", - "username": "", - "password": "", - "realm": "", - "disablepafxfast": false - } - }, + "password": "", + "realm": "", + "disablepafxfast": false + } + }, - // kafka version, if you use sarama, the version must be specified - "version": "2.2.1" - } + // kafka version, if you use sarama, the version must be specified + "version": "2.2.1" }, - "common": { + "task": { + "name": "daily_request", + // kafka topic + "topic": "topic", + // kafka consume from earliest or latest + "earliest": true, + // kafka consumer group + "consumerGroup": "group", + + // message parser + "parser": "json", + + // clickhouse table name + "tableName": "daily", + + // columns of the table + "dims": [ + { + "name": "day", + "type": "Date", + "sourceName": "day" + }, + ... + ], + + // if it's specified, the schema will be auto mapped from clickhouse, + "autoSchema" : true, + // "this columns will be excluded by insert SQL " + "excludeColumns": [] + // batch size to insert into clickhouse "bufferSize": 90000, // min batch size to insert into clickhouse @@ -64,9 +87,10 @@ // interval flush the batch "flushInterval": 5, + }, - // log level - "logLevel": "debug" + // log level + "logLevel": "debug" } } ``` \ No newline at end of file diff --git a/docs/configuration/flag.md b/docs/configuration/flag.md index 204d3ef8..5df377a9 100644 --- a/docs/configuration/flag.md +++ b/docs/configuration/flag.md @@ -4,30 +4,23 @@ ./clickhouse_sinker -h Usage of ./dist/clickhouse_sinker: - -consul-addr string - consul api interface address (default "http://127.0.0.1:8500") - -consul-deregister-critical-services-after string - configure service check DeregisterCriticalServiceAfter (default "30m") - -consul-cfg-enable - register current instance in consul +Usage of dist/clickhouse_sinker: -http-port int http listen port (default 2112) - -local-cfg-dir config.json - local config dir. requires a file named config.json, and some task json files under `tasks` folder (default "/etc/clickhouse_sinker") -local-cfg-file string local config file (default "/etc/clickhouse_sinker.json") -metric-push-gateway-addrs string a list of comma-separated prometheus push gatway address -nacos-addr string a list of comma-separated nacos server addresses (default "127.0.0.1:8848") + -nacos-dataid string + nacos dataid -nacos-group string nacos group name. Empty string doesn't work! (default "DEFAULT_GROUP") -nacos-namespace-id string nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work! -nacos-password string nacos password (default "nacos") - -nacos-cfg-enable - register current instance in nacos -nacos-username string nacos username (default "nacos") -push-interval int diff --git a/docs/configuration/task.md b/docs/configuration/task.md deleted file mode 100644 index 53adc249..00000000 --- a/docs/configuration/task.md +++ /dev/null @@ -1,41 +0,0 @@ -# task config - -``` -{ - "name": "daily_request", - - // kafka cluster - "kafka": "kfk1", - "topic": "topic", - - // kafka consume from earliest or latest - "earliest": true, - // kafka consumer group - "consumerGroup": "group", - - // message parser - "parser": "json", - - // clickhouse cluster - "clickhouse": "ch1", - - // table name - "tableName": "daily", - - // columns of the table - "dims": [ - { - "name": "day", - "type": "Date", - "sourceName": "day" - }, - ... - ], - - // if it's specified, the schema will be auto mapped from clickhouse, - "autoSchema" : true, - // "this columns will be excluded by insert SQL " - "excludeColumns": [] -} - -``` \ No newline at end of file diff --git a/docs/dev/design.md b/docs/dev/design.md index 60f84fdc..8f3dc979 100644 --- a/docs/dev/design.md +++ b/docs/dev/design.md @@ -36,42 +36,3 @@ The flow is: - Generate batches for all shard slots if messages in one shard slot reach batchSize, or flush timer fire. Those batches form a `BatchGroup`. The `before` relationship could be impossilbe if messages of a partition are distributed to multiple batches. So those batches need to be committed after ALL of them have been written to clickhouse. - Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). - -# Remote Config Management - -All clickhouse_sinker instances share(fetched from Nacos, Consul, ZK etc.) Config structure: - -``` -type Config struct { - Kafka map[string]*KafkaConfig - Clickhouse map[string]*ClickHouseConfig - Tasks []*TaskConfig - Common struct { - ... - Replicas int //on how many sinker instances a task runs - } - Assigns map[string][]string //map instance_name to a list of task_name -} -type TaskConfig struct { - ... - Replicas int //on how many sinker instances this task runs -} - -``` - -Each instance can run multiple tasks. -Each task can be assigned to multiple instances. Each task declares how many instances it needs. - -## The coordinator(outside this project) - -- The coordinator provides API and/or webui to add/delete/modify tasks. -- The coordinator watches (do `service discovery`) instance startup/disappear events, and assign tasks to instances (do `publish config`). Refers to `cmd/nacos_publish_config/main.go` to assign tasks (from local config) via consistent-hash to instances(from CLI). - -## The schedule platform(outside this project) -The schedule platform start some clickhouse_sinker instances and start another one if a instance fail. - -## clickhouse_sinker - -- Every clickhouse_sinker instance register itself to a service manager (Nacos, Consul, ZK etc.). -- clickhouse_sinker watches (do `get config`) cofig, compare its assignment with the current one, apply changes. -- clickhouse_sinker shall try to recover a task in endless loop if the task fail. This ensures the tasks run on an instance match the coordinator assigned. \ No newline at end of file diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index cfbd8977..7615f140 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -14,37 +14,30 @@ Refers to [design](./design.md) for how it works. - Support multiple parsers: fastjson(recommended), gjson, csv. - Support multiple Kafka client: kafka-go(recommended), sarama. - Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/SCRAM, SASL/GSSAPI and combinations of them. -- Support multiple sinker tasks, each runs on parallel. -- Support multiple kafka and ClickHouse clusters. - Bulk insert (by config `bufferSize` and `flushInterval`). - Parse messages concurrently. - Write batches concurrently. - Every batch is routed to a determined clickhouse shard. Exit if loop write fail. - Custom sharding policy (by config `shardingKey` and `shardingPolicy`). - Tolerate replica single-point-failure. -- At least once delivery guarantee. -- Dynamic config management with Nacos. +- At-least-once delivery guarantee. +- Config management with local file or Nacos. ## Supported data types - [x] UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 - [x] Float32, Float64 -- [x] String -- [x] FixedString +- [x] String, FixedString, LowCardinality(String) - [x] Date, DateTime, DateTime64 (custom layout parser) -- [x] Array(UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64) -- [x] Array(Float32, Float64) -- [x] Array(String) -- [x] Array(FixedString) -- [x] Nullable +- [x] Array(T), where T is one of above basic types +- [x] Nullable(T), where T is one of above basic types - [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) ## Configuration -Refers to how [integration test](./go.test.sh) use the [example config](./docker/config.json). -Also refers to [code](./config/config.go) for all config items. +Refers to how [integration test](./go.test.sh) use the example config. Also refers to [code](./config/config.go) for all config items. ### Kafka Encryption @@ -55,28 +48,28 @@ clickhouse_sinker supports following encryption mechanisms: An example kafka config: ``` - "kfk1": { - "brokers": "192.168.31.64:9092", - "@version": "Required if you use sarama. It's the the Kafka server version.", - "version": "2.2.1" - } + "kafka": { + "brokers": "192.168.31.64:9092", + "@version": "Required if you use sarama. It's the the Kafka server version.", + "version": "2.2.1" + } ``` - Encryption using SSL An example kafka config: ``` - "kfk2": { - "brokers": "192.168.31.64:9093", - "version": "2.2.1", - "tls": { - "enable": true, - "@caCertFiles": "Required. It's the CA certificate with which Kafka brokers certs be signed. This cert is added to kafka.client.truststore.jks which kafka-console-consumer.sh uses", - "caCertFiles": "/etc/security/ca-cert", - "@insecureSkipVerify": "Whether disable broker FQDN verification. Set it to `true` if kafka-console-consumer.sh uses `ssl.endpoint.identification.algorithm=`.", - "insecureSkipVerify": true - } + "kafka": { + "brokers": "192.168.31.64:9093", + "version": "2.2.1", + "tls": { + "enable": true, + "@caCertFiles": "Required. It's the CA certificate with which Kafka brokers certs be signed. This cert is added to kafka.client.truststore.jks which kafka-console-consumer.sh uses", + "caCertFiles": "/etc/security/ca-cert", + "@insecureSkipVerify": "Whether disable broker FQDN verification. Set it to `true` if kafka-console-consumer.sh uses `ssl.endpoint.identification.algorithm=`.", + "insecureSkipVerify": true } + } ``` FYI. `kafka-console-consumer.sh` works as the following setup: @@ -102,110 +95,110 @@ clickhouse_sinker support following following authentication mechanisms: An example kafka config: ``` - "kfk1": { - "brokers": "192.168.31.64:9092", - "@version": "Required if you use sarama. It's the the Kafka server version.", - "version": "2.2.1" - } + "kafka": { + "brokers": "192.168.31.64:9092", + "@version": "Required if you use sarama. It's the the Kafka server version.", + "version": "2.2.1" + } ``` - SASL/PLAIN An example kafka config: ``` - "kfk3": { - "brokers": "192.168.31.64:9094", - "version": "2.2.1", - "sasl": { - "enable": true, - "mechanism": "PLAIN", - "username": "alice", - "password": "alice-secret" - } + "kafka": { + "brokers": "192.168.31.64:9094", + "version": "2.2.1", + "sasl": { + "enable": true, + "mechanism": "PLAIN", + "username": "alice", + "password": "alice-secret" } + } ``` FYI. Java clients work with the following setup: ``` -$ cat config/PLAINTEXT_PLAIN_client.properties +$ cat config/client_PLAINTEXT_PLAIN.properties security.protocol=SASL_PLAINTEXT sasl.kerberos.service.name=kafka sasl.mechanism=PLAIN sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required username="alice" password="alice-secret"; -$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_PLAIN_client.properties +$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/client_PLAINTEXT_PLAIN.properties -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_PLAIN_client.properties +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/client_PLAINTEXT_PLAIN.properties ``` - SASL/SCRAM An example kafka config: ``` - "kfk4": { - "brokers": "192.168.31.64:9094", - "version": "2.2.1", - "sasl": { - "enable": true, - "@mechanism": "SCRAM-SHA-256 or SCRAM-SHA-512", - "mechanism": "SCRAM-SHA-256", - "username": "alice", - "password": "alice-secret" - } + "kafka": { + "brokers": "192.168.31.64:9094", + "version": "2.2.1", + "sasl": { + "enable": true, + "@mechanism": "SCRAM-SHA-256 or SCRAM-SHA-512", + "mechanism": "SCRAM-SHA-256", + "username": "alice", + "password": "alice-secret" } + } ``` FYI. Java clients work with the following setup: ``` -$ cat config/PLAINTEXT_SCRAM_client.properties +$ cat config/client_PLAINTEXT_SCRAM.properties security.protocol=SASL_PLAINTEXT sasl.kerberos.service.name=kafka sasl.mechanism=SCRAM-SHA-256 sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required username="alice" password="alice-secret"; -$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_SCRAM_client.properties +$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/client_PLAINTEXT_SCRAM.properties -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_SCRAM_client.properties +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/client_PLAINTEXT_SCRAM.properties ``` - SASL/GSSAPI(Kerberos) An example kafka config: ``` - "kfk5": { - "brokers": "192.168.31.64:9094", - "version": "2.2.1", - "sasl": { - "enable": true, - "mechanism": "GSSAPI", - "gssapi": { - "@authtype": "1 - Username and password, 2 - Keytab", - "authtype": 2, - "keytabpath": "/etc/security/mmmtest.keytab", - "kerberosconfigpath": "/etc/krb5.conf", - "servicename": "kafka", - "@username": "`principal` consists of `username` `@` `realm`", - "username": "mmm", - "realm": "ALANWANG.COM" - } + "kafka": { + "brokers": "192.168.31.64:9094", + "version": "2.2.1", + "sasl": { + "enable": true, + "mechanism": "GSSAPI", + "gssapi": { + "@authtype": "1 - Username and password, 2 - Keytab", + "authtype": 2, + "keytabpath": "/etc/security/mmmtest.keytab", + "kerberosconfigpath": "/etc/krb5.conf", + "servicename": "kafka", + "@username": "`principal` consists of `username` `@` `realm`", + "username": "mmm", + "realm": "ALANWANG.COM" } } + } ``` FYI. Java clients work with the following setup: ``` -$ cat config/PLAINTEXT_GSSAPI_client.properties +$ cat config/client_PLAINTEXT_GSSAPI.properties security.protocol=SASL_PLAINTEXT sasl.kerberos.service.name=kafka sasl.mechanism=GSSAPI sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab="/etc/security/mmmtest.keytab" principal="mmm@ALANWANG.COM"; -$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/PLAINTEXT_GSSAPI_client.properties +$ bin/kafka-console-producer.sh --broker-list 192.168.31.64:9094 --topic sunshine --producer.config config/client_PLAINTEXT_GSSAPI.properties -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/PLAINTEXT_GSSAPI_client.properties +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/client_PLAINTEXT_GSSAPI.properties ``` Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://docs.cloudera.com/runtime/7.2.1/kafka-managing/topics/kafka-manage-cli-consumer.html) Kerberos keytab authentication work STRICTLY FOLLOW [this article](https://stackoverflow.com/questions/48744660/kafka-console-consumer-with-kerberos-authentication/49140414#49140414), then test `clickhouse_sinker` Kerberos authentication on the SAME machine which `kafka-console-consumer.sh` runs. I tested sarama Kerberos authentication against Kafka [2.2.1](https://archive.apache.org/dist/kafka/2.2.1/kafka_2.11-2.2.1.tgz). Not sure other Kafka versions work. @@ -226,31 +219,23 @@ This above expression can be customized with `shardingKey` and `shardingPolicy`. The precedence of config items: - CLI parameters > env variables -- Nacos > Consul > Local Config File > Local Config Dir +- Nacos > Local Config File ### Nacos Sinker is able to register with Nacos, get and apply config changes dynamically without restart the whole process. Controled by: -- CLI parameters: `nacos-cfg-enable, nacos-addr, nacos-namespace-id, nacos-group, nacos-username, nacos-password` -- env variables: `NACOS_REGISTER_ENABLE, NACOS_ADDR, NACOS_NAMESPACE_ID, NACOS_GROUP, NACOS_USERNAME, NACOS_PASSWORD` - -### Consul - -Currently sinker is able to register with Consul, but unable to get config. -Controled by: - -- CLI parameters: `consul-cfg-enable, consul-addr, consul-deregister-critical-services-after` -- env variables: `CONSUL_REGISTER_ENABLE, CONSUL_ADDR, CONSUL_DEREGISTER_CRITICAL_SERVICES_AFTER` +- CLI parameters: `nacos-addr, nacos-username, nacos-password, nacos-namespace-id, nacos-group, nacos-dataid` +- env variables: `NACOS_ADDR, NACOS_USERNAME, NACOS_PASSWORD, NACOS_NAMESPACE_ID, NACOS_GROUP, NACOS_DATAID` ### Local Files Currently sinker is able to parse local config files at startup, but unable to detect file changes. Controled by: -- CLI parameters: `local-cfg-file, local-cfg-dir` -- env variables: `LOCAL_CFG_FILE, LOCAL_CFG_DIR` +- CLI parameters: `local-cfg-file` +- env variables: `LOCAL_CFG_FILE` ## Prometheus Metrics @@ -286,14 +271,6 @@ type Inputer interface { // RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... type RemoteConfManager interface { Init(properties map[string]interface{}) error - // Register this instance, and keep-alive via heartbeat. - Register(ip string, port int) error - Deregister(ip string, port int) error - // GetInstances fetchs healthy instances. - // Mature service-discovery solutions(Nacos, Consul etc.) have client side cache - // so that frequent invoking of GetInstances() and GetGlobalConfig() don't harm. - GetInstances() (instances []Instance, err error) - // GetConfig fetchs the config. The manager shall not reference the returned Config object after call. GetConfig() (conf *Config, err error) // PublishConfig publishs the config. The manager shall not reference the passed Config object after call. PublishConfig(conf *Config) (err error) diff --git a/docs/guide/install.md b/docs/guide/install.md index 506f0a23..60b68e9e 100644 --- a/docs/guide/install.md +++ b/docs/guide/install.md @@ -6,7 +6,7 @@ Download the binary files from [release](https://github.com/housepower/clickhous ## By container image -`docker run --volume conf:/etc/clickhouse_sinker quay.io/housepower/clickhouse_sinker` +`docker pull quay.io/housepower/clickhouse_sinker` ## By source diff --git a/docs/guide/run.md b/docs/guide/run.md index 56d1c401..48798b2b 100644 --- a/docs/guide/run.md +++ b/docs/guide/run.md @@ -2,96 +2,54 @@ ## Requirements -Note: we shall enable we have `clickhouse-server` and `kafka` envs, before running clickhouse_sinker. +Note: Ensure `clickhouse-server` and `kafka` work before running clickhouse_sinker. ## Configs -> There are two ways to pass the local config, multiple files and single file. +> There are two ways to get config: a local single config, or Nacos. -- For multiple files: +- For local file: - `./clickhouse_sinker --local-cfg-dir conf` + `clickhouse_sinker --local-cfg-file docker/test_auto_schema.json` - `conf` is the configuration directorys, and it'll read `conf/config.json` as main config and all tasks files in `conf/tasks/*.json` as task configs +- For Nacos: -- For single file: - - `./clickhouse_sinker --local-cfg-file config_single.json`. + `clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema` > Read more detail descriptions of config in [here](docs/config.md) ## Example -* Let's use single file to test `clickhouse_sinker` +Let's follow up a piece of the systest script. - - let's touch config.json - `touch config.json` +* Prepare - - put sample config into the config file + - let's checkout `clickhouse_sinker` + ``` + $ git clone https://github.com/housepower/clickhouse_sinker.git + $ cd clickhouse_sinker + ``` + - let's start standalone clickhouse-server and kafka in container: ``` - { - "clickhouse": { - "default": { - "db": "default", - "hosts": [ - [ - "127.0.0.1" - ] - ], - "port": 9000 - } - }, - "kafka": { - "default": { - "brokers": "127.0.0.1:9092", - "version": "2.2.1" - } - }, - "common": { - "bufferSize": 90000, - "minBufferSize": 1, - "msgSizeHint": 1000, - "flushInterval": 5, - "logLevel": "debug" - }, - "tasks": { - "logstash": { - "name" : "logstash", - "kafkaClient": "kafka-go", - "kafka": "default", - "topic": "logstash", - "consumerGroup" : "logstash_sinker", - "parser" : "json", - "clickhouse" : "default", - "tableName" : "logstash", - - "autoSchema" : true, - "@desc_of_exclude_columns" : "this columns will be excluded by insert SQL ", - "excludeColumns" : ["day"] - } - } - } + $ docker-compose up -d ``` * Create a simple table in Clickhouse - > It's not the duty for clickhouse_sinker to auto create table, so we should maually do that. + > It's not the duty for clickhouse_sinker to auto create table, so we should do that manually. ``` - CREATE TABLE logstash + CREATE TABLE IF NOT EXISTS test_auto_schema ( - `time` DateTime, `day` Date DEFAULT toDate(time), - `request_uri` String, - `age` UInt8 + `time` DateTime, + `name` String, + `value` Float64 ) - ENGINE = Memory - - Ok. - - 0 rows in set. Elapsed: 0.014 sec. - + ENGINE = MergeTree + PARTITION BY day + ORDER BY (time, name); ``` * Enable topic is created in kafka @@ -99,9 +57,9 @@ Note: we shall enable we have `clickhouse-server` and `kafka` envs, before runni > I use [kaf](https://github.com/birdayz/kaf) tool to create topics. ``` - kaf topic create logstash -p 1 -r 1 + kaf topic create topic1 -p 1 -r 1 ✅ Created topic! - Topic Name: logstash + Topic Name: topic1 Partitions: 1 Replication Factor: 1 Cleanup Policy: delete @@ -111,28 +69,21 @@ Note: we shall enable we have `clickhouse-server` and `kafka` envs, before runni * Run clickhouse_sinker ``` - ./clickhouse_sinker --local-cfg-file config.json + clickhouse_sinker --local-cfg-file docker/test_auto_schema.json ``` * Send messages to the topic ``` - echo '{"time" : "2020-12-18T03:38:39.000Z", "age" : 33 }' | kaf -b '127.0.0.1:9092' produce logstash - echo '{"time" : "2020-12-18T03:38:39.000Z", "age" : 33 }' | kaf -b '127.0.0.1:9092' produce logstash - echo '{"time" : "2020-12-18T03:38:39.000Z", "age" : 33 }' | kaf -b '127.0.0.1:9092' produce logstash + echo '{"time" : "2020-12-18T03:38:39.000Z", "name" : "name1", "value" : 1}' | kaf -b '127.0.0.1:9092' produce topic1 + echo '{"time" : "2020-12-18T03:38:39.000Z", "name" : "name2", "value" : 2}' | kaf -b '127.0.0.1:9092' produce topic1 + echo '{"time" : "2020-12-18T03:38:39.000Z", "name" : "name3", "value" : 3}' | kaf -b '127.0.0.1:9092' produce topic1 ``` - Check the data in clickhouse ``` - SELECT * - FROM logstash - - ┌────────────────time─┬────────day─┬─request_uri─┬─age─┐ - │ 2020-12-18 11:38:39 │ 2020-12-18 │ │ 33 │ - │ 2020-12-18 11:38:39 │ 2020-12-18 │ │ 33 │ - │ 2020-12-18 11:38:39 │ 2020-12-18 │ │ 33 │ - └─────────────────────┴────────────┴─────────────┴─────┘ + SELECT count() FROM test_auto_schema; 3 rows in set. Elapsed: 0.016 sec. From 54a2a9ff9a22502a17920d117d7fb2d70245bc9f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 12 Jan 2021 10:07:37 +0800 Subject: [PATCH 044/404] made gh-pages happy --- .github/workflows/gh-pages.yml | 2 +- docs/.vuepress/config.js | 11 +++++++---- docs/configuration/configuration.md | 1 - docs/configuration/flag.md | 3 +-- docs/guide/README.md | 1 - docs/guide/install_run.md | 1 - 6 files changed, 9 insertions(+), 10 deletions(-) delete mode 100644 docs/configuration/configuration.md delete mode 100644 docs/guide/README.md delete mode 100644 docs/guide/install_run.md diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index 25ee5331..e0ef4916 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -11,7 +11,7 @@ jobs: - uses: actions/checkout@v2 - uses: actions/setup-node@v1 with: - node-version: "14.x" + node-version: "15.x" - run: "[[ -z $(git show HEAD -- docs) ]] || npm install" - run: "[[ -z $(git show HEAD -- docs) ]] || npm run docs:build" - run: "[[ -z $(git show HEAD -- docs) ]] || dev/deploy_gh_pages.sh" diff --git a/docs/.vuepress/config.js b/docs/.vuepress/config.js index d34f4678..a6ffd6b0 100644 --- a/docs/.vuepress/config.js +++ b/docs/.vuepress/config.js @@ -55,18 +55,21 @@ module.exports = { children: [ ["flag", "Flag"], ["config", "Config"], - ["task", "Task"], ] } ], "/dev/": [ - ["introduction", "Introduction"], - ["design", "Design"], + { + title: "Development", + children: [ + ["introduction", "Introduction"], + ["design", "Design"], + ] + } ], }, } - }, }, }; diff --git a/docs/configuration/configuration.md b/docs/configuration/configuration.md deleted file mode 100644 index af4abbfe..00000000 --- a/docs/configuration/configuration.md +++ /dev/null @@ -1 +0,0 @@ -# Configuration \ No newline at end of file diff --git a/docs/configuration/flag.md b/docs/configuration/flag.md index 5df377a9..7c92417e 100644 --- a/docs/configuration/flag.md +++ b/docs/configuration/flag.md @@ -3,8 +3,7 @@ ``` ./clickhouse_sinker -h -Usage of ./dist/clickhouse_sinker: -Usage of dist/clickhouse_sinker: +Usage of ./clickhouse_sinker: -http-port int http listen port (default 2112) -local-cfg-file string diff --git a/docs/guide/README.md b/docs/guide/README.md deleted file mode 100644 index 7e0bdcaa..00000000 --- a/docs/guide/README.md +++ /dev/null @@ -1 +0,0 @@ -# Get Started diff --git a/docs/guide/install_run.md b/docs/guide/install_run.md deleted file mode 100644 index 648db68c..00000000 --- a/docs/guide/install_run.md +++ /dev/null @@ -1 +0,0 @@ -# Install and Run \ No newline at end of file From 8e8aa24e9a9141ac3dac212d78c5ce2bd159b61d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 12 Jan 2021 14:48:13 +0800 Subject: [PATCH 045/404] Batch.Commit is not retry-able --- docker-compose.yml | 11 +++-------- model/message.go | 1 + output/clickhouse.go | 32 ++++++++++++++------------------ task/task.go | 4 +--- 4 files changed, 19 insertions(+), 29 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index c11c48fa..7a229c9f 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -31,7 +31,7 @@ services: security_opt: - label:disable clickhouse: - image: yandex/clickhouse-server:19.16 + image: yandex/clickhouse-server:20.8 restart: always ports: - "8123:8123" @@ -45,7 +45,7 @@ services: security_opt: - label:disable tabix: - image: spoonest/clickhouse-tabix-web-client + image: spoonest/clickhouse-tabix-web-client:stable restart: always ports: - "8080:80" @@ -58,13 +58,8 @@ services: CH_PASSWORD: "" security_opt: - label:disable - consul: - image: consul:latest - restart: always - network_mode: host - command: "agent -dev -ui -client 0.0.0.0" nacos: - image: nacos/nacos-server:latest + image: nacos/nacos-server:1.4.0 container_name: nacos-standalone environment: - PREFER_HOST_MODE=hostname diff --git a/model/message.go b/model/message.go index 62c60c82..ccc849e2 100644 --- a/model/message.go +++ b/model/message.go @@ -129,6 +129,7 @@ func (b *Batch) Size() int { return len(*b.Rows) } +// Commit is not retry-able! func (b *Batch) Commit() error { for _, row := range *b.Rows { PutRow(row) diff --git a/output/clickhouse.go b/output/clickhouse.go index 09420d87..92a76d67 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -20,6 +20,7 @@ import ( "database/sql" std_errors "errors" "fmt" + "io" "os" "regexp" "strings" @@ -68,10 +69,10 @@ func (c *ClickHouse) Init() (err error) { } // Send a batch to clickhouse -func (c *ClickHouse) Send(batch *model.Batch, callback func(batch *model.Batch) error) { +func (c *ClickHouse) Send(batch *model.Batch) { statistics.WritingPoolBacklog.WithLabelValues(c.cfg.Task.Name).Inc() _ = util.GlobalWritingPool.Submit(func() { - c.loopWrite(batch, callback) + c.loopWrite(batch) statistics.WritingPoolBacklog.WithLabelValues(c.cfg.Task.Name).Dec() }) } @@ -130,27 +131,22 @@ func shouldReconnect(err error) bool { } // LoopWrite will dead loop to write the records -func (c *ClickHouse) loopWrite(batch *model.Batch, callback func(batch *model.Batch) error) { +func (c *ClickHouse) loopWrite(batch *model.Batch) { var err error var times int for { if err = c.write(batch); err == nil { - for { - if err = callback(batch); err == nil { - return - } - if std_errors.Is(err, context.Canceled) { - log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) - return - } - log.Errorf("%s: committing offset(try #%d) failed with error %+v", c.cfg.Task.Name, times, err) - times++ - if c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes { - time.Sleep(10 * time.Second) - } else { - os.Exit(-1) - } + if err = batch.Commit(); err == nil { + return } + // TODO: kafka_go and sarama commit give different error when context is cancceled. + // How to unify them? + if std_errors.Is(err, context.Canceled) || std_errors.Is(err, io.ErrClosedPipe) { + log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) + return + } + log.Errorf("%s: committing offset failed with permanent error %+v", c.cfg.Task.Name, err) + os.Exit(-1) } if std_errors.Is(err, context.Canceled) { log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) diff --git a/task/task.go b/task/task.go index bb7d85ab..9c39c4cd 100644 --- a/task/task.go +++ b/task/task.go @@ -220,9 +220,7 @@ func (service *Service) flush(batch *model.Batch) (err error) { if (len(*batch.Rows)) == 0 { return batch.Commit() } - service.clickhouse.Send(batch, func(batch *model.Batch) error { - return batch.Commit() - }) + service.clickhouse.Send(batch) return nil } From 9198ffa4bec667409d6393c0095dc53fc17fe501 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 12 Jan 2021 14:46:29 +0800 Subject: [PATCH 046/404] dynamic schema --- config/config.go | 7 ++ docker/test_dynamic_schema.json | 37 ++++++++++ go.test.sh | 48 +++++++++---- model/metric.go | 3 + output/clickhouse.go | 64 +++++++++++++++++ parser/csv.go | 5 ++ parser/dummy.go | 9 ++- parser/fastjson.go | 33 +++++++++ parser/gjson.go | 5 ++ task/task.go | 117 +++++++++++++++++++++++++------- 10 files changed, 291 insertions(+), 37 deletions(-) create mode 100644 docker/test_dynamic_schema.json diff --git a/config/config.go b/config/config.go index fedfa26e..7e6f2c70 100644 --- a/config/config.go +++ b/config/config.go @@ -117,6 +117,13 @@ type TaskConfig struct { Type string SourceName string } `json:"dims"` + // DynamicSchema will add columns present in message to clickhouse. Requires AutoSchema be true. + DynamicSchema struct { + Enable bool + Cluster string + DistTableName string + MaxDims int // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack + } // ShardingKey is the column name to which sharding against ShardingKey string `json:"shardingKey,omitempty"` diff --git a/docker/test_dynamic_schema.json b/docker/test_dynamic_schema.json new file mode 100644 index 00000000..6c2e6596 --- /dev/null +++ b/docker/test_dynamic_schema.json @@ -0,0 +1,37 @@ +{ + "clickhouse": { + "hosts": [ + [ + "127.0.0.1" + ] + ], + "port": 9000, + "db": "default", + "username": "", + "password": "", + "retryTimes": 0 + }, + "kafka": { + "brokers": "127.0.0.1:9093", + "version": "2.2.1" + }, + "task": { + "name": "test_dynamic_schema", + "topic": "topic1", + "consumerGroup": "test_dynamic_schema", + "earliest": true, + "parser": "json", + "autoSchema": true, + "tableName": "test_dynamic_schema", + "excludeColumns": [ + "day" + ], + "dynamicSchema": { + "enable": true, + "cluster": "", + "distTableName": "" + }, + "bufferSize": 50000 + }, + "logLevel": "debug" +} \ No newline at end of file diff --git a/go.test.sh b/go.test.sh index 00c2c9b4..75dd69cf 100755 --- a/go.test.sh +++ b/go.test.sh @@ -2,7 +2,7 @@ ## create table curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_fixed_schema' -curl "localhost:8123" -d 'CREATE TABLE IF NOT EXISTS test_fixed_schema +curl "localhost:8123" -d 'CREATE TABLE test_fixed_schema ( `day` Date DEFAULT toDate(time), `time` DateTime, @@ -14,22 +14,28 @@ PARTITION BY day ORDER BY (time, name)' curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_auto_schema' -curl "localhost:8123" -d 'CREATE TABLE IF NOT EXISTS test_auto_schema -( - `day` Date DEFAULT toDate(time), - `time` DateTime, - `name` String, - `value` Float64 -) -ENGINE = MergeTree -PARTITION BY day -ORDER BY (time, name)' +curl "localhost:8123" -d 'CREATE TABLE test_auto_schema AS test_fixed_schema' + +curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_dynamic_schema' +curl "localhost:8123" -d 'CREATE TABLE test_dynamic_schema AS test_fixed_schema' ## send the messages to kafka now=`date --rfc-3339=ns` -for i in `seq 1 100000`;do - echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : \"$i\" }" +for i in `seq 1 10000`;do + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i }" done > a.json +for i in `seq 10001 30000`;do + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey1\" : $i }" +done >> a.json +for i in `seq 30001 50000`;do + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey2\" : $i.123 }" +done >> a.json +for i in `seq 50001 70000`;do + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey3\" : \"name$i\" }" +done >> a.json +for i in `seq 70001 100000`;do + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i }" +done >> a.json echo "generated a.json" echo "cat /tmp/a.json | kafka-console-producer --topic topic1 --broker-list localhost:9092" > send.sh sudo docker cp a.json kafka:/tmp/ @@ -39,6 +45,7 @@ sudo docker exec kafka sh /tmp/send.sh ## start clickhouse_sinker to consume timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/test_fixed_schema.json timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/test_auto_schema.json +timeout 60 ./dist/clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.json ## check result count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` @@ -49,22 +56,33 @@ count=`curl "localhost:8123" -d 'select count() from test_auto_schema'` echo "Got test_auto_schema count => $count" [ $count -eq 100000 ] || exit 1 +schema=`curl "localhost:8123" -d 'DESC test_dynamic_schema' 2>/dev/null | grep newkey | sort | tr -d '\t' | tr '\n' ','` +echo "Got test_dynamic_schema schema => $schema" +[ $schema = "newkey1Nullable(Int64),newkey2Nullable(Float64),newkey3Nullable(String)," ] || exit 1 +count=`curl "localhost:8123" -d 'SELECT count() FROM test_dynamic_schema'` +echo "Got test_dynamic_schema count => $count" +[ $count -eq 100000 ] || exit 1 + ## reset kafka consumer-group offsets sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_fixed_schema --all-topics --to-earliest sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_auto_schema --all-topics --to-earliest +sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_dynamic_schema --all-topics --to-earliest ## truncate tables curl "localhost:8123" -d 'TRUNCATE TABLE test_fixed_schema' curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' +curl "localhost:8123" -d 'TRUNCATE TABLE test_dynamic_schema' ## publish clickhouse_sinker config ./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_fixed_schema.json ./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_auto_schema.json +./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_dynamic_schema.json ## start clickhouse_sinker to consume timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema +timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema ## check result count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` @@ -74,3 +92,7 @@ echo "Got test_fixed_schema count => $count" count=`curl "localhost:8123" -d 'select count() from test_auto_schema'` echo "Got test_auto_schema count => $count" [ $count -eq 100000 ] || exit 1 + +count=`curl "localhost:8123" -d 'SELECT count() FROM test_dynamic_schema'` +echo "Got test_dynamic_schema count => $count" +[ $count -eq 100000 ] || exit 1 diff --git a/model/metric.go b/model/metric.go index 8a8aa22f..fb4c7234 100644 --- a/model/metric.go +++ b/model/metric.go @@ -15,6 +15,8 @@ limitations under the License. package model +import "sync" + // Metric interface for metric collection type Metric interface { Get(key string) interface{} @@ -26,6 +28,7 @@ type Metric interface { GetDateTime(key string, nullable bool) interface{} GetDateTime64(key string, nullable bool) interface{} GetElasticDateTime(key string, nullable bool) interface{} + GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool } // DimMetrics diff --git a/output/clickhouse.go b/output/clickhouse.go index 92a76d67..00e864db 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -21,9 +21,11 @@ import ( std_errors "errors" "fmt" "io" + "math" "os" "regexp" "strings" + "sync" "time" "github.com/housepower/clickhouse_sinker/config" @@ -218,3 +220,65 @@ func (c *ClickHouse) initSchema() (err error) { log.Infof("%s: Prepare sql=> %s", c.cfg.Task.Name, c.prepareSQL) return nil } + +func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { + var sqls []string + var onCluster string + taskCfg := &c.cfg.Task + chCfg := &c.cfg.Clickhouse + if taskCfg.DynamicSchema.Cluster != "" { + onCluster = fmt.Sprintf("ON CLUSTER %s", taskCfg.DynamicSchema.Cluster) + } + maxDims := math.MaxInt16 + if taskCfg.DynamicSchema.MaxDims > 0 { + maxDims = taskCfg.DynamicSchema.MaxDims + } + newKeysQuota := maxDims - len(c.Dims) + if newKeysQuota <= 0 { + log.Warnf("number of columns reaches upper limit %d", maxDims) + return + } + var i int + newKeys.Range(func(key, value interface{}) bool { + i++ + if i > newKeysQuota { + log.Warnf("number of columns reaches upper limit %d", maxDims) + return false + } + strKey := key.(string) + strVal := value.(string) + switch strVal { + case "int": + strVal = "Nullable(Int64)" + case "float": + strVal = "Nullable(Float64)" + case "string": + strVal = "Nullable(String)" + default: + err = errors.Errorf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) + return false + } + sql := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS %s %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) + sqls = append(sqls, sql) + return true + }) + if err != nil { + return + } + if taskCfg.DynamicSchema.DistTableName != "" { + sqls = append(sqls, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s %s", chCfg.DB, taskCfg.DynamicSchema.DistTableName, onCluster)) + sqls = append(sqls, fmt.Sprintf("CREATE TABLE %s.%s %s AS %s ENGINE = Distributed(%s, %s, %s);", + chCfg.DB, taskCfg.DynamicSchema.DistTableName, onCluster, taskCfg.TableName, + taskCfg.DynamicSchema.Cluster, chCfg.DB, taskCfg.TableName)) + } + + conn := pool.GetConn(0) + for _, sql := range sqls { + log.Infof("%s: executing sql=> %s", taskCfg.Name, sql) + if _, err = conn.Exec(sql); err != nil { + err = errors.Wrapf(err, sql) + return err + } + } + return +} diff --git a/parser/csv.go b/parser/csv.go index 815b758c..77c1eede 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -19,6 +19,7 @@ import ( "encoding/csv" "strconv" "strings" + "sync" "time" "github.com/housepower/clickhouse_sinker/model" @@ -177,3 +178,7 @@ func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) interface{} { return t.Unix() } + +func (c *CsvMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool { + return false +} diff --git a/parser/dummy.go b/parser/dummy.go index 4d6939ed..952cb628 100644 --- a/parser/dummy.go +++ b/parser/dummy.go @@ -15,7 +15,10 @@ limitations under the License. package parser -import "time" +import ( + "sync" + "time" +) type DummyMetric struct { } @@ -60,3 +63,7 @@ func (c *DummyMetric) GetDateTime64(key string) (t time.Time) { func (c *DummyMetric) GetElasticDateTime(key string) int64 { return 0 } + +func (c *DummyMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool { + return false +} diff --git a/parser/fastjson.go b/parser/fastjson.go index 497eebb1..74ef2a34 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -16,10 +16,13 @@ limitations under the License. package parser import ( + "fmt" + "sync" "time" "github.com/housepower/clickhouse_sinker/model" "github.com/pkg/errors" + log "github.com/sirupsen/logrus" "github.com/valyala/fastjson" ) @@ -164,3 +167,33 @@ func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) interface return t.Unix() } + +func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (foundNew bool) { + var obj *fastjson.Object + var err error + if obj, err = c.value.Object(); err != nil { + return + } + obj.Visit(func(key []byte, v *fastjson.Value) { + strKey := string(key) + if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { + v := c.value.Get(strKey) + if v == nil { + panic(fmt.Sprintf("BUG: fastjson.Object.Visit got an unexpected key: %s", strKey)) + } + if _, err = v.Int64(); err == nil { + newKeys.Store(strKey, "int") + foundNew = true + } else if _, err = v.Float64(); err == nil { + newKeys.Store(strKey, "float") + foundNew = true + } else if _, err = v.StringBytes(); err == nil { + newKeys.Store(strKey, "string") + foundNew = true + } else { + log.Warnf("FastjsonMetric.GetNewKeys found a kv not be int/float/string, key: %s, value: %s", strKey, v.String()) + } + } + }) + return +} diff --git a/parser/gjson.go b/parser/gjson.go index 00bdcb85..6c41f194 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -16,6 +16,7 @@ limitations under the License. package parser import ( + "sync" "time" "github.com/tidwall/gjson" @@ -148,3 +149,7 @@ func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) interface{} t, _ := time.Parse(time.RFC3339, r.String()) return t.Unix() } + +func (c *GjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool { + return false +} diff --git a/task/task.go b/task/task.go index 9c39c4cd..f4b30ce2 100644 --- a/task/task.go +++ b/task/task.go @@ -18,12 +18,16 @@ package task import ( "context" "fmt" + "math" + "os" "runtime" "strconv" "strings" "sync" + "sync/atomic" "time" + "github.com/fagongzi/goetty" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/input" "github.com/housepower/clickhouse_sinker/model" @@ -41,6 +45,7 @@ import ( type Service struct { sync.Mutex + parentCtx context.Context ctx context.Context cancel context.CancelFunc started bool @@ -51,6 +56,11 @@ type Service struct { cfg *config.Config dims []*model.ColumnWithType + knownKeys sync.Map + newKeys sync.Map + cntNewKeys int32 // size of newKeys + tid goetty.Timeout + rings []*Ring sharder *Sharder batchChan chan *model.Batch @@ -84,13 +94,37 @@ func (service *Service) Init() (err error) { service.limiter2 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter3 = rate.NewLimiter(rate.Every(10*time.Second), 1) + service.rings = make([]*Ring, 0) if service.cfg.Task.ShardingKey != "" { if service.sharder, err = NewSharder(service); err != nil { return } } - err = service.inputer.Init(service.cfg, service.cfg.Task.Name, service.put) + if err = service.inputer.Init(service.cfg, service.cfg.Task.Name, service.put); err != nil { + return + } + + taskCfg := &service.cfg.Task + if taskCfg.DynamicSchema.Enable { + maxDims := math.MaxInt16 + if service.cfg.Task.DynamicSchema.MaxDims > 0 { + maxDims = taskCfg.DynamicSchema.MaxDims + } + if maxDims <= len(service.dims) { + service.cfg.Task.DynamicSchema.Enable = false + log.Warnf("%s: disabled DynamicSchema since the number of columns reaches upper limit %d", taskCfg.Name, maxDims) + } else { + for _, dim := range service.dims { + service.knownKeys.Store(dim.SourceName, nil) + } + for _, dim := range taskCfg.ExcludeColumns { + service.knownKeys.Store(dim, nil) + } + service.newKeys = sync.Map{} + atomic.StoreInt32(&service.cntNewKeys, 0) + } + } return } @@ -98,6 +132,7 @@ func (service *Service) Init() (err error) { func (service *Service) Run(ctx context.Context) { var err error service.started = true + service.parentCtx = ctx service.ctx, service.cancel = context.WithCancel(ctx) log.Infof("%s: task started", service.cfg.Task.Name) go service.inputer.Run(service.ctx) @@ -129,7 +164,8 @@ func (service *Service) fnCommit(partition int, offset int64) error { } func (service *Service) put(msg model.InputMessage) { - statistics.ConsumeMsgsTotal.WithLabelValues(service.cfg.Task.Name).Inc() + taskCfg := &service.cfg.Task + statistics.ConsumeMsgsTotal.WithLabelValues(taskCfg.Name).Inc() // ensure ring for this message exist service.Lock() var ring *Ring @@ -143,7 +179,7 @@ func (service *Service) put(msg model.InputMessage) { var err error if ring == nil { - batchSizeShift := util.GetShift(service.cfg.Task.BufferSize) + batchSizeShift := util.GetShift(taskCfg.BufferSize) ringCap := int64(1 << (batchSizeShift + 1)) ring := &Ring{ ringBuf: make([]model.MsgRow, ringCap), @@ -155,13 +191,13 @@ func (service *Service) put(msg model.InputMessage) { idleCnt: 0, isIdle: false, partition: msg.Partition, - batchSys: model.NewBatchSys(&service.cfg.Task, service.fnCommit), + batchSys: model.NewBatchSys(taskCfg, service.fnCommit), service: service, } // schedule a delayed ForceBatchOrShard - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(service.cfg.Task.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { + if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", service.cfg.Task.Name, err) + log.Fatalf("%s: got error %+v", taskCfg.Name, err) } service.rings[msg.Partition] = ring service.Unlock() @@ -172,18 +208,18 @@ func (service *Service) put(msg model.InputMessage) { ringGroundOff, ringFilledOffset = ring.ringGroundOff, ring.ringFilledOffset ring.mux.Unlock() if msg.Offset < ringFilledOffset { - statistics.RingMsgsOffTooSmallErrorTotal.WithLabelValues(service.cfg.Task.Name).Inc() + statistics.RingMsgsOffTooSmallErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter2.Allow() { log.Warnf("%s: got a message(topic %v, partition %d, offset %v) left to %v", - service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, ringFilledOffset) + taskCfg.Name, msg.Topic, msg.Partition, msg.Offset, ringFilledOffset) } return } - if msg.Offset >= ringGroundOff+ring.ringCap { + if msg.Offset >= ringGroundOff+ring.ringCap && atomic.LoadInt32(&service.cntNewKeys) == 0 { statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(service.cfg.Task.Name).Inc() if service.limiter3.Allow() { log.Warnf("%s: got a message(topic %v, partition %d, offset %v) right to the range [%v, %v)", - service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap) + taskCfg.Name, msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap) } time.Sleep(1 * time.Second) ring.ForceBatchOrShard(&msg) @@ -191,13 +227,13 @@ func (service *Service) put(msg model.InputMessage) { } // submit message to a goroutine pool - statistics.ParsingPoolBacklog.WithLabelValues(service.cfg.Task.Name).Inc() + statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Inc() _ = util.GlobalParsingPool.Submit(func() { var row *model.Row p := service.pp.Get() metric, err := p.Parse(msg.Value) if err != nil { - statistics.ParseMsgsErrorTotal.WithLabelValues(service.cfg.Task.Name).Inc() + statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter1.Allow() { log.Errorf("%s: failed to parse message(topic %v, partition %d, offset %v) %+v, string(value) <<<%+v>>>, got error %+v", service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, msg, string(msg.Value), err) @@ -205,14 +241,37 @@ func (service *Service) put(msg model.InputMessage) { } else { row = model.MetricToRow(metric, msg, service.dims) } - service.pp.Put(p) - var ring *Ring - service.Lock() - ring = service.rings[msg.Partition] - service.Unlock() - ring.PutElem(model.MsgRow{Msg: &msg, Row: row}) - statistics.ParsingPoolBacklog.WithLabelValues(service.cfg.Task.Name).Dec() + + if taskCfg.DynamicSchema.Enable { + found := metric.GetNewKeys(&service.knownKeys, &service.newKeys) + if found { + cntNewKeys := atomic.AddInt32(&service.cntNewKeys, 1) + if cntNewKeys == 1 { + // The first message which contains new keys triggers flushing + // all messages and scheduling a delayed func to apply schema change. + for _, ring := range service.rings { + if ring != nil { + ring.ForceBatchOrShard(nil) + } + } + if service.sharder != nil { + service.sharder.ForceFlush(nil) + } + if service.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.changeSchema, nil); err != nil { + log.Fatalf("got error %+v", err) + os.Exit(-1) + } + } + } + } + if atomic.LoadInt32(&service.cntNewKeys) == 0 { + var ring *Ring + service.Lock() + ring = service.rings[msg.Partition] + service.Unlock() + ring.PutElem(model.MsgRow{Msg: &msg, Row: row}) + } }) } @@ -224,10 +283,21 @@ func (service *Service) flush(batch *model.Batch) (err error) { return nil } -// NotifyStop notify task to stop, This is non-blocking. -func (service *Service) NotifyStop() { - log.Infof("%s: notified to stop", service.cfg.Task.Name) - service.cancel() +func (service *Service) changeSchema(arg interface{}) { + var err error + taskCfg := &service.cfg.Task + // change schema + if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { + log.Fatalf("%s: clickhouse.ChangeSchema failed with error: %+v", taskCfg.Name, err) + os.Exit(-1) + } + // restart myself + service.Stop() + if err = service.Init(); err != nil { + log.Fatalf("%s: init failed with error: %+v", taskCfg.Name, err) + os.Exit(-1) + } + go service.Run(service.parentCtx) } // Stop stop kafka and clickhouse client. This is blocking. @@ -250,6 +320,7 @@ func (service *Service) Stop() { ring.tid.Stop() } } + service.tid.Stop() log.Infof("%s: stopped internal timers", service.cfg.Task.Name) if service.started { From 24c152595dcd08a684bc158a7b616a917bda24a6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 13 Jan 2021 15:24:19 +0800 Subject: [PATCH 047/404] doc change --- docs/configuration/config.md | 34 +++++++++++++++++++++++++++++----- docs/dev/introduction.md | 6 +++--- 2 files changed, 32 insertions(+), 8 deletions(-) diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 983e5d9e..131eb1f5 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -1,7 +1,6 @@ -# basic configs +# Config Items > Here we use json with comments for documentation - ``` { // clickhouse configs, it's map[string]ClickHouse for multiple clickhouse @@ -26,12 +25,32 @@ "kafka": { "brokers": "127.0.0.1:9093", - // somethings about sasl + // SSL + "tls": { + "enable": false, + // Required. It's the CA certificate with which Kafka brokers certs be signed. + "caCertFiles": "/etc/security/ca-cert", + // Required if Kafka brokers require client authentication. + clientCertFile: "", + // Required if and only if ClientCertFile is present. + clientKeyFile: "", + // Whether disable broker FQDN verification. Set it to `true` if kafka-console-consumer.sh uses `ssl.endpoint.identification.algorithm=`. + "insecureSkipVerify": true + } + + // SASL "sasl": { "enable": false, - "password": "", + // Mechanism is the name of the enabled SASL mechanism. + // Possible values: PLAIN, SCRAM-SHA-256, SCRAM-SHA-512, GSSAPI (defaults to PLAIN) + "mechanism": "PLAIN", + // Username is the authentication identity (authcid) to present for + // SASL/PLAIN or SASL/SCRAM authentication "username": "", + // Password for SASL/PLAIN or SASL/SCRAM authentication + "password": "", "gssapi": { + // authtype - 1. KRB5_USER_AUTH, 2. KRB5_KEYTAB_AUTH "authtype": 0, "keytabpath": "", "kerberosconfigpath": "", @@ -77,6 +96,11 @@ // "this columns will be excluded by insert SQL " "excludeColumns": [] + // shardingKey is the column name to which sharding against + "shardingKey": "", + // shardingPolicy is `stripe,`(requires ShardingKey be numerical) or `hash`(requires ShardingKey be string) + "shardingPolicy": "", + // batch size to insert into clickhouse "bufferSize": 90000, // min batch size to insert into clickhouse @@ -93,4 +117,4 @@ "logLevel": "debug" } } -``` \ No newline at end of file +``` diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 7615f140..806f8ff4 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -37,7 +37,7 @@ Refers to [design](./design.md) for how it works. ## Configuration -Refers to how [integration test](./go.test.sh) use the example config. Also refers to [code](./config/config.go) for all config items. +Refers to how [integration test](https://github.com/housepower/clickhouse_sinker/blob/master/go.test.sh) use the example config. Also refers to [code](https://github.com/housepower/clickhouse_sinker/blob/master/config/config.go) for all config items. ### Kafka Encryption @@ -75,13 +75,13 @@ An example kafka config: FYI. `kafka-console-consumer.sh` works as the following setup: ``` -$ cat config/SSL_NOAUTH_client.properties +$ cat config/client_SSL_NOAUTH.properties security.protocol=SSL ssl.truststore.location=/etc/security/kafka.client.truststore.jks ssl.truststore.password=123456 ssl.endpoint.identification.algorithm= -$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/SSL_NOAUTH_client.properties +$ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic sunshine --group test-consumer-group --from-beginning --consumer.config config/client_SSL_NOAUTH.properties ``` Please follow [`Kafka SSL setup`](https://kafka.apache.org/documentation/#security_ssl). Use `-keyalg RSA` when you create the broker keystore, otherwise there will be no cipher suites in common between the keystore and those Golang supports. See [this](https://github.com/Shopify/sarama/issues/643#issuecomment-216839760) for reference. From 44d672cb48d6b4db01f27e4a6f58cc598e9d9736 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 15 Jan 2021 11:20:26 +0800 Subject: [PATCH 048/404] ClickHouse SetMaxIdleConns zero --- pool/conn.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pool/conn.go b/pool/conn.go index 58bf9b69..af1655b4 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -98,10 +98,12 @@ func InitConn(hosts [][]string, port int, db, username, password, dsnParams stri return } -// TODO, pool this +// TODO: ClickHouse creates a thread for each TCP/HTTP connection. +// If the number of sinkers is close to clickhouse max_concurrent_queries(default 100), user queries could be blocked or refused. func setDBParams(sqlDB *sql.DB) { - sqlDB.SetMaxIdleConns(1) - sqlDB.SetConnMaxLifetime(120 * time.Second) + sqlDB.SetMaxOpenConns(1) + sqlDB.SetMaxIdleConns(0) + sqlDB.SetConnMaxIdleTime(10 * time.Second) } func FreeConn() { From 0030d9c956a735ef352194613934b95e8b81ab66 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 15 Jan 2021 13:44:07 +0800 Subject: [PATCH 049/404] detect basic config errors --- config/config.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/config/config.go b/config/config.go index 7e6f2c70..1d2995c1 100644 --- a/config/config.go +++ b/config/config.go @@ -167,6 +167,10 @@ func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { // normallize and validate configuration func (cfg *Config) Normallize() (err error) { + if len(cfg.Clickhouse.Hosts) == 0 || cfg.Kafka.Brokers == "" || cfg.Task.Name == "" { + err = errors.Errorf("invalid configuration") + return + } if cfg.Kafka.Version == "" { cfg.Kafka.Version = "2.2.1" } From f902968e84361893fdfb844d92da2ea8a22dd0b2 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 19 Jan 2021 15:33:26 +0800 Subject: [PATCH 050/404] introduced TaskCfg.DynamicSchema.DistTblPrefix --- config/config.go | 6 +++++- docker/test_dynamic_schema.json | 2 +- output/clickhouse.go | 8 ++++---- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/config/config.go b/config/config.go index 1d2995c1..9765a839 100644 --- a/config/config.go +++ b/config/config.go @@ -121,7 +121,7 @@ type TaskConfig struct { DynamicSchema struct { Enable bool Cluster string - DistTableName string + DistTblPrefix string MaxDims int // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack } @@ -147,6 +147,7 @@ const ( defaultLayoutDate = "2006-01-02" defaultLayoutDateTime = time.RFC3339 defaultLayoutDateTime64 = time.RFC3339 + defaultDistTblPrefix = "dist_" defaultLogLevel = "info" ) @@ -228,6 +229,9 @@ func (cfg *Config) Normallize() (err error) { if cfg.Task.LayoutDateTime64 == "" { cfg.Task.LayoutDateTime64 = defaultLayoutDateTime64 } + if cfg.Task.DynamicSchema.Enable && cfg.Task.DynamicSchema.DistTblPrefix == "" { + cfg.Task.DynamicSchema.DistTblPrefix = defaultDistTblPrefix + } switch strings.ToLower(cfg.LogLevel) { case "panic", "fatal", "error", "warn", "warning", "info", "debug", "trace": default: diff --git a/docker/test_dynamic_schema.json b/docker/test_dynamic_schema.json index 6c2e6596..12fa59ff 100644 --- a/docker/test_dynamic_schema.json +++ b/docker/test_dynamic_schema.json @@ -29,7 +29,7 @@ "dynamicSchema": { "enable": true, "cluster": "", - "distTableName": "" + "distTblPrefix": "" }, "bufferSize": 50000 }, diff --git a/output/clickhouse.go b/output/clickhouse.go index 00e864db..dd38ca48 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -265,13 +265,13 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { if err != nil { return } - if taskCfg.DynamicSchema.DistTableName != "" { - sqls = append(sqls, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s %s", chCfg.DB, taskCfg.DynamicSchema.DistTableName, onCluster)) + if taskCfg.DynamicSchema.Cluster != "" { + distTableName := taskCfg.DynamicSchema.DistTblPrefix + taskCfg.TableName + sqls = append(sqls, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s %s", chCfg.DB, distTableName, onCluster)) sqls = append(sqls, fmt.Sprintf("CREATE TABLE %s.%s %s AS %s ENGINE = Distributed(%s, %s, %s);", - chCfg.DB, taskCfg.DynamicSchema.DistTableName, onCluster, taskCfg.TableName, + chCfg.DB, distTableName, onCluster, taskCfg.TableName, taskCfg.DynamicSchema.Cluster, chCfg.DB, taskCfg.TableName)) } - conn := pool.GetConn(0) for _, sql := range sqls { log.Infof("%s: executing sql=> %s", taskCfg.Name, sql) From 82d5823eeb80d74a61bb4fd32aba142af201970f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 19 Jan 2021 17:48:56 +0800 Subject: [PATCH 051/404] extract cert and key from JKS --- config/config.go | 15 +++++++---- docs/configuration/config.md | 2 -- docs/dev/introduction.md | 21 +++++++++++++-- input/kafka_go.go | 12 ++++++++- input/kafka_sarama.go | 12 ++++++++- util/common.go | 52 +++++++++++++++++++++++++++++++++++- util/common_test.go | 24 +++++++++++++++++ 7 files changed, 126 insertions(+), 12 deletions(-) create mode 100644 util/common_test.go diff --git a/config/config.go b/config/config.go index 9765a839..e2d13103 100644 --- a/config/config.go +++ b/config/config.go @@ -48,11 +48,16 @@ type KafkaConfig struct { Brokers string Version string TLS struct { - Enable bool - CaCertFiles string // Required. It's the CA certificate with which Kafka brokers certs be signed. - ClientCertFile string // Required if Kafka brokers require client authentication. - ClientKeyFile string // Required if and only if ClientCertFile is present. - InsecureSkipVerify bool // Whether disable broker FQDN verification. + Enable bool + CaCertFiles string // Required. It's the CA cert.pem with which Kafka brokers certs be signed. + ClientCertFile string // Required for client authentication. It's client cert.pem. + ClientKeyFile string // Required if and only if ClientCertFile is present. It's client key.pem. + + TrustStoreLocation string //JKS format of CA certificate, used to extract CA cert.pem. + TrustStorePassword string + KeystoreLocation string //JKS format of client certificate and key, used to extrace client cert.pem and key.pem. + KeystorePassword string + EndpIdentAlgo string } //simplified sarama.Config.Net.SASL to only support SASL/PLAIN and SASL/GSSAPI(Kerberos) Sasl struct { diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 131eb1f5..5873afb8 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -34,8 +34,6 @@ clientCertFile: "", // Required if and only if ClientCertFile is present. clientKeyFile: "", - // Whether disable broker FQDN verification. Set it to `true` if kafka-console-consumer.sh uses `ssl.endpoint.identification.algorithm=`. - "insecureSkipVerify": true } // SASL diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 806f8ff4..b52492d3 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -58,6 +58,23 @@ An example kafka config: - Encryption using SSL An example kafka config: +``` + "kafka": { + "brokers": "192.168.31.64:9093", + "version": "2.2.1", + "tls": { + "enable": true, + "@trustStoreLocation": "ssl.truststore.location which kafka-console-consumer.sh uses", + "trustStoreLocation": "/etc/security/kafka.client.truststore.jks", + "@trustStorePassword": "ssl.truststore.password which kafka-console-consumer.sh uses", + "trustStorePassword": "123456", + "@endpIdentAlgo": "ssl.endpoint.identification.algorithm which kafka-console-consumer.sh uses", + "endpIdentAlgo": "" + } + } +``` + +Or if you have extracted certificates from JKS, use the following config: ``` "kafka": { "brokers": "192.168.31.64:9093", @@ -66,8 +83,8 @@ An example kafka config: "enable": true, "@caCertFiles": "Required. It's the CA certificate with which Kafka brokers certs be signed. This cert is added to kafka.client.truststore.jks which kafka-console-consumer.sh uses", "caCertFiles": "/etc/security/ca-cert", - "@insecureSkipVerify": "Whether disable broker FQDN verification. Set it to `true` if kafka-console-consumer.sh uses `ssl.endpoint.identification.algorithm=`.", - "insecureSkipVerify": true + "@endpIdentAlgo": "ssl.endpoint.identification.algorithm which kafka-console-consumer.sh uses", + "endpIdentAlgo": "" } } ``` diff --git a/input/kafka_go.go b/input/kafka_go.go index 66ded4c7..af6f6e1e 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -70,10 +70,20 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model CommitInterval: time.Second, // flushes commits to Kafka every second ErrorLogger: log.StandardLogger(), //kafka-go INFO log is too verbose } + if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { + if kfkCfg.TLS.CaCertFiles, _, err = util.JksToPem(kfkCfg.TLS.TrustStoreLocation, kfkCfg.TLS.TrustStorePassword, false); err != nil { + return + } + } + if kfkCfg.TLS.ClientKeyFile == "" && kfkCfg.TLS.KeystoreLocation != "" { + if kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, err = util.JksToPem(kfkCfg.TLS.KeystoreLocation, kfkCfg.TLS.KeystorePassword, false); err != nil { + return + } + } var dialer *kafka.Dialer if kfkCfg.TLS.Enable { var tlsConfig *tls.Config - if tlsConfig, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.InsecureSkipVerify); err != nil { + if tlsConfig, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.EndpIdentAlgo == ""); err != nil { return } dialer = &kafka.Dialer{ diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 295bb2d9..37f5f57e 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -91,9 +91,19 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg m return } sarama.Logger = log.StandardLogger() + if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { + if kfkCfg.TLS.CaCertFiles, _, err = util.JksToPem(kfkCfg.TLS.TrustStoreLocation, kfkCfg.TLS.TrustStorePassword, false); err != nil { + return + } + } + if kfkCfg.TLS.ClientKeyFile == "" && kfkCfg.TLS.KeystoreLocation != "" { + if kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, err = util.JksToPem(kfkCfg.TLS.KeystoreLocation, kfkCfg.TLS.KeystorePassword, false); err != nil { + return + } + } if kfkCfg.TLS.Enable { config.Net.TLS.Enable = true - if config.Net.TLS.Config, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.InsecureSkipVerify); err != nil { + if config.Net.TLS.Config, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.EndpIdentAlgo == ""); err != nil { return } } diff --git a/util/common.go b/util/common.go index 16be5669..b26930a1 100644 --- a/util/common.go +++ b/util/common.go @@ -16,18 +16,22 @@ limitations under the License. package util import ( + "bytes" "crypto/tls" "crypto/x509" "fmt" "io/ioutil" - "log" "net" "os" + "os/exec" + "path/filepath" "runtime" "strconv" "strings" "time" + log "github.com/sirupsen/logrus" + "github.com/fagongzi/goetty" "github.com/pkg/errors" ) @@ -163,3 +167,49 @@ func EnvBoolVar(value *bool, key string) { *value = true } } + +// JksToPem converts JKS to PEM +// Refers to: +// https://serverfault.com/questions/715827/how-to-generate-key-and-crt-file-from-jks-file-for-httpd-apache-server +func JksToPem(jksPath, jksPassword string, overwrite bool) (certPemPath, keyPemPath string, err error) { + dir, fn := filepath.Split(jksPath) + certPemPath = filepath.Join(dir, fn+".cert.pem") + keyPemPath = filepath.Join(dir, fn+".key.pem") + pkcs12Path := filepath.Join(dir, fn+".p12") + if overwrite { + for _, fp := range []string{certPemPath, keyPemPath, pkcs12Path} { + if err = os.RemoveAll(fp); err != nil { + err = errors.Wrapf(err, "") + return + } + } + } else { + for _, fp := range []string{certPemPath, keyPemPath, pkcs12Path} { + if _, err = os.Stat(fp); err == nil { + return + } + } + } + cmds := [][]string{ + {"keytool", "-importkeystore", "-srckeystore", jksPath, "-destkeystore", pkcs12Path, "-deststoretype", "PKCS12"}, + {"openssl", "pkcs12", "-in", pkcs12Path, "-nokeys", "-out", certPemPath, "-passin", "env:password"}, + {"openssl", "pkcs12", "-in", pkcs12Path, "-nodes", "-nocerts", "-out", keyPemPath, "-passin", "env:password"}, + } + for _, cmd := range cmds { + log.Infof(strings.Join(cmd, " ")) + exe := exec.Command(cmd[0], cmd[1:]...) + if cmd[0] == "keytool" { + exe.Stdin = bytes.NewReader([]byte(jksPassword + "\n" + jksPassword + "\n" + jksPassword)) + } else if cmd[0] == "openssl" { + exe.Env = []string{fmt.Sprintf("password=%s", jksPassword)} + } + var out []byte + out, err = exe.CombinedOutput() + log.Infof(string(out)) + if err != nil { + err = errors.Wrapf(err, "") + return + } + } + return +} diff --git a/util/common_test.go b/util/common_test.go new file mode 100644 index 00000000..e45fe4a9 --- /dev/null +++ b/util/common_test.go @@ -0,0 +1,24 @@ +package util + +import ( + "fmt" + "os" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestJksToPem(t *testing.T) { + var err error + var certPemPath, keyPemPath string + jksPaths := []string{"kafka.client.truststore.jks", "kafka.client.keystore.jks"} + jksPassword := "123456" + for _, jksPath := range jksPaths { + if _, err = os.Stat(jksPath); err != nil { + continue + } + certPemPath, keyPemPath, err = JksToPem(jksPath, jksPassword, true) + require.Nil(t, err, "err should be nothing") + fmt.Printf("converted %s to %s, %s\n", jksPath, certPemPath, keyPemPath) + } +} From 4c13aa0a311ea3ccb2747712631930b06076eb0a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 25 Jan 2021 13:25:52 +0800 Subject: [PATCH 052/404] support timezone config --- cmd/clickhouse_sinker/main.go | 2 +- config/config.go | 9 +++++++-- parser/csv.go | 14 ++++++-------- parser/csv_test.go | 18 +++++++++--------- parser/fastjson.go | 14 ++++++-------- parser/fastjson_test.go | 18 +++++++++--------- parser/gjson.go | 12 +++++------- parser/gjson_test.go | 18 +++++++++--------- parser/parser.go | 17 +++++++++-------- 9 files changed, 61 insertions(+), 61 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index af110e64..38e8385c 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -128,7 +128,7 @@ func init() { func GenTask(cfg *config.Config) (taskImpl *task.Service) { taskCfg := &cfg.Task ck := output.NewClickHouse(cfg) - pp := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, []string{taskCfg.LayoutDate, taskCfg.LayoutDateTime, taskCfg.LayoutDateTime64}) + pp := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, []string{taskCfg.LayoutDate, taskCfg.LayoutDateTime, taskCfg.LayoutDateTime64, taskCfg.TimeZone}) inputer := input.NewInputer(taskCfg.KafkaClient) taskImpl = task.NewTaskService(inputer, ck, pp, cfg) return diff --git a/config/config.go b/config/config.go index e2d13103..75ef3b96 100644 --- a/config/config.go +++ b/config/config.go @@ -142,16 +142,18 @@ type TaskConfig struct { LayoutDate string `json:"layoutDate,omitempty"` LayoutDateTime string `json:"layoutDateTime,omitempty"` LayoutDateTime64 string `json:"layoutDateTime64,omitempty"` + TimeZone string `json:"timezone"` } const ( defaultFlushInterval = 3 defaultBufferSize = 1 << 20 //1048576 - defaultMinBufferSize = 1 << 13 // 8196 + defaultMinBufferSize = 1 << 14 // 16384 defaultMsgSizeHint = 1000 defaultLayoutDate = "2006-01-02" defaultLayoutDateTime = time.RFC3339 - defaultLayoutDateTime64 = time.RFC3339 + defaultLayoutDateTime64 = time.RFC3339Nano + defaultTimeZone = "Local" defaultDistTblPrefix = "dist_" defaultLogLevel = "info" ) @@ -234,6 +236,9 @@ func (cfg *Config) Normallize() (err error) { if cfg.Task.LayoutDateTime64 == "" { cfg.Task.LayoutDateTime64 = defaultLayoutDateTime64 } + if cfg.Task.TimeZone == "" { + cfg.Task.TimeZone = defaultTimeZone + } if cfg.Task.DynamicSchema.Enable && cfg.Task.DynamicSchema.DistTblPrefix == "" { cfg.Task.DynamicSchema.DistTblPrefix = defaultDistTblPrefix } diff --git a/parser/csv.go b/parser/csv.go index 77c1eede..7cdd2e53 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -32,7 +32,6 @@ var _ Parser = (*CsvParser)(nil) type CsvParser struct { title []string delimiter string - tsLayout []string } // Parse extract a list of comma-separated values from the data @@ -47,15 +46,14 @@ func (p *CsvParser) Parse(bs []byte) (metric model.Metric, err error) { err = errors.Wrap(err, "") return } - metric = &CsvMetric{p.title, value, p.tsLayout} + metric = &CsvMetric{p.title, value} return } // CsvMetic type CsvMetric struct { - titles []string - values []string - tsLayout []string + titles []string + values []string } // Get returns the value corresponding to a column expects called @@ -155,19 +153,19 @@ func (c *CsvMetric) GetDate(key string, nullable bool) interface{} { _ = nullable // nullable can not be supported with csv val := c.GetString(key, false).(string) - t, _ := time.Parse(c.tsLayout[0], val) + t, _ := time.ParseInLocation(TSLayout[0], val, TimeZone) return t } func (c *CsvMetric) GetDateTime(key string, nullable bool) interface{} { val := c.GetString(key, false).(string) - t, _ := time.Parse(c.tsLayout[1], val) + t, _ := time.ParseInLocation(TSLayout[1], val, TimeZone) return t } func (c *CsvMetric) GetDateTime64(key string, nullable bool) interface{} { val := c.GetString(key, false).(string) - t, _ := time.Parse(c.tsLayout[2], val) + t, _ := time.ParseInLocation(TSLayout[2], val, TimeZone) return t } diff --git a/parser/csv_test.go b/parser/csv_test.go index ce25393b..427af7b2 100644 --- a/parser/csv_test.go +++ b/parser/csv_test.go @@ -22,7 +22,7 @@ import ( ) func TestCsvInt(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -42,7 +42,7 @@ func TestCsvInt(t *testing.T) { } func TestCsvFloat(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -62,7 +62,7 @@ func TestCsvFloat(t *testing.T) { } func TestCsvString(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -82,14 +82,14 @@ func TestCsvString(t *testing.T) { } func TestCsvDate(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) require.Nil(t, err) var exp, act time.Time - exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.UTC) + exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) act = metric.GetDate("time1", false).(time.Time) require.Equal(t, exp, act) @@ -102,7 +102,7 @@ func TestCsvDate(t *testing.T) { } func TestCsvDateTime(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -122,7 +122,7 @@ func TestCsvDateTime(t *testing.T) { } func TestCsvDateTime64(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -142,7 +142,7 @@ func TestCsvDateTime64(t *testing.T) { } func TestCsvElasticDateTime(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -164,7 +164,7 @@ func TestCsvElasticDateTime(t *testing.T) { } func TestCsvArray(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", DefaultTSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) diff --git a/parser/fastjson.go b/parser/fastjson.go index 74ef2a34..965edc08 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -30,8 +30,7 @@ var _ Parser = (*FastjsonParser)(nil) // FastjsonParser, parser for get data in json format type FastjsonParser struct { - tsLayout []string - fjp fastjson.Parser + fjp fastjson.Parser } func (p *FastjsonParser) Parse(bs []byte) (metric model.Metric, err error) { @@ -40,13 +39,12 @@ func (p *FastjsonParser) Parse(bs []byte) (metric model.Metric, err error) { err = errors.Wrapf(err, "") return } - metric = &FastjsonMetric{value: value, tsLayout: p.tsLayout} + metric = &FastjsonMetric{value: value} return } type FastjsonMetric struct { - value *fastjson.Value - tsLayout []string + value *fastjson.Value } func (c *FastjsonMetric) Get(key string) interface{} { @@ -126,7 +124,7 @@ func (c *FastjsonMetric) GetDate(key string, nullable bool) interface{} { } val := c.GetString(key, false).(string) - t, _ := time.Parse(c.tsLayout[0], val) + t, _ := time.ParseInLocation(TSLayout[0], val, TimeZone) return t } @@ -140,7 +138,7 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) interface{} { } val := c.GetString(key, false).(string) - t, _ := time.Parse(c.tsLayout[1], val) + t, _ := time.ParseInLocation(TSLayout[1], val, TimeZone) return t } @@ -154,7 +152,7 @@ func (c *FastjsonMetric) GetDateTime64(key string, nullable bool) interface{} { } val := c.GetString(key, false).(string) - t, _ := time.Parse(c.tsLayout[2], val) + t, _ := time.ParseInLocation(TSLayout[2], val, TimeZone) return t } diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go index 9256b801..840acd2a 100644 --- a/parser/fastjson_test.go +++ b/parser/fastjson_test.go @@ -8,7 +8,7 @@ import ( ) func TestFastjsonInt(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -27,7 +27,7 @@ func TestFastjsonInt(t *testing.T) { } func TestFastjsonFloat(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -46,7 +46,7 @@ func TestFastjsonFloat(t *testing.T) { } func TestFastjsonString(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -65,13 +65,13 @@ func TestFastjsonString(t *testing.T) { } func TestFastjsonDate(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) var exp, act time.Time - exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.UTC) + exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) act = metric.GetDate("time1", false).(time.Time) require.Equal(t, exp, act) @@ -84,7 +84,7 @@ func TestFastjsonDate(t *testing.T) { } func TestFastjsonDateTime(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -103,7 +103,7 @@ func TestFastjsonDateTime(t *testing.T) { } func TestFastjsonDateTime64(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -122,7 +122,7 @@ func TestFastjsonDateTime64(t *testing.T) { } func TestFastjsonElasticDateTime(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -143,7 +143,7 @@ func TestFastjsonElasticDateTime(t *testing.T) { } func TestFastjsonArray(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", DefaultTSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) diff --git a/parser/gjson.go b/parser/gjson.go index 6c41f194..56f506bb 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -27,17 +27,15 @@ import ( var _ Parser = (*GjsonParser)(nil) type GjsonParser struct { - tsLayout []string } func (p *GjsonParser) Parse(bs []byte) (metric model.Metric, err error) { - metric = &GjsonMetric{string(bs), p.tsLayout} + metric = &GjsonMetric{string(bs)} return } type GjsonMetric struct { - raw string - tsLayout []string + raw string } func (c *GjsonMetric) Get(key string) interface{} { @@ -106,7 +104,7 @@ func (c *GjsonMetric) GetDate(key string, nullable bool) interface{} { } val := r.String() - t, _ := time.Parse(c.tsLayout[0], val) + t, _ := time.ParseInLocation(TSLayout[0], val, TimeZone) return t } @@ -121,7 +119,7 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) interface{} { } val := r.String() - t, _ := time.Parse(c.tsLayout[1], val) + t, _ := time.ParseInLocation(TSLayout[1], val, TimeZone) return t } @@ -136,7 +134,7 @@ func (c *GjsonMetric) GetDateTime64(key string, nullable bool) interface{} { } val := r.String() - t, _ := time.Parse(c.tsLayout[2], val) + t, _ := time.ParseInLocation(TSLayout[2], val, TimeZone) return t } diff --git a/parser/gjson_test.go b/parser/gjson_test.go index c2a2b647..4fd58250 100644 --- a/parser/gjson_test.go +++ b/parser/gjson_test.go @@ -8,7 +8,7 @@ import ( ) func TestGjsonInt(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -27,7 +27,7 @@ func TestGjsonInt(t *testing.T) { } func TestGjsonFloat(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -46,7 +46,7 @@ func TestGjsonFloat(t *testing.T) { } func TestGjsonString(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -65,13 +65,13 @@ func TestGjsonString(t *testing.T) { } func TestGjsonDate(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) var exp, act time.Time - exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.UTC) + exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) act = metric.GetDate("time1", false).(time.Time) require.Equal(t, exp, act) @@ -84,7 +84,7 @@ func TestGjsonDate(t *testing.T) { } func TestGjsonDateTime(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -103,7 +103,7 @@ func TestGjsonDateTime(t *testing.T) { } func TestGjsonDateTime64(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -122,7 +122,7 @@ func TestGjsonDateTime64(t *testing.T) { } func TestGjsonElasticDateTime(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -143,7 +143,7 @@ func TestGjsonElasticDateTime(t *testing.T) { } func TestGjsonArray(t *testing.T) { - pp := NewParserPool("gjson", nil, "", DefaultTSLayout) + pp := NewParserPool("gjson", nil, "", TSLayout) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) diff --git a/parser/parser.go b/parser/parser.go index 177aacc4..47e78c32 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -23,8 +23,9 @@ import ( ) var ( - DefaultTSLayout = []string{"2006-01-02", time.RFC3339Nano, time.RFC3339Nano} - Epoch = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC) + TSLayout = []string{"2006-01-02", time.RFC3339Nano, time.RFC3339Nano, "Local"} + TimeZone *time.Location + Epoch = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC) ) // Parse is the Parser interface @@ -37,17 +38,17 @@ type Pool struct { name string csvFormat []string delimiter string - tsLayout []string pool sync.Pool } // NewParserPool create a parser pool func NewParserPool(name string, csvFormat []string, delimiter string, tsLayout []string) *Pool { + TSLayout = tsLayout + TimeZone, _ = time.LoadLocation(TSLayout[3]) return &Pool{ name: name, csvFormat: csvFormat, delimiter: delimiter, - tsLayout: tsLayout, } } @@ -59,13 +60,13 @@ func (pp *Pool) Get() Parser { if v == nil { switch pp.name { case "gjson": - return &GjsonParser{pp.tsLayout} + return &GjsonParser{} case "json", "fastjson": - return &FastjsonParser{tsLayout: pp.tsLayout} + return &FastjsonParser{} case "csv": - return &CsvParser{pp.csvFormat, pp.delimiter, pp.tsLayout} + return &CsvParser{pp.csvFormat, pp.delimiter} default: - return &FastjsonParser{tsLayout: pp.tsLayout} + return &FastjsonParser{} } } return v.(Parser) From 3cf33ecbe334d53d63dbb3855c7659da2a242b6f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 25 Jan 2021 14:28:40 +0800 Subject: [PATCH 053/404] faster type lookup --- model/metric.go | 3 +- model/value.go | 154 ++++++++++++++++++++++++++++--------------- output/clickhouse.go | 9 ++- 3 files changed, 108 insertions(+), 58 deletions(-) diff --git a/model/metric.go b/model/metric.go index fb4c7234..308e7193 100644 --- a/model/metric.go +++ b/model/metric.go @@ -40,6 +40,7 @@ type DimMetrics struct { // ColumnWithType type ColumnWithType struct { Name string - Type string + Type int + Nullable bool SourceName string } diff --git a/model/value.go b/model/value.go index 7f43ad9c..26de1a03 100644 --- a/model/value.go +++ b/model/value.go @@ -20,70 +20,116 @@ import ( "github.com/ClickHouse/clickhouse-go" ) +const ( + Int = iota + Float + String + IntArray + FloatArray + StringArray + Date + DateTime + DateTime64 + ElasticDateTime +) + +type TypeInfo struct { + Type int + Nullable bool +} + +var ( + typeInfo map[string]TypeInfo +) + // There are only three cases for the value type of metric, (float64, string, map [string] interface {}) func GetValueByType(metric Metric, cwt *ColumnWithType) interface{} { - swType, nullable := switchType(cwt.Type) name := cwt.SourceName - switch swType { - case "int": - return metric.GetInt(name, nullable) - case "float": - return metric.GetFloat(name, nullable) - case "string": - return metric.GetString(name, nullable) - case "stringArray": - return clickhouse.Array(metric.GetArray(name, "string")) - case "intArray": + switch cwt.Type { + case Int: + return metric.GetInt(name, cwt.Nullable) + case Float: + return metric.GetFloat(name, cwt.Nullable) + case String: + return metric.GetString(name, cwt.Nullable) + case IntArray: return clickhouse.Array(metric.GetArray(name, "int")) - case "floatArray": + case FloatArray: return clickhouse.Array(metric.GetArray(name, "float")) - case "Date": - return metric.GetDate(name, nullable) - case "DateTime": - return metric.GetDateTime(name, nullable) - case "DateTime64": - return metric.GetDateTime64(name, nullable) - case "ElasticDateTime": - return metric.GetElasticDateTime(name, nullable) - - //never happen + case StringArray: + return clickhouse.Array(metric.GetArray(name, "string")) + case Date: + return metric.GetDate(name, cwt.Nullable) + case DateTime: + return metric.GetDateTime(name, cwt.Nullable) + case DateTime64: + return metric.GetDateTime64(name, cwt.Nullable) + case ElasticDateTime: + return metric.GetElasticDateTime(name, cwt.Nullable) default: - return "" + panic("BUG: reached switch default condition") } } -func switchType(typ string) (dataType string, nullable bool) { - nullable = strings.HasPrefix(typ, "Nullable") - - switch typ { - case "UInt8", "UInt16", "UInt32", "UInt64", "Int8", - "Int16", "Int32", "Int64", - "Nullable(UInt8)", "Nullable(UInt16)", "Nullable(UInt32)", "Nullable(UInt64)", - "Nullable(Int8)", "Nullable(Int16)", "Nullable(Int32)", "Nullable(Int64)": - return "int", nullable - case "Array(UInt8)", "Array(UInt16)", "Array(UInt32)", - "Array(UInt64)", "Array(Int8)", "Array(Int16)", "Array(Int32)", "Array(Int64)": - return "intArray", false - case "String", "FixedString", "Nullable(String)": - return "string", nullable - case "Array(String)", "Array(FixedString)": - return "stringArray", false - case "Float32", "Float64", "Nullable(Float32)", "Nullable(Float64)": - return "float", nullable - case "Array(Float32)", "Array(Float64)": - return "floatArray", false - case "Date", "Nullable(Date)": - return "Date", nullable - case "DateTime", "Nullable(DateTime)": - return "DateTime", nullable - case "DateTime64", "Nullable(DateTime64)": - return "DateTime64", nullable - case "ElasticDateTime", "Nullable(ElasticDateTime)": - return "ElasticDateTime", nullable - default: +func WhichType(typ string) (dataType int, nullable bool) { + ti, ok := typeInfo[typ] + if ok { + dataType, nullable = ti.Type, ti.Nullable + return + } + nullable = strings.HasPrefix(typ, "Nullable(") + if nullable { + typ = typ[len("Nullable(") : len(typ)-1] } if strings.HasPrefix(typ, "DateTime64") { - return "DateTime64", nullable + dataType = DateTime64 + } else if strings.HasPrefix(typ, "DateTime") { + dataType = DateTime + } else { + panic("unsupported ClickHouse data type " + typ) + } + typeInfo[typ] = TypeInfo{Type: dataType, Nullable: nullable} + return +} + +func init() { + typeInfo = make(map[string]TypeInfo) + for _, t := range []string{"UInt8", "UInt16", "UInt32", "UInt64", "Int8", + "Int16", "Int32", "Int64"} { + typeInfo[t] = TypeInfo{Type: Int, Nullable: false} + } + for _, t := range []string{"Nullable(UInt8)", "Nullable(UInt16)", "Nullable(UInt32)", "Nullable(UInt64)", + "Nullable(Int8)", "Nullable(Int16)", "Nullable(Int32)", "Nullable(Int64)"} { + typeInfo[t] = TypeInfo{Type: Int, Nullable: true} + } + for _, t := range []string{"Float32", "Float64"} { + typeInfo[t] = TypeInfo{Type: Float, Nullable: false} + } + for _, t := range []string{"Nullable(Float32)", "Nullable(Float64)"} { + typeInfo[t] = TypeInfo{Type: Float, Nullable: true} + } + for _, t := range []string{"String", "FixedString"} { + typeInfo[t] = TypeInfo{Type: String, Nullable: false} + } + for _, t := range []string{"Nullable(String)", "Nullable(FixedString)"} { + typeInfo[t] = TypeInfo{Type: String, Nullable: true} + } + for _, t := range []string{"Array(UInt8)", "Array(UInt16)", "Array(UInt32)", + "Array(UInt64)", "Array(Int8)", "Array(Int16)", "Array(Int32)", "Array(Int64)"} { + typeInfo[t] = TypeInfo{Type: IntArray, Nullable: false} + } + for _, t := range []string{"Array(Float32)", "Array(Float64)"} { + typeInfo[t] = TypeInfo{Type: FloatArray, Nullable: false} + } + for _, t := range []string{"Array(String)", "Array(FixedString)"} { + typeInfo[t] = TypeInfo{Type: String, Nullable: false} } - panic("unsupported type " + typ) + typeInfo["Date"] = TypeInfo{Type: Date, Nullable: false} + typeInfo["Nullable(Date)"] = TypeInfo{Type: Date, Nullable: true} + typeInfo["DateTime"] = TypeInfo{Type: Date, Nullable: false} + typeInfo["Nullable(DateTime)"] = TypeInfo{Type: Date, Nullable: true} + typeInfo["DateTime64"] = TypeInfo{Type: Date, Nullable: false} + typeInfo["Nullable(DateTime64)"] = TypeInfo{Type: Date, Nullable: true} + typeInfo["ElasticDateTime"] = TypeInfo{Type: Date, Nullable: false} + typeInfo["Nullable(ElasticDateTime)"] = TypeInfo{Type: Date, Nullable: true} } diff --git a/output/clickhouse.go b/output/clickhouse.go index dd38ca48..91b1cba5 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -190,20 +190,23 @@ func (c *ClickHouse) initSchema() (err error) { } typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") if !util.StringContains(c.cfg.Task.ExcludeColumns, name) && defaultKind != "MATERIALIZED" { - c.Dims = append(c.Dims, &model.ColumnWithType{Name: name, Type: typ, SourceName: util.GetSourceName(name)}) + tp, nullable := model.WhichType(typ) + c.Dims = append(c.Dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, SourceName: util.GetSourceName(name)}) } } } else { c.Dims = make([]*model.ColumnWithType, 0) for _, dim := range c.cfg.Task.Dims { + tp, nullable := model.WhichType(dim.Type) c.Dims = append(c.Dims, &model.ColumnWithType{ Name: dim.Name, - Type: dim.Type, + Type: tp, + Nullable: nullable, SourceName: dim.SourceName, }) } } - //根据 Dms 生成prepare的sql语句 + // Generate SQL for INSERT c.Dms = make([]string, 0, len(c.Dims)) quotedDms := make([]string, 0, len(c.Dims)) for _, d := range c.Dims { From 76dd64be4e589f3b74250a2df2f63abba49fb049 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 24 Feb 2021 15:36:30 +0800 Subject: [PATCH 054/404] fix #110 --- go.sum | 24 ------------------------ model/value.go | 12 ++++++------ 2 files changed, 6 insertions(+), 30 deletions(-) diff --git a/go.sum b/go.sum index 2ffddb90..f48f15aa 100644 --- a/go.sum +++ b/go.sum @@ -10,7 +10,6 @@ github.com/Shopify/sarama v1.27.2 h1:1EyY1dsxNDUQEv0O/4TsjosHI2CgB1uo9H/v56xzTxc github.com/Shopify/sarama v1.27.2/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt/Mc93II= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= @@ -56,7 +55,6 @@ github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2 github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= @@ -66,7 +64,6 @@ github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= @@ -78,12 +75,10 @@ github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gops v0.3.12 h1:aDEe2KOOYPOOcvhmCzmlrO9YJbLVJIe0ilqlzDrH+eA= github.com/google/gops v0.3.12/go.mod h1:38bMPVKFh+1X106CPpbLAWtZIR1+xwgzT9gew0kn6w4= @@ -108,10 +103,8 @@ github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/u github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19 h1:WjT3fLi9n8YWh/Ih8Q1LHAPsTqGddPcHqscN+PJ3i68= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.9.8 h1:VMAMUUOh+gaxKTMk+zqbjsSjsIcUcL/LF4o63i82QyA= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.11.0 h1:wJbzvpYMVGG9iTI9VxpnNZfd4DzMPoCWze3GgSqz8yg= github.com/klauspost/compress v1.11.0/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= @@ -122,7 +115,6 @@ github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORN github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= @@ -147,12 +139,10 @@ github.com/nacos-group/nacos-sdk-go v1.0.1 h1:VNmXGlSS28xOmkO5Nxk5WRp6f1HMosAmG9 github.com/nacos-group/nacos-sdk-go v1.0.1/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.5.2+incompatible h1:WCjObylUIOlKy/+7Abdn34TLIkXiA4UWUMhxq9m9ZXI= github.com/pierrec/lz4 v2.5.2+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -167,7 +157,6 @@ github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1: github.com/prometheus/client_model v0.1.0 h1:ElTg5tNp4DqfV7UQjDqv2+RJlNzsDtvNAWccbItceIE= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.7.0 h1:L+1lyG48J1zAQXA3RBX/nG/B3gjlHq0zTt2tlbJLyCY= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.8.0 h1:bLkjvFe2ZRX1DpcgZcdf7j/+MnusEps5hktST/FHA34= github.com/prometheus/common v0.8.0/go.mod h1:PC/OgXc+UN7B4ALwvn1yzVZmVwvhXp5JsbBv6wSv6i0= @@ -180,7 +169,6 @@ github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqn github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= -github.com/shirou/gopsutil v2.20.4+incompatible h1:cMT4rxS55zx9NVUnCkrmXCsEB/RNfG9SwHY9evtX8Ng= github.com/shirou/gopsutil v2.20.4+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= @@ -195,10 +183,8 @@ github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasO github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5NYJe/zRYDwOu9ku6YHy+Iw7l5DM= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= @@ -220,7 +206,6 @@ github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c h1:u40Z8hqBAAQyv+vATcGgV github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= github.com/xdg/stringprep v1.0.0 h1:d9X0esnoa3dFsV0FG35rAT0RIhYFlPq7MiP+DW89La0= github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= -github.com/xlab/treeprint v1.0.0 h1:J0TkWtiuYgtdlrkkrDLISYBQ92M+X5m4LrIIMKrbDTs= github.com/xlab/treeprint v1.0.0/go.mod h1:IoImgRak9i3zJyuxOKUP1v4UZd1tMoKkq/Cimt1uhCg= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -234,7 +219,6 @@ golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnf golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190506204251-e1dfcc566284/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rBCcS0QyQY66Mpf/7BZbInM= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= @@ -244,7 +228,6 @@ golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKG golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190613194153-d28f0bde5980 h1:dfGZHvZk057jK2MCeWus/TowKpJ8y4AmooUzdBSR9GU= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200904194848-62affa334b73 h1:MXfv8rhZWmFeqX3GNZRsd6vOLoaCHjYEX3qkRo3YBUA= @@ -252,7 +235,6 @@ golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -261,11 +243,9 @@ golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9 h1:YTzHMGlqJu67/uEo1lBv0n3wBXhXNeUbB1XfN2vmTm0= golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200917061948-648f2a039071 h1:t0H7WMwCt9t0LnLSYz5zdZ/OiAtROxc5cHb5iHt3Xyw= golang.org/x/sys v0.0.0-20200917061948-648f2a039071/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= @@ -273,7 +253,6 @@ golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e h1:EHBhcS0mlXEAVwNyO2dLfjTo golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190328211700-ab21143f2384 h1:TFlARGu6Czu1z7q93HTxcP1P+/ZFC/IKythI5RzrnRg= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= @@ -310,14 +289,11 @@ gopkg.in/jcmturner/gokrb5.v7 v7.5.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuv gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU= gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -rsc.io/goversion v1.2.0 h1:SPn+NLTiAG7w30IRK/DKp1BjvpWabYgxlLp/+kx5J8w= rsc.io/goversion v1.2.0/go.mod h1:Eih9y/uIBS3ulggl7KNJ09xGSLcuNaLgmvvqa07sgfo= diff --git a/model/value.go b/model/value.go index 26de1a03..faf5dc99 100644 --- a/model/value.go +++ b/model/value.go @@ -126,10 +126,10 @@ func init() { } typeInfo["Date"] = TypeInfo{Type: Date, Nullable: false} typeInfo["Nullable(Date)"] = TypeInfo{Type: Date, Nullable: true} - typeInfo["DateTime"] = TypeInfo{Type: Date, Nullable: false} - typeInfo["Nullable(DateTime)"] = TypeInfo{Type: Date, Nullable: true} - typeInfo["DateTime64"] = TypeInfo{Type: Date, Nullable: false} - typeInfo["Nullable(DateTime64)"] = TypeInfo{Type: Date, Nullable: true} - typeInfo["ElasticDateTime"] = TypeInfo{Type: Date, Nullable: false} - typeInfo["Nullable(ElasticDateTime)"] = TypeInfo{Type: Date, Nullable: true} + typeInfo["DateTime"] = TypeInfo{Type: DateTime, Nullable: false} + typeInfo["Nullable(DateTime)"] = TypeInfo{Type: DateTime, Nullable: true} + typeInfo["DateTime64"] = TypeInfo{Type: DateTime64, Nullable: false} + typeInfo["Nullable(DateTime64)"] = TypeInfo{Type: DateTime64, Nullable: true} + typeInfo["ElasticDateTime"] = TypeInfo{Type: ElasticDateTime, Nullable: false} + typeInfo["Nullable(ElasticDateTime)"] = TypeInfo{Type: ElasticDateTime, Nullable: true} } From f79c8ce226e036888641a3074f99283c91abfa9c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 24 Feb 2021 15:55:16 +0800 Subject: [PATCH 055/404] listen on wildcard IP, close #111 --- cmd/clickhouse_sinker/main.go | 8 +++----- util/common.go | 4 ++-- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 38e8385c..3d565dff 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -62,7 +62,6 @@ type CmdOptions struct { var ( cmdOps CmdOptions selfIP string - selfAddr string httpMetrics = promhttp.Handler() runner *Sinker ) @@ -120,8 +119,7 @@ func init() { os.Exit(0) } selfIP = util.GetOutboundIP().String() - cmdOps.HTTPPort = util.GetSpareTCPPort(selfIP, cmdOps.HTTPPort) - selfAddr = fmt.Sprintf("%s:%d", selfIP, cmdOps.HTTPPort) + cmdOps.HTTPPort = util.GetSpareTCPPort(cmdOps.HTTPPort) } // GenTask generate a task via config @@ -188,8 +186,8 @@ func main() { mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) mux.HandleFunc("/debug/pprof/trace", pprof.Trace) - log.Infof("Run http server http://%s", selfAddr) - log.Error(http.ListenAndServe(selfAddr, mux)) + log.Infof("Run http server http://%s:%d", selfIP, cmdOps.HTTPPort) + log.Error(http.ListenAndServe(fmt.Sprintf(":%d", cmdOps.HTTPPort), mux)) }() runner.Run() diff --git a/util/common.go b/util/common.go index b26930a1..bd65e3ba 100644 --- a/util/common.go +++ b/util/common.go @@ -97,10 +97,10 @@ func GetOutboundIP() net.IP { } // GetSpareTCPPort find a spare TCP port -func GetSpareTCPPort(ip string, portBegin int) (port int) { +func GetSpareTCPPort(portBegin int) (port int) { LOOP: for port = portBegin; ; port++ { - addr := fmt.Sprintf("%s:%d", ip, port) + addr := fmt.Sprintf(":%d", port) ln, err := net.Listen("tcp", addr) if err == nil { ln.Close() From f97c0cb05bb02debe65e0bab7f8a6eb459ae266b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 24 Feb 2021 16:47:11 +0800 Subject: [PATCH 056/404] make lint happy --- .github/workflows/lint.yml | 4 ++-- Makefile | 2 +- cmd/clickhouse_sinker/main.go | 2 +- config/build.go | 17 +++++++++-------- output/clickhouse.go | 5 +---- util/common.go | 3 +-- util/common_test.go | 3 +-- 7 files changed, 16 insertions(+), 20 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 6b90fa74..231f0665 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -16,13 +16,13 @@ jobs: uses: golangci/golangci-lint-action@v2 with: # Required: the version of golangci-lint is required and must be specified without patch version: we always use the latest patch version. - version: v1.33 + version: v1.37 # Optional: working directory, useful for monorepos # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true \ No newline at end of file diff --git a/Makefile b/Makefile index 79ee0beb..c0fe8d52 100644 --- a/Makefile +++ b/Makefile @@ -25,7 +25,7 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-dir conf/ diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 3d565dff..3908b2d3 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -115,7 +115,7 @@ func initCmdOptions() { func init() { initCmdOptions() if cmdOps.ShowVer { - config.PrintSinkerInfo() + log.Infoln(config.GetSinkerInfo()) os.Exit(0) } selfIP = util.GetOutboundIP().String() diff --git a/config/build.go b/config/build.go index a275e94c..21ed3828 100644 --- a/config/build.go +++ b/config/build.go @@ -5,16 +5,17 @@ import "fmt" var ( // SinkerReleaseVersion information. SinkerReleaseVersion = "None" - SinkerBuildTS = "None" + SinkerEdition = "None" SinkerGitHash = "None" SinkerGitBranch = "None" - SinkerEdition = "None" + SinkerBuildTS = "None" ) -func PrintSinkerInfo() { - fmt.Println("Release Version:", SinkerReleaseVersion) - fmt.Println("Edition:", SinkerEdition) - fmt.Println("Git Commit Hash:", SinkerGitHash) - fmt.Println("Git Branch:", SinkerGitBranch) - fmt.Println("UTC Build Time: ", SinkerBuildTS) +func GetSinkerInfo() string { + return fmt.Sprintf("Release Version: %s, Edition: %s, Git Commit Hash: %s, Git Branch: %s, Build At: %s", + SinkerReleaseVersion, + SinkerEdition, + SinkerGitHash, + SinkerGitBranch, + SinkerBuildTS) } diff --git a/output/clickhouse.go b/output/clickhouse.go index 91b1cba5..72623e83 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -64,10 +64,7 @@ func (c *ClickHouse) Init() (err error) { if err = pool.InitConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams); err != nil { return } - if err = c.initSchema(); err != nil { - return err - } - return nil + return c.initSchema() } // Send a batch to clickhouse diff --git a/util/common.go b/util/common.go index bd65e3ba..c917e33d 100644 --- a/util/common.go +++ b/util/common.go @@ -162,8 +162,7 @@ func EnvIntVar(value *int, key string) { func EnvBoolVar(value *bool, key string) { realKey := strings.ReplaceAll(strings.ToUpper(key), "-", "_") - _, found := os.LookupEnv(realKey) - if found { + if _, found := os.LookupEnv(realKey); found { *value = true } } diff --git a/util/common_test.go b/util/common_test.go index e45fe4a9..1a03b6f7 100644 --- a/util/common_test.go +++ b/util/common_test.go @@ -1,7 +1,6 @@ package util import ( - "fmt" "os" "testing" @@ -19,6 +18,6 @@ func TestJksToPem(t *testing.T) { } certPemPath, keyPemPath, err = JksToPem(jksPath, jksPassword, true) require.Nil(t, err, "err should be nothing") - fmt.Printf("converted %s to %s, %s\n", jksPath, certPemPath, keyPemPath) + t.Logf("converted %s to %s, %s\n", jksPath, certPemPath, keyPemPath) } } From a3e9c567260694f9cbf14935458d3c064cf1d52a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 25 Feb 2021 16:19:16 +0800 Subject: [PATCH 057/404] updated config.md --- docs/configuration/config.md | 32 ++++++++++++++++++++++++-------- 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 5873afb8..39ff9e59 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -94,25 +94,41 @@ // "this columns will be excluded by insert SQL " "excludeColumns": [] + // (experiment feature) detect new fields and their type, and add columns to the ClickHouse table accordingly. supported types are Int64, Float64, String. + "dynamicSchema": { + // whether enable this feature, default to false + "enable": true, + // cluster the ClickHouse node belongs + "cluster": "test", + // distributed table name prefix, default to "dist_" + "distTblPrefix": "" + }, + // shardingKey is the column name to which sharding against "shardingKey": "", // shardingPolicy is `stripe,`(requires ShardingKey be numerical) or `hash`(requires ShardingKey be string) "shardingPolicy": "", - // batch size to insert into clickhouse + // interval of flushing the batch + "flushInterval": 5, + // batch size to insert into clickhouse. sinker will round upward it to the the nearest 2^n. "bufferSize": 90000, - // min batch size to insert into clickhouse + // min batch size to insert into clickhouse. sinker will round upward it to the the nearest 2^n. "minBufferSize": 1, - - // msg bytes per message + // estimated avg message size. kafka-go needs this to determize receive buffer size. default to 1000. "msgSizeHint": 1000, - // interval flush the batch - "flushInterval": 5, + // Date format in message, default to "2006-01-02". + "layoutDate": "", + // DateTime format in message, default to "2006-01-02T15:04:05Z07:00" (aka time.RFC3339). + "layoutDateTime": "", + // DateTime64 format in message, default to "2006-01-02T15:04:05.999999999Z07:00" (aka time.RFC3339Nano). + "layoutDateTime64": "", + // In the absence of time zone information, interprets the time as in the given location. Default to "Local" (aka /etc/localtime of the machine on which sinker runs) + "timezone": "" }, - // log level + // log level, possible value: panic, fatal, error, warn, warning, info, debug, trace "logLevel": "debug" - } } ``` From 3a4ba8d7862db5b5343fb977e93a6b0c3e528164 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 25 Feb 2021 17:11:50 +0800 Subject: [PATCH 058/404] fix Array(String) --- model/value.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/model/value.go b/model/value.go index faf5dc99..39b645f6 100644 --- a/model/value.go +++ b/model/value.go @@ -122,7 +122,7 @@ func init() { typeInfo[t] = TypeInfo{Type: FloatArray, Nullable: false} } for _, t := range []string{"Array(String)", "Array(FixedString)"} { - typeInfo[t] = TypeInfo{Type: String, Nullable: false} + typeInfo[t] = TypeInfo{Type: StringArray, Nullable: false} } typeInfo["Date"] = TypeInfo{Type: Date, Nullable: false} typeInfo["Nullable(Date)"] = TypeInfo{Type: Date, Nullable: true} From a8c5dc5ce299f06928bd886d7747634ffee0717d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 25 Feb 2021 21:33:12 +0800 Subject: [PATCH 059/404] fastjson bool --- parser/fastjson.go | 18 +++++++++++++++--- parser/fastjson_test.go | 8 ++++++++ parser/gjson_test.go | 8 ++++++++ parser/parser_test.go | 4 +++- 4 files changed, 34 insertions(+), 4 deletions(-) diff --git a/parser/fastjson.go b/parser/fastjson.go index 965edc08..6d8cb725 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -67,8 +67,15 @@ func (c *FastjsonMetric) GetFloat(key string, nullable bool) interface{} { } func (c *FastjsonMetric) GetInt(key string, nullable bool) interface{} { - if nullable && !c.value.Exists(key) { - return nil + v := c.value.Get(key) + if v == nil { + if nullable { + return nil + } + return int64(0) + } + if v.Type() == fastjson.TypeTrue { + return int64(1) } return c.value.GetInt64(key) } @@ -91,7 +98,12 @@ func (c *FastjsonMetric) GetArray(key string, t string) interface{} { case "int": results := make([]int64, 0, len(array)) for _, e := range array { - v, _ := e.Int64() + var v int64 + if e.Type() == fastjson.TypeTrue { + v = 1 + } else { + v, _ = e.Int64() + } results = append(results, v) } return results diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go index 840acd2a..643666f9 100644 --- a/parser/fastjson_test.go +++ b/parser/fastjson_test.go @@ -24,6 +24,14 @@ func TestFastjsonInt(t *testing.T) { actual := metric.GetInt("its_not_exist", true) require.Nil(t, actual, "err should be nothing") + + exp = 0 + act = metric.GetInt("bool_false", false).(int64) + require.Equal(t, exp, act) + + exp = 1 + act = metric.GetInt("bool_true", false).(int64) + require.Equal(t, exp, act) } func TestFastjsonFloat(t *testing.T) { diff --git a/parser/gjson_test.go b/parser/gjson_test.go index 4fd58250..1172b5c1 100644 --- a/parser/gjson_test.go +++ b/parser/gjson_test.go @@ -24,6 +24,14 @@ func TestGjsonInt(t *testing.T) { actual := metric.GetInt("its_not_exist", true) require.Nil(t, actual, "err should be nothing") + + exp = 0 + act = metric.GetInt("bool_false", false).(int64) + require.Equal(t, exp, act) + + exp = 1 + act = metric.GetInt("bool_true", false).(int64) + require.Equal(t, exp, act) } func TestGjsonFloat(t *testing.T) { diff --git a/parser/parser_test.go b/parser/parser_test.go index e77248e5..f22dbdab 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -41,7 +41,9 @@ var jsonSample = []byte(`{ "array_int": [1,2,3], "array_float": [1.1,2.2,3.3], "array_string": ["aa","bb","cc"], - "array_empty": [] + "array_empty": [], + "bool_true": true, + "bool_false": false }`) var csvSampleSchema = []string{"its", "percent", "channel", "time1", "time2", "time3", "array_int", "array_float", "array_string", "array_empty"} From 82c04d6c8dac58b2ad8937213e220b27c6781010 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 25 Feb 2021 22:39:07 +0800 Subject: [PATCH 060/404] fix fastjson GetNewKeys panic --- parser/fastjson.go | 5 ----- parser/fastjson_test.go | 28 ++++++++++++++++++++++++++++ parser/parser_test.go | 2 ++ task/task.go | 38 ++++++++++++++++++++------------------ 4 files changed, 50 insertions(+), 23 deletions(-) diff --git a/parser/fastjson.go b/parser/fastjson.go index 6d8cb725..b4fb4789 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -16,7 +16,6 @@ limitations under the License. package parser import ( - "fmt" "sync" "time" @@ -187,10 +186,6 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (fou obj.Visit(func(key []byte, v *fastjson.Value) { strKey := string(key) if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { - v := c.value.Get(strKey) - if v == nil { - panic(fmt.Sprintf("BUG: fastjson.Object.Visit got an unexpected key: %s", strKey)) - } if _, err = v.Int64(); err == nil { newKeys.Store(strKey, "int") foundNew = true diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go index 643666f9..832bfd11 100644 --- a/parser/fastjson_test.go +++ b/parser/fastjson_test.go @@ -1,10 +1,14 @@ package parser import ( + "fmt" + "strings" "testing" "time" + log "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" + "github.com/valyala/fastjson" ) func TestFastjsonInt(t *testing.T) { @@ -179,4 +183,28 @@ func TestFastjsonArray(t *testing.T) { actSE := metric.GetArray("array_empty", "string").([]string) expSE := []string{} require.Equal(t, expSE, actSE) + + metric, _ = parser.Parse(jsonSample2) + actS = metric.GetArray("listvalue", "string").([]string) + expS = []string{"aaa", "bbb", "ccc"} + require.Equal(t, expS, actS) +} + +func TestFastjsonDetectSchema(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", TSLayout) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var nameAndTypes []string + c := metric.(*FastjsonMetric) + var obj *fastjson.Object + var err error + if obj, err = c.value.Object(); err != nil { + return + } + obj.Visit(func(key []byte, v *fastjson.Value) { + nameAndTypes = append(nameAndTypes, fmt.Sprintf("%s: %s", string(key), v.Type().String())) + }) + log.Infof("detected json schema: {%s}", strings.Join(nameAndTypes, ", ")) } diff --git a/parser/parser_test.go b/parser/parser_test.go index f22dbdab..48ec72dd 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -46,6 +46,8 @@ var jsonSample = []byte(`{ "bool_false": false }`) +var jsonSample2 = []byte(`{"date":"2021-01-02","ip":"192.168.0.3","floatvalue":425.633,"doublevalue":571.2464722672763,"novalue":" ","metric":"CPU_Idle_Time","service":"Web3","listvalue":["aaa","bbb","ccc"],"addint":123,"adddouble":571.2464722672763,"addstring":"add","value":123,"timestamp":"2021-01-02 21:06:00"}`) + var csvSampleSchema = []string{"its", "percent", "channel", "time1", "time2", "time3", "array_int", "array_float", "array_string", "array_empty"} var csvSample = []byte(`1536813227,"0.11","escaped_""ws",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30.123Z,"[1,2,3]","[1.1,2.2,3.3]","[aa,bb,cc]","[]"`) diff --git a/task/task.go b/task/task.go index f4b30ce2..a4611dfb 100644 --- a/task/task.go +++ b/task/task.go @@ -230,6 +230,7 @@ func (service *Service) put(msg model.InputMessage) { statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Inc() _ = util.GlobalParsingPool.Submit(func() { var row *model.Row + var foundNewKeys bool p := service.pp.Get() metric, err := p.Parse(msg.Value) if err != nil { @@ -241,28 +242,29 @@ func (service *Service) put(msg model.InputMessage) { } else { row = model.MetricToRow(metric, msg, service.dims) } + if taskCfg.DynamicSchema.Enable { + foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys) + } + // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. service.pp.Put(p) - if taskCfg.DynamicSchema.Enable { - found := metric.GetNewKeys(&service.knownKeys, &service.newKeys) - if found { - cntNewKeys := atomic.AddInt32(&service.cntNewKeys, 1) - if cntNewKeys == 1 { - // The first message which contains new keys triggers flushing - // all messages and scheduling a delayed func to apply schema change. - for _, ring := range service.rings { - if ring != nil { - ring.ForceBatchOrShard(nil) - } - } - if service.sharder != nil { - service.sharder.ForceFlush(nil) - } - if service.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.changeSchema, nil); err != nil { - log.Fatalf("got error %+v", err) - os.Exit(-1) + if foundNewKeys { + cntNewKeys := atomic.AddInt32(&service.cntNewKeys, 1) + if cntNewKeys == 1 { + // The first message which contains new keys triggers flushing + // all messages and scheduling a delayed func to apply schema change. + for _, ring := range service.rings { + if ring != nil { + ring.ForceBatchOrShard(nil) } } + if service.sharder != nil { + service.sharder.ForceFlush(nil) + } + if service.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.changeSchema, nil); err != nil { + log.Fatalf("got error %+v", err) + os.Exit(-1) + } } } if atomic.LoadInt32(&service.cntNewKeys) == 0 { From 53185143d8387babd0469db4b29bfa199a61ef51 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 26 Feb 2021 10:15:32 +0800 Subject: [PATCH 061/404] updated dynamicSchema doc --- config/config.go | 10 ++++++++-- docs/configuration/config.md | 2 +- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/config/config.go b/config/config.go index 75ef3b96..8491ed4d 100644 --- a/config/config.go +++ b/config/config.go @@ -239,8 +239,14 @@ func (cfg *Config) Normallize() (err error) { if cfg.Task.TimeZone == "" { cfg.Task.TimeZone = defaultTimeZone } - if cfg.Task.DynamicSchema.Enable && cfg.Task.DynamicSchema.DistTblPrefix == "" { - cfg.Task.DynamicSchema.DistTblPrefix = defaultDistTblPrefix + if cfg.Task.DynamicSchema.Enable { + if cfg.Task.Parser != "fastjson" { + err = errors.Errorf("Parser %s doesn't support DynamicSchema", cfg.Task.Parser) + return + } + if cfg.Task.DynamicSchema.DistTblPrefix == "" { + cfg.Task.DynamicSchema.DistTblPrefix = defaultDistTblPrefix + } } switch strings.ToLower(cfg.LogLevel) { case "panic", "fatal", "error", "warn", "warning", "info", "debug", "trace": diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 39ff9e59..bcbb9108 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -94,7 +94,7 @@ // "this columns will be excluded by insert SQL " "excludeColumns": [] - // (experiment feature) detect new fields and their type, and add columns to the ClickHouse table accordingly. supported types are Int64, Float64, String. + // (experiment feature) detect new fields and their type, and add columns to the ClickHouse table accordingly. This feature requires parser be "fastjson", and support following ClickHouse data types: Int64, Float64, String. "dynamicSchema": { // whether enable this feature, default to false "enable": true, From f826dddd1d626822b22aaaa2c26be6555dc344b5 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 26 Feb 2021 13:07:46 +0800 Subject: [PATCH 062/404] fix systest --- config/config.go | 2 +- docker-compose.yml | 2 +- go.test.sh | 6 +++++- parser/parser.go | 2 +- 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/config/config.go b/config/config.go index 8491ed4d..0124c2fb 100644 --- a/config/config.go +++ b/config/config.go @@ -201,7 +201,7 @@ func (cfg *Config) Normallize() (err error) { } else if cfg.Task.KafkaClient == "" { cfg.Task.KafkaClient = "kafka-go" } - if cfg.Task.Parser == "" { + if cfg.Task.Parser == "" || cfg.Task.Parser == "json" { cfg.Task.Parser = "fastjson" } diff --git a/docker-compose.yml b/docker-compose.yml index 7a229c9f..e1b3a166 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -31,7 +31,7 @@ services: security_opt: - label:disable clickhouse: - image: yandex/clickhouse-server:20.8 + image: yandex/clickhouse-server:20.8.5.45 restart: always ports: - "8123:8123" diff --git a/go.test.sh b/go.test.sh index 75dd69cf..74690bec 100755 --- a/go.test.sh +++ b/go.test.sh @@ -19,6 +19,10 @@ curl "localhost:8123" -d 'CREATE TABLE test_auto_schema AS test_fixed_schema' curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_dynamic_schema' curl "localhost:8123" -d 'CREATE TABLE test_dynamic_schema AS test_fixed_schema' +counts=`curl "localhost:8123" -d 'SELECT count() FROM test_fixed_schema UNION ALL SELECT count() FROM test_auto_schema UNION ALL SELECT count() FROM test_dynamic_schema' 2>/dev/null | tr '\n' ','` +echo "Got initial row counts => $counts" +[ $counts = "0,0,0," ] || exit 1 + ## send the messages to kafka now=`date --rfc-3339=ns` for i in `seq 1 10000`;do @@ -40,7 +44,7 @@ echo "generated a.json" echo "cat /tmp/a.json | kafka-console-producer --topic topic1 --broker-list localhost:9092" > send.sh sudo docker cp a.json kafka:/tmp/ sudo docker cp send.sh kafka:/tmp/ -sudo docker exec kafka sh /tmp/send.sh +sudo docker exec kafka sh /tmp/send.sh ## start clickhouse_sinker to consume timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/test_fixed_schema.json diff --git a/parser/parser.go b/parser/parser.go index 47e78c32..0639cf54 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -61,7 +61,7 @@ func (pp *Pool) Get() Parser { switch pp.name { case "gjson": return &GjsonParser{} - case "json", "fastjson": + case "fastjson": return &FastjsonParser{} case "csv": return &CsvParser{pp.csvFormat, pp.delimiter} From 246c08f2b281f033f6841a7dcaea6b719163c6e6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 1 Mar 2021 15:48:52 +0800 Subject: [PATCH 063/404] simplified docker-compose.yml --- docker-compose.yml | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index e1b3a166..8d235e63 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -44,20 +44,6 @@ services: - "./docker-entrypoint-initdb.d:/docker-entrypoint-initdb.d" security_opt: - label:disable - tabix: - image: spoonest/clickhouse-tabix-web-client:stable - restart: always - ports: - - "8080:80" - depends_on: - - clickhouse - environment: - CH_NAME: "dev" - CH_HOST: "http://127.0.0.1:8123" - CH_LOGIN: "default" - CH_PASSWORD: "" - security_opt: - - label:disable nacos: image: nacos/nacos-server:1.4.0 container_name: nacos-standalone From b81cf13bae0d7bdfdf202620da052d24fe7b68eb Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 4 Mar 2021 21:33:33 +0800 Subject: [PATCH 064/404] more parser test cases --- parser/csv_test.go | 85 +++++++++++++++++++++++--------------- parser/fastjson_test.go | 90 ++++++++++++++++++++++++++++------------- parser/gjson_test.go | 88 +++++++++++++++++++++++++++------------- parser/parser.go | 14 +++++-- parser/parser_test.go | 27 ++++++++++--- 5 files changed, 206 insertions(+), 98 deletions(-) diff --git a/parser/csv_test.go b/parser/csv_test.go index 427af7b2..581f7c88 100644 --- a/parser/csv_test.go +++ b/parser/csv_test.go @@ -22,7 +22,7 @@ import ( ) func TestCsvInt(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", nil) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -34,15 +34,15 @@ func TestCsvInt(t *testing.T) { require.Equal(t, exp, act) exp = 0 - act = metric.GetInt("its_not_exist", false).(int64) + act = metric.GetInt("not_exist", false).(int64) require.Equal(t, exp, act) - act = metric.GetInt("its_not_exist", true).(int64) + act = metric.GetInt("not_exist", true).(int64) require.Equal(t, exp, act) } func TestCsvFloat(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", nil) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -54,15 +54,15 @@ func TestCsvFloat(t *testing.T) { require.Equal(t, exp, act) exp = 0.0 - act = metric.GetFloat("percent_not_exist", false).(float64) + act = metric.GetFloat("not_exist", false).(float64) require.Equal(t, exp, act) - act = metric.GetFloat("percent_not_exist", true).(float64) + act = metric.GetFloat("not_exist", true).(float64) require.Equal(t, exp, act) } func TestCsvString(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", nil) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -74,15 +74,15 @@ func TestCsvString(t *testing.T) { require.Equal(t, exp, act) exp = "" - act = metric.GetString("channel_not_exist", false).(string) + act = metric.GetString("not_exist", false).(string) require.Equal(t, exp, act) - act = metric.GetString("channel_not_exist", true).(string) + act = metric.GetString("not_exist", true).(string) require.Equal(t, exp, act) } func TestCsvDate(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -90,19 +90,12 @@ func TestCsvDate(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - act = metric.GetDate("time1", false).(time.Time) - require.Equal(t, exp, act) - - exp = time.Time{} - act = metric.GetDate("time1_not_exist", false).(time.Time) - require.Equal(t, exp, act) - - act = metric.GetDate("time1_not_exist", true).(time.Time) + act = metric.GetDate("date1", false).(time.Time) require.Equal(t, exp, act) } -func TestCsvDateTime(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) +func TestCsvDateTimeStd(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -110,19 +103,33 @@ func TestCsvDateTime(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - act = metric.GetDateTime("time2", false).(time.Time) + act = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) + act = metric.GetDateTime("time_sec_rfc3339_2", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDateTime("time2_not_exist", false).(time.Time) + act = metric.GetDateTime("not_exist", false).(time.Time) require.Equal(t, exp, act) +} - act = metric.GetDateTime("time2_not_exist", true).(time.Time) +func TestCsvDateTimeCh(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutCh) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) + act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) } -func TestCsvDateTime64(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) +func TestCsvDateTime64Std(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -130,19 +137,33 @@ func TestCsvDateTime64(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - act = metric.GetDateTime64("time3", false).(time.Time) + act = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) require.Equal(t, exp, act) + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) + act = metric.GetDateTime64("time_ms_rfc3339_2", false).(time.Time).In(time.UTC) + require.EqualValues(t, exp, act) + exp = time.Time{} - act = metric.GetDateTime64("time3_not_exist", false).(time.Time) + act = metric.GetDateTime64("not_exist", false).(time.Time) require.Equal(t, exp, act) +} - act = metric.GetDateTime64("time3_not_exist", true).(time.Time) +func TestCsvDateTime64Ch(t *testing.T) { + pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutCh) + parser := pp.Get() + defer pp.Put(parser) + metric, err := parser.Parse(csvSample) + require.Nil(t, err) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) + act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) } func TestCsvElasticDateTime(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) + pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -152,14 +173,14 @@ func TestCsvElasticDateTime(t *testing.T) { // {"date": "2019-12-16T12:10:30Z"} // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC exp = 1576498230 - act = metric.GetElasticDateTime("time2", false).(int64) + act = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) require.Equal(t, exp, act) exp = -62135596800 - act = metric.GetElasticDateTime("time2_not_exist", false).(int64) + act = metric.GetElasticDateTime("not_exist", false).(int64) require.Equal(t, exp, act) - act = metric.GetElasticDateTime("time2_not_exist", true).(int64) + act = metric.GetElasticDateTime("not_exist", true).(int64) require.Equal(t, exp, act) } diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go index 832bfd11..2323a6c3 100644 --- a/parser/fastjson_test.go +++ b/parser/fastjson_test.go @@ -12,7 +12,7 @@ import ( ) func TestFastjsonInt(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayout) + pp := NewParserPool("fastjson", nil, "", nil) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -23,10 +23,10 @@ func TestFastjsonInt(t *testing.T) { require.Equal(t, exp, act) exp = 0 - act = metric.GetInt("its_not_exist", false).(int64) + act = metric.GetInt("not_exist", false).(int64) require.Equal(t, exp, act) - actual := metric.GetInt("its_not_exist", true) + actual := metric.GetInt("not_exist", true) require.Nil(t, actual, "err should be nothing") exp = 0 @@ -39,7 +39,7 @@ func TestFastjsonInt(t *testing.T) { } func TestFastjsonFloat(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayout) + pp := NewParserPool("fastjson", nil, "", nil) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -50,15 +50,15 @@ func TestFastjsonFloat(t *testing.T) { require.Equal(t, exp, act) exp = 0.0 - act = metric.GetFloat("percent_not_exist", false).(float64) + act = metric.GetFloat("not_exist", false).(float64) require.Equal(t, exp, act) - actual := metric.GetFloat("percent_not_exist", true) + actual := metric.GetFloat("not_exist", true) require.Nil(t, actual, "err should be nothing") } func TestFastjsonString(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayout) + pp := NewParserPool("fastjson", nil, "", nil) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -69,72 +69,104 @@ func TestFastjsonString(t *testing.T) { require.Equal(t, exp, act) exp = "" - act = metric.GetString("channel_not_exist", false).(string) + act = metric.GetString("not_exist", false).(string) require.Equal(t, exp, act) - actual := metric.GetString("channel_not_exist", true) + actual := metric.GetString("not_exist", true) require.Nil(t, actual, "err should be nothing") } func TestFastjsonDate(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) var exp, act time.Time exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - act = metric.GetDate("time1", false).(time.Time) + act = metric.GetDate("date1", false).(time.Time) require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDate("time1_not_exist", false).(time.Time) + act = metric.GetDate("not_exist", false).(time.Time) require.Equal(t, exp, act) - actual := metric.GetDate("time1_not_exist", true) + actual := metric.GetDate("not_exist", true) require.Nil(t, actual, "err should be nothing") } -func TestFastjsonDateTime(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayout) +func TestFastjsonDateTimeStd(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - act = metric.GetDateTime("time2", false).(time.Time) + act = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) + act = metric.GetDateTime("time_sec_rfc3339_2", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDateTime("time2_not_exist", false).(time.Time) + act = metric.GetDateTime("not_exist", false).(time.Time) require.Equal(t, exp, act) - actual := metric.GetDateTime("time2_not_exist", true) + actual := metric.GetDateTime("not_exist", true) require.Nil(t, actual, "err should be nothing") } -func TestFastjsonDateTime64(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayout) +func TestFastjsonDateTimeCh(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", TSLayoutCh) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) + act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) + require.Equal(t, exp, act) +} + +func TestFastjsonDateTime64Std(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - act = metric.GetDateTime64("time3", false).(time.Time) + act = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) require.Equal(t, exp, act) + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) + act = metric.GetDateTime64("time_ms_rfc3339_2", false).(time.Time).In(time.UTC) + require.EqualValues(t, exp, act) + exp = time.Time{} - act = metric.GetDateTime64("time3_not_exist", false).(time.Time) + act = metric.GetDateTime64("not_exist", false).(time.Time) require.Equal(t, exp, act) - actual := metric.GetDateTime64("time3_not_exist", true) + actual := metric.GetDateTime64("not_exist", true) require.Nil(t, actual, "err should be nothing") } +func TestFastjsonDateTime64Ch(t *testing.T) { + pp := NewParserPool("fastjson", nil, "", TSLayoutCh) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) + act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) + require.Equal(t, exp, act) +} + func TestFastjsonElasticDateTime(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayout) + pp := NewParserPool("fastjson", nil, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -143,19 +175,19 @@ func TestFastjsonElasticDateTime(t *testing.T) { // {"date": "2019-12-16T12:10:30Z"} // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC exp = 1576498230 - act = metric.GetElasticDateTime("time2", false).(int64) + act = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) require.Equal(t, exp, act) exp = -62135596800 - act = metric.GetElasticDateTime("time2_not_exist", false).(int64) + act = metric.GetElasticDateTime("not_exist", false).(int64) require.Equal(t, exp, act) - actual := metric.GetElasticDateTime("time2_not_exist", true) + actual := metric.GetElasticDateTime("not_exist", true) require.Nil(t, actual, "err should be nothing") } func TestFastjsonArray(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayout) + pp := NewParserPool("fastjson", nil, "", nil) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -191,7 +223,7 @@ func TestFastjsonArray(t *testing.T) { } func TestFastjsonDetectSchema(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayout) + pp := NewParserPool("fastjson", nil, "", nil) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) diff --git a/parser/gjson_test.go b/parser/gjson_test.go index 1172b5c1..d7d40a9a 100644 --- a/parser/gjson_test.go +++ b/parser/gjson_test.go @@ -8,7 +8,7 @@ import ( ) func TestGjsonInt(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayout) + pp := NewParserPool("gjson", nil, "", nil) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -19,10 +19,10 @@ func TestGjsonInt(t *testing.T) { require.Equal(t, exp, act) exp = 0 - act = metric.GetInt("its_not_exist", false).(int64) + act = metric.GetInt("not_exist", false).(int64) require.Equal(t, exp, act) - actual := metric.GetInt("its_not_exist", true) + actual := metric.GetInt("not_exist", true) require.Nil(t, actual, "err should be nothing") exp = 0 @@ -35,7 +35,7 @@ func TestGjsonInt(t *testing.T) { } func TestGjsonFloat(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayout) + pp := NewParserPool("gjson", nil, "", nil) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -46,15 +46,15 @@ func TestGjsonFloat(t *testing.T) { require.Equal(t, exp, act) exp = 0.0 - act = metric.GetFloat("percent_not_exist", false).(float64) + act = metric.GetFloat("not_exist", false).(float64) require.Equal(t, exp, act) - actual := metric.GetFloat("percent_not_exist", true) + actual := metric.GetFloat("not_exist", true) require.Nil(t, actual, "err should be nothing") } func TestGjsonString(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayout) + pp := NewParserPool("gjson", nil, "", nil) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -65,72 +65,104 @@ func TestGjsonString(t *testing.T) { require.Equal(t, exp, act) exp = "" - act = metric.GetString("channel_not_exist", false).(string) + act = metric.GetString("not_exist", false).(string) require.Equal(t, exp, act) - actual := metric.GetString("channel_not_exist", true) + actual := metric.GetString("not_exist", true) require.Nil(t, actual, "err should be nothing") } func TestGjsonDate(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayout) + pp := NewParserPool("gjson", nil, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) var exp, act time.Time exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - act = metric.GetDate("time1", false).(time.Time) + act = metric.GetDate("date1", false).(time.Time) require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDate("time1_not_exist", false).(time.Time) + act = metric.GetDate("not_exist", false).(time.Time) require.Equal(t, exp, act) - actual := metric.GetDate("time1_not_exist", true) + actual := metric.GetDate("not_exist", true) require.Nil(t, actual, "err should be nothing") } -func TestGjsonDateTime(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayout) +func TestGjsonDateTimeStd(t *testing.T) { + pp := NewParserPool("gjson", nil, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - act = metric.GetDateTime("time2", false).(time.Time) + act = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) + require.Equal(t, exp, act) + + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) + act = metric.GetDateTime("time_sec_rfc3339_2", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDateTime("time2_not_exist", false).(time.Time) + act = metric.GetDateTime("not_exist", false).(time.Time) require.Equal(t, exp, act) - actual := metric.GetDateTime("time2_not_exist", true) + actual := metric.GetDateTime("not_exist", true) require.Nil(t, actual, "err should be nothing") } -func TestGjsonDateTime64(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayout) +func TestGjsonDateTimeCh(t *testing.T) { + pp := NewParserPool("gjson", nil, "", TSLayoutCh) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) + act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) + require.Equal(t, exp, act) +} + +func TestGjsonDateTime64Std(t *testing.T) { + pp := NewParserPool("gjson", nil, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - act = metric.GetDateTime64("time3", false).(time.Time) + act = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) require.Equal(t, exp, act) + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) + act = metric.GetDateTime64("time_ms_rfc3339_2", false).(time.Time).In(time.UTC) + require.EqualValues(t, exp, act) + exp = time.Time{} - act = metric.GetDateTime64("time3_not_exist", false).(time.Time) + act = metric.GetDateTime64("not_exist", false).(time.Time) require.Equal(t, exp, act) - actual := metric.GetDateTime64("time3_not_exist", true) + actual := metric.GetDateTime64("not_exist", true) require.Nil(t, actual, "err should be nothing") } +func TestGjsonDateTime64Ch(t *testing.T) { + pp := NewParserPool("gjson", nil, "", TSLayoutCh) + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + var exp, act time.Time + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) + act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) + require.Equal(t, exp, act) +} + func TestGjsonElasticDateTime(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayout) + pp := NewParserPool("gjson", nil, "", TSLayoutStd) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -139,19 +171,19 @@ func TestGjsonElasticDateTime(t *testing.T) { // {"date": "2019-12-16T12:10:30Z"} // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC exp = 1576498230 - act = metric.GetElasticDateTime("time2", false).(int64) + act = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) require.Equal(t, exp, act) exp = -62135596800 - act = metric.GetElasticDateTime("time2_not_exist", false).(int64) + act = metric.GetElasticDateTime("not_exist", false).(int64) require.Equal(t, exp, act) - actual := metric.GetElasticDateTime("time2_not_exist", true) + actual := metric.GetElasticDateTime("not_exist", true) require.Nil(t, actual, "err should be nothing") } func TestGjsonArray(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayout) + pp := NewParserPool("gjson", nil, "", nil) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) diff --git a/parser/parser.go b/parser/parser.go index 0639cf54..f9af10a9 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -23,9 +23,11 @@ import ( ) var ( - TSLayout = []string{"2006-01-02", time.RFC3339Nano, time.RFC3339Nano, "Local"} - TimeZone *time.Location - Epoch = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC) + TSLayoutCh = []string{"2006-01-02", "2006-01-02 15:04:05", "2006-01-02 15:04:05.999999999", "Local"} + TSLayoutStd = []string{"2006-01-02", time.RFC3339Nano, time.RFC3339Nano, "Local"} + TSLayout = []string{"2006-01-02", time.RFC3339Nano, time.RFC3339Nano, "Local"} + TimeZone *time.Location + Epoch = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC) ) // Parse is the Parser interface @@ -43,7 +45,11 @@ type Pool struct { // NewParserPool create a parser pool func NewParserPool(name string, csvFormat []string, delimiter string, tsLayout []string) *Pool { - TSLayout = tsLayout + if tsLayout == nil { + TSLayout = TSLayoutStd + } else { + TSLayout = tsLayout + } TimeZone, _ = time.LoadLocation(TSLayout[3]) return &Pool{ name: name, diff --git a/parser/parser_test.go b/parser/parser_test.go index 48ec72dd..f5a4f987 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -35,9 +35,13 @@ var jsonSample = []byte(`{ "success":0, "percent":0.11, "mp": {"i" : [1,2,3], "f": [1.1,2.2,3.3], "s": ["aa","bb","cc"], "e": []}, - "time1": "2019-12-16", - "time2": "2019-12-16T12:10:30Z", - "time3": "2019-12-16T12:10:30.123Z", + "date1": "2019-12-16", + "time_sec_rfc3339_1": "2019-12-16T12:10:30Z", + "time_sec_rfc3339_2": "2019-12-16T12:10:30+08:00", + "time_sec_clickhouse_1": "2019-12-16 12:10:30", + "time_ms_rfc3339_1": "2019-12-16T12:10:30.123Z", + "time_ms_rfc3339_2": "2019-12-16T12:10:30.123+08:00", + "time_ms_clickhouse_1": "2019-12-16 12:10:30.123", "array_int": [1,2,3], "array_float": [1.1,2.2,3.3], "array_string": ["aa","bb","cc"], @@ -48,8 +52,21 @@ var jsonSample = []byte(`{ var jsonSample2 = []byte(`{"date":"2021-01-02","ip":"192.168.0.3","floatvalue":425.633,"doublevalue":571.2464722672763,"novalue":" ","metric":"CPU_Idle_Time","service":"Web3","listvalue":["aaa","bbb","ccc"],"addint":123,"adddouble":571.2464722672763,"addstring":"add","value":123,"timestamp":"2021-01-02 21:06:00"}`) -var csvSampleSchema = []string{"its", "percent", "channel", "time1", "time2", "time3", "array_int", "array_float", "array_string", "array_empty"} -var csvSample = []byte(`1536813227,"0.11","escaped_""ws",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30.123Z,"[1,2,3]","[1.1,2.2,3.3]","[aa,bb,cc]","[]"`) +var csvSampleSchema = []string{"its", + "percent", + "channel", + "date1", + "time_sec_rfc3339_1", + "time_sec_rfc3339_2", + "time_sec_clickhouse_1", + "time_ms_rfc3339_1", + "time_ms_rfc3339_2", + "time_ms_clickhouse_1", + "array_int", + "array_float", + "array_string", + "array_empty"} +var csvSample = []byte(`1536813227,"0.11","escaped_""ws",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[aa,bb,cc]","[]"`) func BenchmarkUnmarshalljson(b *testing.B) { mp := map[string]interface{}{} From 46f9c175d6c0e8f82b88f81e5a30042605538165 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 9 Mar 2021 15:04:06 +0800 Subject: [PATCH 065/404] fix typo --- docs/dev/design.md | 2 +- docs/dev/introduction.md | 2 +- go.sum | 32 ++++++++++++++++++++++++++++++++ 3 files changed, 34 insertions(+), 2 deletions(-) diff --git a/docs/dev/design.md b/docs/dev/design.md index 8f3dc979..806d1e94 100644 --- a/docs/dev/design.md +++ b/docs/dev/design.md @@ -16,7 +16,7 @@ The flow is: - Fetch message via kafka-go or samara, which starts internally an goroutine for each partition. - Parse messages in a global goroutine pool(pool size is customizable), fill the result to a ring according to the message's partition and offset. - Generate a batch if messages in a ring reach a batchSize bondary, or flush timer fire. This ensures offset/batchSize be same for all messages inside a batch. -- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). Batch is routed according to `(kafka_offset/roundup(batch_size))%clickhouse_shards`. +- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). Batch is routed according to `(kafka_offset/roundup(buffer_size))%clickhouse_shards`. ## Sharding with custom key and policy diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index b52492d3..53672bf1 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -224,7 +224,7 @@ Kerberos setup is complex. Please ensure [`kafka-console-consumer.sh`](https://d Every message is routed to a determined ClickHouse shard. -By default, the shard number is caculated by `(kafka_offset/roundup(batch_size))%clickhouse_shards`, where `roundup()` round upward an unsigned integer to the the nearest 2^n. +By default, the shard number is caculated by `(kafka_offset/roundup(buffer_size))%clickhouse_shards`, where `roundup()` round upward an unsigned integer to the the nearest 2^n. This above expression can be customized with `shardingKey` and `shardingPolicy`. `shardingKey` value is a column name. `shardingPolicy` value could be: diff --git a/go.sum b/go.sum index f48f15aa..0036ee7f 100644 --- a/go.sum +++ b/go.sum @@ -10,10 +10,13 @@ github.com/Shopify/sarama v1.27.2 h1:1EyY1dsxNDUQEv0O/4TsjosHI2CgB1uo9H/v56xzTxc github.com/Shopify/sarama v1.27.2/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt/Mc93II= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= +github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4 h1:Hs82Z41s6SdL1CELW+XaDYmOH4hkBN4/N9og/AsOv7E= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 h1:zOVTBdCKFd9JbCKz9/nt+FovbjPFmb7mUnp8nH9fQBA= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= @@ -31,6 +34,7 @@ github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+ github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= +github.com/creack/pty v1.1.9 h1:uDmaGzcdjhF4i/plgjmEsriH11Y0o7RKapEf/LDaM3w= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -52,13 +56,20 @@ github.com/frankban/quicktest v1.10.2/go.mod h1:K+q6oSqb0W0Ininfk863uOk1lMy69l/P github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0 h1:wDJmvq38kDhkVxi50ni9ykkdUr1PKgqKOoi01fa0Mdk= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0 h1:MP4Eh7ZCb31lleYCFuwm0oe4/YGak+5l1vA2NOE80nA= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= +github.com/go-sql-driver/mysql v1.4.0 h1:7LxgVwFb2hIQtMm87NdgAVfXjnt4OePseqT1tKx+opk= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= +github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gogo/protobuf v1.1.1 h1:72R+M5VuhED/KujmZVcIquuo8mBgX4oVda//DQb3PXo= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d h1:lBXNCxVENCipq4D1Is42JVOP4eQjlB8TQ6H69Yx5J9Q= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= @@ -79,9 +90,11 @@ github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMyw github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gops v0.3.12 h1:aDEe2KOOYPOOcvhmCzmlrO9YJbLVJIe0ilqlzDrH+eA= github.com/google/gops v0.3.12/go.mod h1:38bMPVKFh+1X106CPpbLAWtZIR1+xwgzT9gew0kn6w4= +github.com/google/renameio v0.1.0 h1:GOZbcHa3HfsPKPlmyPyN2KEohoMXOhdMbHrvbpl2QaA= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= @@ -93,6 +106,7 @@ github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uc github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869/go.mod h1:cJ6Cj7dQo+O6GJNiMx+Pa94qKj+TG8ONdKHgMNIyyag= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= +github.com/jmoiron/sqlx v1.2.0 h1:41Ip0zITnmWNR/vHV+S4m+VoUivnWY5E4OJfLZjCJMA= github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= @@ -102,18 +116,23 @@ github.com/json-iterator/go v1.1.8 h1:QiWkFLKq0T7mpzwOTu6BzNDbfTE8OLrYhVKYMLF46O github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/julienschmidt/httprouter v1.2.0 h1:TDTW5Yz1mjftljbcKqRcrYhd4XeOoI98t+9HbQbYf7g= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19 h1:WjT3fLi9n8YWh/Ih8Q1LHAPsTqGddPcHqscN+PJ3i68= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= +github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.11.0 h1:wJbzvpYMVGG9iTI9VxpnNZfd4DzMPoCWze3GgSqz8yg= github.com/klauspost/compress v1.11.0/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pty v1.1.1 h1:VkoXIwSboBpnk99O/KFauAEILuNHv5DVFKZMBN/gUgw= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -124,7 +143,9 @@ github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f h1:sgU github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f/go.mod h1:UGmTpUd3rjbtfIpwAPrcfmGf/Z1HS95TATB+m57TPB8= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 h1:Bvq8AziQ5jFF4BHGAEDSqwPW1NJS3XshxbRCxtjFAZc= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042/go.mod h1:TPpsiPUEh0zFL1Snz4crhMlBe60PYxRHr5oFF3rRYg0= +github.com/lib/pq v1.0.0 h1:X5PMW56eZitiTeO7tKzZxFCSpbFZJtkMMooicw2us9A= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/mattn/go-sqlite3 v1.9.0 h1:pDRiWfl+++eC2FEFRy6jXmQlvp4Yh3z1MJKg4UeYM/4= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -134,6 +155,7 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223 h1:F9x/1yl3T2AeKLr2AMdilSD8+f9bvMnNN8VS5iDtovc= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nacos-group/nacos-sdk-go v1.0.1 h1:VNmXGlSS28xOmkO5Nxk5WRp6f1HMosAmG9pDtcnUFcw= github.com/nacos-group/nacos-sdk-go v1.0.1/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= @@ -166,9 +188,11 @@ github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLk github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 h1:MkV+77GLUNo5oJ0jf870itWm3D0Sjh7+Za9gazKc5LQ= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rogpeppe/go-internal v1.3.0 h1:RR9dF3JtopPvtkroDZuVD7qquD0bnHlKSqaQhgwt8yk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= +github.com/shirou/gopsutil v2.20.4+incompatible h1:cMT4rxS55zx9NVUnCkrmXCsEB/RNfG9SwHY9evtX8Ng= github.com/shirou/gopsutil v2.20.4+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= @@ -183,6 +207,7 @@ github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasO github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5NYJe/zRYDwOu9ku6YHy+Iw7l5DM= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= @@ -206,6 +231,7 @@ github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c h1:u40Z8hqBAAQyv+vATcGgV github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= github.com/xdg/stringprep v1.0.0 h1:d9X0esnoa3dFsV0FG35rAT0RIhYFlPq7MiP+DW89La0= github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= +github.com/xlab/treeprint v1.0.0 h1:J0TkWtiuYgtdlrkkrDLISYBQ92M+X5m4LrIIMKrbDTs= github.com/xlab/treeprint v1.0.0/go.mod h1:IoImgRak9i3zJyuxOKUP1v4UZd1tMoKkq/Cimt1uhCg= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -224,6 +250,7 @@ golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rB golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e h1:JgcxKXxCjrA2tyDP/aNU9K0Ck5Czfk6C7e2tMw7+bSI= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -235,6 +262,7 @@ golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -270,11 +298,13 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/errgo.v2 v2.1.0 h1:0vLT13EuvQ0hNvakwLuFZ/jYrLp5F3kcWHXdRggjCE8= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/ini.v1 v1.42.0 h1:7N3gPTt50s8GuLortA00n8AqRTk75qOP98+mTPpgzRk= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= @@ -290,10 +320,12 @@ gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU= gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.4 h1:/eiJrUcujPVeJ3xlSWaiNi3uSVmDGBK1pDHUHAnao1I= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +rsc.io/goversion v1.2.0 h1:SPn+NLTiAG7w30IRK/DKp1BjvpWabYgxlLp/+kx5J8w= rsc.io/goversion v1.2.0/go.mod h1:Eih9y/uIBS3ulggl7KNJ09xGSLcuNaLgmvvqa07sgfo= From 4e9ce3f87074a8bc10c560f2efe609593f99b042 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 11 Mar 2021 11:25:41 +0800 Subject: [PATCH 066/404] removed gops agent --- cmd/clickhouse_sinker/main.go | 4 ---- go.mod | 1 - go.sum | 41 ----------------------------------- 3 files changed, 46 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 3908b2d3..8610e005 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -39,7 +39,6 @@ import ( "github.com/housepower/clickhouse_sinker/util" _ "github.com/ClickHouse/clickhouse-go" - "github.com/google/gops/agent" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus/promhttp" log "github.com/sirupsen/logrus" @@ -136,9 +135,6 @@ func main() { log.SetFormatter(&log.TextFormatter{ FullTimestamp: true, }) - if err := agent.Listen(agent.Options{}); err != nil { - log.Fatalf("%+v", err) - } util.Run("clickhouse_sinker", func() error { var rcm config.RemoteConfManager diff --git a/go.mod b/go.mod index 6c5f0556..4cd58026 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,6 @@ require ( github.com/cespare/xxhash v1.1.0 github.com/fagongzi/goetty v1.6.0 github.com/golang/protobuf v1.4.2 // indirect - github.com/google/gops v0.3.12 github.com/nacos-group/nacos-sdk-go v1.0.1 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.3.0 diff --git a/go.sum b/go.sum index 0036ee7f..e420ae19 100644 --- a/go.sum +++ b/go.sum @@ -10,13 +10,9 @@ github.com/Shopify/sarama v1.27.2 h1:1EyY1dsxNDUQEv0O/4TsjosHI2CgB1uo9H/v56xzTxc github.com/Shopify/sarama v1.27.2/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt/Mc93II= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4 h1:Hs82Z41s6SdL1CELW+XaDYmOH4hkBN4/N9og/AsOv7E= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 h1:zOVTBdCKFd9JbCKz9/nt+FovbjPFmb7mUnp8nH9fQBA= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= @@ -34,7 +30,6 @@ github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+ github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= -github.com/creack/pty v1.1.9 h1:uDmaGzcdjhF4i/plgjmEsriH11Y0o7RKapEf/LDaM3w= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -56,20 +51,12 @@ github.com/frankban/quicktest v1.10.2/go.mod h1:K+q6oSqb0W0Ininfk863uOk1lMy69l/P github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/kit v0.9.0 h1:wDJmvq38kDhkVxi50ni9ykkdUr1PKgqKOoi01fa0Mdk= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= -github.com/go-logfmt/logfmt v0.4.0 h1:MP4Eh7ZCb31lleYCFuwm0oe4/YGak+5l1vA2NOE80nA= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= -github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= -github.com/go-sql-driver/mysql v1.4.0 h1:7LxgVwFb2hIQtMm87NdgAVfXjnt4OePseqT1tKx+opk= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= -github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/gogo/protobuf v1.1.1 h1:72R+M5VuhED/KujmZVcIquuo8mBgX4oVda//DQb3PXo= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d h1:lBXNCxVENCipq4D1Is42JVOP4eQjlB8TQ6H69Yx5J9Q= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= @@ -90,11 +77,7 @@ github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMyw github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/gops v0.3.12 h1:aDEe2KOOYPOOcvhmCzmlrO9YJbLVJIe0ilqlzDrH+eA= -github.com/google/gops v0.3.12/go.mod h1:38bMPVKFh+1X106CPpbLAWtZIR1+xwgzT9gew0kn6w4= -github.com/google/renameio v0.1.0 h1:GOZbcHa3HfsPKPlmyPyN2KEohoMXOhdMbHrvbpl2QaA= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= @@ -106,7 +89,6 @@ github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uc github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869/go.mod h1:cJ6Cj7dQo+O6GJNiMx+Pa94qKj+TG8ONdKHgMNIyyag= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= -github.com/jmoiron/sqlx v1.2.0 h1:41Ip0zITnmWNR/vHV+S4m+VoUivnWY5E4OJfLZjCJMA= github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= @@ -116,23 +98,17 @@ github.com/json-iterator/go v1.1.8 h1:QiWkFLKq0T7mpzwOTu6BzNDbfTE8OLrYhVKYMLF46O github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/julienschmidt/httprouter v1.2.0 h1:TDTW5Yz1mjftljbcKqRcrYhd4XeOoI98t+9HbQbYf7g= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19 h1:WjT3fLi9n8YWh/Ih8Q1LHAPsTqGddPcHqscN+PJ3i68= -github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= -github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.11.0 h1:wJbzvpYMVGG9iTI9VxpnNZfd4DzMPoCWze3GgSqz8yg= github.com/klauspost/compress v1.11.0/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= -github.com/kr/pty v1.1.1 h1:VkoXIwSboBpnk99O/KFauAEILuNHv5DVFKZMBN/gUgw= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -143,9 +119,7 @@ github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f h1:sgU github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f/go.mod h1:UGmTpUd3rjbtfIpwAPrcfmGf/Z1HS95TATB+m57TPB8= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 h1:Bvq8AziQ5jFF4BHGAEDSqwPW1NJS3XshxbRCxtjFAZc= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042/go.mod h1:TPpsiPUEh0zFL1Snz4crhMlBe60PYxRHr5oFF3rRYg0= -github.com/lib/pq v1.0.0 h1:X5PMW56eZitiTeO7tKzZxFCSpbFZJtkMMooicw2us9A= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= -github.com/mattn/go-sqlite3 v1.9.0 h1:pDRiWfl+++eC2FEFRy6jXmQlvp4Yh3z1MJKg4UeYM/4= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -155,7 +129,6 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223 h1:F9x/1yl3T2AeKLr2AMdilSD8+f9bvMnNN8VS5iDtovc= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nacos-group/nacos-sdk-go v1.0.1 h1:VNmXGlSS28xOmkO5Nxk5WRp6f1HMosAmG9pDtcnUFcw= github.com/nacos-group/nacos-sdk-go v1.0.1/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= @@ -188,12 +161,9 @@ github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLk github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 h1:MkV+77GLUNo5oJ0jf870itWm3D0Sjh7+Za9gazKc5LQ= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= -github.com/rogpeppe/go-internal v1.3.0 h1:RR9dF3JtopPvtkroDZuVD7qquD0bnHlKSqaQhgwt8yk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= -github.com/shirou/gopsutil v2.20.4+incompatible h1:cMT4rxS55zx9NVUnCkrmXCsEB/RNfG9SwHY9evtX8Ng= -github.com/shirou/gopsutil v2.20.4+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= @@ -207,7 +177,6 @@ github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasO github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5NYJe/zRYDwOu9ku6YHy+Iw7l5DM= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= @@ -231,8 +200,6 @@ github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c h1:u40Z8hqBAAQyv+vATcGgV github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= github.com/xdg/stringprep v1.0.0 h1:d9X0esnoa3dFsV0FG35rAT0RIhYFlPq7MiP+DW89La0= github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= -github.com/xlab/treeprint v1.0.0 h1:J0TkWtiuYgtdlrkkrDLISYBQ92M+X5m4LrIIMKrbDTs= -github.com/xlab/treeprint v1.0.0/go.mod h1:IoImgRak9i3zJyuxOKUP1v4UZd1tMoKkq/Cimt1uhCg= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= @@ -250,7 +217,6 @@ golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rB golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e h1:JgcxKXxCjrA2tyDP/aNU9K0Ck5Czfk6C7e2tMw7+bSI= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -262,7 +228,6 @@ golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -271,7 +236,6 @@ golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200917061948-648f2a039071 h1:t0H7WMwCt9t0LnLSYz5zdZ/OiAtROxc5cHb5iHt3Xyw= golang.org/x/sys v0.0.0-20200917061948-648f2a039071/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -298,13 +262,11 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/errgo.v2 v2.1.0 h1:0vLT13EuvQ0hNvakwLuFZ/jYrLp5F3kcWHXdRggjCE8= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/ini.v1 v1.42.0 h1:7N3gPTt50s8GuLortA00n8AqRTk75qOP98+mTPpgzRk= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= @@ -320,12 +282,9 @@ gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU= gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.4 h1:/eiJrUcujPVeJ3xlSWaiNi3uSVmDGBK1pDHUHAnao1I= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -rsc.io/goversion v1.2.0 h1:SPn+NLTiAG7w30IRK/DKp1BjvpWabYgxlLp/+kx5J8w= -rsc.io/goversion v1.2.0/go.mod h1:Eih9y/uIBS3ulggl7KNJ09xGSLcuNaLgmvvqa07sgfo= From ca2442770318ecf950be52787e40282194e7f34c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 1 Apr 2021 13:05:02 +0800 Subject: [PATCH 067/404] detect distributed tables --- config/config.go | 17 ++++----- docker/test_dynamic_schema.json | 5 ++- docs/configuration/config.md | 8 ++--- output/clickhouse.go | 61 ++++++++++++++++++++++++--------- 4 files changed, 57 insertions(+), 34 deletions(-) diff --git a/config/config.go b/config/config.go index 0124c2fb..4703cace 100644 --- a/config/config.go +++ b/config/config.go @@ -87,9 +87,10 @@ type KafkaConfig struct { // ClickHouseConfig configuration parameters type ClickHouseConfig struct { - DB string - Hosts [][]string - Port int + Cluster string + DB string + Hosts [][]string + Port int Username string Password string @@ -124,10 +125,8 @@ type TaskConfig struct { } `json:"dims"` // DynamicSchema will add columns present in message to clickhouse. Requires AutoSchema be true. DynamicSchema struct { - Enable bool - Cluster string - DistTblPrefix string - MaxDims int // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack + Enable bool + MaxDims int // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack } // ShardingKey is the column name to which sharding against @@ -154,7 +153,6 @@ const ( defaultLayoutDateTime = time.RFC3339 defaultLayoutDateTime64 = time.RFC3339Nano defaultTimeZone = "Local" - defaultDistTblPrefix = "dist_" defaultLogLevel = "info" ) @@ -244,9 +242,6 @@ func (cfg *Config) Normallize() (err error) { err = errors.Errorf("Parser %s doesn't support DynamicSchema", cfg.Task.Parser) return } - if cfg.Task.DynamicSchema.DistTblPrefix == "" { - cfg.Task.DynamicSchema.DistTblPrefix = defaultDistTblPrefix - } } switch strings.ToLower(cfg.LogLevel) { case "panic", "fatal", "error", "warn", "warning", "info", "debug", "trace": diff --git a/docker/test_dynamic_schema.json b/docker/test_dynamic_schema.json index 12fa59ff..e74e61a1 100644 --- a/docker/test_dynamic_schema.json +++ b/docker/test_dynamic_schema.json @@ -1,5 +1,6 @@ { "clickhouse": { + "cluster": "", "hosts": [ [ "127.0.0.1" @@ -27,9 +28,7 @@ "day" ], "dynamicSchema": { - "enable": true, - "cluster": "", - "distTblPrefix": "" + "enable": true }, "bufferSize": 50000 }, diff --git a/docs/configuration/config.md b/docs/configuration/config.md index bcbb9108..689c60db 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -5,6 +5,8 @@ { // clickhouse configs, it's map[string]ClickHouse for multiple clickhouse "clickhouse": { + // cluster the ClickHouse node belongs + "cluster": "test", // hosts for connection, it's Array(Array(String)) // we can put hosts with same shard into the inner array // it helps data deduplication for ReplicateMergeTree when driver error occurs @@ -98,10 +100,8 @@ "dynamicSchema": { // whether enable this feature, default to false "enable": true, - // cluster the ClickHouse node belongs - "cluster": "test", - // distributed table name prefix, default to "dist_" - "distTblPrefix": "" + // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack + "maxDims": 1024 }, // shardingKey is the column name to which sharding against diff --git a/output/clickhouse.go b/output/clickhouse.go index 72623e83..47296243 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -222,12 +222,12 @@ func (c *ClickHouse) initSchema() (err error) { } func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { - var sqls []string + var queries []string var onCluster string taskCfg := &c.cfg.Task chCfg := &c.cfg.Clickhouse - if taskCfg.DynamicSchema.Cluster != "" { - onCluster = fmt.Sprintf("ON CLUSTER %s", taskCfg.DynamicSchema.Cluster) + if chCfg.Cluster != "" { + onCluster = fmt.Sprintf("ON CLUSTER %s", chCfg.Cluster) } maxDims := math.MaxInt16 if taskCfg.DynamicSchema.MaxDims > 0 { @@ -258,27 +258,56 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { err = errors.Errorf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) return false } - sql := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS %s %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) - sqls = append(sqls, sql) + query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS %s %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) + queries = append(queries, query) return true }) if err != nil { return } - if taskCfg.DynamicSchema.Cluster != "" { - distTableName := taskCfg.DynamicSchema.DistTblPrefix + taskCfg.TableName - sqls = append(sqls, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s %s", chCfg.DB, distTableName, onCluster)) - sqls = append(sqls, fmt.Sprintf("CREATE TABLE %s.%s %s AS %s ENGINE = Distributed(%s, %s, %s);", - chCfg.DB, distTableName, onCluster, taskCfg.TableName, - taskCfg.DynamicSchema.Cluster, chCfg.DB, taskCfg.TableName)) + if chCfg.Cluster != "" { + var distTbls []string + if distTbls, err = c.getDistTbls(); err != nil { + return + } + for _, distTbl := range distTbls { + queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s %s", chCfg.DB, distTbl, onCluster)) + queries = append(queries, fmt.Sprintf("CREATE TABLE %s.%s %s AS %s ENGINE = Distributed(%s, %s, %s);", + chCfg.DB, distTbl, onCluster, taskCfg.TableName, + chCfg.Cluster, chCfg.DB, taskCfg.TableName)) + } } conn := pool.GetConn(0) - for _, sql := range sqls { - log.Infof("%s: executing sql=> %s", taskCfg.Name, sql) - if _, err = conn.Exec(sql); err != nil { - err = errors.Wrapf(err, sql) - return err + for _, query := range queries { + log.Infof("%s: executing sql=> %s", taskCfg.Name, query) + if _, err = conn.Exec(query); err != nil { + err = errors.Wrapf(err, query) + return + } + } + return +} + +func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { + taskCfg := &c.cfg.Task + chCfg := &c.cfg.Clickhouse + conn := pool.GetConn(0) + query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, chCfg.DB, chCfg.DB, taskCfg.TableName) + log.Infof("%s: executing sql=> %s", taskCfg.Name, query) + + var rows *sql.Rows + if rows, err = conn.Query(query); err != nil { + err = errors.Wrapf(err, "") + return + } + defer rows.Close() + for rows.Next() { + var name string + if err = rows.Scan(&name); err != nil { + err = errors.Wrapf(err, "") + return } + distTbls = append(distTbls, name) } return } From 9b8f6fccf65478b67cb935b424716392cb8093a0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 7 Apr 2021 18:09:19 +0800 Subject: [PATCH 068/404] automatically detect date formats --- cmd/clickhouse_sinker/main.go | 2 +- config/config.go | 38 ++++-------- docs/configuration/config.md | 6 -- docs/dev/introduction.md | 2 +- go.test.sh | 18 +++--- parser/csv.go | 27 +++++---- parser/csv_test.go | 54 ++++++----------- parser/fastjson.go | 10 ++-- parser/fastjson_test.go | 54 ++++++----------- parser/gjson.go | 10 ++-- parser/gjson_test.go | 52 ++++++----------- parser/parser.go | 107 ++++++++++++++++++++++++++-------- task/task.go | 4 +- 13 files changed, 187 insertions(+), 197 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 8610e005..3d1512d3 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -125,7 +125,7 @@ func init() { func GenTask(cfg *config.Config) (taskImpl *task.Service) { taskCfg := &cfg.Task ck := output.NewClickHouse(cfg) - pp := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, []string{taskCfg.LayoutDate, taskCfg.LayoutDateTime, taskCfg.LayoutDateTime64, taskCfg.TimeZone}) + pp, _ := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone) inputer := input.NewInputer(taskCfg.KafkaClient) taskImpl = task.NewTaskService(inputer, ck, pp, cfg) return diff --git a/config/config.go b/config/config.go index 4703cace..a49b2de3 100644 --- a/config/config.go +++ b/config/config.go @@ -19,7 +19,6 @@ import ( "encoding/json" "io/ioutil" "strings" - "time" "github.com/housepower/clickhouse_sinker/util" @@ -134,26 +133,20 @@ type TaskConfig struct { // ShardingPolicy is `stripe,`(requires ShardingKey be numerical) or `hash`(requires ShardingKey be string) ShardingPolicy string `json:"shardingPolicy,omitempty"` - FlushInterval int `json:"flushInterval,omitempty"` - BufferSize int `json:"bufferSize,omitempty"` - MinBufferSize int `json:"minBufferSize,omitempty"` - MsgSizeHint int `json:"msgSizeHint,omitempty"` - LayoutDate string `json:"layoutDate,omitempty"` - LayoutDateTime string `json:"layoutDateTime,omitempty"` - LayoutDateTime64 string `json:"layoutDateTime64,omitempty"` - TimeZone string `json:"timezone"` + FlushInterval int `json:"flushInterval,omitempty"` + BufferSize int `json:"bufferSize,omitempty"` + MinBufferSize int `json:"minBufferSize,omitempty"` + MsgSizeHint int `json:"msgSizeHint,omitempty"` + TimeZone string `json:"timezone"` } const ( - defaultFlushInterval = 3 - defaultBufferSize = 1 << 20 //1048576 - defaultMinBufferSize = 1 << 14 // 16384 - defaultMsgSizeHint = 1000 - defaultLayoutDate = "2006-01-02" - defaultLayoutDateTime = time.RFC3339 - defaultLayoutDateTime64 = time.RFC3339Nano - defaultTimeZone = "Local" - defaultLogLevel = "info" + defaultFlushInterval = 3 + defaultBufferSize = 1 << 20 //1048576 + defaultMinBufferSize = 1 << 14 // 16384 + defaultMsgSizeHint = 1000 + defaultTimeZone = "Local" + defaultLogLevel = "info" ) func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { @@ -225,15 +218,6 @@ func (cfg *Config) Normallize() (err error) { if cfg.Task.MsgSizeHint <= 0 { cfg.Task.MsgSizeHint = defaultMsgSizeHint } - if cfg.Task.LayoutDate == "" { - cfg.Task.LayoutDate = defaultLayoutDate - } - if cfg.Task.LayoutDateTime == "" { - cfg.Task.LayoutDateTime = defaultLayoutDateTime - } - if cfg.Task.LayoutDateTime64 == "" { - cfg.Task.LayoutDateTime64 = defaultLayoutDateTime64 - } if cfg.Task.TimeZone == "" { cfg.Task.TimeZone = defaultTimeZone } diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 689c60db..bcc31fbe 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -118,12 +118,6 @@ // estimated avg message size. kafka-go needs this to determize receive buffer size. default to 1000. "msgSizeHint": 1000, - // Date format in message, default to "2006-01-02". - "layoutDate": "", - // DateTime format in message, default to "2006-01-02T15:04:05Z07:00" (aka time.RFC3339). - "layoutDateTime": "", - // DateTime64 format in message, default to "2006-01-02T15:04:05.999999999Z07:00" (aka time.RFC3339Nano). - "layoutDateTime64": "", // In the absence of time zone information, interprets the time as in the given location. Default to "Local" (aka /etc/localtime of the machine on which sinker runs) "timezone": "" }, diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 53672bf1..54cb62e5 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -28,7 +28,7 @@ Refers to [design](./design.md) for how it works. - [x] UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 - [x] Float32, Float64 - [x] String, FixedString, LowCardinality(String) -- [x] Date, DateTime, DateTime64 (custom layout parser) +- [x] Date, DateTime, DateTime64. Support [these layouts](https://github.com/housepower/clickhouse_sinker/blob/master/parser/parser.go). - [x] Array(T), where T is one of above basic types - [x] Nullable(T), where T is one of above basic types - [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) diff --git a/go.test.sh b/go.test.sh index 74690bec..265562c4 100755 --- a/go.test.sh +++ b/go.test.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -## create table +echo "create tables" curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_fixed_schema' curl "localhost:8123" -d 'CREATE TABLE test_fixed_schema ( @@ -23,7 +23,6 @@ counts=`curl "localhost:8123" -d 'SELECT count() FROM test_fixed_schema UNION AL echo "Got initial row counts => $counts" [ $counts = "0,0,0," ] || exit 1 -## send the messages to kafka now=`date --rfc-3339=ns` for i in `seq 1 10000`;do echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i }" @@ -41,17 +40,18 @@ for i in `seq 70001 100000`;do echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i }" done >> a.json echo "generated a.json" +echo "send messages to kafka" echo "cat /tmp/a.json | kafka-console-producer --topic topic1 --broker-list localhost:9092" > send.sh sudo docker cp a.json kafka:/tmp/ sudo docker cp send.sh kafka:/tmp/ sudo docker exec kafka sh /tmp/send.sh -## start clickhouse_sinker to consume +echo "start clickhouse_sinker to consume" timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/test_fixed_schema.json timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/test_auto_schema.json timeout 60 ./dist/clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.json -## check result +echo "check result" count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` echo "Got test_fixed_schema count => $count" [ $count -eq 100000 ] || exit 1 @@ -68,27 +68,27 @@ echo "Got test_dynamic_schema count => $count" [ $count -eq 100000 ] || exit 1 -## reset kafka consumer-group offsets +echo "reset kafka consumer-group offsets" sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_fixed_schema --all-topics --to-earliest sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_auto_schema --all-topics --to-earliest sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_dynamic_schema --all-topics --to-earliest -## truncate tables +echo "truncate tables" curl "localhost:8123" -d 'TRUNCATE TABLE test_fixed_schema' curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' curl "localhost:8123" -d 'TRUNCATE TABLE test_dynamic_schema' -## publish clickhouse_sinker config +echo "publish clickhouse_sinker config" ./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_fixed_schema.json ./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_auto_schema.json ./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_dynamic_schema.json -## start clickhouse_sinker to consume +echo "start clickhouse_sinker to consume" timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema -## check result +echo "check result" count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` echo "Got test_fixed_schema count => $count" [ $count -eq 100000 ] || exit 1 diff --git a/parser/csv.go b/parser/csv.go index 7cdd2e53..1bedfaad 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -30,36 +30,35 @@ var _ Parser = (*CsvParser)(nil) // CsvParser implementation to parse input from a CSV format per RFC 4180 type CsvParser struct { - title []string - delimiter string + pp *ParserPool } // Parse extract a list of comma-separated values from the data func (p *CsvParser) Parse(bs []byte) (metric model.Metric, err error) { r := csv.NewReader(bytes.NewReader(bs)) - r.FieldsPerRecord = len(p.title) - if len(p.delimiter) > 0 { - r.Comma = rune(p.delimiter[0]) + r.FieldsPerRecord = len(p.pp.csvFormat) + if len(p.pp.delimiter) > 0 { + r.Comma = rune(p.pp.delimiter[0]) } var value []string if value, err = r.Read(); err != nil { err = errors.Wrap(err, "") return } - metric = &CsvMetric{p.title, value} + metric = &CsvMetric{p.pp, value} return } // CsvMetic type CsvMetric struct { - titles []string + pp *ParserPool values []string } // Get returns the value corresponding to a column expects called // interpret the type func (c *CsvMetric) Get(key string) interface{} { - for i, k := range c.titles { + for i, k := range c.pp.csvFormat { if k == key && i < len(c.values) { return c.values[i] } @@ -70,7 +69,7 @@ func (c *CsvMetric) Get(key string) interface{} { // GetString get the value as string func (c *CsvMetric) GetString(key string, nullable bool) interface{} { _ = nullable // nullable can not be supported with csv - for i, k := range c.titles { + for i, k := range c.pp.csvFormat { if k == key && i < len(c.values) { return c.values[i] } @@ -81,7 +80,7 @@ func (c *CsvMetric) GetString(key string, nullable bool) interface{} { // GetFloat returns the value as float func (c *CsvMetric) GetFloat(key string, nullable bool) interface{} { _ = nullable // nullable can not be supported with csv - for i, k := range c.titles { + for i, k := range c.pp.csvFormat { if k == key && i < len(c.values) { n, _ := strconv.ParseFloat(c.values[i], 64) return n @@ -93,7 +92,7 @@ func (c *CsvMetric) GetFloat(key string, nullable bool) interface{} { // GetInt returns int func (c *CsvMetric) GetInt(key string, nullable bool) interface{} { _ = nullable // nullable can not be supported with csv - for i, k := range c.titles { + for i, k := range c.pp.csvFormat { if k == key && i < len(c.values) { n, _ := strconv.ParseInt(c.values[i], 10, 64) return n @@ -153,19 +152,19 @@ func (c *CsvMetric) GetDate(key string, nullable bool) interface{} { _ = nullable // nullable can not be supported with csv val := c.GetString(key, false).(string) - t, _ := time.ParseInLocation(TSLayout[0], val, TimeZone) + t, _ := c.pp.ParseDateTime(key, val) return t } func (c *CsvMetric) GetDateTime(key string, nullable bool) interface{} { val := c.GetString(key, false).(string) - t, _ := time.ParseInLocation(TSLayout[1], val, TimeZone) + t, _ := c.pp.ParseDateTime(key, val) return t } func (c *CsvMetric) GetDateTime64(key string, nullable bool) interface{} { val := c.GetString(key, false).(string) - t, _ := time.ParseInLocation(TSLayout[2], val, TimeZone) + t, _ := c.pp.ParseDateTime(key, val) return t } diff --git a/parser/csv_test.go b/parser/csv_test.go index 581f7c88..8038a3de 100644 --- a/parser/csv_test.go +++ b/parser/csv_test.go @@ -22,7 +22,7 @@ import ( ) func TestCsvInt(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", nil) + pp, _ := NewParserPool("csv", csvSampleSchema, "", "") parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -42,7 +42,7 @@ func TestCsvInt(t *testing.T) { } func TestCsvFloat(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", nil) + pp, _ := NewParserPool("csv", csvSampleSchema, "", "") parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -62,7 +62,7 @@ func TestCsvFloat(t *testing.T) { } func TestCsvString(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", nil) + pp, _ := NewParserPool("csv", csvSampleSchema, "", "") parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -82,7 +82,7 @@ func TestCsvString(t *testing.T) { } func TestCsvDate(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutStd) + pp, _ := NewParserPool("csv", csvSampleSchema, "", "") parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -94,8 +94,8 @@ func TestCsvDate(t *testing.T) { require.Equal(t, exp, act) } -func TestCsvDateTimeStd(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutStd) +func TestCsvDateTime(t *testing.T) { + pp, _ := NewParserPool("csv", csvSampleSchema, "", "") parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -110,26 +110,17 @@ func TestCsvDateTimeStd(t *testing.T) { act = metric.GetDateTime("time_sec_rfc3339_2", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) - exp = time.Time{} - act = metric.GetDateTime("not_exist", false).(time.Time) - require.Equal(t, exp, act) -} - -func TestCsvDateTimeCh(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutCh) - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDateTime("not_exist", false).(time.Time) + require.Equal(t, exp, act) } -func TestCsvDateTime64Std(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutStd) +func TestCsvDateTime64(t *testing.T) { + pp, _ := NewParserPool("csv", csvSampleSchema, "", "") parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -144,26 +135,17 @@ func TestCsvDateTime64Std(t *testing.T) { act = metric.GetDateTime64("time_ms_rfc3339_2", false).(time.Time).In(time.UTC) require.EqualValues(t, exp, act) - exp = time.Time{} - act = metric.GetDateTime64("not_exist", false).(time.Time) - require.Equal(t, exp, act) -} - -func TestCsvDateTime64Ch(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutCh) - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) + + exp = time.Time{} + act = metric.GetDateTime64("not_exist", false).(time.Time) + require.Equal(t, exp, act) } func TestCsvElasticDateTime(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayoutStd) + pp, _ := NewParserPool("csv", csvSampleSchema, "", "") parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) @@ -185,7 +167,7 @@ func TestCsvElasticDateTime(t *testing.T) { } func TestCsvArray(t *testing.T) { - pp := NewParserPool("csv", csvSampleSchema, "", TSLayout) + pp, _ := NewParserPool("csv", csvSampleSchema, "", "") parser := pp.Get() defer pp.Put(parser) metric, err := parser.Parse(csvSample) diff --git a/parser/fastjson.go b/parser/fastjson.go index b4fb4789..907f891d 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -29,6 +29,7 @@ var _ Parser = (*FastjsonParser)(nil) // FastjsonParser, parser for get data in json format type FastjsonParser struct { + pp *ParserPool fjp fastjson.Parser } @@ -38,11 +39,12 @@ func (p *FastjsonParser) Parse(bs []byte) (metric model.Metric, err error) { err = errors.Wrapf(err, "") return } - metric = &FastjsonMetric{value: value} + metric = &FastjsonMetric{pp: p.pp, value: value} return } type FastjsonMetric struct { + pp *ParserPool value *fastjson.Value } @@ -135,7 +137,7 @@ func (c *FastjsonMetric) GetDate(key string, nullable bool) interface{} { } val := c.GetString(key, false).(string) - t, _ := time.ParseInLocation(TSLayout[0], val, TimeZone) + t, _ := c.pp.ParseDateTime(key, val) return t } @@ -149,7 +151,7 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) interface{} { } val := c.GetString(key, false).(string) - t, _ := time.ParseInLocation(TSLayout[1], val, TimeZone) + t, _ := c.pp.ParseDateTime(key, val) return t } @@ -163,7 +165,7 @@ func (c *FastjsonMetric) GetDateTime64(key string, nullable bool) interface{} { } val := c.GetString(key, false).(string) - t, _ := time.ParseInLocation(TSLayout[2], val, TimeZone) + t, _ := c.pp.ParseDateTime(key, val) return t } diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go index 2323a6c3..63e595de 100644 --- a/parser/fastjson_test.go +++ b/parser/fastjson_test.go @@ -12,7 +12,7 @@ import ( ) func TestFastjsonInt(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", nil) + pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -39,7 +39,7 @@ func TestFastjsonInt(t *testing.T) { } func TestFastjsonFloat(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", nil) + pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -58,7 +58,7 @@ func TestFastjsonFloat(t *testing.T) { } func TestFastjsonString(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", nil) + pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -77,7 +77,7 @@ func TestFastjsonString(t *testing.T) { } func TestFastjsonDate(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayoutStd) + pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -95,8 +95,8 @@ func TestFastjsonDate(t *testing.T) { require.Nil(t, actual, "err should be nothing") } -func TestFastjsonDateTimeStd(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayoutStd) +func TestFastjsonDateTime(t *testing.T) { + pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -110,6 +110,10 @@ func TestFastjsonDateTimeStd(t *testing.T) { act = metric.GetDateTime("time_sec_rfc3339_2", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) + act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) + require.Equal(t, exp, act) + exp = time.Time{} act = metric.GetDateTime("not_exist", false).(time.Time) require.Equal(t, exp, act) @@ -118,20 +122,8 @@ func TestFastjsonDateTimeStd(t *testing.T) { require.Nil(t, actual, "err should be nothing") } -func TestFastjsonDateTimeCh(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayoutCh) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) - act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) -} - -func TestFastjsonDateTime64Std(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayoutStd) +func TestFastjsonDateTime64(t *testing.T) { + pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -145,6 +137,10 @@ func TestFastjsonDateTime64Std(t *testing.T) { act = metric.GetDateTime64("time_ms_rfc3339_2", false).(time.Time).In(time.UTC) require.EqualValues(t, exp, act) + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) + act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) + require.Equal(t, exp, act) + exp = time.Time{} act = metric.GetDateTime64("not_exist", false).(time.Time) require.Equal(t, exp, act) @@ -153,20 +149,8 @@ func TestFastjsonDateTime64Std(t *testing.T) { require.Nil(t, actual, "err should be nothing") } -func TestFastjsonDateTime64Ch(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayoutCh) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) - act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) -} - func TestFastjsonElasticDateTime(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", TSLayoutStd) + pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -187,7 +171,7 @@ func TestFastjsonElasticDateTime(t *testing.T) { } func TestFastjsonArray(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", nil) + pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -223,7 +207,7 @@ func TestFastjsonArray(t *testing.T) { } func TestFastjsonDetectSchema(t *testing.T) { - pp := NewParserPool("fastjson", nil, "", nil) + pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) diff --git a/parser/gjson.go b/parser/gjson.go index 56f506bb..dd35377a 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -27,14 +27,16 @@ import ( var _ Parser = (*GjsonParser)(nil) type GjsonParser struct { + pp *ParserPool } func (p *GjsonParser) Parse(bs []byte) (metric model.Metric, err error) { - metric = &GjsonMetric{string(bs)} + metric = &GjsonMetric{p.pp, string(bs)} return } type GjsonMetric struct { + pp *ParserPool raw string } @@ -104,7 +106,7 @@ func (c *GjsonMetric) GetDate(key string, nullable bool) interface{} { } val := r.String() - t, _ := time.ParseInLocation(TSLayout[0], val, TimeZone) + t, _ := c.pp.ParseDateTime(key, val) return t } @@ -119,7 +121,7 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) interface{} { } val := r.String() - t, _ := time.ParseInLocation(TSLayout[1], val, TimeZone) + t, _ := c.pp.ParseDateTime(key, val) return t } @@ -134,7 +136,7 @@ func (c *GjsonMetric) GetDateTime64(key string, nullable bool) interface{} { } val := r.String() - t, _ := time.ParseInLocation(TSLayout[2], val, TimeZone) + t, _ := c.pp.ParseDateTime(key, val) return t } diff --git a/parser/gjson_test.go b/parser/gjson_test.go index d7d40a9a..53b254bc 100644 --- a/parser/gjson_test.go +++ b/parser/gjson_test.go @@ -8,7 +8,7 @@ import ( ) func TestGjsonInt(t *testing.T) { - pp := NewParserPool("gjson", nil, "", nil) + pp, _ := NewParserPool("gjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -35,7 +35,7 @@ func TestGjsonInt(t *testing.T) { } func TestGjsonFloat(t *testing.T) { - pp := NewParserPool("gjson", nil, "", nil) + pp, _ := NewParserPool("gjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -54,7 +54,7 @@ func TestGjsonFloat(t *testing.T) { } func TestGjsonString(t *testing.T) { - pp := NewParserPool("gjson", nil, "", nil) + pp, _ := NewParserPool("gjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -73,7 +73,7 @@ func TestGjsonString(t *testing.T) { } func TestGjsonDate(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayoutStd) + pp, _ := NewParserPool("gjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -91,8 +91,8 @@ func TestGjsonDate(t *testing.T) { require.Nil(t, actual, "err should be nothing") } -func TestGjsonDateTimeStd(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayoutStd) +func TestGjsonDateTime(t *testing.T) { + pp, _ := NewParserPool("gjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -106,6 +106,10 @@ func TestGjsonDateTimeStd(t *testing.T) { act = metric.GetDateTime("time_sec_rfc3339_2", false).(time.Time).In(time.UTC) require.Equal(t, exp, act) + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) + act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) + require.Equal(t, exp, act) + exp = time.Time{} act = metric.GetDateTime("not_exist", false).(time.Time) require.Equal(t, exp, act) @@ -114,20 +118,8 @@ func TestGjsonDateTimeStd(t *testing.T) { require.Nil(t, actual, "err should be nothing") } -func TestGjsonDateTimeCh(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayoutCh) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) - act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) -} - -func TestGjsonDateTime64Std(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayoutStd) +func TestGjsonDateTime64(t *testing.T) { + pp, _ := NewParserPool("gjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -141,6 +133,10 @@ func TestGjsonDateTime64Std(t *testing.T) { act = metric.GetDateTime64("time_ms_rfc3339_2", false).(time.Time).In(time.UTC) require.EqualValues(t, exp, act) + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) + act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) + require.Equal(t, exp, act) + exp = time.Time{} act = metric.GetDateTime64("not_exist", false).(time.Time) require.Equal(t, exp, act) @@ -149,20 +145,8 @@ func TestGjsonDateTime64Std(t *testing.T) { require.Nil(t, actual, "err should be nothing") } -func TestGjsonDateTime64Ch(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayoutCh) - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) - act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) -} - func TestGjsonElasticDateTime(t *testing.T) { - pp := NewParserPool("gjson", nil, "", TSLayoutStd) + pp, _ := NewParserPool("gjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -183,7 +167,7 @@ func TestGjsonElasticDateTime(t *testing.T) { } func TestGjsonArray(t *testing.T) { - pp := NewParserPool("gjson", nil, "", nil) + pp, _ := NewParserPool("gjson", nil, "", "") parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) diff --git a/parser/parser.go b/parser/parser.go index f9af10a9..7400e19b 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -16,18 +16,50 @@ package parser import ( "encoding/json" + "strconv" "sync" "time" "github.com/housepower/clickhouse_sinker/model" + "github.com/pkg/errors" ) var ( - TSLayoutCh = []string{"2006-01-02", "2006-01-02 15:04:05", "2006-01-02 15:04:05.999999999", "Local"} - TSLayoutStd = []string{"2006-01-02", time.RFC3339Nano, time.RFC3339Nano, "Local"} - TSLayout = []string{"2006-01-02", time.RFC3339Nano, time.RFC3339Nano, "Local"} - TimeZone *time.Location - Epoch = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC) + Layouts = []string{ + //DateTime + "2006-01-02T15:04:05.999999999Z07:00", //time.RFC3339Nano, `date --iso-8601=ns` output format + "2006-01-02T15:04:05Z07:00", //time.RFC3339, `date --iso-8601=s` output format + "2006-01-02T15:04:05.999999999", + "2006-01-02T15:04:05", + "2006-01-02 15:04:05.999999999Z07:00", //`date --rfc-3339=ns` output format + "2006-01-02 15:04:05Z07:00", //`date --rfc-3339=s` output format + "2006-01-02 15:04:05.999999999", + "2006-01-02 15:04:05", + "Jan 02, 2006 15:04:05.999999999Z07:00", + "Jan 02, 2006 15:04:05.999999999", + "Jan 02, 2006 15:04:05", + "02/01/2006 15:04:05.999999999", + "02/01/06 15:04:05.999999999", + "02/Jan/2006 15:04:05 Z07:00", + "02/Jan/2006 15:04:05 -0700", + "Mon Jan _2 15:04:05 2006", //time.ANSIC + "Mon Jan _2 15:04:05 MST 2006", //time.UnixDate + "Mon Jan 02 15:04:05 -0700 2006", //time.RubyDate + "02 Jan 06 15:04 MST", //time.RFC822 + "02 Jan 06 15:04 -0700", //time.RFC822Z + "Monday, 02-Jan-06 15:04:05 MST", //time.RFC850 + "Mon, 02 Jan 2006 15:04:05 MST", //time.RFC1123 + "Mon, 02 Jan 2006 15:04:05 -0700", //time.RFC1123Z + "Mon Jan 02 15:04:05 MST 2006", + "Mon 02 Jan 2006 03:04:05 PM MST", // `date` default output format + //Date + "2006-01-02", + "02/01/2006", + "02/Jan/2006", + "Jan 02, 2006", + "Mon Jan 02, 2006", + } + Epoch = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC) ) // Parse is the Parser interface @@ -36,43 +68,48 @@ type Parser interface { } // Pool may be used for pooling Parsers for similarly typed JSONs. -type Pool struct { - name string - csvFormat []string - delimiter string - pool sync.Pool +type ParserPool struct { + name string + csvFormat []string + delimiter string + timeZone *time.Location + knownLayouts sync.Map + pool sync.Pool } -// NewParserPool create a parser pool -func NewParserPool(name string, csvFormat []string, delimiter string, tsLayout []string) *Pool { - if tsLayout == nil { - TSLayout = TSLayoutStd - } else { - TSLayout = tsLayout +// NewParserPool creates a parser pool +func NewParserPool(name string, csvFormat []string, delimiter string, timezone string) (pp *ParserPool, err error) { + var tz *time.Location + if timezone == "" { + tz = time.Local + } else if tz, err = time.LoadLocation(timezone); err != nil { + err = errors.Wrapf(err, "") + return } - TimeZone, _ = time.LoadLocation(TSLayout[3]) - return &Pool{ + pp = &ParserPool{ name: name, csvFormat: csvFormat, delimiter: delimiter, + timeZone: tz, } + return } // Get returns a Parser from pp. // // The Parser must be Put to pp after use. -func (pp *Pool) Get() Parser { +func (pp *ParserPool) Get() Parser { v := pp.pool.Get() if v == nil { switch pp.name { case "gjson": - return &GjsonParser{} + return &GjsonParser{pp: pp} case "fastjson": - return &FastjsonParser{} + return &FastjsonParser{pp: pp} case "csv": - return &CsvParser{pp.csvFormat, pp.delimiter} + return &CsvParser{pp: pp} default: - return &FastjsonParser{} + return &FastjsonParser{pp: pp} } } return v.(Parser) @@ -82,10 +119,32 @@ func (pp *Pool) Get() Parser { // // p and objects recursively returned from p cannot be used after p // is put into pp. -func (pp *Pool) Put(p Parser) { +func (pp *ParserPool) Put(p Parser) { pp.pool.Put(p) } +func (pp *ParserPool) ParseDateTime(key string, val string) (t time.Time, err error) { + var layout string + var lay interface{} + var ok bool + if lay, ok = pp.knownLayouts.Load(key); !ok { + for _, layout = range Layouts { + if t, err = time.ParseInLocation(layout, val, pp.timeZone); err == nil { + pp.knownLayouts.Store(key, layout) + return + } + } + pp.knownLayouts.Store(key, nil) + } + if lay == nil { + err = errors.Errorf("cannot parse time %s at field %s", strconv.Quote(val), key) + return + } + layout = lay.(string) + t, _ = time.ParseInLocation(layout, val, pp.timeZone) + return +} + func GetJSONShortStr(v interface{}) string { bs, _ := json.Marshal(v) return string(bs) diff --git a/task/task.go b/task/task.go index a4611dfb..1005efb8 100644 --- a/task/task.go +++ b/task/task.go @@ -52,7 +52,7 @@ type Service struct { stopped chan struct{} inputer input.Inputer clickhouse *output.ClickHouse - pp *parser.Pool + pp *parser.ParserPool cfg *config.Config dims []*model.ColumnWithType @@ -70,7 +70,7 @@ type Service struct { } // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances -func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *parser.Pool, cfg *config.Config) *Service { +func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *parser.ParserPool, cfg *config.Config) *Service { return &Service{ stopped: make(chan struct{}), inputer: inputer, From d1f7b4a55a3ef8ad896ed61d60fa1fca091f21d0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 7 Apr 2021 18:38:57 +0800 Subject: [PATCH 069/404] happy lint --- .github/workflows/lint.yml | 4 ++-- Makefile | 2 +- config/nacos.go | 8 +++---- docs/dev/introduction.md | 2 +- model/message.go | 2 +- output/clickhouse.go | 7 +++--- parser/csv.go | 14 +++++------ parser/csv_test.go | 46 ++++++++++++++++++------------------ parser/fastjson.go | 14 +++++------ parser/fastjson_test.go | 48 +++++++++++++++++++------------------- parser/gjson.go | 4 ++-- parser/gjson_test.go | 44 +++++++++++++++++----------------- parser/parser.go | 14 +++++------ task/ring.go | 2 +- task/task.go | 4 ++-- util/common.go | 2 +- 16 files changed, 109 insertions(+), 108 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 231f0665..d2cae262 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -16,13 +16,13 @@ jobs: uses: golangci/golangci-lint-action@v2 with: # Required: the version of golangci-lint is required and must be specified without patch version: we always use the latest patch version. - version: v1.37 + version: v1.39 # Optional: working directory, useful for monorepos # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true \ No newline at end of file diff --git a/Makefile b/Makefile index c0fe8d52..db2b9d5b 100644 --- a/Makefile +++ b/Makefile @@ -25,7 +25,7 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-dir conf/ diff --git a/config/nacos.go b/config/nacos.go index 0d606fce..1460a4ba 100644 --- a/config/nacos.go +++ b/config/nacos.go @@ -39,7 +39,7 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) var clientDir string if v, ok := properties["clientDir"]; ok { - clientDir = v.(string) + clientDir, _ = v.(string) } else { clientDir = "/tmp/nacos" } @@ -47,10 +47,10 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) group := constant.DEFAULT_GROUP //Empty string doesn't work! var ok bool if _, ok = properties["namespaceId"]; ok { - namespaceID = properties["namespaceId"].(string) + namespaceID, _ = properties["namespaceId"].(string) } if _, ok = properties["group"]; ok { - group = properties["group"].(string) + group, _ = properties["group"].(string) } cc := constant.ClientConfig{ NamespaceId: namespaceID, @@ -77,7 +77,7 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) ncm.group = group if _, ok = properties["dataId"]; ok { - ncm.dataID = properties["dataId"].(string) + ncm.dataID, _ = properties["dataId"].(string) } return } diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 54cb62e5..c47ca2d2 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -28,7 +28,7 @@ Refers to [design](./design.md) for how it works. - [x] UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 - [x] Float32, Float64 - [x] String, FixedString, LowCardinality(String) -- [x] Date, DateTime, DateTime64. Support [these layouts](https://github.com/housepower/clickhouse_sinker/blob/master/parser/parser.go). +- [x] Date, DateTime, DateTime64. Automatically detect [these date formats](https://github.com/housepower/clickhouse_sinker/blob/master/parser/parser.go). - [x] Array(T), where T is one of above basic types - [x] Nullable(T), where T is one of above basic types - [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) diff --git a/model/message.go b/model/message.go index ccc849e2..90c6f49f 100644 --- a/model/message.go +++ b/model/message.go @@ -70,7 +70,7 @@ func (bs *BatchSys) TryCommit() error { // ensure groups be committed orderly LOOP: for e := bs.groups.Front(); e != nil; { - grp := e.Value.(*BatchGroup) + grp, _ := e.Value.(*BatchGroup) if atomic.LoadInt32(&grp.PendWrite) != 0 { break LOOP } diff --git a/output/clickhouse.go b/output/clickhouse.go index 47296243..7f46b0a1 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -245,8 +245,8 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { log.Warnf("number of columns reaches upper limit %d", maxDims) return false } - strKey := key.(string) - strVal := value.(string) + strKey, _ := key.(string) + strVal, _ := value.(string) switch strVal { case "int": strVal = "Nullable(Int64)" @@ -292,7 +292,8 @@ func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { taskCfg := &c.cfg.Task chCfg := &c.cfg.Clickhouse conn := pool.GetConn(0) - query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, chCfg.DB, chCfg.DB, taskCfg.TableName) + query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, + chCfg.DB, chCfg.DB, taskCfg.TableName) log.Infof("%s: executing sql=> %s", taskCfg.Name, query) var rows *sql.Rows diff --git a/parser/csv.go b/parser/csv.go index 1bedfaad..e3c26b7f 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -30,7 +30,7 @@ var _ Parser = (*CsvParser)(nil) // CsvParser implementation to parse input from a CSV format per RFC 4180 type CsvParser struct { - pp *ParserPool + pp *Pool } // Parse extract a list of comma-separated values from the data @@ -51,7 +51,7 @@ func (p *CsvParser) Parse(bs []byte) (metric model.Metric, err error) { // CsvMetic type CsvMetric struct { - pp *ParserPool + pp *Pool values []string } @@ -106,7 +106,7 @@ func (c *CsvMetric) GetArray(key string, t string) interface{} { var err error var array []string var r *csv.Reader - val := c.GetString(key, false).(string) + val, _ := c.GetString(key, false).(string) valLen := len(val) if val == "" || val[0] != '[' || val[valLen-1] != ']' { goto QUIT @@ -151,26 +151,26 @@ QUIT: func (c *CsvMetric) GetDate(key string, nullable bool) interface{} { _ = nullable // nullable can not be supported with csv - val := c.GetString(key, false).(string) + val, _ := c.GetString(key, false).(string) t, _ := c.pp.ParseDateTime(key, val) return t } func (c *CsvMetric) GetDateTime(key string, nullable bool) interface{} { - val := c.GetString(key, false).(string) + val, _ := c.GetString(key, false).(string) t, _ := c.pp.ParseDateTime(key, val) return t } func (c *CsvMetric) GetDateTime64(key string, nullable bool) interface{} { - val := c.GetString(key, false).(string) + val, _ := c.GetString(key, false).(string) t, _ := c.pp.ParseDateTime(key, val) return t } func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) interface{} { _ = nullable // nullable can not be supported with csv - val := c.GetString(key, false).(string) + val, _ := c.GetString(key, false).(string) t, _ := time.Parse(time.RFC3339, val) return t.Unix() diff --git a/parser/csv_test.go b/parser/csv_test.go index 8038a3de..af4061b2 100644 --- a/parser/csv_test.go +++ b/parser/csv_test.go @@ -30,14 +30,14 @@ func TestCsvInt(t *testing.T) { var exp, act int64 exp = 1536813227 - act = metric.GetInt("its", false).(int64) + act, _ = metric.GetInt("its", false).(int64) require.Equal(t, exp, act) exp = 0 - act = metric.GetInt("not_exist", false).(int64) + act, _ = metric.GetInt("not_exist", false).(int64) require.Equal(t, exp, act) - act = metric.GetInt("not_exist", true).(int64) + act, _ = metric.GetInt("not_exist", true).(int64) require.Equal(t, exp, act) } @@ -50,14 +50,14 @@ func TestCsvFloat(t *testing.T) { var exp, act float64 exp = 0.11 - act = metric.GetFloat("percent", false).(float64) + act, _ = metric.GetFloat("percent", false).(float64) require.Equal(t, exp, act) exp = 0.0 - act = metric.GetFloat("not_exist", false).(float64) + act, _ = metric.GetFloat("not_exist", false).(float64) require.Equal(t, exp, act) - act = metric.GetFloat("not_exist", true).(float64) + act, _ = metric.GetFloat("not_exist", true).(float64) require.Equal(t, exp, act) } @@ -70,14 +70,14 @@ func TestCsvString(t *testing.T) { var exp, act string exp = `escaped_"ws` - act = metric.GetString("channel", false).(string) + act, _ = metric.GetString("channel", false).(string) require.Equal(t, exp, act) exp = "" - act = metric.GetString("not_exist", false).(string) + act, _ = metric.GetString("not_exist", false).(string) require.Equal(t, exp, act) - act = metric.GetString("not_exist", true).(string) + act, _ = metric.GetString("not_exist", true).(string) require.Equal(t, exp, act) } @@ -90,7 +90,7 @@ func TestCsvDate(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - act = metric.GetDate("date1", false).(time.Time) + act, _ = metric.GetDate("date1", false).(time.Time) require.Equal(t, exp, act) } @@ -103,7 +103,7 @@ func TestCsvDateTime(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - act = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) + act, _ = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) require.Equal(t, exp, act) exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) @@ -115,7 +115,7 @@ func TestCsvDateTime(t *testing.T) { require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDateTime("not_exist", false).(time.Time) + act, _ = metric.GetDateTime("not_exist", false).(time.Time) require.Equal(t, exp, act) } @@ -128,7 +128,7 @@ func TestCsvDateTime64(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - act = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) + act, _ = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) require.Equal(t, exp, act) exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) @@ -140,7 +140,7 @@ func TestCsvDateTime64(t *testing.T) { require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDateTime64("not_exist", false).(time.Time) + act, _ = metric.GetDateTime64("not_exist", false).(time.Time) require.Equal(t, exp, act) } @@ -155,14 +155,14 @@ func TestCsvElasticDateTime(t *testing.T) { // {"date": "2019-12-16T12:10:30Z"} // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC exp = 1576498230 - act = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) + act, _ = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) require.Equal(t, exp, act) exp = -62135596800 - act = metric.GetElasticDateTime("not_exist", false).(int64) + act, _ = metric.GetElasticDateTime("not_exist", false).(int64) require.Equal(t, exp, act) - act = metric.GetElasticDateTime("not_exist", true).(int64) + act, _ = metric.GetElasticDateTime("not_exist", true).(int64) require.Equal(t, exp, act) } @@ -173,27 +173,27 @@ func TestCsvArray(t *testing.T) { metric, err := parser.Parse(csvSample) require.Nil(t, err) - actI := metric.GetArray("array_int", "int").([]int64) + actI, _ := metric.GetArray("array_int", "int").([]int64) expI := []int64{1, 2, 3} require.Equal(t, expI, actI) - actF := metric.GetArray("array_float", "float").([]float64) + actF, _ := metric.GetArray("array_float", "float").([]float64) expF := []float64{1.1, 2.2, 3.3} require.Equal(t, expF, actF) - actS := metric.GetArray("array_string", "string").([]string) + actS, _ := metric.GetArray("array_string", "string").([]string) expS := []string{"aa", "bb", "cc"} require.Equal(t, expS, actS) - actIE := metric.GetArray("array_empty", "int").([]int64) + actIE, _ := metric.GetArray("array_empty", "int").([]int64) expIE := []int64{} require.Equal(t, expIE, actIE) - actFE := metric.GetArray("array_empty", "float").([]float64) + actFE, _ := metric.GetArray("array_empty", "float").([]float64) expFE := []float64{} require.Equal(t, expFE, actFE) - actSE := metric.GetArray("array_empty", "string").([]string) + actSE, _ := metric.GetArray("array_empty", "string").([]string) expSE := []string{} require.Equal(t, expSE, actSE) } diff --git a/parser/fastjson.go b/parser/fastjson.go index 907f891d..a74c1275 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -29,7 +29,7 @@ var _ Parser = (*FastjsonParser)(nil) // FastjsonParser, parser for get data in json format type FastjsonParser struct { - pp *ParserPool + pp *Pool fjp fastjson.Parser } @@ -44,7 +44,7 @@ func (p *FastjsonParser) Parse(bs []byte) (metric model.Metric, err error) { } type FastjsonMetric struct { - pp *ParserPool + pp *Pool value *fastjson.Value } @@ -136,7 +136,7 @@ func (c *FastjsonMetric) GetDate(key string, nullable bool) interface{} { return nil } - val := c.GetString(key, false).(string) + val, _ := c.GetString(key, false).(string) t, _ := c.pp.ParseDateTime(key, val) return t } @@ -146,11 +146,11 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) interface{} { return nil } - if v := c.GetFloat(key, false).(float64); v != 0 { + if v, _ := c.GetFloat(key, false).(float64); v != 0 { return time.Unix(int64(v), int64(v*1e9)%1e9) } - val := c.GetString(key, false).(string) + val, _ := c.GetString(key, false).(string) t, _ := c.pp.ParseDateTime(key, val) return t } @@ -160,11 +160,11 @@ func (c *FastjsonMetric) GetDateTime64(key string, nullable bool) interface{} { return nil } - if v := c.GetFloat(key, false).(float64); v != 0 { + if v, _ := c.GetFloat(key, false).(float64); v != 0 { return time.Unix(int64(v), int64(v*1e9)%1e9) } - val := c.GetString(key, false).(string) + val, _ := c.GetString(key, false).(string) t, _ := c.pp.ParseDateTime(key, val) return t } diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go index 63e595de..e26fa4f1 100644 --- a/parser/fastjson_test.go +++ b/parser/fastjson_test.go @@ -19,22 +19,22 @@ func TestFastjsonInt(t *testing.T) { var exp, act int64 exp = 1536813227 - act = metric.GetInt("its", false).(int64) + act, _ = metric.GetInt("its", false).(int64) require.Equal(t, exp, act) exp = 0 - act = metric.GetInt("not_exist", false).(int64) + act, _ = metric.GetInt("not_exist", false).(int64) require.Equal(t, exp, act) actual := metric.GetInt("not_exist", true) require.Nil(t, actual, "err should be nothing") exp = 0 - act = metric.GetInt("bool_false", false).(int64) + act, _ = metric.GetInt("bool_false", false).(int64) require.Equal(t, exp, act) exp = 1 - act = metric.GetInt("bool_true", false).(int64) + act, _ = metric.GetInt("bool_true", false).(int64) require.Equal(t, exp, act) } @@ -46,11 +46,11 @@ func TestFastjsonFloat(t *testing.T) { var exp, act float64 exp = 0.11 - act = metric.GetFloat("percent", false).(float64) + act, _ = metric.GetFloat("percent", false).(float64) require.Equal(t, exp, act) exp = 0.0 - act = metric.GetFloat("not_exist", false).(float64) + act, _ = metric.GetFloat("not_exist", false).(float64) require.Equal(t, exp, act) actual := metric.GetFloat("not_exist", true) @@ -65,11 +65,11 @@ func TestFastjsonString(t *testing.T) { var exp, act string exp = "ws" - act = metric.GetString("channel", false).(string) + act, _ = metric.GetString("channel", false).(string) require.Equal(t, exp, act) exp = "" - act = metric.GetString("not_exist", false).(string) + act, _ = metric.GetString("not_exist", false).(string) require.Equal(t, exp, act) actual := metric.GetString("not_exist", true) @@ -84,11 +84,11 @@ func TestFastjsonDate(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - act = metric.GetDate("date1", false).(time.Time) + act, _ = metric.GetDate("date1", false).(time.Time) require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDate("not_exist", false).(time.Time) + act, _ = metric.GetDate("not_exist", false).(time.Time) require.Equal(t, exp, act) actual := metric.GetDate("not_exist", true) @@ -103,7 +103,7 @@ func TestFastjsonDateTime(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - act = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) + act, _ = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) require.Equal(t, exp, act) exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) @@ -115,7 +115,7 @@ func TestFastjsonDateTime(t *testing.T) { require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDateTime("not_exist", false).(time.Time) + act, _ = metric.GetDateTime("not_exist", false).(time.Time) require.Equal(t, exp, act) actual := metric.GetDateTime("not_exist", true) @@ -130,7 +130,7 @@ func TestFastjsonDateTime64(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - act = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) + act, _ = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) require.Equal(t, exp, act) exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) @@ -142,7 +142,7 @@ func TestFastjsonDateTime64(t *testing.T) { require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDateTime64("not_exist", false).(time.Time) + act, _ = metric.GetDateTime64("not_exist", false).(time.Time) require.Equal(t, exp, act) actual := metric.GetDateTime64("not_exist", true) @@ -159,11 +159,11 @@ func TestFastjsonElasticDateTime(t *testing.T) { // {"date": "2019-12-16T12:10:30Z"} // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC exp = 1576498230 - act = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) + act, _ = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) require.Equal(t, exp, act) exp = -62135596800 - act = metric.GetElasticDateTime("not_exist", false).(int64) + act, _ = metric.GetElasticDateTime("not_exist", false).(int64) require.Equal(t, exp, act) actual := metric.GetElasticDateTime("not_exist", true) @@ -176,32 +176,32 @@ func TestFastjsonArray(t *testing.T) { defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - actI := metric.GetArray("array_int", "int").([]int64) + actI, _ := metric.GetArray("array_int", "int").([]int64) expI := []int64{1, 2, 3} require.Equal(t, expI, actI) - actF := metric.GetArray("array_float", "float").([]float64) + actF, _ := metric.GetArray("array_float", "float").([]float64) expF := []float64{1.1, 2.2, 3.3} require.Equal(t, expF, actF) - actS := metric.GetArray("array_string", "string").([]string) + actS, _ := metric.GetArray("array_string", "string").([]string) expS := []string{"aa", "bb", "cc"} require.Equal(t, expS, actS) - actIE := metric.GetArray("array_empty", "int").([]int64) + actIE, _ := metric.GetArray("array_empty", "int").([]int64) expIE := []int64{} require.Equal(t, expIE, actIE) - actFE := metric.GetArray("array_empty", "float").([]float64) + actFE, _ := metric.GetArray("array_empty", "float").([]float64) expFE := []float64{} require.Equal(t, expFE, actFE) - actSE := metric.GetArray("array_empty", "string").([]string) + actSE, _ := metric.GetArray("array_empty", "string").([]string) expSE := []string{} require.Equal(t, expSE, actSE) metric, _ = parser.Parse(jsonSample2) - actS = metric.GetArray("listvalue", "string").([]string) + actS, _ = metric.GetArray("listvalue", "string").([]string) expS = []string{"aaa", "bbb", "ccc"} require.Equal(t, expS, actS) } @@ -213,7 +213,7 @@ func TestFastjsonDetectSchema(t *testing.T) { metric, _ := parser.Parse(jsonSample) var nameAndTypes []string - c := metric.(*FastjsonMetric) + c, _ := metric.(*FastjsonMetric) var obj *fastjson.Object var err error if obj, err = c.value.Object(); err != nil { diff --git a/parser/gjson.go b/parser/gjson.go index dd35377a..cdd4e559 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -27,7 +27,7 @@ import ( var _ Parser = (*GjsonParser)(nil) type GjsonParser struct { - pp *ParserPool + pp *Pool } func (p *GjsonParser) Parse(bs []byte) (metric model.Metric, err error) { @@ -36,7 +36,7 @@ func (p *GjsonParser) Parse(bs []byte) (metric model.Metric, err error) { } type GjsonMetric struct { - pp *ParserPool + pp *Pool raw string } diff --git a/parser/gjson_test.go b/parser/gjson_test.go index 53b254bc..d1ca7468 100644 --- a/parser/gjson_test.go +++ b/parser/gjson_test.go @@ -15,22 +15,22 @@ func TestGjsonInt(t *testing.T) { var exp, act int64 exp = 1536813227 - act = metric.GetInt("its", false).(int64) + act, _ = metric.GetInt("its", false).(int64) require.Equal(t, exp, act) exp = 0 - act = metric.GetInt("not_exist", false).(int64) + act, _ = metric.GetInt("not_exist", false).(int64) require.Equal(t, exp, act) actual := metric.GetInt("not_exist", true) require.Nil(t, actual, "err should be nothing") exp = 0 - act = metric.GetInt("bool_false", false).(int64) + act, _ = metric.GetInt("bool_false", false).(int64) require.Equal(t, exp, act) exp = 1 - act = metric.GetInt("bool_true", false).(int64) + act, _ = metric.GetInt("bool_true", false).(int64) require.Equal(t, exp, act) } @@ -42,11 +42,11 @@ func TestGjsonFloat(t *testing.T) { var exp, act float64 exp = 0.11 - act = metric.GetFloat("percent", false).(float64) + act, _ = metric.GetFloat("percent", false).(float64) require.Equal(t, exp, act) exp = 0.0 - act = metric.GetFloat("not_exist", false).(float64) + act, _ = metric.GetFloat("not_exist", false).(float64) require.Equal(t, exp, act) actual := metric.GetFloat("not_exist", true) @@ -61,11 +61,11 @@ func TestGjsonString(t *testing.T) { var exp, act string exp = "ws" - act = metric.GetString("channel", false).(string) + act, _ = metric.GetString("channel", false).(string) require.Equal(t, exp, act) exp = "" - act = metric.GetString("not_exist", false).(string) + act, _ = metric.GetString("not_exist", false).(string) require.Equal(t, exp, act) actual := metric.GetString("not_exist", true) @@ -80,11 +80,11 @@ func TestGjsonDate(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - act = metric.GetDate("date1", false).(time.Time) + act, _ = metric.GetDate("date1", false).(time.Time) require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDate("not_exist", false).(time.Time) + act, _ = metric.GetDate("not_exist", false).(time.Time) require.Equal(t, exp, act) actual := metric.GetDate("not_exist", true) @@ -99,7 +99,7 @@ func TestGjsonDateTime(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - act = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) + act, _ = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) require.Equal(t, exp, act) exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) @@ -111,7 +111,7 @@ func TestGjsonDateTime(t *testing.T) { require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDateTime("not_exist", false).(time.Time) + act, _ = metric.GetDateTime("not_exist", false).(time.Time) require.Equal(t, exp, act) actual := metric.GetDateTime("not_exist", true) @@ -126,7 +126,7 @@ func TestGjsonDateTime64(t *testing.T) { var exp, act time.Time exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - act = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) + act, _ = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) require.Equal(t, exp, act) exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) @@ -138,7 +138,7 @@ func TestGjsonDateTime64(t *testing.T) { require.Equal(t, exp, act) exp = time.Time{} - act = metric.GetDateTime64("not_exist", false).(time.Time) + act, _ = metric.GetDateTime64("not_exist", false).(time.Time) require.Equal(t, exp, act) actual := metric.GetDateTime64("not_exist", true) @@ -155,11 +155,11 @@ func TestGjsonElasticDateTime(t *testing.T) { // {"date": "2019-12-16T12:10:30Z"} // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC exp = 1576498230 - act = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) + act, _ = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) require.Equal(t, exp, act) exp = -62135596800 - act = metric.GetElasticDateTime("not_exist", false).(int64) + act, _ = metric.GetElasticDateTime("not_exist", false).(int64) require.Equal(t, exp, act) actual := metric.GetElasticDateTime("not_exist", true) @@ -172,27 +172,27 @@ func TestGjsonArray(t *testing.T) { defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - actI := metric.GetArray("mp.i", "int").([]int64) + actI, _ := metric.GetArray("mp.i", "int").([]int64) expI := []int64{1, 2, 3} require.Equal(t, actI, expI) - actF := metric.GetArray("mp.f", "float").([]float64) + actF, _ := metric.GetArray("mp.f", "float").([]float64) expF := []float64{1.1, 2.2, 3.3} require.Equal(t, expF, actF) - actS := metric.GetArray("mp.s", "string").([]string) + actS, _ := metric.GetArray("mp.s", "string").([]string) expS := []string{"aa", "bb", "cc"} require.Equal(t, expS, actS) - actIE := metric.GetArray("mp.e", "int").([]int64) + actIE, _ := metric.GetArray("mp.e", "int").([]int64) expIE := []int64{} require.Equal(t, expIE, actIE) - actFE := metric.GetArray("mp.e", "float").([]float64) + actFE, _ := metric.GetArray("mp.e", "float").([]float64) expFE := []float64{} require.Equal(t, expFE, actFE) - actSE := metric.GetArray("mp.e", "string").([]string) + actSE, _ := metric.GetArray("mp.e", "string").([]string) expSE := []string{} require.Equal(t, expSE, actSE) } diff --git a/parser/parser.go b/parser/parser.go index 7400e19b..e1fb2a9f 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -68,7 +68,7 @@ type Parser interface { } // Pool may be used for pooling Parsers for similarly typed JSONs. -type ParserPool struct { +type Pool struct { name string csvFormat []string delimiter string @@ -78,7 +78,7 @@ type ParserPool struct { } // NewParserPool creates a parser pool -func NewParserPool(name string, csvFormat []string, delimiter string, timezone string) (pp *ParserPool, err error) { +func NewParserPool(name string, csvFormat []string, delimiter string, timezone string) (pp *Pool, err error) { var tz *time.Location if timezone == "" { tz = time.Local @@ -86,7 +86,7 @@ func NewParserPool(name string, csvFormat []string, delimiter string, timezone s err = errors.Wrapf(err, "") return } - pp = &ParserPool{ + pp = &Pool{ name: name, csvFormat: csvFormat, delimiter: delimiter, @@ -98,7 +98,7 @@ func NewParserPool(name string, csvFormat []string, delimiter string, timezone s // Get returns a Parser from pp. // // The Parser must be Put to pp after use. -func (pp *ParserPool) Get() Parser { +func (pp *Pool) Get() Parser { v := pp.pool.Get() if v == nil { switch pp.name { @@ -119,11 +119,11 @@ func (pp *ParserPool) Get() Parser { // // p and objects recursively returned from p cannot be used after p // is put into pp. -func (pp *ParserPool) Put(p Parser) { +func (pp *Pool) Put(p Parser) { pp.pool.Put(p) } -func (pp *ParserPool) ParseDateTime(key string, val string) (t time.Time, err error) { +func (pp *Pool) ParseDateTime(key string, val string) (t time.Time, err error) { var layout string var lay interface{} var ok bool @@ -140,7 +140,7 @@ func (pp *ParserPool) ParseDateTime(key string, val string) (t time.Time, err er err = errors.Errorf("cannot parse time %s at field %s", strconv.Quote(val), key) return } - layout = lay.(string) + layout, _ = lay.(string) t, _ = time.ParseInLocation(layout, val, pp.timeZone) return } diff --git a/task/ring.go b/task/ring.go index a663a563..4839febe 100644 --- a/task/ring.go +++ b/task/ring.go @@ -89,7 +89,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { ring.mux.Lock() defer ring.mux.Unlock() if arg != nil { - newMsg = arg.(*model.InputMessage) + newMsg, _ = arg.(*model.InputMessage) log.Warnf("%s: Ring.ForceBatchOrShard partition %d message range [%d, %d)", taskCfg.Name, newMsg.Partition, ring.ringGroundOff, newMsg.Offset) } if !ring.isIdle { diff --git a/task/task.go b/task/task.go index 1005efb8..a4611dfb 100644 --- a/task/task.go +++ b/task/task.go @@ -52,7 +52,7 @@ type Service struct { stopped chan struct{} inputer input.Inputer clickhouse *output.ClickHouse - pp *parser.ParserPool + pp *parser.Pool cfg *config.Config dims []*model.ColumnWithType @@ -70,7 +70,7 @@ type Service struct { } // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances -func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *parser.ParserPool, cfg *config.Config) *Service { +func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *parser.Pool, cfg *config.Config) *Service { return &Service{ stopped: make(chan struct{}), inputer: inputer, diff --git a/util/common.go b/util/common.go index c917e33d..2da39faf 100644 --- a/util/common.go +++ b/util/common.go @@ -92,7 +92,7 @@ func GetOutboundIP() net.IP { log.Fatal(err) } defer conn.Close() - localAddr := conn.LocalAddr().(*net.UDPAddr) + localAddr, _ := conn.LocalAddr().(*net.UDPAddr) return localAddr.IP } From 134cd884d3e5e43a47e9fb260690351dd66c57ec Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 12 Apr 2021 15:50:27 +0800 Subject: [PATCH 070/404] a bit cleanup --- Makefile | 5 +---- config.json | 43 ------------------------------------------- 2 files changed, 1 insertion(+), 47 deletions(-) delete mode 100644 config.json diff --git a/Makefile b/Makefile index db2b9d5b..c3e06272 100644 --- a/Makefile +++ b/Makefile @@ -27,7 +27,4 @@ systest: build lint: golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr run: pre - go run cmd/clickhouse_sinker/main.go --local-cfg-dir conf/ - -docker-run: - docker run --net=host -e "CONFIG=`cat conf/config.json`" -e "TASK=`cat conf/tasks/logstash_sample.json`" --rm -it `docker build -q .` \ No newline at end of file + go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.json diff --git a/config.json b/config.json deleted file mode 100644 index cf32567e..00000000 --- a/config.json +++ /dev/null @@ -1,43 +0,0 @@ -{ - "clickhouse": { - "default": { - "db": "default", - "hosts": [ - [ - "127.0.0.1" - ] - ], - "port": 9000 - } - }, - "kafka": { - "default": { - "brokers": "146.196.79.232:9094", - "version": "2.2.1" - } - }, - "common": { - "bufferSize": 90000, - "minBufferSize": 1, - "msgSizeHint": 1000, - "flushInterval": 5, - "logLevel": "debug" - }, - "tasks": { - "logstash": { - "name": "logstash", - "kafkaClient": "kafka-go", - "kafka": "default", - "topic": "logstash", - "consumerGroup": "logstash_sinker", - "parser": "json", - "clickhouse": "default", - "tableName": "logstash", - "autoSchema": true, - "@desc_of_exclude_columns": "this columns will be excluded by insert SQL ", - "excludeColumns": [ - "day" - ] - } - } -} From b5fe04626aa5d47642e27d908638ec35adfc53e5 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 15 Apr 2021 18:07:41 +0800 Subject: [PATCH 071/404] fix nacos dataID --- cmd/clickhouse_sinker/main.go | 9 ++++++--- cmd/nacos_publish_config/main.go | 4 ++-- config/nacos.go | 7 +------ docker-compose.yml | 2 +- go.test.sh | 6 +++--- 5 files changed, 13 insertions(+), 15 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 3d1512d3..561a2665 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -140,6 +140,7 @@ func main() { var rcm config.RemoteConfManager var properties map[string]interface{} if cmdOps.NacosDataID != "" { + log.Infof("get config from nacos serverAddrs %s, namespaceId %s, group %s, dataId %s", cmdOps.NacosAddr, cmdOps.NacosNamespaceID, cmdOps.NacosGroup, cmdOps.NacosDataID) rcm = &config.NacosConfManager{} properties = make(map[string]interface{}) properties["serverAddrs"] = cmdOps.NacosAddr @@ -148,6 +149,8 @@ func main() { properties["namespaceId"] = cmdOps.NacosNamespaceID properties["group"] = cmdOps.NacosGroup properties["dataId"] = cmdOps.NacosDataID + } else { + log.Infof("get config from local file %s", cmdOps.LocalCfgFile) } if rcm != nil { if err := rcm.Init(properties); err != nil { @@ -254,15 +257,15 @@ func (s *Sinker) Run() { return case <-time.After(5 * time.Second): if newCfg, err = s.rcm.GetConfig(); err != nil { - log.Fatalf("%+v", err) + log.Errorf("%+v", err) return } if err = newCfg.Normallize(); err != nil { - log.Fatalf("%+v", err) + log.Errorf("%+v", err) return } if err = s.applyConfig(newCfg); err != nil { - log.Fatalf("%+v", err) + log.Errorf("%+v", err) return } } diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index f443c308..9564cd03 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -37,8 +37,8 @@ var ( `nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work!`) nacosGroup = flag.String("nacos-group", "DEFAULT_GROUP", `nacos group name. Empty string doesn't work!`) - nacosDataID = flag.String("nacos-dataid", "", - `nacos data id, the task name`) + nacosDataID = flag.String("nacos-dataid", "clickhouse_sinker.json", + `nacos data id`) localCfgFile = flag.String("local-cfg-file", "/etc/clickhouse_sinker.json", "local config file") ) diff --git a/config/nacos.go b/config/nacos.go index 1460a4ba..f8c00776 100644 --- a/config/nacos.go +++ b/config/nacos.go @@ -97,10 +97,6 @@ func (ncm *NacosConfManager) GetConfig() (conf *Config, err error) { err = errors.Wrapf(err, "") return } - if ncm.dataID != conf.Task.Name { - err = errors.Errorf("DataId %s doesn't match with config: %s", ncm.dataID, content) - return - } return } @@ -112,7 +108,7 @@ func (ncm *NacosConfManager) PublishConfig(conf *Config) (err error) { } content := string(bs) _, err = ncm.configClient.PublishConfig(vo.ConfigParam{ - DataId: conf.Task.Name, + DataId: ncm.dataID, Group: ncm.group, Content: content, }) @@ -120,6 +116,5 @@ func (ncm *NacosConfManager) PublishConfig(conf *Config) (err error) { err = errors.Wrapf(err, "") return } - ncm.dataID = conf.Task.Name return } diff --git a/docker-compose.yml b/docker-compose.yml index 8d235e63..b56fa9b7 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -45,7 +45,7 @@ services: security_opt: - label:disable nacos: - image: nacos/nacos-server:1.4.0 + image: nacos/nacos-server:1.4.1 container_name: nacos-standalone environment: - PREFER_HOST_MODE=hostname diff --git a/go.test.sh b/go.test.sh index 265562c4..574247a9 100755 --- a/go.test.sh +++ b/go.test.sh @@ -79,9 +79,9 @@ curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' curl "localhost:8123" -d 'TRUNCATE TABLE test_dynamic_schema' echo "publish clickhouse_sinker config" -./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_fixed_schema.json -./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_auto_schema.json -./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --local-cfg-file docker/test_dynamic_schema.json +./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema --local-cfg-file docker/test_fixed_schema.json +./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema --local-cfg-file docker/test_auto_schema.json +./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.json echo "start clickhouse_sinker to consume" timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema From 489065135a72ad5cbd2a0abf9b872670821cfaba Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Thu, 15 Apr 2021 14:17:04 +0800 Subject: [PATCH 072/404] convert kafka security config from java style into sinker --- .gitignore | 1 + config/config.go | 120 ++++++++++++++++++++++++++++++++--- docs/configuration/config.md | 8 +++ docs/dev/introduction.md | 24 +++---- task/ring.go | 2 +- util/common.go | 2 +- 6 files changed, 134 insertions(+), 23 deletions(-) diff --git a/.gitignore b/.gitignore index dcd08b24..2000211d 100644 --- a/.gitignore +++ b/.gitignore @@ -10,4 +10,5 @@ package-lock.json *.bin .goreleaser.yml dist +vendor diff --git a/config/config.go b/config/config.go index a49b2de3..b89f382a 100644 --- a/config/config.go +++ b/config/config.go @@ -44,9 +44,10 @@ type Config struct { // KafkaConfig configuration parameters type KafkaConfig struct { - Brokers string - Version string - TLS struct { + Brokers string + Version string + Security map[string]string + TLS struct { Enable bool CaCertFiles string // Required. It's the CA cert.pem with which Kafka brokers certs be signed. ClientCertFile string // Required for client authentication. It's client cert.pem. @@ -141,12 +142,13 @@ type TaskConfig struct { } const ( - defaultFlushInterval = 3 - defaultBufferSize = 1 << 20 //1048576 - defaultMinBufferSize = 1 << 14 // 16384 - defaultMsgSizeHint = 1000 - defaultTimeZone = "Local" - defaultLogLevel = "info" + defaultFlushInterval = 3 + defaultBufferSize = 1 << 20 //1048576 + defaultMinBufferSize = 1 << 14 // 16384 + defaultMsgSizeHint = 1000 + defaultTimeZone = "Local" + defaultLogLevel = "info" + defaultKerberosConfigPath = "/etc/krb5.conf" ) func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { @@ -173,6 +175,8 @@ func (cfg *Config) Normallize() (err error) { if cfg.Kafka.Version == "" { cfg.Kafka.Version = "2.2.1" } + + cfg.convertKfkSecurity() if cfg.Kafka.Sasl.Enable { cfg.Kafka.Sasl.Mechanism = strings.ToUpper(cfg.Kafka.Sasl.Mechanism) switch cfg.Kafka.Sasl.Mechanism { @@ -234,3 +238,101 @@ func (cfg *Config) Normallize() (err error) { } return } + +//convert java client style configuration into sinker +func (cfg *Config) convertKfkSecurity() { + if protocal, ok := cfg.Kafka.Security["security.protocol"]; ok { + if strings.Contains(protocal, "SASL") { + cfg.Kafka.Sasl.Enable = true + } + if strings.Contains(protocal, "SSL") { + cfg.Kafka.TLS.Enable = true + } + } + + if cfg.Kafka.TLS.Enable { + if endpIdentAlgo, ok := cfg.Kafka.Security["ssl.endpoint.identification.algorithm"]; ok { + cfg.Kafka.TLS.EndpIdentAlgo = endpIdentAlgo + } + if trustStoreLocation, ok := cfg.Kafka.Security["ssl.truststore.location"]; ok { + cfg.Kafka.TLS.TrustStoreLocation = trustStoreLocation + } + if trustStorePassword, ok := cfg.Kafka.Security["ssl.truststore.password"]; ok { + cfg.Kafka.TLS.TrustStorePassword = trustStorePassword + } + if keyStoreLocation, ok := cfg.Kafka.Security["ssl.keystore.location"]; ok { + cfg.Kafka.TLS.KeystoreLocation = keyStoreLocation + } + if keyStorePassword, ok := cfg.Kafka.Security["ssl.keystore.password"]; ok { + cfg.Kafka.TLS.KeystorePassword = keyStorePassword + } + } + if cfg.Kafka.Sasl.Enable { + if mechanism, ok := cfg.Kafka.Security["sasl.mechanism"]; ok { + cfg.Kafka.Sasl.Mechanism = mechanism + } + if config, ok := cfg.Kafka.Security["sasl.jaas.config"]; ok { + configMap := readConfig(config) + if strings.Contains(cfg.Kafka.Sasl.Mechanism, "SCRAM") { + // SCRAM-SHA-256 or SCRAM-SHA-512 + if username, ok := configMap["username"]; ok { + cfg.Kafka.Sasl.Username = username + } + if password, ok := configMap["password"]; ok { + cfg.Kafka.Sasl.Password = password + } + } + if strings.Contains(cfg.Kafka.Sasl.Mechanism, "GSSAPI") { + // GSSAPI + if useKeyTab, ok := configMap["useKeyTab"]; ok { + if useKeyTab == "true" { + cfg.Kafka.Sasl.GSSAPI.AuthType = 2 + } else { + cfg.Kafka.Sasl.GSSAPI.AuthType = 1 + } + } + if cfg.Kafka.Sasl.GSSAPI.AuthType == 1 { + //Username and password + if username, ok := configMap["username"]; ok { + cfg.Kafka.Sasl.GSSAPI.Username = username + } + if password, ok := configMap["password"]; ok { + cfg.Kafka.Sasl.GSSAPI.Password = password + } + } else { + //Keytab + if keyTab, ok := configMap["keyTab"]; ok { + cfg.Kafka.Sasl.GSSAPI.KeyTabPath = keyTab + } + if principal, ok := configMap["principal"]; ok { + username := strings.Split(principal, "@")[0] + realm := strings.Split(principal, "@")[1] + cfg.Kafka.Sasl.GSSAPI.Username = username + cfg.Kafka.Sasl.GSSAPI.Realm = realm + } + if servicename, ok := cfg.Kafka.Security["sasl.kerberos.service.name"]; ok { + cfg.Kafka.Sasl.GSSAPI.ServiceName = servicename + } + if cfg.Kafka.Sasl.GSSAPI.KerberosConfigPath == "" { + cfg.Kafka.Sasl.GSSAPI.KerberosConfigPath = defaultKerberosConfigPath + } + } + } + } + } +} + +func readConfig(config string) map[string]string { + configMap := make(map[string]string) + config = strings.TrimSuffix(config, ";") + fields := strings.Split(config, " ") + for _, field := range fields { + if strings.Contains(field, "=") { + key := strings.Split(field, "=")[0] + value := strings.Split(field, "=")[1] + value = strings.Trim(value, "\"") + configMap[key] = value + } + } + return configMap +} diff --git a/docs/configuration/config.md b/docs/configuration/config.md index bcc31fbe..b39b8ee2 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -26,6 +26,14 @@ // kafka configs "kafka": { "brokers": "127.0.0.1:9093", + + // jave client style security authentication + "security":{ + "security.protocol": "SASL_PLAINTEXT", + "sasl.kerberos.service.name": "kafka", + "sasl.mechanism":"GSSAPI", + "sasl.jaas.config":"com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab=\"/etc/security/mmmtest.keytab\" principal=\"mmm@ALANWANG.COM\";" + } // SSL "tls": { diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index c47ca2d2..a9f41236 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -47,7 +47,7 @@ clickhouse_sinker supports following encryption mechanisms: An example kafka config: -``` +```json "kafka": { "brokers": "192.168.31.64:9092", "@version": "Required if you use sarama. It's the the Kafka server version.", @@ -58,7 +58,7 @@ An example kafka config: - Encryption using SSL An example kafka config: -``` +```json "kafka": { "brokers": "192.168.31.64:9093", "version": "2.2.1", @@ -75,7 +75,7 @@ An example kafka config: ``` Or if you have extracted certificates from JKS, use the following config: -``` +```json "kafka": { "brokers": "192.168.31.64:9093", "version": "2.2.1", @@ -91,7 +91,7 @@ Or if you have extracted certificates from JKS, use the following config: FYI. `kafka-console-consumer.sh` works as the following setup: -``` +```bash $ cat config/client_SSL_NOAUTH.properties security.protocol=SSL ssl.truststore.location=/etc/security/kafka.client.truststore.jks @@ -111,7 +111,7 @@ clickhouse_sinker support following following authentication mechanisms: An example kafka config: -``` +```json "kafka": { "brokers": "192.168.31.64:9092", "@version": "Required if you use sarama. It's the the Kafka server version.", @@ -122,7 +122,7 @@ An example kafka config: - SASL/PLAIN An example kafka config: -``` +```json "kafka": { "brokers": "192.168.31.64:9094", "version": "2.2.1", @@ -137,7 +137,7 @@ An example kafka config: FYI. Java clients work with the following setup: -``` +```bash $ cat config/client_PLAINTEXT_PLAIN.properties security.protocol=SASL_PLAINTEXT sasl.kerberos.service.name=kafka @@ -152,7 +152,7 @@ $ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic su - SASL/SCRAM An example kafka config: -``` +```json "kafka": { "brokers": "192.168.31.64:9094", "version": "2.2.1", @@ -168,7 +168,7 @@ An example kafka config: FYI. Java clients work with the following setup: -``` +```bash $ cat config/client_PLAINTEXT_SCRAM.properties security.protocol=SASL_PLAINTEXT sasl.kerberos.service.name=kafka @@ -183,7 +183,7 @@ $ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic su - SASL/GSSAPI(Kerberos) An example kafka config: -``` +```json "kafka": { "brokers": "192.168.31.64:9094", "version": "2.2.1", @@ -206,7 +206,7 @@ An example kafka config: FYI. Java clients work with the following setup: -``` +```bash $ cat config/client_PLAINTEXT_GSSAPI.properties security.protocol=SASL_PLAINTEXT sasl.kerberos.service.name=kafka @@ -273,7 +273,7 @@ If CLI `--metric-push-gateway-addrs` or env `METRIC_PUSH_GATEWAY_ADDRS` (a list There are several abstract interfaces which you can implement to support more message format, message queue and config management mechanism. -``` +```go type Parser interface { Parse(bs []byte) model.Metric } diff --git a/task/ring.go b/task/ring.go index 4839febe..ef33c286 100644 --- a/task/ring.go +++ b/task/ring.go @@ -20,7 +20,7 @@ type Ring struct { ringGroundOff int64 //min message offset inside the ring ringCeilingOff int64 //1 + max message offset inside the ring ringFilledOffset int64 //every message which's offset inside range [ringGroundOff, ringFilledOffset) is in the ring - batchSizeShift int //the shift of desired batch size + batchSizeShift uint //the shift of desired batch size tid goetty.Timeout idleCnt int isIdle bool diff --git a/util/common.go b/util/common.go index 2da39faf..784de22f 100644 --- a/util/common.go +++ b/util/common.go @@ -78,7 +78,7 @@ func GetSourceName(name string) (sourcename string) { } // GetShift returns the smallest `shift` which 1< Date: Sat, 17 Apr 2021 11:36:55 +0800 Subject: [PATCH 073/404] go lint --- config/config.go | 6 +++--- task/ring.go | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/config/config.go b/config/config.go index b89f382a..e5e91d7c 100644 --- a/config/config.go +++ b/config/config.go @@ -241,11 +241,11 @@ func (cfg *Config) Normallize() (err error) { //convert java client style configuration into sinker func (cfg *Config) convertKfkSecurity() { - if protocal, ok := cfg.Kafka.Security["security.protocol"]; ok { - if strings.Contains(protocal, "SASL") { + if protocol, ok := cfg.Kafka.Security["security.protocol"]; ok { + if strings.Contains(protocol, "SASL") { cfg.Kafka.Sasl.Enable = true } - if strings.Contains(protocal, "SSL") { + if strings.Contains(protocol, "SSL") { cfg.Kafka.TLS.Enable = true } } diff --git a/task/ring.go b/task/ring.go index ef33c286..9fd53172 100644 --- a/task/ring.go +++ b/task/ring.go @@ -20,7 +20,7 @@ type Ring struct { ringGroundOff int64 //min message offset inside the ring ringCeilingOff int64 //1 + max message offset inside the ring ringFilledOffset int64 //every message which's offset inside range [ringGroundOff, ringFilledOffset) is in the ring - batchSizeShift uint //the shift of desired batch size + batchSizeShift uint //the shift of desired batch size tid goetty.Timeout idleCnt int isIdle bool From 17c1f6209330d0b8f4844dc856467d5c077022f1 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 17 Apr 2021 23:04:47 +0800 Subject: [PATCH 074/404] gracefully stop task --- cmd/clickhouse_sinker/main.go | 42 +++++++++++++++++++---------------- 1 file changed, 23 insertions(+), 19 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 561a2665..0aeb3034 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -209,8 +209,7 @@ type Sinker struct { // NewSinker get an instance of sinker with the task list func NewSinker(rcm config.RemoteConfManager) *Sinker { - parent := context.Background() - ctx, cancel := context.WithCancel(parent) + ctx, cancel := context.WithCancel(context.Background()) s := &Sinker{rcm: rcm, ctx: ctx, cancel: cancel} return s } @@ -275,16 +274,19 @@ func (s *Sinker) Run() { // Close shutdown task func (s *Sinker) Close() { - s.cancel() - s.task.Stop() - + // Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). + log.Infof("%s: stopping parsing pool", s.curCfg.Task.Name) util.GlobalParsingPool.StopWait() + log.Infof("%s: stopping writing pool", s.curCfg.Task.Name) util.GlobalWritingPool.StopWait() + log.Infof("%s: stopping timer wheel", s.curCfg.Task.Name) util.GlobalTimerWheel.Stop() + s.task.Stop() if s.pusher != nil { s.pusher.Stop() } + s.cancel() } func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { @@ -343,29 +345,31 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { log.Infof("going to apply a different config: %+v", string(bsNewCfg)) if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { - // 1. Stop task + // 1. Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). + log.Infof("%s: stopping parsing pool", s.curCfg.Task.Name) + util.GlobalParsingPool.StopWait() + log.Infof("%s: stopping writing pool", s.curCfg.Task.Name) + util.GlobalWritingPool.StopWait() + log.Infof("%s: stopping timer wheel", s.curCfg.Task.Name) + util.GlobalTimerWheel.Stop() s.task.Stop() - // 2. Generate, initialize and start task + // 2. Generate and initialize task t := GenTask(newCfg) if err = t.Init(); err != nil { return } - go t.Run(s.ctx) - s.task = t - // 3. Resize goroutine pools. - concurrentParsers := 10 - if runtime.NumCPU() >= 2 { - if concurrentParsers > runtime.NumCPU()/2 { - concurrentParsers = runtime.NumCPU() / 2 - } - } else { - concurrentParsers = 1 - } - util.GlobalParsingPool.Resize(concurrentParsers) + // 3. Restart goroutine pools. + util.InitGlobalTimerWheel() + util.GlobalParsingPool.Restart() totalConn := pool.GetTotalConn() util.GlobalWritingPool.Resize(totalConn) + util.GlobalWritingPool.Restart() + + // 4. Start task + go t.Run(s.ctx) + s.task = t } // Record the new config s.curCfg = newCfg From 79d71b6d46e66b1898e064d52ef052c68a158f00 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 18 Apr 2021 22:05:21 +0800 Subject: [PATCH 075/404] added gitleaks into workflow --- .github/workflows/lint.yml | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index d2cae262..dde41c61 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -1,14 +1,12 @@ -name: golangci-lint +name: lint on: push: - tags: - - v* - branches: - - master + tags: [ v* ] + branches: [ master ] pull_request: + branches: [ master ] jobs: - golangci: - name: lint + golangci-lint: runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 @@ -25,4 +23,12 @@ jobs: args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr # Optional: show only new issues if it's a pull request. The default value is `false`. - # only-new-issues: true \ No newline at end of file + # only-new-issues: true + gitleaks: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v2 + with: + fetch-depth: '0' + - name: gitleaks-action + uses: zricethezav/gitleaks-action@master From d57b802fdebb76b2a19d9dd7375117589312d8af Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 21 Apr 2021 15:46:07 +0800 Subject: [PATCH 076/404] replaced logrus with zap --- cmd/clickhouse_sinker/main.go | 67 ++++++++++++++++---------------- cmd/nacos_publish_config/main.go | 22 +++++------ config/config.go | 6 ++- docker/test_auto_schema.json | 3 +- docker/test_dynamic_schema.json | 3 +- docker/test_fixed_schema.json | 3 +- docs/configuration/config.md | 9 +++-- go.mod | 3 +- go.sum | 5 ++- input/input.go | 4 +- input/kafka_go.go | 10 ++--- input/kafka_sarama.go | 10 ++--- output/clickhouse.go | 25 ++++++------ parser/fastjson.go | 4 +- parser/fastjson_test.go | 9 ++--- parser/parser_test.go | 30 +++++++++++++- pool/conn.go | 9 ++--- statistics/statistics.go | 6 +-- task/ring.go | 17 ++++---- task/sharding.go | 7 ++-- task/task.go | 34 ++++++++-------- util/app.go | 41 ++++--------------- util/common.go | 51 ++++++++++++++++++++---- 23 files changed, 204 insertions(+), 174 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 0aeb3034..4fe9efd0 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -20,6 +20,8 @@ import ( "encoding/json" "flag" "fmt" + "log" + "net" "net/http" "net/http/pprof" "os" @@ -37,11 +39,11 @@ import ( "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/task" "github.com/housepower/clickhouse_sinker/util" + "go.uber.org/zap" _ "github.com/ClickHouse/clickhouse-go" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus/promhttp" - log "github.com/sirupsen/logrus" ) type CmdOptions struct { @@ -112,12 +114,18 @@ func initCmdOptions() { } func init() { + util.InitLogger("info", []string{"stdout"}) initCmdOptions() if cmdOps.ShowVer { - log.Infoln(config.GetSinkerInfo()) + util.Logger.Info(config.GetSinkerInfo()) os.Exit(0) } - selfIP = util.GetOutboundIP().String() + var err error + var ip net.IP + if ip, err = util.GetOutboundIP(); err != nil { + log.Fatal("unable to determine self ip", err) + } + selfIP = ip.String() cmdOps.HTTPPort = util.GetSpareTCPPort(cmdOps.HTTPPort) } @@ -132,15 +140,11 @@ func GenTask(cfg *config.Config) (taskImpl *task.Service) { } func main() { - log.SetFormatter(&log.TextFormatter{ - FullTimestamp: true, - }) - util.Run("clickhouse_sinker", func() error { var rcm config.RemoteConfManager var properties map[string]interface{} if cmdOps.NacosDataID != "" { - log.Infof("get config from nacos serverAddrs %s, namespaceId %s, group %s, dataId %s", cmdOps.NacosAddr, cmdOps.NacosNamespaceID, cmdOps.NacosGroup, cmdOps.NacosDataID) + util.Logger.Infof("get config from nacos serverAddrs %s, namespaceId %s, group %s, dataId %s", cmdOps.NacosAddr, cmdOps.NacosNamespaceID, cmdOps.NacosGroup, cmdOps.NacosDataID) rcm = &config.NacosConfManager{} properties = make(map[string]interface{}) properties["serverAddrs"] = cmdOps.NacosAddr @@ -150,11 +154,11 @@ func main() { properties["group"] = cmdOps.NacosGroup properties["dataId"] = cmdOps.NacosDataID } else { - log.Infof("get config from local file %s", cmdOps.LocalCfgFile) + util.Logger.Infof("get config from local file %s", cmdOps.LocalCfgFile) } if rcm != nil { if err := rcm.Init(properties); err != nil { - log.Fatalf("%+v", err) + util.Logger.Fatal("rcm.Init failed", zap.Error(err)) } } runner = NewSinker(rcm) @@ -185,8 +189,8 @@ func main() { mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) mux.HandleFunc("/debug/pprof/trace", pprof.Trace) - log.Infof("Run http server http://%s:%d", selfIP, cmdOps.HTTPPort) - log.Error(http.ListenAndServe(fmt.Sprintf(":%d", cmdOps.HTTPPort), mux)) + util.Logger.Infof("Run http server http://%s:%d", selfIP, cmdOps.HTTPPort) + util.Logger.Error(http.ListenAndServe(fmt.Sprintf(":%d", cmdOps.HTTPPort), mux)) }() runner.Run() @@ -233,19 +237,19 @@ func (s *Sinker) Run() { if s.rcm == nil { if _, err = os.Stat(cmdOps.LocalCfgFile); err == nil { if newCfg, err = config.ParseLocalCfgFile(cmdOps.LocalCfgFile); err != nil { - log.Fatalf("%+v", err) + util.Logger.Fatalf("%+v", err) return } } else { - log.Fatalf("expect --local-cfg-file or --local-cfg-dir") + util.Logger.Fatalf("expect --local-cfg-file or --local-cfg-dir") return } if err = newCfg.Normallize(); err != nil { - log.Fatalf("%+v", err) + util.Logger.Fatalf("%+v", err) return } if err = s.applyConfig(newCfg); err != nil { - log.Fatalf("%+v", err) + util.Logger.Fatalf("%+v", err) return } <-s.ctx.Done() @@ -256,15 +260,15 @@ func (s *Sinker) Run() { return case <-time.After(5 * time.Second): if newCfg, err = s.rcm.GetConfig(); err != nil { - log.Errorf("%+v", err) + util.Logger.Errorf("%+v", err) return } if err = newCfg.Normallize(); err != nil { - log.Errorf("%+v", err) + util.Logger.Errorf("%+v", err) return } if err = s.applyConfig(newCfg); err != nil { - log.Errorf("%+v", err) + util.Logger.Errorf("%+v", err) return } } @@ -275,11 +279,11 @@ func (s *Sinker) Run() { // Close shutdown task func (s *Sinker) Close() { // Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). - log.Infof("%s: stopping parsing pool", s.curCfg.Task.Name) + util.Logger.Infof("%s: stopping parsing pool", s.curCfg.Task.Name) util.GlobalParsingPool.StopWait() - log.Infof("%s: stopping writing pool", s.curCfg.Task.Name) + util.Logger.Infof("%s: stopping writing pool", s.curCfg.Task.Name) util.GlobalWritingPool.StopWait() - log.Infof("%s: stopping timer wheel", s.curCfg.Task.Name) + util.Logger.Infof("%s: stopping timer wheel", s.curCfg.Task.Name) util.GlobalTimerWheel.Stop() s.task.Stop() @@ -290,16 +294,13 @@ func (s *Sinker) Close() { } func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { - var lvl log.Level - if lvl, err = log.ParseLevel(newCfg.LogLevel); err != nil { - err = errors.Wrapf(err, "") - return + if s.curCfg == nil || newCfg.LogLevel != s.curCfg.LogLevel || !reflect.DeepEqual(newCfg.LogPaths, s.curCfg.LogPaths) { + util.InitLogger(newCfg.LogLevel, newCfg.LogPaths) } - log.SetLevel(lvl) if s.curCfg == nil { // The first time invoking of applyConfig err = s.applyFirstConfig(newCfg) - } else if !reflect.DeepEqual(newCfg, s.curCfg) { + } else if !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { err = s.applyAnotherConfig(newCfg) } return @@ -311,7 +312,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { err = errors.Wrapf(err, "") return } - log.Infof("going to apply the first config: %+v", string(bsNewCfg)) + util.Logger.Infof("going to apply the first config: %+v", string(bsNewCfg)) util.InitGlobalTimerWheel() t := GenTask(newCfg) @@ -342,15 +343,15 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { err = errors.Wrapf(err, "") return } - log.Infof("going to apply a different config: %+v", string(bsNewCfg)) + util.Logger.Infof("going to apply a different config: %+v", string(bsNewCfg)) if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { // 1. Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). - log.Infof("%s: stopping parsing pool", s.curCfg.Task.Name) + util.Logger.Infof("%s: stopping parsing pool", s.curCfg.Task.Name) util.GlobalParsingPool.StopWait() - log.Infof("%s: stopping writing pool", s.curCfg.Task.Name) + util.Logger.Infof("%s: stopping writing pool", s.curCfg.Task.Name) util.GlobalWritingPool.StopWait() - log.Infof("%s: stopping timer wheel", s.curCfg.Task.Name) + util.Logger.Infof("%s: stopping timer wheel", s.curCfg.Task.Name) util.GlobalTimerWheel.Stop() s.task.Stop() diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 9564cd03..80e623c8 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -23,7 +23,7 @@ import ( _ "github.com/ClickHouse/clickhouse-go" "github.com/housepower/clickhouse_sinker/config" - log "github.com/sirupsen/logrus" + "github.com/housepower/clickhouse_sinker/util" ) var ( @@ -60,44 +60,42 @@ func PublishSinkerConfig() { var cfg *config.Config if _, err = os.Stat(*localCfgFile); err == nil { if cfg, err = config.ParseLocalCfgFile(*localCfgFile); err != nil { - log.Fatalf("%+v", err) + util.Logger.Fatalf("%+v", err) return } } else { - log.Fatalf("expect --local-cfg-file") + util.Logger.Fatalf("expect --local-cfg-file") return } if err = cfg.Normallize(); err != nil { - log.Fatalf("%+v", err) + util.Logger.Fatalf("%+v", err) return } ncm := config.NacosConfManager{} properties := getProperties() if err = ncm.Init(properties); err != nil { - log.Fatalf("%+v", err) + util.Logger.Fatalf("%+v", err) } if err = ncm.PublishConfig(cfg); err != nil { - log.Fatalf("%+v", err) + util.Logger.Fatalf("%+v", err) } - log.Infof("sleep a while") + util.Logger.Infof("sleep a while") time.Sleep(10 * time.Second) var newCfg *config.Config if newCfg, err = ncm.GetConfig(); err != nil { - log.Fatalf("%+v", err) + util.Logger.Fatalf("%+v", err) } if !reflect.DeepEqual(newCfg, cfg) { - log.Fatalf("got different config: %+v", newCfg) + util.Logger.Fatalf("got different config: %+v", newCfg) } } func main() { - log.SetFormatter(&log.TextFormatter{ - FullTimestamp: true, - }) + util.InitLogger("info", []string{"stdout"}) flag.Parse() PublishSinkerConfig() } diff --git a/config/config.go b/config/config.go index e5e91d7c..5721bc88 100644 --- a/config/config.go +++ b/config/config.go @@ -40,6 +40,7 @@ type Config struct { Clickhouse ClickHouseConfig Task TaskConfig LogLevel string + LogPaths []string } // KafkaConfig configuration parameters @@ -232,10 +233,13 @@ func (cfg *Config) Normallize() (err error) { } } switch strings.ToLower(cfg.LogLevel) { - case "panic", "fatal", "error", "warn", "warning", "info", "debug", "trace": + case "debug", "info", "warn", "error", "dpanic", "panic", "fatal": default: cfg.LogLevel = defaultLogLevel } + if len(cfg.LogPaths) == 0 { + cfg.LogPaths = []string{"stdout"} + } return } diff --git a/docker/test_auto_schema.json b/docker/test_auto_schema.json index ed4e330f..72f1cb59 100644 --- a/docker/test_auto_schema.json +++ b/docker/test_auto_schema.json @@ -28,5 +28,6 @@ ], "bufferSize": 50000 }, - "logLevel": "debug" + "logLevel": "debug", + "logPaths": ["stdout", "test_auto_schema.log"] } \ No newline at end of file diff --git a/docker/test_dynamic_schema.json b/docker/test_dynamic_schema.json index e74e61a1..d251cd04 100644 --- a/docker/test_dynamic_schema.json +++ b/docker/test_dynamic_schema.json @@ -32,5 +32,6 @@ }, "bufferSize": 50000 }, - "logLevel": "debug" + "logLevel": "debug", + "logPaths": ["stdout", "test_dynamic_schema.log"] } \ No newline at end of file diff --git a/docker/test_fixed_schema.json b/docker/test_fixed_schema.json index 52a6d479..d9f11164 100644 --- a/docker/test_fixed_schema.json +++ b/docker/test_fixed_schema.json @@ -38,5 +38,6 @@ ], "bufferSize": 50000 }, - "logLevel": "debug" + "logLevel": "debug", + "logPaths": ["stdout", "test_fixed_schema.log"] } \ No newline at end of file diff --git a/docs/configuration/config.md b/docs/configuration/config.md index b39b8ee2..9b0efa37 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -75,7 +75,7 @@ }, "task": { - "name": "daily_request", + "name": "test_dynamic_schema", // kafka topic "topic": "topic", // kafka consume from earliest or latest @@ -130,7 +130,10 @@ "timezone": "" }, - // log level, possible value: panic, fatal, error, warn, warning, info, debug, trace - "logLevel": "debug" + // log level, possible value: "debug", "info", "warn", "error", "dpanic", "panic", "fatal" + "logLevel": "debug", + + // log output paths, possible value: "stdout", "stderr", relative file path, absoute file path. Log files will be rotated every 100MB, keep 10 old ones. + "logPaths": ["stdout", "test_dynamic_schema.log"] } ``` diff --git a/go.mod b/go.mod index 4cd58026..d0994484 100644 --- a/go.mod +++ b/go.mod @@ -13,7 +13,6 @@ require ( github.com/prometheus/client_golang v1.3.0 github.com/prometheus/common v0.8.0 github.com/segmentio/kafka-go v0.4.8 - github.com/sirupsen/logrus v1.4.2 github.com/smartystreets/goconvey v1.6.4 // indirect github.com/stretchr/testify v1.6.1 github.com/tidwall/gjson v1.1.3 @@ -21,6 +20,8 @@ require ( github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/valyala/fastjson v1.4.1 github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c + go.uber.org/zap v1.15.0 golang.org/x/sys v0.0.0-20200917061948-648f2a039071 // indirect golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e + gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) diff --git a/go.sum b/go.sum index e420ae19..0e80d535 100644 --- a/go.sum +++ b/go.sum @@ -103,7 +103,6 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.11.0 h1:wJbzvpYMVGG9iTI9VxpnNZfd4DzMPoCWze3GgSqz8yg= github.com/klauspost/compress v1.11.0/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -165,7 +164,6 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= @@ -280,8 +278,11 @@ gopkg.in/jcmturner/gokrb5.v7 v7.5.0 h1:a9tsXlIDD9SKxotJMK3niV7rPZAJeX2aD/0yg3qlI gopkg.in/jcmturner/gokrb5.v7 v7.5.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM= gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU= gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= +gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= +gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.4 h1:/eiJrUcujPVeJ3xlSWaiNi3uSVmDGBK1pDHUHAnao1I= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= diff --git a/input/input.go b/input/input.go index 28a3555e..45615ddb 100644 --- a/input/input.go +++ b/input/input.go @@ -2,10 +2,10 @@ package input import ( "context" - "log" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" ) const ( @@ -28,7 +28,7 @@ func NewInputer(typ string) Inputer { case TypeKafkaSarama: return NewKafkaSarama() default: - log.Fatalf("%s is not a supported input type", typ) + util.Logger.Fatalf("%s is not a supported input type", typ) return nil } } diff --git a/input/kafka_go.go b/input/kafka_go.go index af6f6e1e..33fdaaa2 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -26,7 +26,6 @@ import ( "github.com/segmentio/kafka-go" "github.com/segmentio/kafka-go/sasl/plain" "github.com/segmentio/kafka-go/sasl/scram" - log "github.com/sirupsen/logrus" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" @@ -67,8 +66,7 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model MinBytes: k.cfg.Task.MinBufferSize * k.cfg.Task.MsgSizeHint, MaxBytes: k.cfg.Task.BufferSize * k.cfg.Task.MsgSizeHint, MaxWait: time.Duration(k.cfg.Task.FlushInterval) * time.Second, - CommitInterval: time.Second, // flushes commits to Kafka every second - ErrorLogger: log.StandardLogger(), //kafka-go INFO log is too verbose + CommitInterval: time.Second, // flushes commits to Kafka every second } if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { if kfkCfg.TLS.CaCertFiles, _, err = util.JksToPem(kfkCfg.TLS.TrustStoreLocation, kfkCfg.TLS.TrustStorePassword, false); err != nil { @@ -130,15 +128,15 @@ LOOP_KAFKA_GO: var msg kafka.Message if msg, err = k.r.FetchMessage(ctx); err != nil { if errors.Is(err, context.Canceled) { - log.Infof("%s: Kafka.Run quit due to context has been canceled", k.cfg.Task.Name) + util.Logger.Infof("%s: Kafka.Run quit due to context has been canceled", k.cfg.Task.Name) break LOOP_KAFKA_GO } else if errors.Is(err, io.EOF) { - log.Infof("%s: Kafka.Run quit due to reader has been closed", k.cfg.Task.Name) + util.Logger.Infof("%s: Kafka.Run quit due to reader has been closed", k.cfg.Task.Name) break LOOP_KAFKA_GO } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.cfg.Task.Name).Inc() err = errors.Wrap(err, "") - log.Errorf("%s: Kafka.Run got error %+v", k.cfg.Task.Name, err) + util.Logger.Errorf("%s: Kafka.Run got error %+v", k.cfg.Task.Name, err) continue } } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 37f5f57e..d4c1ee51 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -25,7 +25,6 @@ import ( "github.com/Shopify/sarama" "github.com/pkg/errors" - log "github.com/sirupsen/logrus" "github.com/xdg/scram" "github.com/housepower/clickhouse_sinker/config" @@ -59,7 +58,7 @@ func (h MyConsumerGroupHandler) Setup(sess sarama.ConsumerGroupSession) error { return nil } func (h MyConsumerGroupHandler) Cleanup(_ sarama.ConsumerGroupSession) error { - log.Infof("%s: consumer group %s cleanup", h.k.cfg.Task.Name, h.k.cfg.Task.ConsumerGroup) + util.Logger.Infof("%s: consumer group %s cleanup", h.k.cfg.Task.Name, h.k.cfg.Task.ConsumerGroup) time.Sleep(5 * time.Second) return nil } @@ -90,7 +89,6 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg m err = errors.Wrapf(err, "") return } - sarama.Logger = log.StandardLogger() if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { if kfkCfg.TLS.CaCertFiles, _, err = util.JksToPem(kfkCfg.TLS.TrustStoreLocation, kfkCfg.TLS.TrustStorePassword, false); err != nil { return @@ -148,15 +146,15 @@ LOOP_SARAMA: // recreated to get the new claims if err := k.cg.Consume(ctx, []string{taskCfg.Topic}, handler); err != nil { if errors.Is(err, context.Canceled) { - log.Infof("%s: Kafka.Run quit due to context has been canceled", taskCfg.Name) + util.Logger.Infof("%s: Kafka.Run quit due to context has been canceled", taskCfg.Name) break LOOP_SARAMA } else if errors.Is(err, sarama.ErrClosedConsumerGroup) { - log.Infof("%s: Kafka.Run quit due to consumer group has been closed", taskCfg.Name) + util.Logger.Infof("%s: Kafka.Run quit due to consumer group has been closed", taskCfg.Name) break LOOP_SARAMA } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() err = errors.Wrap(err, "") - log.Errorf("%s: Kafka.Run got error %+v", taskCfg.Name, err) + util.Logger.Errorf("%s: Kafka.Run got error %+v", taskCfg.Name, err) continue } } diff --git a/output/clickhouse.go b/output/clickhouse.go index 7f46b0a1..ee32ffe1 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -34,8 +34,6 @@ import ( "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" - - log "github.com/sirupsen/logrus" ) var ( @@ -101,7 +99,7 @@ func (c *ClickHouse) write(batch *model.Batch) error { } } if err != nil { - log.Errorf("%s: stmt.Exec failed %d times with errors %+v", c.cfg.Task.Name, numErr, err) + util.Logger.Errorf("%s: stmt.Exec failed %d times with errors %+v", c.cfg.Task.Name, numErr, err) goto ERR } @@ -125,7 +123,7 @@ func shouldReconnect(err error) bool { if strings.Contains(err.Error(), "connection refused") || strings.Contains(err.Error(), "bad connection") { return true } - log.Infof("permanent error: %v", err.Error()) + util.Logger.Infof("permanent error: %v", err.Error()) return false } @@ -141,17 +139,16 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { // TODO: kafka_go and sarama commit give different error when context is cancceled. // How to unify them? if std_errors.Is(err, context.Canceled) || std_errors.Is(err, io.ErrClosedPipe) { - log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) + util.Logger.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) return } - log.Errorf("%s: committing offset failed with permanent error %+v", c.cfg.Task.Name, err) - os.Exit(-1) + util.Logger.Fatalf("%s: committing offset failed with permanent error %+v", c.cfg.Task.Name, err) } if std_errors.Is(err, context.Canceled) { - log.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) + util.Logger.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) return } - log.Errorf("%s: flush batch(try #%d) failed with error %+v", c.cfg.Task.Name, c.cfg.Clickhouse.RetryTimes-times, err) + util.Logger.Errorf("%s: flush batch(try #%d) failed with error %+v", c.cfg.Task.Name, c.cfg.Clickhouse.RetryTimes-times, err) statistics.FlushMsgsErrorTotal.WithLabelValues(c.cfg.Task.Name).Add(float64(batch.RealSize)) times++ if shouldReconnect(err) && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { @@ -217,7 +214,7 @@ func (c *ClickHouse) initSchema() (err error) { c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.cfg.Task.TableName + " (" + strings.Join(quotedDms, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" - log.Infof("%s: Prepare sql=> %s", c.cfg.Task.Name, c.prepareSQL) + util.Logger.Infof("%s: Prepare sql=> %s", c.cfg.Task.Name, c.prepareSQL) return nil } @@ -235,14 +232,14 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } newKeysQuota := maxDims - len(c.Dims) if newKeysQuota <= 0 { - log.Warnf("number of columns reaches upper limit %d", maxDims) + util.Logger.Warnf("number of columns reaches upper limit %d", maxDims) return } var i int newKeys.Range(func(key, value interface{}) bool { i++ if i > newKeysQuota { - log.Warnf("number of columns reaches upper limit %d", maxDims) + util.Logger.Warnf("number of columns reaches upper limit %d", maxDims) return false } strKey, _ := key.(string) @@ -279,7 +276,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } conn := pool.GetConn(0) for _, query := range queries { - log.Infof("%s: executing sql=> %s", taskCfg.Name, query) + util.Logger.Infof("%s: executing sql=> %s", taskCfg.Name, query) if _, err = conn.Exec(query); err != nil { err = errors.Wrapf(err, query) return @@ -294,7 +291,7 @@ func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { conn := pool.GetConn(0) query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, chCfg.DB, chCfg.DB, taskCfg.TableName) - log.Infof("%s: executing sql=> %s", taskCfg.Name, query) + util.Logger.Infof("%s: executing sql=> %s", taskCfg.Name, query) var rows *sql.Rows if rows, err = conn.Query(query); err != nil { diff --git a/parser/fastjson.go b/parser/fastjson.go index a74c1275..7523af18 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -20,8 +20,8 @@ import ( "time" "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" - log "github.com/sirupsen/logrus" "github.com/valyala/fastjson" ) @@ -198,7 +198,7 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (fou newKeys.Store(strKey, "string") foundNew = true } else { - log.Warnf("FastjsonMetric.GetNewKeys found a kv not be int/float/string, key: %s, value: %s", strKey, v.String()) + util.Logger.Warnf("FastjsonMetric.GetNewKeys found a kv not be int/float/string, key: %s, value: %s", strKey, v.String()) } } }) diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go index e26fa4f1..a51b2a9c 100644 --- a/parser/fastjson_test.go +++ b/parser/fastjson_test.go @@ -1,12 +1,9 @@ package parser import ( - "fmt" - "strings" "testing" "time" - log "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" "github.com/valyala/fastjson" ) @@ -212,7 +209,7 @@ func TestFastjsonDetectSchema(t *testing.T) { defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) - var nameAndTypes []string + act := make(map[string]string) c, _ := metric.(*FastjsonMetric) var obj *fastjson.Object var err error @@ -220,7 +217,7 @@ func TestFastjsonDetectSchema(t *testing.T) { return } obj.Visit(func(key []byte, v *fastjson.Value) { - nameAndTypes = append(nameAndTypes, fmt.Sprintf("%s: %s", string(key), v.Type().String())) + act[string(key)] = v.Type().String() }) - log.Infof("detected json schema: {%s}", strings.Join(nameAndTypes, ", ")) + require.Equal(t, jsonSchema, act) } diff --git a/parser/parser_test.go b/parser/parser_test.go index f5a4f987..b27298c4 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -16,7 +16,6 @@ package parser import ( "encoding/json" - "log" "testing" "github.com/tidwall/gjson" @@ -50,6 +49,33 @@ var jsonSample = []byte(`{ "bool_false": false }`) +var jsonSchema = map[string]string{ + "its": "number", + "_ip": "string", + "cgi": "string", + "channel": "string", + "platform": "string", + "experiment": "string", + "ip": "string", + "version": "string", + "success": "number", + "percent": "number", + "mp": "object", + "date1": "string", + "time_sec_rfc3339_1": "string", + "time_sec_rfc3339_2": "string", + "time_sec_clickhouse_1": "string", + "time_ms_rfc3339_1": "string", + "time_ms_rfc3339_2": "string", + "time_ms_clickhouse_1": "string", + "array_int": "array", + "array_float": "array", + "array_string": "array", + "array_empty": "array", + "bool_true": "true", + "bool_false": "false", +} + var jsonSample2 = []byte(`{"date":"2021-01-02","ip":"192.168.0.3","floatvalue":425.633,"doublevalue":571.2464722672763,"novalue":" ","metric":"CPU_Idle_Time","service":"Web3","listvalue":["aaa","bbb","ccc"],"addint":123,"adddouble":571.2464722672763,"addstring":"add","value":123,"timestamp":"2021-01-02 21:06:00"}`) var csvSampleSchema = []string{"its", @@ -83,7 +109,7 @@ func BenchmarkUnmarshallFastJson(b *testing.B) { for i := 0; i < b.N; i++ { v, err := p.Parse(str) if err != nil { - log.Fatal(err) + panic(err) } v.GetInt("its") v.GetStringBytes("_ip") diff --git a/pool/conn.go b/pool/conn.go index af1655b4..11324f1c 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -29,8 +29,6 @@ import ( "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" "github.com/troian/healthcheck" - - log "github.com/sirupsen/logrus" ) const ( @@ -52,11 +50,11 @@ type Connection struct { func (c *Connection) ReConnect() error { sqlDB, err := sql.Open("clickhouse", c.dsn) if err != nil { - log.Info("reconnect to ", c.dsn, err.Error()) + util.Logger.Info("reconnect to ", c.dsn, err.Error()) return err } setDBParams(sqlDB) - log.Info("reconnect success to ", c.dsn) + util.Logger.Info("reconnect success to ", c.dsn) c.DB = sqlDB return nil } @@ -111,8 +109,7 @@ func FreeConn() { defer lock.Unlock() for _, conn := range connections { if err := health.Health.RemoveReadinessCheck(conn.dsn); err != nil { - err = errors.Wrapf(err, conn.dsn) - log.Errorf("got error: %+v", err) + util.Logger.Error(conn.dsn+" RemoveReadinessCheck failed", err) } conn.DB.Close() } diff --git a/statistics/statistics.go b/statistics/statistics.go index 617e498d..1a27f536 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -20,11 +20,11 @@ import ( "net" "time" + "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" "github.com/prometheus/common/expfmt" - log "github.com/sirupsen/logrus" ) var ( @@ -215,11 +215,11 @@ FOR: err := p.pusher.Push() if err != nil { err = errors.Wrapf(err, "") - log.Infof("pushing metrics failed. %v", err) + util.Logger.Error("pushing metrics failed", err) p.reconnect() } case <-ctx.Done(): - log.Warnf("metric pusher quit due to context has been canceled") + util.Logger.Warn("metric pusher quit due to context has been canceled") break FOR } } diff --git a/task/ring.go b/task/ring.go index 9fd53172..64000228 100644 --- a/task/ring.go +++ b/task/ring.go @@ -6,7 +6,6 @@ import ( "github.com/fagongzi/goetty" "github.com/pkg/errors" - log "github.com/sirupsen/logrus" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/statistics" @@ -44,7 +43,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { ring.idleCnt = 0 ring.isIdle = false ring.ringBuf = make([]model.MsgRow, ring.ringCap) - log.Infof("%s: topic %s partition %d quit idle", taskCfg.Name, taskCfg.Topic, ring.partition) + util.Logger.Infof("%s: topic %s partition %d quit idle", taskCfg.Name, taskCfg.Topic, ring.partition) } // assert(msgOffset < ring.ringGroundOff + ring.ringCap) if msgOffset >= ring.ringCeilingOff { @@ -53,7 +52,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { if ring.service.sharder != nil && msgRow.Row != nil { if msgRow.Shard, err = ring.service.sharder.Calc(msgRow.Row); err != nil { - log.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) } } statistics.RingMsgs.WithLabelValues(taskCfg.Name).Inc() @@ -66,7 +65,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { ring.tid.Stop() if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) } } } @@ -81,7 +80,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { taskCfg := &ring.service.cfg.Task select { case <-ring.service.ctx.Done(): - log.Errorf("%s: Ring.ForceBatchOrShard quit due to the context has been canceled", taskCfg.Name) + util.Logger.Errorf("%s: Ring.ForceBatchOrShard quit due to the context has been canceled", taskCfg.Name) return default: } @@ -90,7 +89,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { defer ring.mux.Unlock() if arg != nil { newMsg, _ = arg.(*model.InputMessage) - log.Warnf("%s: Ring.ForceBatchOrShard partition %d message range [%d, %d)", taskCfg.Name, newMsg.Partition, ring.ringGroundOff, newMsg.Offset) + util.Logger.Warnf("%s: Ring.ForceBatchOrShard partition %d message range [%d, %d)", taskCfg.Name, newMsg.Partition, ring.ringGroundOff, newMsg.Offset) } if !ring.isIdle { if newMsg == nil { @@ -103,7 +102,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { ring.idleCnt = 0 ring.isIdle = true ring.ringBuf = nil - log.Infof("%s: topic %s partition %d enter idle", taskCfg.Name, taskCfg.Topic, ring.partition) + util.Logger.Infof("%s: topic %s partition %d enter idle", taskCfg.Name, taskCfg.Topic, ring.partition) } } } else { @@ -127,7 +126,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { var err error if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) } } @@ -179,7 +178,7 @@ func (ring *Ring) genBatchOrShard(expNewGroundOff int64) { } if batch.RealSize > 0 { - log.Debugf("%s: going to flush a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", + util.Logger.Debugf("%s: going to flush a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", taskCfg.Name, taskCfg.Topic, ring.partition, endOff-1, batch.RealSize, gaps, parseErrs) diff --git a/task/sharding.go b/task/sharding.go index a9751ed1..59f4fd42 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -13,7 +13,6 @@ import ( "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" - log "github.com/sirupsen/logrus" ) type ShardingPolicy struct { @@ -183,7 +182,7 @@ func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOf maxBatchSize = batchSize } } - log.Debugf("%s: sharded a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", + util.Logger.Debugf("%s: sharded a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", taskCfg.Name, taskCfg.Topic, partition, endOff-1, msgCnt, gaps, parseErrs) if maxBatchSize >= taskCfg.BufferSize { @@ -218,7 +217,7 @@ func (sh *Sharder) doFlush(_ interface{}) { } } if msgCnt > 0 { - log.Debugf("%s: going to flush batch group for topic %v, offsets %+v, messages %d", taskCfg.Name, taskCfg.Name, sh.offsets, msgCnt) + util.Logger.Debugf("%s: going to flush batch group for topic %v, offsets %+v, messages %d", taskCfg.Name, taskCfg.Name, sh.offsets, msgCnt) sh.batchSys.CreateBatchGroupMulti(batches, sh.offsets) sh.offsets = sh.offsets[:0] // ALL batches in a group shall be populated before sending any one to next stage. @@ -232,6 +231,6 @@ func (sh *Sharder) doFlush(_ interface{}) { sh.tid.Stop() if sh.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, sh.ForceFlush, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) } } diff --git a/task/task.go b/task/task.go index a4611dfb..114722fe 100644 --- a/task/task.go +++ b/task/task.go @@ -37,8 +37,6 @@ import ( "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" "golang.org/x/time/rate" - - log "github.com/sirupsen/logrus" ) // TaskService holds the configuration for each task @@ -113,7 +111,7 @@ func (service *Service) Init() (err error) { } if maxDims <= len(service.dims) { service.cfg.Task.DynamicSchema.Enable = false - log.Warnf("%s: disabled DynamicSchema since the number of columns reaches upper limit %d", taskCfg.Name, maxDims) + util.Logger.Warnf("%s: disabled DynamicSchema since the number of columns reaches upper limit %d", taskCfg.Name, maxDims) } else { for _, dim := range service.dims { service.knownKeys.Store(dim.SourceName, nil) @@ -134,13 +132,13 @@ func (service *Service) Run(ctx context.Context) { service.started = true service.parentCtx = ctx service.ctx, service.cancel = context.WithCancel(ctx) - log.Infof("%s: task started", service.cfg.Task.Name) + util.Logger.Infof("%s: task started", service.cfg.Task.Name) go service.inputer.Run(service.ctx) if service.sharder != nil { // schedule a delayed ForceFlush if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(service.cfg.Task.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", service.cfg.Task.Name, err) + util.Logger.Fatalf("%s: got error %+v", service.cfg.Task.Name, err) } } @@ -151,7 +149,7 @@ LOOP: break LOOP case batch := <-service.batchChan: if err := service.flush(batch); err != nil { - log.Errorf("%s: got error %+v", service.cfg.Task.Name, err) + util.Logger.Errorf("%s: got error %+v", service.cfg.Task.Name, err) } } } @@ -197,7 +195,7 @@ func (service *Service) put(msg model.InputMessage) { // schedule a delayed ForceBatchOrShard if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - log.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) } service.rings[msg.Partition] = ring service.Unlock() @@ -210,7 +208,7 @@ func (service *Service) put(msg model.InputMessage) { if msg.Offset < ringFilledOffset { statistics.RingMsgsOffTooSmallErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter2.Allow() { - log.Warnf("%s: got a message(topic %v, partition %d, offset %v) left to %v", + util.Logger.Warnf("%s: got a message(topic %v, partition %d, offset %v) left to %v", taskCfg.Name, msg.Topic, msg.Partition, msg.Offset, ringFilledOffset) } return @@ -218,7 +216,7 @@ func (service *Service) put(msg model.InputMessage) { if msg.Offset >= ringGroundOff+ring.ringCap && atomic.LoadInt32(&service.cntNewKeys) == 0 { statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(service.cfg.Task.Name).Inc() if service.limiter3.Allow() { - log.Warnf("%s: got a message(topic %v, partition %d, offset %v) right to the range [%v, %v)", + util.Logger.Warnf("%s: got a message(topic %v, partition %d, offset %v) right to the range [%v, %v)", taskCfg.Name, msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap) } time.Sleep(1 * time.Second) @@ -236,7 +234,7 @@ func (service *Service) put(msg model.InputMessage) { if err != nil { statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter1.Allow() { - log.Errorf("%s: failed to parse message(topic %v, partition %d, offset %v) %+v, string(value) <<<%+v>>>, got error %+v", + util.Logger.Errorf("%s: failed to parse message(topic %v, partition %d, offset %v) %+v, string(value) <<<%+v>>>, got error %+v", service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, msg, string(msg.Value), err) } } else { @@ -262,7 +260,7 @@ func (service *Service) put(msg model.InputMessage) { service.sharder.ForceFlush(nil) } if service.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.changeSchema, nil); err != nil { - log.Fatalf("got error %+v", err) + util.Logger.Fatalf("got error %+v", err) os.Exit(-1) } } @@ -290,13 +288,13 @@ func (service *Service) changeSchema(arg interface{}) { taskCfg := &service.cfg.Task // change schema if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { - log.Fatalf("%s: clickhouse.ChangeSchema failed with error: %+v", taskCfg.Name, err) + util.Logger.Fatalf("%s: clickhouse.ChangeSchema failed with error: %+v", taskCfg.Name, err) os.Exit(-1) } // restart myself service.Stop() if err = service.Init(); err != nil { - log.Fatalf("%s: init failed with error: %+v", taskCfg.Name, err) + util.Logger.Fatalf("%s: init failed with error: %+v", taskCfg.Name, err) os.Exit(-1) } go service.Run(service.parentCtx) @@ -304,15 +302,15 @@ func (service *Service) changeSchema(arg interface{}) { // Stop stop kafka and clickhouse client. This is blocking. func (service *Service) Stop() { - log.Infof("%s: stopping task service...", service.cfg.Task.Name) + util.Logger.Infof("%s: stopping task service...", service.cfg.Task.Name) service.cancel() if err := service.inputer.Stop(); err != nil { panic(err) } - log.Infof("%s: stopped input", service.cfg.Task.Name) + util.Logger.Infof("%s: stopped input", service.cfg.Task.Name) _ = service.clickhouse.Stop() - log.Infof("%s: stopped output", service.cfg.Task.Name) + util.Logger.Infof("%s: stopped output", service.cfg.Task.Name) if service.sharder != nil { service.sharder.tid.Stop() @@ -323,12 +321,12 @@ func (service *Service) Stop() { } } service.tid.Stop() - log.Infof("%s: stopped internal timers", service.cfg.Task.Name) + util.Logger.Infof("%s: stopped internal timers", service.cfg.Task.Name) if service.started { <-service.stopped } - log.Infof("%s: stopped", service.cfg.Task.Name) + util.Logger.Infof("%s: stopped", service.cfg.Task.Name) } // GoID returns goroutine id diff --git a/util/app.go b/util/app.go index fcba7f6a..f4501c1a 100644 --- a/util/app.go +++ b/util/app.go @@ -16,50 +16,25 @@ limitations under the License. package util import ( - "log" + "go.uber.org/zap" ) func Run(appName string, initFunc, jobFunc, cleanupFunc func() error) { - log.Printf("Initial [%s]", appName) + Logger.Info(appName + " initialization") if err := initFunc(); err != nil { - log.Printf("Initial [%s] failure: [%s]", appName, err) - panic(err) + Logger.Fatal(appName+" initialization failed", zap.Error(err)) } - log.Printf("Initial [%s] complete", appName) + Logger.Info(appName + " initialization completed") go func() { if err := jobFunc(); err != nil { - log.Printf("[%s] run error: [%v]", appName, err) - panic(err) + Logger.Fatal(appName+" run failed", zap.Error(err)) } }() WaitForExitSign() - log.Printf("[%s] watched the exit signal, start to clean", appName) + Logger.Info(appName + " got the exit signal, start to clean") if err := cleanupFunc(); err != nil { - log.Printf("[%s] clean failed: [%v]", appName, err) - panic(err) - } - log.Printf("[%s] clean complete, exited", appName) -} - -func Funcs(funcs ...func() error) func() error { - return func() error { - for _, fun := range funcs { - if err := fun(); err != nil { - return err - } - } - return nil - } -} -func LogWrapper(msg string, fun func() error) func() error { - return func() error { - log.Println(msg + " start") - if err := fun(); err != nil { - log.Printf("%s failed: %v", msg, err) - return err - } - log.Println(msg + " success") - return nil + Logger.Fatal(appName+" clean failed", zap.Error(err)) } + Logger.Info(appName + " clean completed, exit") } diff --git a/util/common.go b/util/common.go index 784de22f..43af69aa 100644 --- a/util/common.go +++ b/util/common.go @@ -30,7 +30,9 @@ import ( "strings" "time" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "gopkg.in/natefinch/lumberjack.v2" "github.com/fagongzi/goetty" "github.com/pkg/errors" @@ -40,6 +42,7 @@ var ( GlobalTimerWheel *goetty.TimeoutWheel //the global timer wheel GlobalParsingPool *WorkerPool //for all tasks' parsing, cpu intensive GlobalWritingPool *WorkerPool //the all tasks' writing ClickHouse, cpu-net balance + Logger *zap.SugaredLogger ) // InitGlobalTimerWheel initialize the global timer wheel @@ -86,14 +89,16 @@ func GetShift(s int) (shift uint) { // GetOutboundIP get preferred outbound ip of this machine //https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go -func GetOutboundIP() net.IP { - conn, err := net.Dial("udp", "8.8.8.8:80") - if err != nil { - log.Fatal(err) +func GetOutboundIP() (ip net.IP, err error) { + var conn net.Conn + if conn, err = net.Dial("udp", "8.8.8.8:80"); err != nil { + err = errors.Wrapf(err, "") + return } defer conn.Close() localAddr, _ := conn.LocalAddr().(*net.UDPAddr) - return localAddr.IP + ip = localAddr.IP + return } // GetSpareTCPPort find a spare TCP port @@ -195,7 +200,7 @@ func JksToPem(jksPath, jksPassword string, overwrite bool) (certPemPath, keyPemP {"openssl", "pkcs12", "-in", pkcs12Path, "-nodes", "-nocerts", "-out", keyPemPath, "-passin", "env:password"}, } for _, cmd := range cmds { - log.Infof(strings.Join(cmd, " ")) + Logger.Info(strings.Join(cmd, " ")) exe := exec.Command(cmd[0], cmd[1:]...) if cmd[0] == "keytool" { exe.Stdin = bytes.NewReader([]byte(jksPassword + "\n" + jksPassword + "\n" + jksPassword)) @@ -204,7 +209,7 @@ func JksToPem(jksPath, jksPassword string, overwrite bool) (certPemPath, keyPemP } var out []byte out, err = exe.CombinedOutput() - log.Infof(string(out)) + Logger.Info(string(out)) if err != nil { err = errors.Wrapf(err, "") return @@ -212,3 +217,33 @@ func JksToPem(jksPath, jksPassword string, overwrite bool) (certPemPath, keyPemP } return } + +func InitLogger(logLevel string, logPaths []string) { + var lvl zapcore.Level + if err := lvl.Set(logLevel); err != nil { + lvl = zap.InfoLevel + } + var syncers []zapcore.WriteSyncer + for _, p := range logPaths { + switch p { + case "stdout": + syncers = append(syncers, zapcore.AddSync(os.Stdout)) + case "stderr": + syncers = append(syncers, zapcore.AddSync(os.Stderr)) + default: + writeFile := zapcore.AddSync(&lumberjack.Logger{ + Filename: p, + MaxSize: 100, // megabytes + MaxBackups: 10, + }) + syncers = append(syncers, writeFile) + } + } + + core := zapcore.NewCore( + zapcore.NewJSONEncoder(zap.NewProductionEncoderConfig()), + zapcore.NewMultiWriteSyncer(syncers...), + lvl, + ) + Logger = zap.New(core).Sugar() +} From 104123bb018a441cd25683b9bf5cf7ec2471203d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 22 Apr 2021 17:37:10 +0800 Subject: [PATCH 077/404] parser treats type mismatch as error --- model/message.go | 9 +- model/metric.go | 21 +-- model/value.go | 10 +- output/clickhouse.go | 3 +- parser/csv.go | 184 ++++++++++--------- parser/csv_test.go | 199 --------------------- parser/dummy.go | 69 -------- parser/fastjson.go | 192 +++++++++++--------- parser/fastjson_test.go | 196 --------------------- parser/gjson.go | 188 +++++++++++--------- parser/gjson_test.go | 198 --------------------- parser/parser.go | 9 +- parser/parser_test.go | 381 +++++++++++++++++++++++++++++++++++++++- task/task.go | 20 +-- 14 files changed, 735 insertions(+), 944 deletions(-) delete mode 100644 parser/csv_test.go delete mode 100644 parser/dummy.go delete mode 100644 parser/gjson_test.go diff --git a/model/message.go b/model/message.go index 90c6f49f..c6e60911 100644 --- a/model/message.go +++ b/model/message.go @@ -170,8 +170,9 @@ func PutRow(r *Row) { rowPool.Put(r) } -func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType) (row *Row) { +func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType) (row *Row, err error) { row = GetRow() + var val interface{} for _, dim := range dims { if strings.HasPrefix(dim.Name, "__kafka") { if strings.HasSuffix(dim.Name, "_topic") { @@ -182,7 +183,11 @@ func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType) (row * *row = append(*row, msg.Offset) } } else { - *row = append(*row, GetValueByType(metric, dim)) + if val, err = GetValueByType(metric, dim); err != nil { + PutRow(row) + return + } + *row = append(*row, val) } } return diff --git a/model/metric.go b/model/metric.go index 308e7193..42d5c232 100644 --- a/model/metric.go +++ b/model/metric.go @@ -15,19 +15,20 @@ limitations under the License. package model -import "sync" +import ( + "sync" +) // Metric interface for metric collection type Metric interface { - Get(key string) interface{} - GetString(key string, nullable bool) interface{} - GetArray(key string, t string) interface{} - GetFloat(key string, nullable bool) interface{} - GetInt(key string, nullable bool) interface{} - GetDate(key string, nullable bool) interface{} - GetDateTime(key string, nullable bool) interface{} - GetDateTime64(key string, nullable bool) interface{} - GetElasticDateTime(key string, nullable bool) interface{} + GetString(key string, nullable bool) (val interface{}, err error) + GetFloat(key string, nullable bool) (val interface{}, err error) + GetInt(key string, nullable bool) (val interface{}, err error) + GetDate(key string, nullable bool) (val interface{}, err error) + GetDateTime(key string, nullable bool) (val interface{}, err error) + GetDateTime64(key string, nullable bool) (val interface{}, err error) + GetElasticDateTime(key string, nullable bool) (val interface{}, err error) + GetArray(key string, t string) (val interface{}, err error) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool } diff --git a/model/value.go b/model/value.go index 39b645f6..f14f4e39 100644 --- a/model/value.go +++ b/model/value.go @@ -16,8 +16,6 @@ package model import ( "strings" - - "github.com/ClickHouse/clickhouse-go" ) const ( @@ -43,7 +41,7 @@ var ( ) // There are only three cases for the value type of metric, (float64, string, map [string] interface {}) -func GetValueByType(metric Metric, cwt *ColumnWithType) interface{} { +func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}, err error) { name := cwt.SourceName switch cwt.Type { case Int: @@ -53,11 +51,11 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) interface{} { case String: return metric.GetString(name, cwt.Nullable) case IntArray: - return clickhouse.Array(metric.GetArray(name, "int")) + return metric.GetArray(name, "int") case FloatArray: - return clickhouse.Array(metric.GetArray(name, "float")) + return metric.GetArray(name, "float") case StringArray: - return clickhouse.Array(metric.GetArray(name, "string")) + return metric.GetArray(name, "string") case Date: return metric.GetDate(name, cwt.Nullable) case DateTime: diff --git a/output/clickhouse.go b/output/clickhouse.go index ee32ffe1..958101fa 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -22,7 +22,6 @@ import ( "fmt" "io" "math" - "os" "regexp" "strings" "sync" @@ -154,7 +153,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { if shouldReconnect(err) && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { time.Sleep(10 * time.Second) } else { - os.Exit(-1) + util.Logger.Fatalf("%s: ClickHouse.loopWrite failed", c.cfg.Task.Name) } } } diff --git a/parser/csv.go b/parser/csv.go index e3c26b7f..c6fb37d3 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -17,6 +17,7 @@ package parser import ( "bytes" "encoding/csv" + "io" "strconv" "strings" "sync" @@ -55,125 +56,142 @@ type CsvMetric struct { values []string } -// Get returns the value corresponding to a column expects called -// interpret the type -func (c *CsvMetric) Get(key string) interface{} { - for i, k := range c.pp.csvFormat { - if k == key && i < len(c.values) { - return c.values[i] +// GetString get the value as string +func (c *CsvMetric) GetString(key string, nullable bool) (val interface{}, err error) { + var idx int + var ok bool + if idx, ok = c.pp.csvFormat[key]; !ok { + if nullable { + return } + val = "" + return } - return nil + val = c.values[idx] + return } -// GetString get the value as string -func (c *CsvMetric) GetString(key string, nullable bool) interface{} { - _ = nullable // nullable can not be supported with csv - for i, k := range c.pp.csvFormat { - if k == key && i < len(c.values) { - return c.values[i] +// GetFloat returns the value as float +func (c *CsvMetric) GetFloat(key string, nullable bool) (val interface{}, err error) { + var idx int + var ok bool + if idx, ok = c.pp.csvFormat[key]; !ok { + if nullable { + return } + val = float64(0.0) + return } - return "" + val, err = strconv.ParseFloat(c.values[idx], 64) + return } -// GetFloat returns the value as float -func (c *CsvMetric) GetFloat(key string, nullable bool) interface{} { - _ = nullable // nullable can not be supported with csv - for i, k := range c.pp.csvFormat { - if k == key && i < len(c.values) { - n, _ := strconv.ParseFloat(c.values[i], 64) - return n +// GetInt returns int +func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}, err error) { + var idx int + var ok bool + if idx, ok = c.pp.csvFormat[key]; !ok { + if nullable { + return } + val = int64(0) + return } - return float64(0) + val, err = strconv.ParseInt(c.values[idx], 10, 64) + return } -// GetInt returns int -func (c *CsvMetric) GetInt(key string, nullable bool) interface{} { - _ = nullable // nullable can not be supported with csv - for i, k := range c.pp.csvFormat { - if k == key && i < len(c.values) { - n, _ := strconv.ParseInt(c.values[i], 10, 64) - return n +func (c *CsvMetric) GetDate(key string, nullable bool) (val interface{}, err error) { + return c.GetDateTime(key, nullable) +} + +func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}, err error) { + var idx int + var ok bool + if idx, ok = c.pp.csvFormat[key]; !ok { + if nullable { + return } + val = Epoch + return + } + val, err = c.pp.ParseDateTime(key, c.values[idx]) + return +} + +func (c *CsvMetric) GetDateTime64(key string, nullable bool) (val interface{}, err error) { + return c.GetDateTime(key, nullable) +} + +func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) (val interface{}, err error) { + var t interface{} + if t, err = c.GetDateTime(key, nullable); err != nil { + return + } + if t != nil { + val = t.(time.Time).Unix() } - return int64(0) + return } // GetArray parse an CSV encoded array -func (c *CsvMetric) GetArray(key string, t string) interface{} { - var err error +func (c *CsvMetric) GetArray(key string, t string) (val interface{}, err error) { + var s interface{} var array []string var r *csv.Reader - val, _ := c.GetString(key, false).(string) - valLen := len(val) - if val == "" || val[0] != '[' || val[valLen-1] != ']' { - goto QUIT + if s, err = c.GetString(key, false); err != nil { + return + } + str, _ := s.(string) + strLen := len(str) + if str == "" || str[0] != '[' || str[strLen-1] != ']' { + err = errors.Errorf("GetArray %s got unexpected value %s", key, str) + return } - r = csv.NewReader(strings.NewReader(val[1 : valLen-1])) + r = csv.NewReader(strings.NewReader(str[1 : strLen-1])) if array, err = r.Read(); err != nil { - goto QUIT + if errors.Is(err, io.EOF) { + err = nil + switch t { + case "int": + val = []int64{} + case "float": + val = []float64{} + case "string": + val = []string{} + default: + panic("LOGIC ERROR: not supported array type " + t) + } + } + return } switch t { case "int": results := make([]int64, 0, len(array)) + var v int64 for _, e := range array { - v, _ := strconv.ParseInt(e, 10, 64) + if v, err = strconv.ParseInt(e, 10, 64); err != nil { + return + } results = append(results, v) } - return results + val = results case "float": results := make([]float64, 0, len(array)) + var v float64 for _, e := range array { - v, _ := strconv.ParseFloat(e, 64) + if v, err = strconv.ParseFloat(e, 64); err != nil { + return + } results = append(results, v) } - return results - case "string": - return array - default: - panic("not supported array type " + t) - } -QUIT: - switch t { - case "int": - return []int64{} - case "float": - return []float64{} + val = results case "string": - return []string{} + val = array default: - return nil + panic("LOGIC ERROR: not supported array type " + t) } -} - -func (c *CsvMetric) GetDate(key string, nullable bool) interface{} { - _ = nullable // nullable can not be supported with csv - - val, _ := c.GetString(key, false).(string) - t, _ := c.pp.ParseDateTime(key, val) - return t -} - -func (c *CsvMetric) GetDateTime(key string, nullable bool) interface{} { - val, _ := c.GetString(key, false).(string) - t, _ := c.pp.ParseDateTime(key, val) - return t -} - -func (c *CsvMetric) GetDateTime64(key string, nullable bool) interface{} { - val, _ := c.GetString(key, false).(string) - t, _ := c.pp.ParseDateTime(key, val) - return t -} - -func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) interface{} { - _ = nullable // nullable can not be supported with csv - val, _ := c.GetString(key, false).(string) - t, _ := time.Parse(time.RFC3339, val) - - return t.Unix() + return } func (c *CsvMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool { diff --git a/parser/csv_test.go b/parser/csv_test.go deleted file mode 100644 index af4061b2..00000000 --- a/parser/csv_test.go +++ /dev/null @@ -1,199 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ -package parser - -import ( - "testing" - "time" - - "github.com/stretchr/testify/require" -) - -func TestCsvInt(t *testing.T) { - pp, _ := NewParserPool("csv", csvSampleSchema, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - var exp, act int64 - exp = 1536813227 - act, _ = metric.GetInt("its", false).(int64) - require.Equal(t, exp, act) - - exp = 0 - act, _ = metric.GetInt("not_exist", false).(int64) - require.Equal(t, exp, act) - - act, _ = metric.GetInt("not_exist", true).(int64) - require.Equal(t, exp, act) -} - -func TestCsvFloat(t *testing.T) { - pp, _ := NewParserPool("csv", csvSampleSchema, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - var exp, act float64 - exp = 0.11 - act, _ = metric.GetFloat("percent", false).(float64) - require.Equal(t, exp, act) - - exp = 0.0 - act, _ = metric.GetFloat("not_exist", false).(float64) - require.Equal(t, exp, act) - - act, _ = metric.GetFloat("not_exist", true).(float64) - require.Equal(t, exp, act) -} - -func TestCsvString(t *testing.T) { - pp, _ := NewParserPool("csv", csvSampleSchema, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - var exp, act string - exp = `escaped_"ws` - act, _ = metric.GetString("channel", false).(string) - require.Equal(t, exp, act) - - exp = "" - act, _ = metric.GetString("not_exist", false).(string) - require.Equal(t, exp, act) - - act, _ = metric.GetString("not_exist", true).(string) - require.Equal(t, exp, act) -} - -func TestCsvDate(t *testing.T) { - pp, _ := NewParserPool("csv", csvSampleSchema, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - act, _ = metric.GetDate("date1", false).(time.Time) - require.Equal(t, exp, act) -} - -func TestCsvDateTime(t *testing.T) { - pp, _ := NewParserPool("csv", csvSampleSchema, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - act, _ = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) - require.Equal(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) - act = metric.GetDateTime("time_sec_rfc3339_2", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) - act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) - - exp = time.Time{} - act, _ = metric.GetDateTime("not_exist", false).(time.Time) - require.Equal(t, exp, act) -} - -func TestCsvDateTime64(t *testing.T) { - pp, _ := NewParserPool("csv", csvSampleSchema, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - act, _ = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) - require.Equal(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) - act = metric.GetDateTime64("time_ms_rfc3339_2", false).(time.Time).In(time.UTC) - require.EqualValues(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) - act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) - - exp = time.Time{} - act, _ = metric.GetDateTime64("not_exist", false).(time.Time) - require.Equal(t, exp, act) -} - -func TestCsvElasticDateTime(t *testing.T) { - pp, _ := NewParserPool("csv", csvSampleSchema, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - var exp, act int64 - // {"date": "2019-12-16T12:10:30Z"} - // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC - exp = 1576498230 - act, _ = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) - require.Equal(t, exp, act) - - exp = -62135596800 - act, _ = metric.GetElasticDateTime("not_exist", false).(int64) - require.Equal(t, exp, act) - - act, _ = metric.GetElasticDateTime("not_exist", true).(int64) - require.Equal(t, exp, act) -} - -func TestCsvArray(t *testing.T) { - pp, _ := NewParserPool("csv", csvSampleSchema, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, err := parser.Parse(csvSample) - require.Nil(t, err) - - actI, _ := metric.GetArray("array_int", "int").([]int64) - expI := []int64{1, 2, 3} - require.Equal(t, expI, actI) - - actF, _ := metric.GetArray("array_float", "float").([]float64) - expF := []float64{1.1, 2.2, 3.3} - require.Equal(t, expF, actF) - - actS, _ := metric.GetArray("array_string", "string").([]string) - expS := []string{"aa", "bb", "cc"} - require.Equal(t, expS, actS) - - actIE, _ := metric.GetArray("array_empty", "int").([]int64) - expIE := []int64{} - require.Equal(t, expIE, actIE) - - actFE, _ := metric.GetArray("array_empty", "float").([]float64) - expFE := []float64{} - require.Equal(t, expFE, actFE) - - actSE, _ := metric.GetArray("array_empty", "string").([]string) - expSE := []string{} - require.Equal(t, expSE, actSE) -} diff --git a/parser/dummy.go b/parser/dummy.go deleted file mode 100644 index 952cb628..00000000 --- a/parser/dummy.go +++ /dev/null @@ -1,69 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package parser - -import ( - "sync" - "time" -) - -type DummyMetric struct { -} - -func (c *DummyMetric) Get(key string) interface{} { - return "" -} - -func (c *DummyMetric) GetString(key string) string { - return "" -} - -func (c *DummyMetric) GetFloat(key string) float64 { - return 0 -} - -func (c *DummyMetric) GetInt(key string) int64 { - return 0 -} - -// GetArray is Empty implemented for DummyMetric -func (c *DummyMetric) GetArray(key string, t string) interface{} { - return []string{} -} - -func (c *DummyMetric) String() string { - return "_dummy" -} - -func (c *DummyMetric) GetDate(key string) (t time.Time) { - return -} - -func (c *DummyMetric) GetDateTime(key string) (t time.Time) { - return -} - -func (c *DummyMetric) GetDateTime64(key string) (t time.Time) { - return -} - -func (c *DummyMetric) GetElasticDateTime(key string) int64 { - return 0 -} - -func (c *DummyMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool { - return false -} diff --git a/parser/fastjson.go b/parser/fastjson.go index 7523af18..ac1b90d6 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -48,52 +48,121 @@ type FastjsonMetric struct { value *fastjson.Value } -func (c *FastjsonMetric) Get(key string) interface{} { - return c.value.Get(key) +func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}, err error) { + v := c.value.Get(key) + if v == nil { + if nullable { + return + } + val = "" + return + } + var b []byte + if b, err = v.StringBytes(); err != nil { + return + } + val = string(b) + return } -func (c *FastjsonMetric) GetString(key string, nullable bool) interface{} { - v := c.value.GetStringBytes(key) - if nullable && v == nil { - return nil +func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}, err error) { + v := c.value.Get(key) + if v == nil { + if nullable { + return + } + val = float64(0.0) + return } - return string(v) + val, err = v.Float64() + return } -func (c *FastjsonMetric) GetFloat(key string, nullable bool) interface{} { - if nullable && !c.value.Exists(key) { - return nil +func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}, err error) { + v := c.value.Get(key) + if v == nil { + if nullable { + return + } + val = int64(0) + return + } + switch v.Type() { + case fastjson.TypeTrue: + val = int64(1) + case fastjson.TypeFalse: + val = int64(0) + default: + val, err = v.Int64() } - return c.value.GetFloat64(key) + return +} + +func (c *FastjsonMetric) GetDate(key string, nullable bool) (val interface{}, err error) { + return c.GetDateTime(key, nullable) } -func (c *FastjsonMetric) GetInt(key string, nullable bool) interface{} { +func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, err error) { v := c.value.Get(key) if v == nil { if nullable { - return nil + return } - return int64(0) + val = Epoch + return + } + switch v.Type() { + case fastjson.TypeNumber: + var f float64 + if f, err = v.Float64(); err != nil { + return + } + val = time.Unix(int64(f), int64(f*1e9)%1e9) + case fastjson.TypeString: + var b []byte + if b, err = v.StringBytes(); err != nil { + return + } + val, err = c.pp.ParseDateTime(key, string(b)) + default: + err = errors.Errorf("GetDateTime %s got unexpected type %s", key, v.Type().String()) + } + return +} + +func (c *FastjsonMetric) GetDateTime64(key string, nullable bool) (val interface{}, err error) { + return c.GetDateTime(key, nullable) +} + +func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}, err error) { + var t interface{} + if t, err = c.GetDateTime(key, nullable); err != nil { + return } - if v.Type() == fastjson.TypeTrue { - return int64(1) + if t != nil { + val = t.(time.Time).Unix() } - return c.value.GetInt64(key) + return } -func (c *FastjsonMetric) GetArray(key string, t string) interface{} { - array := c.value.GetArray(key) - if array == nil { +func (c *FastjsonMetric) GetArray(key string, t string) (val interface{}, err error) { + v := c.value.Get(key) + if v == nil { switch t { case "int": - return []int64{} + val = []int64{} case "float": - return []float64{} + val = []float64{} case "string": - return []string{} + val = []string{} default: - return nil + panic("LOGIC ERROR: not supported array type " + t) } + return + } + var array []*fastjson.Value + if array, err = v.Array(); err != nil { + return } switch t { case "int": @@ -103,80 +172,37 @@ func (c *FastjsonMetric) GetArray(key string, t string) interface{} { if e.Type() == fastjson.TypeTrue { v = 1 } else { - v, _ = e.Int64() + if v, err = e.Int64(); err != nil { + return + } } results = append(results, v) } - return results + val = results case "float": results := make([]float64, 0, len(array)) for _, e := range array { - v, _ := e.Float64() + var v float64 + if v, err = e.Float64(); err != nil { + return + } results = append(results, v) } - return results + val = results case "string": results := make([]string, 0, len(array)) for _, e := range array { - v, _ := e.StringBytes() + var v []byte + if v, err = e.StringBytes(); err != nil { + return + } results = append(results, string(v)) } - return results + val = results default: - panic("not supported array type " + t) - } -} - -func (c *FastjsonMetric) String() string { - return c.value.String() -} - -func (c *FastjsonMetric) GetDate(key string, nullable bool) interface{} { - if nullable && !c.value.Exists(key) { - return nil - } - - val, _ := c.GetString(key, false).(string) - t, _ := c.pp.ParseDateTime(key, val) - return t -} - -func (c *FastjsonMetric) GetDateTime(key string, nullable bool) interface{} { - if nullable && !c.value.Exists(key) { - return nil - } - - if v, _ := c.GetFloat(key, false).(float64); v != 0 { - return time.Unix(int64(v), int64(v*1e9)%1e9) - } - - val, _ := c.GetString(key, false).(string) - t, _ := c.pp.ParseDateTime(key, val) - return t -} - -func (c *FastjsonMetric) GetDateTime64(key string, nullable bool) interface{} { - if nullable && !c.value.Exists(key) { - return nil + panic("LOGIC ERROR: not supported array type " + t) } - - if v, _ := c.GetFloat(key, false).(float64); v != 0 { - return time.Unix(int64(v), int64(v*1e9)%1e9) - } - - val, _ := c.GetString(key, false).(string) - t, _ := c.pp.ParseDateTime(key, val) - return t -} - -func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) interface{} { - val := c.GetString(key, nullable) - if val == nil { - return nil - } - t, _ := time.Parse(time.RFC3339, val.(string)) - - return t.Unix() + return } func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (foundNew bool) { diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go index a51b2a9c..1b0c8a89 100644 --- a/parser/fastjson_test.go +++ b/parser/fastjson_test.go @@ -2,207 +2,11 @@ package parser import ( "testing" - "time" "github.com/stretchr/testify/require" "github.com/valyala/fastjson" ) -func TestFastjsonInt(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act int64 - exp = 1536813227 - act, _ = metric.GetInt("its", false).(int64) - require.Equal(t, exp, act) - - exp = 0 - act, _ = metric.GetInt("not_exist", false).(int64) - require.Equal(t, exp, act) - - actual := metric.GetInt("not_exist", true) - require.Nil(t, actual, "err should be nothing") - - exp = 0 - act, _ = metric.GetInt("bool_false", false).(int64) - require.Equal(t, exp, act) - - exp = 1 - act, _ = metric.GetInt("bool_true", false).(int64) - require.Equal(t, exp, act) -} - -func TestFastjsonFloat(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act float64 - exp = 0.11 - act, _ = metric.GetFloat("percent", false).(float64) - require.Equal(t, exp, act) - - exp = 0.0 - act, _ = metric.GetFloat("not_exist", false).(float64) - require.Equal(t, exp, act) - - actual := metric.GetFloat("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestFastjsonString(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act string - exp = "ws" - act, _ = metric.GetString("channel", false).(string) - require.Equal(t, exp, act) - - exp = "" - act, _ = metric.GetString("not_exist", false).(string) - require.Equal(t, exp, act) - - actual := metric.GetString("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestFastjsonDate(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - act, _ = metric.GetDate("date1", false).(time.Time) - require.Equal(t, exp, act) - - exp = time.Time{} - act, _ = metric.GetDate("not_exist", false).(time.Time) - require.Equal(t, exp, act) - - actual := metric.GetDate("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestFastjsonDateTime(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - act, _ = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) - require.Equal(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) - act = metric.GetDateTime("time_sec_rfc3339_2", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) - act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) - - exp = time.Time{} - act, _ = metric.GetDateTime("not_exist", false).(time.Time) - require.Equal(t, exp, act) - - actual := metric.GetDateTime("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestFastjsonDateTime64(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - act, _ = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) - require.Equal(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) - act = metric.GetDateTime64("time_ms_rfc3339_2", false).(time.Time).In(time.UTC) - require.EqualValues(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) - act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) - - exp = time.Time{} - act, _ = metric.GetDateTime64("not_exist", false).(time.Time) - require.Equal(t, exp, act) - - actual := metric.GetDateTime64("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestFastjsonElasticDateTime(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act int64 - // {"date": "2019-12-16T12:10:30Z"} - // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC - exp = 1576498230 - act, _ = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) - require.Equal(t, exp, act) - - exp = -62135596800 - act, _ = metric.GetElasticDateTime("not_exist", false).(int64) - require.Equal(t, exp, act) - - actual := metric.GetElasticDateTime("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestFastjsonArray(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - actI, _ := metric.GetArray("array_int", "int").([]int64) - expI := []int64{1, 2, 3} - require.Equal(t, expI, actI) - - actF, _ := metric.GetArray("array_float", "float").([]float64) - expF := []float64{1.1, 2.2, 3.3} - require.Equal(t, expF, actF) - - actS, _ := metric.GetArray("array_string", "string").([]string) - expS := []string{"aa", "bb", "cc"} - require.Equal(t, expS, actS) - - actIE, _ := metric.GetArray("array_empty", "int").([]int64) - expIE := []int64{} - require.Equal(t, expIE, actIE) - - actFE, _ := metric.GetArray("array_empty", "float").([]float64) - expFE := []float64{} - require.Equal(t, expFE, actFE) - - actSE, _ := metric.GetArray("array_empty", "string").([]string) - expSE := []string{} - require.Equal(t, expSE, actSE) - - metric, _ = parser.Parse(jsonSample2) - actS, _ = metric.GetArray("listvalue", "string").([]string) - expS = []string{"aaa", "bbb", "ccc"} - require.Equal(t, expS, actS) -} - func TestFastjsonDetectSchema(t *testing.T) { pp, _ := NewParserPool("fastjson", nil, "", "") parser := pp.Get() diff --git a/parser/gjson.go b/parser/gjson.go index cdd4e559..f810666d 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -19,6 +19,7 @@ import ( "sync" "time" + "github.com/pkg/errors" "github.com/tidwall/gjson" "github.com/housepower/clickhouse_sinker/model" @@ -40,114 +41,143 @@ type GjsonMetric struct { raw string } -func (c *GjsonMetric) Get(key string) interface{} { - return gjson.Get(c.raw, key).Value() -} - -func (c *GjsonMetric) GetString(key string, nullable bool) interface{} { +func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if nullable && !r.Exists() { - return nil - } - - return r.String() -} - -func (c *GjsonMetric) GetArray(key string, t string) interface{} { - slice := gjson.Get(c.raw, key).Array() - switch t { - case "string": - results := make([]string, 0, len(slice)) - for _, s := range slice { - results = append(results, s.String()) - } - return results - - case "float": - results := make([]float64, 0, len(slice)) - - for _, s := range slice { - results = append(results, s.Float()) + if !r.Exists() { + if nullable { + return } - return results - - case "int": - results := make([]int64, 0, len(slice)) - for _, s := range slice { - results = append(results, s.Int()) - } - return results - + val = "" + return + } + switch r.Type { + case gjson.String: + val = r.Str default: - panic("not supported array type " + t) + err = errors.Errorf("GetString %s got unexpected type %s", key, r.Type.String()) } + return } -func (c *GjsonMetric) GetFloat(key string, nullable bool) interface{} { +func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if nullable && !r.Exists() { - return nil + if !r.Exists() { + if nullable { + return + } + val = float64(0.0) + return } - return r.Float() -} - -func (c *GjsonMetric) GetInt(key string, nullable bool) interface{} { - r := gjson.Get(c.raw, key) - if nullable && !r.Exists() { - return nil + switch r.Type { + case gjson.Number: + val = r.Num + default: + err = errors.Errorf("GetFloat %s got unexpected type %s", key, r.Type.String()) } - return r.Int() + return } -func (c *GjsonMetric) GetDate(key string, nullable bool) interface{} { +func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if nullable && !r.Exists() { - return nil + if !r.Exists() { + if nullable { + return + } + val = int64(0) + return } + switch r.Type { + case gjson.Number: + val = int64(r.Num) + default: + err = errors.Errorf("GetInt %s got unexpected type %s", key, r.Type.String()) + } + return +} - val := r.String() - t, _ := c.pp.ParseDateTime(key, val) - return t +func (c *GjsonMetric) GetDate(key string, nullable bool) (val interface{}, err error) { + return c.GetDateTime(key, nullable) } -func (c *GjsonMetric) GetDateTime(key string, nullable bool) interface{} { +func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if nullable && !r.Exists() { - return nil + if !r.Exists() { + if nullable { + return + } + val = Epoch + return } - - if v := r.Float(); v != 0 { - return time.Unix(int64(v), int64(v*1e9)%1e9) + switch r.Type { + case gjson.Number: + val = time.Unix(int64(r.Num), int64(r.Num*1e9)%1e9) + case gjson.String: + val, err = c.pp.ParseDateTime(key, r.Str) + default: + err = errors.Errorf("GetDateTime %s got unexpected type %s", key, r.Type.String()) } + return +} - val := r.String() - t, _ := c.pp.ParseDateTime(key, val) - return t +func (c *GjsonMetric) GetDateTime64(key string, nullable bool) (val interface{}, err error) { + return c.GetDateTime(key, nullable) } -func (c *GjsonMetric) GetDateTime64(key string, nullable bool) interface{} { - r := gjson.Get(c.raw, key) - if nullable && !r.Exists() { - return nil +func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}, err error) { + var t interface{} + if t, err = c.GetDateTime(key, nullable); err != nil { + return } - - if v := r.Float(); v != 0 { - return time.Unix(int64(v), int64(v*1e9)%1e9) + if t != nil { + val = t.(time.Time).Unix() } - - val := r.String() - t, _ := c.pp.ParseDateTime(key, val) - return t + return } -func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) interface{} { +func (c *GjsonMetric) GetArray(key string, t string) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if nullable && !r.Exists() { - return nil + if !r.Exists() { + switch t { + case "int": + val = []int64{} + case "float": + val = []float64{} + case "string": + val = []string{} + default: + panic("LOGIC ERROR: not supported array type " + t) + } + return } + if r.Type != gjson.JSON { + err = errors.Errorf("GetArray %s got unexpected type %s", key, r.Type.String()) + return + } + array := r.Array() + switch t { + case "string": + results := make([]string, 0, len(array)) + for _, s := range array { + results = append(results, s.String()) + } + val = results + case "float": + results := make([]float64, 0, len(array)) - t, _ := time.Parse(time.RFC3339, r.String()) - return t.Unix() + for _, s := range array { + results = append(results, s.Float()) + } + val = results + case "int": + results := make([]int64, 0, len(array)) + for _, s := range array { + results = append(results, s.Int()) + } + val = results + default: + panic("LOGIC ERROR: not supported array type " + t) + } + return } func (c *GjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool { diff --git a/parser/gjson_test.go b/parser/gjson_test.go deleted file mode 100644 index d1ca7468..00000000 --- a/parser/gjson_test.go +++ /dev/null @@ -1,198 +0,0 @@ -package parser - -import ( - "testing" - "time" - - "github.com/stretchr/testify/require" -) - -func TestGjsonInt(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act int64 - exp = 1536813227 - act, _ = metric.GetInt("its", false).(int64) - require.Equal(t, exp, act) - - exp = 0 - act, _ = metric.GetInt("not_exist", false).(int64) - require.Equal(t, exp, act) - - actual := metric.GetInt("not_exist", true) - require.Nil(t, actual, "err should be nothing") - - exp = 0 - act, _ = metric.GetInt("bool_false", false).(int64) - require.Equal(t, exp, act) - - exp = 1 - act, _ = metric.GetInt("bool_true", false).(int64) - require.Equal(t, exp, act) -} - -func TestGjsonFloat(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act float64 - exp = 0.11 - act, _ = metric.GetFloat("percent", false).(float64) - require.Equal(t, exp, act) - - exp = 0.0 - act, _ = metric.GetFloat("not_exist", false).(float64) - require.Equal(t, exp, act) - - actual := metric.GetFloat("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestGjsonString(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act string - exp = "ws" - act, _ = metric.GetString("channel", false).(string) - require.Equal(t, exp, act) - - exp = "" - act, _ = metric.GetString("not_exist", false).(string) - require.Equal(t, exp, act) - - actual := metric.GetString("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestGjsonDate(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - act, _ = metric.GetDate("date1", false).(time.Time) - require.Equal(t, exp, act) - - exp = time.Time{} - act, _ = metric.GetDate("not_exist", false).(time.Time) - require.Equal(t, exp, act) - - actual := metric.GetDate("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestGjsonDateTime(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - act, _ = metric.GetDateTime("time_sec_rfc3339_1", false).(time.Time) - require.Equal(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) - act = metric.GetDateTime("time_sec_rfc3339_2", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) - act = metric.GetDateTime("time_sec_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) - - exp = time.Time{} - act, _ = metric.GetDateTime("not_exist", false).(time.Time) - require.Equal(t, exp, act) - - actual := metric.GetDateTime("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestGjsonDateTime64(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act time.Time - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - act, _ = metric.GetDateTime64("time_ms_rfc3339_1", false).(time.Time) - require.Equal(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) - act = metric.GetDateTime64("time_ms_rfc3339_2", false).(time.Time).In(time.UTC) - require.EqualValues(t, exp, act) - - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) - act = metric.GetDateTime64("time_ms_clickhouse_1", false).(time.Time).In(time.UTC) - require.Equal(t, exp, act) - - exp = time.Time{} - act, _ = metric.GetDateTime64("not_exist", false).(time.Time) - require.Equal(t, exp, act) - - actual := metric.GetDateTime64("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestGjsonElasticDateTime(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - var exp, act int64 - // {"date": "2019-12-16T12:10:30Z"} - // TZ=UTC date -d @1576498230 => Mon 16 Dec 2019 12:10:30 PM UTC - exp = 1576498230 - act, _ = metric.GetElasticDateTime("time_sec_rfc3339_1", false).(int64) - require.Equal(t, exp, act) - - exp = -62135596800 - act, _ = metric.GetElasticDateTime("not_exist", false).(int64) - require.Equal(t, exp, act) - - actual := metric.GetElasticDateTime("not_exist", true) - require.Nil(t, actual, "err should be nothing") -} - -func TestGjsonArray(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - actI, _ := metric.GetArray("mp.i", "int").([]int64) - expI := []int64{1, 2, 3} - require.Equal(t, actI, expI) - - actF, _ := metric.GetArray("mp.f", "float").([]float64) - expF := []float64{1.1, 2.2, 3.3} - require.Equal(t, expF, actF) - - actS, _ := metric.GetArray("mp.s", "string").([]string) - expS := []string{"aa", "bb", "cc"} - require.Equal(t, expS, actS) - - actIE, _ := metric.GetArray("mp.e", "int").([]int64) - expIE := []int64{} - require.Equal(t, expIE, actIE) - - actFE, _ := metric.GetArray("mp.e", "float").([]float64) - expFE := []float64{} - require.Equal(t, expFE, actFE) - - actSE, _ := metric.GetArray("mp.e", "string").([]string) - expSE := []string{} - require.Equal(t, expSE, actSE) -} diff --git a/parser/parser.go b/parser/parser.go index e1fb2a9f..d344febe 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -70,7 +70,7 @@ type Parser interface { // Pool may be used for pooling Parsers for similarly typed JSONs. type Pool struct { name string - csvFormat []string + csvFormat map[string]int delimiter string timeZone *time.Location knownLayouts sync.Map @@ -88,10 +88,15 @@ func NewParserPool(name string, csvFormat []string, delimiter string, timezone s } pp = &Pool{ name: name, - csvFormat: csvFormat, delimiter: delimiter, timeZone: tz, } + if csvFormat != nil { + pp.csvFormat = make(map[string]int) + for i, title := range csvFormat { + pp.csvFormat[title] = i + } + } return } diff --git a/parser/parser_test.go b/parser/parser_test.go index b27298c4..85f8a7e2 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -16,8 +16,12 @@ package parser import ( "encoding/json" + "sync" "testing" + "time" + "github.com/housepower/clickhouse_sinker/model" + "github.com/stretchr/testify/require" "github.com/tidwall/gjson" "github.com/valyala/fastjson" ) @@ -26,7 +30,7 @@ var jsonSample = []byte(`{ "its":1536813227, "_ip":"112.96.65.228", "cgi":"/commui/queryhttpdns", - "channel":"ws", + "channel":"escaped_\"ws", "platform":"adr", "experiment":"default", "ip":"36.248.20.69", @@ -76,9 +80,10 @@ var jsonSchema = map[string]string{ "bool_false": "false", } -var jsonSample2 = []byte(`{"date":"2021-01-02","ip":"192.168.0.3","floatvalue":425.633,"doublevalue":571.2464722672763,"novalue":" ","metric":"CPU_Idle_Time","service":"Web3","listvalue":["aaa","bbb","ccc"],"addint":123,"adddouble":571.2464722672763,"addstring":"add","value":123,"timestamp":"2021-01-02 21:06:00"}`) +var csvSample = []byte(`1536813227,"0.11","escaped_""ws",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[aa,bb,cc]","[]"`) -var csvSampleSchema = []string{"its", +var csvSchema = []string{ + "its", "percent", "channel", "date1", @@ -91,8 +96,374 @@ var csvSampleSchema = []string{"its", "array_int", "array_float", "array_string", - "array_empty"} -var csvSample = []byte(`1536813227,"0.11","escaped_""ws",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[aa,bb,cc]","[]"`) + "array_empty", +} + +var initialize sync.Once +var initErr error +var names = []string{"csv", "fastjson", "gjson"} +var pools []*Pool +var parsers []Parser +var metrics []model.Metric + +func initMetrics() { + var pp *Pool + var parser Parser + var metric model.Metric + + pp, _ = NewParserPool("csv", csvSchema, "", "") + parser = pp.Get() + if metric, initErr = parser.Parse(csvSample); initErr != nil { + return + } + metrics = append(metrics, metric) + pools = append(pools, pp) + parsers = append(parsers, parser) + + pp, _ = NewParserPool("fastjson", nil, "", "") + parser = pp.Get() + if metric, initErr = parser.Parse(jsonSample); initErr != nil { + return + } + metrics = append(metrics, metric) + pools = append(pools, pp) + parsers = append(parsers, parser) + + pp, _ = NewParserPool("gjson", nil, "", "") + parser = pp.Get() + if metric, initErr = parser.Parse(jsonSample); initErr != nil { + return + } + metrics = append(metrics, metric) + pools = append(pools, pp) + parsers = append(parsers, parser) +} + +func TestParserInt(t *testing.T) { + initialize.Do(initMetrics) + require.Nil(t, initErr) + + var err error + var v interface{} + var exp, act int64 + var desc string + for i := range names { + name := names[i] + metric := metrics[i] + + desc = name + ` GetInt("its", false)` + exp = 1536813227 + v, err = metric.GetInt("its", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act, _ = v.(int64) + require.Equal(t, exp, act, desc) + + desc = name + ` GetInt("not_exist", false)` + exp = int64(0) + v, err = metric.GetInt("not_exist", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act, _ = v.(int64) + require.Equal(t, exp, act, desc) + + desc = name + ` GetInt("not_exist", true)` + v, err = metric.GetInt("not_exist", true) + require.Nil(t, err, desc) + require.Nil(t, v, desc) + + // Verify parsers treat type mismatch as error. + desc = name + ` GetInt("channel", false)` + _, err = metric.GetInt("channel", false) + require.NotNil(t, err, desc) + } +} + +func TestParserFloat(t *testing.T) { + initialize.Do(initMetrics) + require.Nil(t, initErr) + + var err error + var v interface{} + var exp, act float64 + var desc string + for i := range names { + name := names[i] + metric := metrics[i] + + desc = name + ` GetFloat("percent", false)` + exp = 0.11 + v, err = metric.GetFloat("percent", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act, _ = v.(float64) + require.Equal(t, exp, act, desc) + + desc = name + ` GetFloat("not_exist", false)` + exp = 0.0 + v, err = metric.GetFloat("not_exist", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act, _ = v.(float64) + require.Equal(t, exp, act, desc) + + desc = name + ` GetFloat("not_exist", true)` + v, err = metric.GetFloat("not_exist", true) + require.Nil(t, err, desc) + require.Nil(t, v, desc) + + // Verify parsers treat type mismatch as error. + desc = name + ` GetFloat("channel", false)` + _, err = metric.GetFloat("channel", false) + require.NotNil(t, err, desc) + } +} + +func TestParserString(t *testing.T) { + initialize.Do(initMetrics) + require.Nil(t, initErr) + + var err error + var v interface{} + var exp, act string + var desc string + for i := range names { + name := names[i] + metric := metrics[i] + + desc = name + ` GetString("channel", false)` + exp = "escaped_\"ws" + v, err = metric.GetString("channel", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act, _ = v.(string) + require.Equal(t, exp, act, desc) + + desc = name + ` GetString("not_exist", false)` + exp = "" + v, err = metric.GetString("not_exist", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act, _ = v.(string) + require.Equal(t, exp, act, desc) + + desc = name + ` GetString("not_exist", true)` + v, err = metric.GetString("not_exist", true) + require.Nil(t, err, desc) + require.Nil(t, v, desc) + + // Verify parsers treat type mismatch as error. + desc = name + ` GetString("its", false)` + _, err = metric.GetString("its", false) + switch name { + case "csv": + require.Nil(t, err, desc) + default: + require.NotNil(t, err, desc) + } + } +} + +func TestParserDateTime(t *testing.T) { + initialize.Do(initMetrics) + require.Nil(t, initErr) + + var err error + var v interface{} + var exp, act time.Time + var desc string + for i := range names { + name := names[i] + metric := metrics[i] + + desc = name + ` GetDate("date1", false)` + exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) + v, err = metric.GetDate("date1", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act = v.(time.Time).In(time.Local) + require.Equal(t, exp, act, desc) + + desc = name + ` GetDateTime("time_sec_rfc3339_1", false)` + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) + v, err = metric.GetDateTime("time_sec_rfc3339_1", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act = v.(time.Time).In(time.UTC) + require.Equal(t, exp, act, desc) + + desc = name + ` GetDateTime("time_sec_rfc3339_2", false)` + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) + v, err = metric.GetDateTime("time_sec_rfc3339_2", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act = v.(time.Time).In(time.UTC) + require.Equal(t, exp, act, desc) + + desc = name + ` GetDateTime("time_sec_clickhouse_1", false)` + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) + v, err = metric.GetDateTime("time_sec_clickhouse_1", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act = v.(time.Time).In(time.UTC) + require.Equal(t, exp, act, desc) + + desc = name + ` GetDateTime("not_exist", false)` + exp = Epoch + v, err = metric.GetDateTime("not_exist", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act = v.(time.Time).In(time.UTC) + require.Equal(t, exp, act, desc) + + // Verify parsers treat type mismatch as error. + desc = name + ` GetDateTime("array_int", false)` + _, err = metric.GetDateTime("array_int", false) + require.NotNil(t, err, desc) + } +} + +func TestParserDateTime64(t *testing.T) { + initialize.Do(initMetrics) + require.Nil(t, initErr) + + var err error + var v interface{} + var exp, act time.Time + var desc string + for i := range names { + name := names[i] + metric := metrics[i] + + desc = name + ` GetDateTime64("time_ms_rfc3339_1", false)` + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) + v, err = metric.GetDateTime64("time_ms_rfc3339_1", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act = v.(time.Time).In(time.UTC) + require.Equal(t, exp, act, desc) + + desc = name + ` GetDateTime64("time_ms_rfc3339_2", false)` + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) + v, err = metric.GetDateTime64("time_ms_rfc3339_2", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act = v.(time.Time).In(time.UTC) + require.Equal(t, exp, act, desc) + + desc = name + ` GetDateTime64("time_ms_clickhouse_1", false)` + exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) + v, err = metric.GetDateTime64("time_ms_clickhouse_1", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act = v.(time.Time).In(time.UTC) + require.Equal(t, exp, act, desc) + + desc = name + ` GetDateTime64("not_exist", false)` + exp = Epoch + v, err = metric.GetDateTime64("not_exist", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act = v.(time.Time).In(time.UTC) + require.Equal(t, exp, act, desc) + } +} + +func TestParserElasticDateTime(t *testing.T) { + initialize.Do(initMetrics) + require.Nil(t, initErr) + + var err error + var v interface{} + var exp, act int64 + var desc string + for i := range names { + name := names[i] + metric := metrics[i] + + desc = name + ` GetElasticDateTime("time_sec_rfc3339_1", false)` + exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC).Unix() + v, err = metric.GetElasticDateTime("time_sec_rfc3339_1", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act, _ = v.(int64) + require.Equal(t, exp, act, desc) + + desc = name + ` GetElasticDateTime("not_exist", false)` + exp = 0 + v, err = metric.GetElasticDateTime("not_exist", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act, _ = v.(int64) + require.Equal(t, exp, act, desc) + + desc = name + ` GetElasticDateTime("not_exist", true)` + v, err = metric.GetElasticDateTime("not_exist", true) + require.Nil(t, err, desc) + require.Nil(t, v, desc) + } +} + +func TestParserArray(t *testing.T) { + initialize.Do(initMetrics) + require.Nil(t, initErr) + + var err error + var v interface{} + var desc string + for i := range names { + name := names[i] + metric := metrics[i] + + desc = name + ` GetArray("array_int", "int")` + expI := []int64{1, 2, 3} + v, err = metric.GetArray("array_int", "int") + require.Nil(t, err, desc) + require.IsType(t, expI, v, desc) + actI, _ := v.([]int64) + require.Equal(t, expI, actI, desc) + + desc = name + ` GetArray("array_float", "float")` + expF := []float64{1.1, 2.2, 3.3} + v, err = metric.GetArray("array_float", "float") + require.Nil(t, err, desc) + require.IsType(t, expF, v, desc) + actF, _ := v.([]float64) + require.Equal(t, expF, actF, desc) + + desc = name + ` GetArray("array_string", "string")` + expS := []string{"aa", "bb", "cc"} + v, err = metric.GetArray("array_string", "string") + require.Nil(t, err, desc) + require.IsType(t, expS, v, desc) + actS, _ := v.([]string) + require.Equal(t, expS, actS, desc) + + desc = name + ` GetArray("array_empty", "int")` + expIE := []int64{} + v, err = metric.GetArray("array_empty", "int") + require.Nil(t, err, desc) + require.IsType(t, expIE, v, desc) + actIE, _ := v.([]int64) + require.Equal(t, expIE, actIE, desc) + + desc = name + ` GetArray("array_empty", "float")` + expFE := []float64{} + v, err = metric.GetArray("array_empty", "float") + require.Nil(t, err, desc) + require.IsType(t, expFE, v, desc) + actFE, _ := v.([]float64) + require.Equal(t, expFE, actFE, desc) + + desc = name + ` GetArray("array_empty", "string")` + expSE := []string{} + v, err = metric.GetArray("array_empty", "string") + require.Nil(t, err, desc) + require.IsType(t, expSE, v, desc) + actSE, _ := v.([]string) + require.Equal(t, expSE, actSE, desc) + } +} func BenchmarkUnmarshalljson(b *testing.B) { mp := map[string]interface{}{} diff --git a/task/task.go b/task/task.go index 114722fe..4599d973 100644 --- a/task/task.go +++ b/task/task.go @@ -19,7 +19,6 @@ import ( "context" "fmt" "math" - "os" "runtime" "strconv" "strings" @@ -229,23 +228,27 @@ func (service *Service) put(msg model.InputMessage) { _ = util.GlobalParsingPool.Submit(func() { var row *model.Row var foundNewKeys bool + var metric model.Metric p := service.pp.Get() - metric, err := p.Parse(msg.Value) + metric, err = p.Parse(msg.Value) + if err == nil { + row, err = model.MetricToRow(metric, msg, service.dims) + } if err != nil { statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter1.Allow() { util.Logger.Errorf("%s: failed to parse message(topic %v, partition %d, offset %v) %+v, string(value) <<<%+v>>>, got error %+v", service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, msg, string(msg.Value), err) } - } else { - row = model.MetricToRow(metric, msg, service.dims) + // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. + service.pp.Put(p) + return } + service.pp.Put(p) + if taskCfg.DynamicSchema.Enable { foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys) } - // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. - service.pp.Put(p) - if foundNewKeys { cntNewKeys := atomic.AddInt32(&service.cntNewKeys, 1) if cntNewKeys == 1 { @@ -261,7 +264,6 @@ func (service *Service) put(msg model.InputMessage) { } if service.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.changeSchema, nil); err != nil { util.Logger.Fatalf("got error %+v", err) - os.Exit(-1) } } } @@ -289,13 +291,11 @@ func (service *Service) changeSchema(arg interface{}) { // change schema if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { util.Logger.Fatalf("%s: clickhouse.ChangeSchema failed with error: %+v", taskCfg.Name, err) - os.Exit(-1) } // restart myself service.Stop() if err = service.Init(); err != nil { util.Logger.Fatalf("%s: init failed with error: %+v", taskCfg.Name, err) - os.Exit(-1) } go service.Run(service.parentCtx) } From f75cb3a99e40d81764c0709913a1ef1d1d263815 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 23 Apr 2021 08:36:32 +0800 Subject: [PATCH 078/404] dsn escape --- Makefile | 4 ++-- cmd/clickhouse_sinker/main.go | 2 +- parser/parser.go | 3 ++- pool/conn.go | 3 ++- 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/Makefile b/Makefile index c3e06272..a623bf3d 100644 --- a/Makefile +++ b/Makefile @@ -1,8 +1,8 @@ PKG := github.com/housepower/clickhouse_sinker EDITION ?= housepower -SINKER_LDFLAGS += -X "$(PKG)/config.SinkerReleaseVersion=$(git describe --tags --dirty)" -SINKER_LDFLAGS += -X "$(PKG)/config.SinkerBuildTS=$(shell date -u '+%Y-%m-%d %I:%M:%S')" +SINKER_LDFLAGS += -X "$(PKG)/config.SinkerReleaseVersion=$(shell git describe --tags --dirty)" +SINKER_LDFLAGS += -X "$(PKG)/config.SinkerBuildTS=$(shell date --iso-8601=s)" SINKER_LDFLAGS += -X "$(PKG)/config.SinkerGitHash=$(shell git rev-parse HEAD)" SINKER_LDFLAGS += -X "$(PKG)/config.SinkerGitBranch=$(shell git rev-parse --abbrev-ref HEAD)" SINKER_LDFLAGS += -X "$(PKG)/config.SinkerEdition=$(EDITION)" diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 4fe9efd0..e8e9ccd1 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -116,8 +116,8 @@ func initCmdOptions() { func init() { util.InitLogger("info", []string{"stdout"}) initCmdOptions() + util.Logger.Info(config.GetSinkerInfo()) if cmdOps.ShowVer { - util.Logger.Info(config.GetSinkerInfo()) os.Exit(0) } var err error diff --git a/parser/parser.go b/parser/parser.go index d344febe..11239c11 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -28,7 +28,8 @@ var ( Layouts = []string{ //DateTime "2006-01-02T15:04:05.999999999Z07:00", //time.RFC3339Nano, `date --iso-8601=ns` output format - "2006-01-02T15:04:05Z07:00", //time.RFC3339, `date --iso-8601=s` output format + "2006-01-02T15:04:05Z07:00", //time.RFC3339, `date --iso-8601=s` on Ubuntu 20.04 + "2006-01-02T15:04:05-0700", //`date --iso-8601=s` on CentOS 7.6 "2006-01-02T15:04:05.999999999", "2006-01-02T15:04:05", "2006-01-02 15:04:05.999999999Z07:00", //`date --rfc-3339=ns` output format diff --git a/pool/conn.go b/pool/conn.go index 11324f1c..a6ffc724 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -21,6 +21,7 @@ package pool import ( "database/sql" "fmt" + "net/url" "strings" "sync" "time" @@ -75,7 +76,7 @@ func InitConn(hosts [][]string, port int, db, username, password, dsnParams stri replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) } dsn := fmt.Sprintf("tcp://%s?database=%s&username=%s&password=%s&block_size=%d", - replicaAddrs[0], db, username, password, BlockSize) + replicaAddrs[0], url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password), BlockSize) if numReplicas > 1 { dsn += "&connection_open_strategy=in_order&alt_hosts=" + strings.Join(replicaAddrs[1:numReplicas], ",") } From db8d614fa952dd4b4704440aa43d8b5bd1fd1839 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 23 Apr 2021 14:57:53 +0800 Subject: [PATCH 079/404] more time format --- parser/parser.go | 1 + 1 file changed, 1 insertion(+) diff --git a/parser/parser.go b/parser/parser.go index 11239c11..ddd8d3da 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -39,6 +39,7 @@ var ( "Jan 02, 2006 15:04:05.999999999Z07:00", "Jan 02, 2006 15:04:05.999999999", "Jan 02, 2006 15:04:05", + "Jan 02, 2006 03:04:05 PM", "02/01/2006 15:04:05.999999999", "02/01/06 15:04:05.999999999", "02/Jan/2006 15:04:05 Z07:00", From 101b1bb4239ab7f148f7dfd922c393c0f652de19 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 25 Apr 2021 17:22:24 +0800 Subject: [PATCH 080/404] show version built by goreleaser --- Makefile | 12 ++++-------- cmd/clickhouse_sinker/main.go | 12 +++++++++++- config/build.go | 21 --------------------- 3 files changed, 15 insertions(+), 30 deletions(-) delete mode 100644 config/build.go diff --git a/Makefile b/Makefile index a623bf3d..6f6e8df9 100644 --- a/Makefile +++ b/Makefile @@ -1,11 +1,7 @@ -PKG := github.com/housepower/clickhouse_sinker -EDITION ?= housepower - -SINKER_LDFLAGS += -X "$(PKG)/config.SinkerReleaseVersion=$(shell git describe --tags --dirty)" -SINKER_LDFLAGS += -X "$(PKG)/config.SinkerBuildTS=$(shell date --iso-8601=s)" -SINKER_LDFLAGS += -X "$(PKG)/config.SinkerGitHash=$(shell git rev-parse HEAD)" -SINKER_LDFLAGS += -X "$(PKG)/config.SinkerGitBranch=$(shell git rev-parse --abbrev-ref HEAD)" -SINKER_LDFLAGS += -X "$(PKG)/config.SinkerEdition=$(EDITION)" +SINKER_LDFLAGS += -X "main.version=$(shell git describe --tags --dirty)" +SINKER_LDFLAGS += -X "main.date=$(shell date --iso-8601=s)" +SINKER_LDFLAGS += -X "main.commit=$(shell git rev-parse HEAD)" +SINKER_LDFLAGS += -X "main.builtBy=$(shell echo `whoami`@`hostname`)" GO := CGO_ENABLED=0 go GOBUILD := $(GO) build $(BUILD_FLAG) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index e8e9ccd1..26e4be46 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -61,6 +61,12 @@ type CmdOptions struct { } var ( + //goreleaser fill following info per https://goreleaser.com/customization/build/. + version = "None" + commit = "None" + date = "None" + builtBy = "None" + cmdOps CmdOptions selfIP string httpMetrics = promhttp.Handler() @@ -113,10 +119,14 @@ func initCmdOptions() { flag.Parse() } +func getVersion() string { + return fmt.Sprintf("version %s, commit %s, date %s, builtBy %s", version, commit, date, builtBy) +} + func init() { util.InitLogger("info", []string{"stdout"}) initCmdOptions() - util.Logger.Info(config.GetSinkerInfo()) + util.Logger.Info(getVersion()) if cmdOps.ShowVer { os.Exit(0) } diff --git a/config/build.go b/config/build.go deleted file mode 100644 index 21ed3828..00000000 --- a/config/build.go +++ /dev/null @@ -1,21 +0,0 @@ -package config - -import "fmt" - -var ( - // SinkerReleaseVersion information. - SinkerReleaseVersion = "None" - SinkerEdition = "None" - SinkerGitHash = "None" - SinkerGitBranch = "None" - SinkerBuildTS = "None" -) - -func GetSinkerInfo() string { - return fmt.Sprintf("Release Version: %s, Edition: %s, Git Commit Hash: %s, Git Branch: %s, Build At: %s", - SinkerReleaseVersion, - SinkerEdition, - SinkerGitHash, - SinkerGitBranch, - SinkerBuildTS) -} From 45762ac6803550f990f9859d67fd3855c077ff96 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 25 Apr 2021 18:11:40 +0800 Subject: [PATCH 081/404] treat value null as not exist --- model/value.go | 26 ++++++++++++++++---------- parser/fastjson.go | 12 ++++++------ parser/gjson.go | 20 ++++++++++---------- task/task.go | 4 ++-- 4 files changed, 34 insertions(+), 28 deletions(-) diff --git a/model/value.go b/model/value.go index f14f4e39..3658b60d 100644 --- a/model/value.go +++ b/model/value.go @@ -16,6 +16,8 @@ package model import ( "strings" + + "github.com/pkg/errors" ) const ( @@ -45,28 +47,32 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}, err er name := cwt.SourceName switch cwt.Type { case Int: - return metric.GetInt(name, cwt.Nullable) + val, err = metric.GetInt(name, cwt.Nullable) case Float: - return metric.GetFloat(name, cwt.Nullable) + val, err = metric.GetFloat(name, cwt.Nullable) case String: - return metric.GetString(name, cwt.Nullable) + val, err = metric.GetString(name, cwt.Nullable) case IntArray: - return metric.GetArray(name, "int") + val, err = metric.GetArray(name, "int") case FloatArray: - return metric.GetArray(name, "float") + val, err = metric.GetArray(name, "float") case StringArray: - return metric.GetArray(name, "string") + val, err = metric.GetArray(name, "string") case Date: - return metric.GetDate(name, cwt.Nullable) + val, err = metric.GetDate(name, cwt.Nullable) case DateTime: - return metric.GetDateTime(name, cwt.Nullable) + val, err = metric.GetDateTime(name, cwt.Nullable) case DateTime64: - return metric.GetDateTime64(name, cwt.Nullable) + val, err = metric.GetDateTime64(name, cwt.Nullable) case ElasticDateTime: - return metric.GetElasticDateTime(name, cwt.Nullable) + val, err = metric.GetElasticDateTime(name, cwt.Nullable) default: panic("BUG: reached switch default condition") } + if err != nil { + err = errors.Errorf(name + " " + err.Error()) + } + return } func WhichType(typ string) (dataType int, nullable bool) { diff --git a/parser/fastjson.go b/parser/fastjson.go index ac1b90d6..5b041dd7 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -50,7 +50,7 @@ type FastjsonMetric struct { func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}, err error) { v := c.value.Get(key) - if v == nil { + if v == nil || v.Type() == fastjson.TypeNull { if nullable { return } @@ -67,7 +67,7 @@ func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}, func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}, err error) { v := c.value.Get(key) - if v == nil { + if v == nil || v.Type() == fastjson.TypeNull { if nullable { return } @@ -80,7 +80,7 @@ func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}, e func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}, err error) { v := c.value.Get(key) - if v == nil { + if v == nil || v.Type() == fastjson.TypeNull { if nullable { return } @@ -104,7 +104,7 @@ func (c *FastjsonMetric) GetDate(key string, nullable bool) (val interface{}, er func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, err error) { v := c.value.Get(key) - if v == nil { + if v == nil || v.Type() == fastjson.TypeNull { if nullable { return } @@ -125,7 +125,7 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} } val, err = c.pp.ParseDateTime(key, string(b)) default: - err = errors.Errorf("GetDateTime %s got unexpected type %s", key, v.Type().String()) + err = errors.Errorf("value doesn't contain number nor string, it contains %s", v.Type().String()) } return } @@ -147,7 +147,7 @@ func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) (val inte func (c *FastjsonMetric) GetArray(key string, t string) (val interface{}, err error) { v := c.value.Get(key) - if v == nil { + if v == nil || v.Type() == fastjson.TypeNull { switch t { case "int": val = []int64{} diff --git a/parser/gjson.go b/parser/gjson.go index f810666d..50d123a2 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -43,7 +43,7 @@ type GjsonMetric struct { func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if !r.Exists() { + if !r.Exists() || r.Type == gjson.Null { if nullable { return } @@ -54,14 +54,14 @@ func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}, err case gjson.String: val = r.Str default: - err = errors.Errorf("GetString %s got unexpected type %s", key, r.Type.String()) + err = errors.Errorf("value doesn't contain string, it contains %s", r.Type.String()) } return } func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if !r.Exists() { + if !r.Exists() || r.Type == gjson.Null { if nullable { return } @@ -72,14 +72,14 @@ func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}, err case gjson.Number: val = r.Num default: - err = errors.Errorf("GetFloat %s got unexpected type %s", key, r.Type.String()) + err = errors.Errorf("value doesn't contain number, it contains %s", r.Type.String()) } return } func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if !r.Exists() { + if !r.Exists() || r.Type == gjson.Null { if nullable { return } @@ -90,7 +90,7 @@ func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}, err er case gjson.Number: val = int64(r.Num) default: - err = errors.Errorf("GetInt %s got unexpected type %s", key, r.Type.String()) + err = errors.Errorf("value doesn't contain number, it contains %s", r.Type.String()) } return } @@ -101,7 +101,7 @@ func (c *GjsonMetric) GetDate(key string, nullable bool) (val interface{}, err e func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if !r.Exists() { + if !r.Exists() || r.Type == gjson.Null { if nullable { return } @@ -114,7 +114,7 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, e case gjson.String: val, err = c.pp.ParseDateTime(key, r.Str) default: - err = errors.Errorf("GetDateTime %s got unexpected type %s", key, r.Type.String()) + err = errors.Errorf("value doesn't contain string, it contains %s", r.Type.String()) } return } @@ -136,7 +136,7 @@ func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) (val interfa func (c *GjsonMetric) GetArray(key string, t string) (val interface{}, err error) { r := gjson.Get(c.raw, key) - if !r.Exists() { + if !r.Exists() || r.Type == gjson.Null { switch t { case "int": val = []int64{} @@ -150,7 +150,7 @@ func (c *GjsonMetric) GetArray(key string, t string) (val interface{}, err error return } if r.Type != gjson.JSON { - err = errors.Errorf("GetArray %s got unexpected type %s", key, r.Type.String()) + err = errors.Errorf("value doesn't contain json, it contains %s", r.Type.String()) return } array := r.Array() diff --git a/task/task.go b/task/task.go index 4599d973..9e13b4f2 100644 --- a/task/task.go +++ b/task/task.go @@ -237,8 +237,8 @@ func (service *Service) put(msg model.InputMessage) { if err != nil { statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter1.Allow() { - util.Logger.Errorf("%s: failed to parse message(topic %v, partition %d, offset %v) %+v, string(value) <<<%+v>>>, got error %+v", - service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, msg, string(msg.Value), err) + util.Logger.Errorf("%s: failed to parse message(topic %v, partition %d, offset %v) string(value) <<<%s>>>, got error %+v", + service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, string(msg.Value), err) } // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. service.pp.Put(p) From 189adb0b897731cb4106f5a583f9ec9ff6f0d051 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 26 Apr 2021 10:45:10 +0800 Subject: [PATCH 082/404] don't use zap Sugar logger --- cmd/clickhouse_sinker/main.go | 54 +++++++++++++------------------- cmd/nacos_publish_config/main.go | 17 +++++----- input/input.go | 3 +- input/kafka_go.go | 7 +++-- input/kafka_sarama.go | 9 +++--- output/clickhouse.go | 25 ++++++++------- parser/fastjson.go | 3 +- pool/conn.go | 7 +++-- statistics/statistics.go | 3 +- task/ring.go | 22 +++++++------ task/sharding.go | 12 ++++--- task/task.go | 40 ++++++++++++----------- util/common.go | 8 +++-- 13 files changed, 108 insertions(+), 102 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 26e4be46..55a89181 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -17,7 +17,6 @@ package main import ( "context" - "encoding/json" "flag" "fmt" "log" @@ -42,7 +41,6 @@ import ( "go.uber.org/zap" _ "github.com/ClickHouse/clickhouse-go" - "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus/promhttp" ) @@ -154,7 +152,8 @@ func main() { var rcm config.RemoteConfManager var properties map[string]interface{} if cmdOps.NacosDataID != "" { - util.Logger.Infof("get config from nacos serverAddrs %s, namespaceId %s, group %s, dataId %s", cmdOps.NacosAddr, cmdOps.NacosNamespaceID, cmdOps.NacosGroup, cmdOps.NacosDataID) + util.Logger.Info(fmt.Sprintf("get config from nacos serverAddrs %s, namespaceId %s, group %s, dataId %s", + cmdOps.NacosAddr, cmdOps.NacosNamespaceID, cmdOps.NacosGroup, cmdOps.NacosDataID)) rcm = &config.NacosConfManager{} properties = make(map[string]interface{}) properties["serverAddrs"] = cmdOps.NacosAddr @@ -164,7 +163,7 @@ func main() { properties["group"] = cmdOps.NacosGroup properties["dataId"] = cmdOps.NacosDataID } else { - util.Logger.Infof("get config from local file %s", cmdOps.LocalCfgFile) + util.Logger.Info(fmt.Sprintf("get config from local file %s", cmdOps.LocalCfgFile)) } if rcm != nil { if err := rcm.Init(properties); err != nil { @@ -199,8 +198,10 @@ func main() { mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) mux.HandleFunc("/debug/pprof/trace", pprof.Trace) - util.Logger.Infof("Run http server http://%s:%d", selfIP, cmdOps.HTTPPort) - util.Logger.Error(http.ListenAndServe(fmt.Sprintf(":%d", cmdOps.HTTPPort), mux)) + util.Logger.Info(fmt.Sprintf("Run http server http://%s:%d", selfIP, cmdOps.HTTPPort)) + if err := http.ListenAndServe(fmt.Sprintf(":%d", cmdOps.HTTPPort), mux); err != nil { + util.Logger.Error("http.ListenAndServe failed", zap.Error(err)) + } }() runner.Run() @@ -247,19 +248,19 @@ func (s *Sinker) Run() { if s.rcm == nil { if _, err = os.Stat(cmdOps.LocalCfgFile); err == nil { if newCfg, err = config.ParseLocalCfgFile(cmdOps.LocalCfgFile); err != nil { - util.Logger.Fatalf("%+v", err) + util.Logger.Fatal("config.ParseLocalCfgFile failed", zap.Error(err)) return } } else { - util.Logger.Fatalf("expect --local-cfg-file or --local-cfg-dir") + util.Logger.Fatal("expect --local-cfg-file or --local-cfg-dir") return } if err = newCfg.Normallize(); err != nil { - util.Logger.Fatalf("%+v", err) + util.Logger.Fatal("newCfg.Normallize failed", zap.Error(err)) return } if err = s.applyConfig(newCfg); err != nil { - util.Logger.Fatalf("%+v", err) + util.Logger.Fatal("s.applyConfig failed", zap.Error(err)) return } <-s.ctx.Done() @@ -270,15 +271,15 @@ func (s *Sinker) Run() { return case <-time.After(5 * time.Second): if newCfg, err = s.rcm.GetConfig(); err != nil { - util.Logger.Errorf("%+v", err) + util.Logger.Error("s.rcm.GetConfig failed", zap.Error(err)) return } if err = newCfg.Normallize(); err != nil { - util.Logger.Errorf("%+v", err) + util.Logger.Error("newCfg.Normallize failed", zap.Error(err)) return } if err = s.applyConfig(newCfg); err != nil { - util.Logger.Errorf("%+v", err) + util.Logger.Error("s.applyConfig failed", zap.Error(err)) return } } @@ -289,11 +290,11 @@ func (s *Sinker) Run() { // Close shutdown task func (s *Sinker) Close() { // Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). - util.Logger.Infof("%s: stopping parsing pool", s.curCfg.Task.Name) + util.Logger.Info("stopping parsing pool", zap.String("task", s.curCfg.Task.Name)) util.GlobalParsingPool.StopWait() - util.Logger.Infof("%s: stopping writing pool", s.curCfg.Task.Name) + util.Logger.Info("stopping writing pool", zap.String("task", s.curCfg.Task.Name)) util.GlobalWritingPool.StopWait() - util.Logger.Infof("%s: stopping timer wheel", s.curCfg.Task.Name) + util.Logger.Info("stopping timer wheel", zap.String("task", s.curCfg.Task.Name)) util.GlobalTimerWheel.Stop() s.task.Stop() @@ -317,13 +318,7 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { } func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { - var bsNewCfg []byte - if bsNewCfg, err = json.Marshal(newCfg); err != nil { - err = errors.Wrapf(err, "") - return - } - util.Logger.Infof("going to apply the first config: %+v", string(bsNewCfg)) - + util.Logger.Info("going to apply the first config", zap.Reflect("config", newCfg)) util.InitGlobalTimerWheel() t := GenTask(newCfg) if err = t.Init(); err != nil { @@ -348,20 +343,15 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { } func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { - var bsNewCfg []byte - if bsNewCfg, err = json.Marshal(newCfg); err != nil { - err = errors.Wrapf(err, "") - return - } - util.Logger.Infof("going to apply a different config: %+v", string(bsNewCfg)) + util.Logger.Info("going to apply another config", zap.Reflect("config", newCfg)) if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { // 1. Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). - util.Logger.Infof("%s: stopping parsing pool", s.curCfg.Task.Name) + util.Logger.Info("stopping parsing pool", zap.String("task", s.curCfg.Task.Name)) util.GlobalParsingPool.StopWait() - util.Logger.Infof("%s: stopping writing pool", s.curCfg.Task.Name) + util.Logger.Info("stopping writing pool", zap.String("task", s.curCfg.Task.Name)) util.GlobalWritingPool.StopWait() - util.Logger.Infof("%s: stopping timer wheel", s.curCfg.Task.Name) + util.Logger.Info("stopping timer wheel", zap.String("task", s.curCfg.Task.Name)) util.GlobalTimerWheel.Stop() s.task.Stop() diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 80e623c8..b510c571 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -24,6 +24,7 @@ import ( _ "github.com/ClickHouse/clickhouse-go" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/util" + "go.uber.org/zap" ) var ( @@ -60,37 +61,37 @@ func PublishSinkerConfig() { var cfg *config.Config if _, err = os.Stat(*localCfgFile); err == nil { if cfg, err = config.ParseLocalCfgFile(*localCfgFile); err != nil { - util.Logger.Fatalf("%+v", err) + util.Logger.Fatal("config.ParseLocalCfgFile failed", zap.Error(err)) return } } else { - util.Logger.Fatalf("expect --local-cfg-file") + util.Logger.Fatal("expect --local-cfg-file") return } if err = cfg.Normallize(); err != nil { - util.Logger.Fatalf("%+v", err) + util.Logger.Fatal("cfg.Normallize failed", zap.Error(err)) return } ncm := config.NacosConfManager{} properties := getProperties() if err = ncm.Init(properties); err != nil { - util.Logger.Fatalf("%+v", err) + util.Logger.Fatal("ncm.Init failed", zap.Error(err)) } if err = ncm.PublishConfig(cfg); err != nil { - util.Logger.Fatalf("%+v", err) + util.Logger.Fatal("ncm.PublishConfig failed", zap.Error(err)) } - util.Logger.Infof("sleep a while") + util.Logger.Info("sleep a while") time.Sleep(10 * time.Second) var newCfg *config.Config if newCfg, err = ncm.GetConfig(); err != nil { - util.Logger.Fatalf("%+v", err) + util.Logger.Fatal("ncm.GetConfig failed", zap.Error(err)) } if !reflect.DeepEqual(newCfg, cfg) { - util.Logger.Fatalf("got different config: %+v", newCfg) + util.Logger.Fatal("BUG: got different config", zap.Reflect("cfg", cfg), zap.Reflect("newCfg", newCfg)) } } diff --git a/input/input.go b/input/input.go index 45615ddb..ac4cde40 100644 --- a/input/input.go +++ b/input/input.go @@ -2,6 +2,7 @@ package input import ( "context" + "fmt" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" @@ -28,7 +29,7 @@ func NewInputer(typ string) Inputer { case TypeKafkaSarama: return NewKafkaSarama() default: - util.Logger.Fatalf("%s is not a supported input type", typ) + util.Logger.Fatal(fmt.Sprintf("BUG: %s is not a supported input type", typ)) return nil } } diff --git a/input/kafka_go.go b/input/kafka_go.go index 33fdaaa2..efdcac8b 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -26,6 +26,7 @@ import ( "github.com/segmentio/kafka-go" "github.com/segmentio/kafka-go/sasl/plain" "github.com/segmentio/kafka-go/sasl/scram" + "go.uber.org/zap" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" @@ -128,15 +129,15 @@ LOOP_KAFKA_GO: var msg kafka.Message if msg, err = k.r.FetchMessage(ctx); err != nil { if errors.Is(err, context.Canceled) { - util.Logger.Infof("%s: Kafka.Run quit due to context has been canceled", k.cfg.Task.Name) + util.Logger.Info("Kafka.Run quit due to context has been canceled", zap.String("task", k.cfg.Task.Name)) break LOOP_KAFKA_GO } else if errors.Is(err, io.EOF) { - util.Logger.Infof("%s: Kafka.Run quit due to reader has been closed", k.cfg.Task.Name) + util.Logger.Info("Kafka.Run quit due to reader has been closed", zap.String("task", k.cfg.Task.Name)) break LOOP_KAFKA_GO } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.cfg.Task.Name).Inc() err = errors.Wrap(err, "") - util.Logger.Errorf("%s: Kafka.Run got error %+v", k.cfg.Task.Name, err) + util.Logger.Error("k.r.FetchMessage failed", zap.String("task", k.cfg.Task.Name), zap.Error(err)) continue } } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index d4c1ee51..805f7722 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -26,6 +26,7 @@ import ( "github.com/Shopify/sarama" "github.com/pkg/errors" "github.com/xdg/scram" + "go.uber.org/zap" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" @@ -58,7 +59,7 @@ func (h MyConsumerGroupHandler) Setup(sess sarama.ConsumerGroupSession) error { return nil } func (h MyConsumerGroupHandler) Cleanup(_ sarama.ConsumerGroupSession) error { - util.Logger.Infof("%s: consumer group %s cleanup", h.k.cfg.Task.Name, h.k.cfg.Task.ConsumerGroup) + util.Logger.Info("consumer group cleanup", zap.String("task", h.k.cfg.Task.Name), zap.String("consumer group", h.k.cfg.Task.ConsumerGroup)) time.Sleep(5 * time.Second) return nil } @@ -146,15 +147,15 @@ LOOP_SARAMA: // recreated to get the new claims if err := k.cg.Consume(ctx, []string{taskCfg.Topic}, handler); err != nil { if errors.Is(err, context.Canceled) { - util.Logger.Infof("%s: Kafka.Run quit due to context has been canceled", taskCfg.Name) + util.Logger.Info("Kafka.Run quit due to context has been canceled", zap.String("task", k.cfg.Task.Name)) break LOOP_SARAMA } else if errors.Is(err, sarama.ErrClosedConsumerGroup) { - util.Logger.Infof("%s: Kafka.Run quit due to consumer group has been closed", taskCfg.Name) + util.Logger.Info("Kafka.Run quit due to consumer group has been closed", zap.String("task", k.cfg.Task.Name)) break LOOP_SARAMA } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() err = errors.Wrap(err, "") - util.Logger.Errorf("%s: Kafka.Run got error %+v", taskCfg.Name, err) + util.Logger.Error("Kafka.Run got error %+v", zap.String("task", k.cfg.Task.Name), zap.Error(err)) continue } } diff --git a/output/clickhouse.go b/output/clickhouse.go index 958101fa..413172c1 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -33,6 +33,7 @@ import ( "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" + "go.uber.org/zap" ) var ( @@ -98,7 +99,7 @@ func (c *ClickHouse) write(batch *model.Batch) error { } } if err != nil { - util.Logger.Errorf("%s: stmt.Exec failed %d times with errors %+v", c.cfg.Task.Name, numErr, err) + util.Logger.Error("stmt.Exec failed", zap.String("task", c.cfg.Task.Name), zap.Int("times", numErr), zap.Error(err)) goto ERR } @@ -122,7 +123,7 @@ func shouldReconnect(err error) bool { if strings.Contains(err.Error(), "connection refused") || strings.Contains(err.Error(), "bad connection") { return true } - util.Logger.Infof("permanent error: %v", err.Error()) + util.Logger.Info("this is a permanent error", zap.Error(err)) return false } @@ -138,22 +139,22 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { // TODO: kafka_go and sarama commit give different error when context is cancceled. // How to unify them? if std_errors.Is(err, context.Canceled) || std_errors.Is(err, io.ErrClosedPipe) { - util.Logger.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) + util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.cfg.Task.Name)) return } - util.Logger.Fatalf("%s: committing offset failed with permanent error %+v", c.cfg.Task.Name, err) + util.Logger.Fatal("committing offset failed with permanent error %+v", zap.String("task", c.cfg.Task.Name), zap.Error(err)) } if std_errors.Is(err, context.Canceled) { - util.Logger.Infof("%s: ClickHouse.loopWrite quit due to the context has been cancelled", c.cfg.Task.Name) + util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.cfg.Task.Name)) return } - util.Logger.Errorf("%s: flush batch(try #%d) failed with error %+v", c.cfg.Task.Name, c.cfg.Clickhouse.RetryTimes-times, err) + util.Logger.Error("flush batch failed", zap.String("task", c.cfg.Task.Name), zap.Int("try", c.cfg.Clickhouse.RetryTimes-times), zap.Error(err)) statistics.FlushMsgsErrorTotal.WithLabelValues(c.cfg.Task.Name).Add(float64(batch.RealSize)) times++ if shouldReconnect(err) && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { time.Sleep(10 * time.Second) } else { - util.Logger.Fatalf("%s: ClickHouse.loopWrite failed", c.cfg.Task.Name) + util.Logger.Fatal("ClickHouse.loopWrite failed", zap.String("task", c.cfg.Task.Name)) } } } @@ -213,7 +214,7 @@ func (c *ClickHouse) initSchema() (err error) { c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.cfg.Task.TableName + " (" + strings.Join(quotedDms, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" - util.Logger.Infof("%s: Prepare sql=> %s", c.cfg.Task.Name, c.prepareSQL) + util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.cfg.Task.Name)) return nil } @@ -231,14 +232,14 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } newKeysQuota := maxDims - len(c.Dims) if newKeysQuota <= 0 { - util.Logger.Warnf("number of columns reaches upper limit %d", maxDims) + util.Logger.Warn("number of columns reaches upper limit", zap.Int("limit", maxDims), zap.Int("current", len(c.Dims))) return } var i int newKeys.Range(func(key, value interface{}) bool { i++ if i > newKeysQuota { - util.Logger.Warnf("number of columns reaches upper limit %d", maxDims) + util.Logger.Warn("number of columns reaches upper limit", zap.Int("limit", maxDims), zap.Int("current", i)) return false } strKey, _ := key.(string) @@ -275,7 +276,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } conn := pool.GetConn(0) for _, query := range queries { - util.Logger.Infof("%s: executing sql=> %s", taskCfg.Name, query) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) if _, err = conn.Exec(query); err != nil { err = errors.Wrapf(err, query) return @@ -290,7 +291,7 @@ func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { conn := pool.GetConn(0) query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, chCfg.DB, chCfg.DB, taskCfg.TableName) - util.Logger.Infof("%s: executing sql=> %s", taskCfg.Name, query) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows *sql.Rows if rows, err = conn.Query(query); err != nil { diff --git a/parser/fastjson.go b/parser/fastjson.go index 5b041dd7..2c4dc628 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -23,6 +23,7 @@ import ( "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" "github.com/valyala/fastjson" + "go.uber.org/zap" ) var _ Parser = (*FastjsonParser)(nil) @@ -224,7 +225,7 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (fou newKeys.Store(strKey, "string") foundNew = true } else { - util.Logger.Warnf("FastjsonMetric.GetNewKeys found a kv not be int/float/string, key: %s, value: %s", strKey, v.String()) + util.Logger.Warn("FastjsonMetric.GetNewKeys found a kv not be int/float/string", zap.String("key", strKey), zap.String("value", v.String())) } } }) diff --git a/pool/conn.go b/pool/conn.go index a6ffc724..98993e5b 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -30,6 +30,7 @@ import ( "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" "github.com/troian/healthcheck" + "go.uber.org/zap" ) const ( @@ -51,11 +52,11 @@ type Connection struct { func (c *Connection) ReConnect() error { sqlDB, err := sql.Open("clickhouse", c.dsn) if err != nil { - util.Logger.Info("reconnect to ", c.dsn, err.Error()) + util.Logger.Info("sql.Open failed", zap.String("dsn", c.dsn), zap.Error(err)) return err } setDBParams(sqlDB) - util.Logger.Info("reconnect success to ", c.dsn) + util.Logger.Info("sql.Open succeeded", zap.String("dsn", c.dsn)) c.DB = sqlDB return nil } @@ -110,7 +111,7 @@ func FreeConn() { defer lock.Unlock() for _, conn := range connections { if err := health.Health.RemoveReadinessCheck(conn.dsn); err != nil { - util.Logger.Error(conn.dsn+" RemoveReadinessCheck failed", err) + util.Logger.Error("health.Health.RemoveReadinessCheck failed", zap.Error(err)) } conn.DB.Close() } diff --git a/statistics/statistics.go b/statistics/statistics.go index 1a27f536..7e052796 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -25,6 +25,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" "github.com/prometheus/common/expfmt" + "go.uber.org/zap" ) var ( @@ -215,7 +216,7 @@ FOR: err := p.pusher.Push() if err != nil { err = errors.Wrapf(err, "") - util.Logger.Error("pushing metrics failed", err) + util.Logger.Error("pushing metrics failed", zap.Error(err)) p.reconnect() } case <-ctx.Done(): diff --git a/task/ring.go b/task/ring.go index 64000228..29ae21d9 100644 --- a/task/ring.go +++ b/task/ring.go @@ -1,11 +1,13 @@ package task import ( + "fmt" "sync" "time" "github.com/fagongzi/goetty" "github.com/pkg/errors" + "go.uber.org/zap" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/statistics" @@ -43,7 +45,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { ring.idleCnt = 0 ring.isIdle = false ring.ringBuf = make([]model.MsgRow, ring.ringCap) - util.Logger.Infof("%s: topic %s partition %d quit idle", taskCfg.Name, taskCfg.Topic, ring.partition) + util.Logger.Info(fmt.Sprintf("topic %s partition %d became busy", taskCfg.Topic, ring.partition), zap.String("task", taskCfg.Name)) } // assert(msgOffset < ring.ringGroundOff + ring.ringCap) if msgOffset >= ring.ringCeilingOff { @@ -52,7 +54,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { if ring.service.sharder != nil && msgRow.Row != nil { if msgRow.Shard, err = ring.service.sharder.Calc(msgRow.Row); err != nil { - util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatal("shard number calculation failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } statistics.RingMsgs.WithLabelValues(taskCfg.Name).Inc() @@ -65,7 +67,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { ring.tid.Stop() if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatal("sheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } } @@ -80,7 +82,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { taskCfg := &ring.service.cfg.Task select { case <-ring.service.ctx.Done(): - util.Logger.Errorf("%s: Ring.ForceBatchOrShard quit due to the context has been canceled", taskCfg.Name) + util.Logger.Error("Ring.ForceBatchOrShard quit due to the context has been canceled", zap.String("task", taskCfg.Name)) return default: } @@ -89,7 +91,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { defer ring.mux.Unlock() if arg != nil { newMsg, _ = arg.(*model.InputMessage) - util.Logger.Warnf("%s: Ring.ForceBatchOrShard partition %d message range [%d, %d)", taskCfg.Name, newMsg.Partition, ring.ringGroundOff, newMsg.Offset) + util.Logger.Warn(fmt.Sprintf("Ring.ForceBatchOrShard partition %d message range [%d, %d)", newMsg.Partition, ring.ringGroundOff, newMsg.Offset), zap.String("task", taskCfg.Name)) } if !ring.isIdle { if newMsg == nil { @@ -102,7 +104,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { ring.idleCnt = 0 ring.isIdle = true ring.ringBuf = nil - util.Logger.Infof("%s: topic %s partition %d enter idle", taskCfg.Name, taskCfg.Topic, ring.partition) + util.Logger.Info(fmt.Sprintf("topic %s partition %d became idle", taskCfg.Topic, ring.partition), zap.String("task", taskCfg.Name)) } } } else { @@ -126,7 +128,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { var err error if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) } } @@ -178,9 +180,9 @@ func (ring *Ring) genBatchOrShard(expNewGroundOff int64) { } if batch.RealSize > 0 { - util.Logger.Debugf("%s: going to flush a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", - taskCfg.Name, taskCfg.Topic, ring.partition, endOff-1, - batch.RealSize, gaps, parseErrs) + util.Logger.Debug(fmt.Sprintf("going to flush a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", taskCfg.Topic, ring.partition, endOff-1, + batch.RealSize, gaps, parseErrs), + zap.String("task", taskCfg.Name)) batch.BatchIdx = (endOff - 1) >> ring.batchSizeShift ring.batchSys.CreateBatchGroupSingle(batch, ring.partition, endOff-1) diff --git a/task/sharding.go b/task/sharding.go index 59f4fd42..6fb40849 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -1,6 +1,7 @@ package task import ( + "fmt" "strconv" "strings" "sync" @@ -13,6 +14,7 @@ import ( "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" + "go.uber.org/zap" ) type ShardingPolicy struct { @@ -182,9 +184,9 @@ func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOf maxBatchSize = batchSize } } - util.Logger.Debugf("%s: sharded a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", - taskCfg.Name, taskCfg.Topic, partition, endOff-1, - msgCnt, gaps, parseErrs) + util.Logger.Debug(fmt.Sprintf("sharded a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", + taskCfg.Topic, partition, endOff-1, msgCnt, gaps, parseErrs), + zap.String("task", taskCfg.Name)) if maxBatchSize >= taskCfg.BufferSize { sh.doFlush(nil) } @@ -217,7 +219,7 @@ func (sh *Sharder) doFlush(_ interface{}) { } } if msgCnt > 0 { - util.Logger.Debugf("%s: going to flush batch group for topic %v, offsets %+v, messages %d", taskCfg.Name, taskCfg.Name, sh.offsets, msgCnt) + util.Logger.Debug(fmt.Sprintf("going to flush batch group for topic %v, offsets %+v, messages %d", taskCfg.Topic, sh.offsets, msgCnt), zap.String("task", taskCfg.Name)) sh.batchSys.CreateBatchGroupMulti(batches, sh.offsets) sh.offsets = sh.offsets[:0] // ALL batches in a group shall be populated before sending any one to next stage. @@ -231,6 +233,6 @@ func (sh *Sharder) doFlush(_ interface{}) { sh.tid.Stop() if sh.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, sh.ForceFlush, nil); err != nil { err = errors.Wrap(err, "") - util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatal("scheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } diff --git a/task/task.go b/task/task.go index 9e13b4f2..d153752f 100644 --- a/task/task.go +++ b/task/task.go @@ -35,6 +35,7 @@ import ( "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" + "go.uber.org/zap" "golang.org/x/time/rate" ) @@ -110,7 +111,7 @@ func (service *Service) Init() (err error) { } if maxDims <= len(service.dims) { service.cfg.Task.DynamicSchema.Enable = false - util.Logger.Warnf("%s: disabled DynamicSchema since the number of columns reaches upper limit %d", taskCfg.Name, maxDims) + util.Logger.Warn(fmt.Sprintf("disabled DynamicSchema since the number of columns reaches upper limit %d", maxDims), zap.String("task", taskCfg.Name)) } else { for _, dim := range service.dims { service.knownKeys.Store(dim.SourceName, nil) @@ -131,13 +132,13 @@ func (service *Service) Run(ctx context.Context) { service.started = true service.parentCtx = ctx service.ctx, service.cancel = context.WithCancel(ctx) - util.Logger.Infof("%s: task started", service.cfg.Task.Name) + util.Logger.Info("task started", zap.String("task", service.cfg.Task.Name)) go service.inputer.Run(service.ctx) if service.sharder != nil { // schedule a delayed ForceFlush if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(service.cfg.Task.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { err = errors.Wrap(err, "") - util.Logger.Fatalf("%s: got error %+v", service.cfg.Task.Name, err) + util.Logger.Fatal("scheduling timer failed", zap.String("task", service.cfg.Task.Name), zap.Error(err)) } } @@ -148,7 +149,7 @@ LOOP: break LOOP case batch := <-service.batchChan: if err := service.flush(batch); err != nil { - util.Logger.Errorf("%s: got error %+v", service.cfg.Task.Name, err) + util.Logger.Fatal("service.flush failed", zap.String("task", service.cfg.Task.Name), zap.Error(err)) } } } @@ -194,7 +195,7 @@ func (service *Service) put(msg model.InputMessage) { // schedule a delayed ForceBatchOrShard if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { err = errors.Wrap(err, "") - util.Logger.Fatalf("%s: got error %+v", taskCfg.Name, err) + util.Logger.Fatal("scheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) } service.rings[msg.Partition] = ring service.Unlock() @@ -207,16 +208,16 @@ func (service *Service) put(msg model.InputMessage) { if msg.Offset < ringFilledOffset { statistics.RingMsgsOffTooSmallErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter2.Allow() { - util.Logger.Warnf("%s: got a message(topic %v, partition %d, offset %v) left to %v", - taskCfg.Name, msg.Topic, msg.Partition, msg.Offset, ringFilledOffset) + util.Logger.Warn(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) left to %v", + msg.Topic, msg.Partition, msg.Offset, ringFilledOffset), zap.String("task", taskCfg.Name)) } return } if msg.Offset >= ringGroundOff+ring.ringCap && atomic.LoadInt32(&service.cntNewKeys) == 0 { statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(service.cfg.Task.Name).Inc() if service.limiter3.Allow() { - util.Logger.Warnf("%s: got a message(topic %v, partition %d, offset %v) right to the range [%v, %v)", - taskCfg.Name, msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap) + util.Logger.Warn(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) right to the range [%v, %v)", + msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap), zap.String("task", taskCfg.Name)) } time.Sleep(1 * time.Second) ring.ForceBatchOrShard(&msg) @@ -237,8 +238,8 @@ func (service *Service) put(msg model.InputMessage) { if err != nil { statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter1.Allow() { - util.Logger.Errorf("%s: failed to parse message(topic %v, partition %d, offset %v) string(value) <<<%s>>>, got error %+v", - service.cfg.Task.Name, msg.Topic, msg.Partition, msg.Offset, string(msg.Value), err) + util.Logger.Error(fmt.Sprintf("failed to parse message(topic %v, partition %d, offset %v)", + msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", service.cfg.Task.Name), zap.Error(err)) } // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. service.pp.Put(p) @@ -263,7 +264,7 @@ func (service *Service) put(msg model.InputMessage) { service.sharder.ForceFlush(nil) } if service.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.changeSchema, nil); err != nil { - util.Logger.Fatalf("got error %+v", err) + util.Logger.Fatal("scheduling timer failed", zap.String("task", service.cfg.Task.Name), zap.Error(err)) } } } @@ -290,27 +291,28 @@ func (service *Service) changeSchema(arg interface{}) { taskCfg := &service.cfg.Task // change schema if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { - util.Logger.Fatalf("%s: clickhouse.ChangeSchema failed with error: %+v", taskCfg.Name, err) + util.Logger.Fatal("clickhouse.ChangeSchema failed", zap.String("task", taskCfg.Name), zap.Error(err)) } // restart myself service.Stop() if err = service.Init(); err != nil { - util.Logger.Fatalf("%s: init failed with error: %+v", taskCfg.Name, err) + util.Logger.Fatal("service.Init failed", zap.String("task", taskCfg.Name), zap.Error(err)) } go service.Run(service.parentCtx) } // Stop stop kafka and clickhouse client. This is blocking. func (service *Service) Stop() { - util.Logger.Infof("%s: stopping task service...", service.cfg.Task.Name) + taskCfg := &service.cfg.Task + util.Logger.Info("stopping task service...", zap.String("task", taskCfg.Name)) service.cancel() if err := service.inputer.Stop(); err != nil { panic(err) } - util.Logger.Infof("%s: stopped input", service.cfg.Task.Name) + util.Logger.Info("stopped input", zap.String("task", taskCfg.Name)) _ = service.clickhouse.Stop() - util.Logger.Infof("%s: stopped output", service.cfg.Task.Name) + util.Logger.Info("stopped output", zap.String("task", taskCfg.Name)) if service.sharder != nil { service.sharder.tid.Stop() @@ -321,12 +323,12 @@ func (service *Service) Stop() { } } service.tid.Stop() - util.Logger.Infof("%s: stopped internal timers", service.cfg.Task.Name) + util.Logger.Info("stopped internal timers", zap.String("task", taskCfg.Name)) if service.started { <-service.stopped } - util.Logger.Infof("%s: stopped", service.cfg.Task.Name) + util.Logger.Info("stopped", zap.String("task", taskCfg.Name)) } // GoID returns goroutine id diff --git a/util/common.go b/util/common.go index 43af69aa..b8dfafe2 100644 --- a/util/common.go +++ b/util/common.go @@ -42,7 +42,7 @@ var ( GlobalTimerWheel *goetty.TimeoutWheel //the global timer wheel GlobalParsingPool *WorkerPool //for all tasks' parsing, cpu intensive GlobalWritingPool *WorkerPool //the all tasks' writing ClickHouse, cpu-net balance - Logger *zap.SugaredLogger + Logger *zap.Logger ) // InitGlobalTimerWheel initialize the global timer wheel @@ -240,10 +240,12 @@ func InitLogger(logLevel string, logPaths []string) { } } + cfg := zap.NewProductionEncoderConfig() + cfg.EncodeTime = zapcore.RFC3339NanoTimeEncoder core := zapcore.NewCore( - zapcore.NewJSONEncoder(zap.NewProductionEncoderConfig()), + zapcore.NewJSONEncoder(cfg), zapcore.NewMultiWriteSyncer(syncers...), lvl, ) - Logger = zap.New(core).Sugar() + Logger = zap.New(core) } From bada22274d1b9c648aae2e9ec0ed1d1c7c58214c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 27 Apr 2021 09:01:15 +0800 Subject: [PATCH 083/404] Always PutElem even if there's parsing error. Fixed bug introduced by 104123bb. --- model/message.go | 1 + task/task.go | 5 ++--- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/model/message.go b/model/message.go index c6e60911..16e8d141 100644 --- a/model/message.go +++ b/model/message.go @@ -185,6 +185,7 @@ func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType) (row * } else { if val, err = GetValueByType(metric, dim); err != nil { PutRow(row) + row = nil return } *row = append(*row, val) diff --git a/task/task.go b/task/task.go index d153752f..d3a64e41 100644 --- a/task/task.go +++ b/task/task.go @@ -235,16 +235,15 @@ func (service *Service) put(msg model.InputMessage) { if err == nil { row, err = model.MetricToRow(metric, msg, service.dims) } + // WARNNING: Always PutElem even if there's parsing error, so that this message can be acked to Kafka and skipped writting to ClickHouse. if err != nil { statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter1.Allow() { util.Logger.Error(fmt.Sprintf("failed to parse message(topic %v, partition %d, offset %v)", msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", service.cfg.Task.Name), zap.Error(err)) } - // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. - service.pp.Put(p) - return } + // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. service.pp.Put(p) if taskCfg.DynamicSchema.Enable { From b0789a0d69bf145c41458d31445476bd06eb6298 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 27 Apr 2021 10:14:00 +0800 Subject: [PATCH 084/404] minor log change --- pool/conn.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pool/conn.go b/pool/conn.go index 98993e5b..6ac64ace 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -52,11 +52,11 @@ type Connection struct { func (c *Connection) ReConnect() error { sqlDB, err := sql.Open("clickhouse", c.dsn) if err != nil { - util.Logger.Info("sql.Open failed", zap.String("dsn", c.dsn), zap.Error(err)) + util.Logger.Debug("sql.Open failed", zap.String("dsn", c.dsn), zap.Error(err)) return err } setDBParams(sqlDB) - util.Logger.Info("sql.Open succeeded", zap.String("dsn", c.dsn)) + util.Logger.Debug("sql.Open succeeded", zap.String("dsn", c.dsn)) c.DB = sqlDB return nil } @@ -84,6 +84,7 @@ func InitConn(hosts [][]string, port int, db, username, password, dsnParams stri if dsnParams != "" { dsn += "&" + dsnParams } + util.Logger.Debug("sql.Open", zap.String("dsn", dsn)) if sqlDB, err = sql.Open("clickhouse", dsn); err != nil { err = errors.Wrapf(err, "") return From 47d479c0f22f2a0f4215fa084cc824e99c7ca63f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 27 Apr 2021 13:32:04 +0800 Subject: [PATCH 085/404] everything can be converted to string --- parser/fastjson.go | 14 +++++-- parser/gjson.go | 8 +--- parser/parser_test.go | 89 +++++++++++++++++++++---------------------- 3 files changed, 56 insertions(+), 55 deletions(-) diff --git a/parser/fastjson.go b/parser/fastjson.go index 2c4dc628..95ecb4d2 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -58,11 +58,17 @@ func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}, val = "" return } - var b []byte - if b, err = v.StringBytes(); err != nil { - return + // Everything can be converted to string. + switch v.Type() { + case fastjson.TypeString: + var b []byte + if b, err = v.StringBytes(); err != nil { + return + } + val = string(b) + default: + val = v.String() } - val = string(b) return } diff --git a/parser/gjson.go b/parser/gjson.go index 50d123a2..24530bfb 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -50,12 +50,8 @@ func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}, err val = "" return } - switch r.Type { - case gjson.String: - val = r.Str - default: - err = errors.Errorf("value doesn't contain string, it contains %s", r.Type.String()) - } + // Everything can be converted to string. + val = r.String() return } diff --git a/parser/parser_test.go b/parser/parser_test.go index 85f8a7e2..ef9ecf78 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -80,12 +80,13 @@ var jsonSchema = map[string]string{ "bool_false": "false", } -var csvSample = []byte(`1536813227,"0.11","escaped_""ws",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[aa,bb,cc]","[]"`) +var csvSample = []byte(`1536813227,"0.11","escaped_""ws",{""i"" : [1,2,3], ""f"": [1.1,2.2,3.3], ""s"": [""aa"",""bb"",""cc""], ""e"": []},2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[aa,bb,cc]","[]", "true", "false"`) var csvSchema = []string{ "its", "percent", "channel", + "mp", "date1", "time_sec_rfc3339_1", "time_sec_rfc3339_2", @@ -97,46 +98,45 @@ var csvSchema = []string{ "array_float", "array_string", "array_empty", + "bool_true", + "bool_false", } var initialize sync.Once var initErr error var names = []string{"csv", "fastjson", "gjson"} -var pools []*Pool -var parsers []Parser -var metrics []model.Metric +var pools map[string]*Pool +var parsers map[string]Parser +var metrics map[string]model.Metric func initMetrics() { var pp *Pool var parser Parser var metric model.Metric - - pp, _ = NewParserPool("csv", csvSchema, "", "") - parser = pp.Get() - if metric, initErr = parser.Parse(csvSample); initErr != nil { - return - } - metrics = append(metrics, metric) - pools = append(pools, pp) - parsers = append(parsers, parser) - - pp, _ = NewParserPool("fastjson", nil, "", "") - parser = pp.Get() - if metric, initErr = parser.Parse(jsonSample); initErr != nil { - return - } - metrics = append(metrics, metric) - pools = append(pools, pp) - parsers = append(parsers, parser) - - pp, _ = NewParserPool("gjson", nil, "", "") - parser = pp.Get() - if metric, initErr = parser.Parse(jsonSample); initErr != nil { - return + var sample []byte + pools = make(map[string]*Pool) + parsers = make(map[string]Parser) + metrics = make(map[string]model.Metric) + for _, name := range names { + switch name { + case "csv": + pp, _ = NewParserPool("csv", csvSchema, "", "") + sample = csvSample + case "fastjson": + pp, _ = NewParserPool("fastjson", nil, "", "") + sample = jsonSample + case "gjson": + pp, _ = NewParserPool("gjson", nil, "", "") + sample = jsonSample + } + parser = pp.Get() + if metric, initErr = parser.Parse(sample); initErr != nil { + return + } + pools[name] = pp + parsers[name] = parser + metrics[name] = metric } - metrics = append(metrics, metric) - pools = append(pools, pp) - parsers = append(parsers, parser) } func TestParserInt(t *testing.T) { @@ -149,7 +149,7 @@ func TestParserInt(t *testing.T) { var desc string for i := range names { name := names[i] - metric := metrics[i] + metric := metrics[name] desc = name + ` GetInt("its", false)` exp = 1536813227 @@ -189,7 +189,7 @@ func TestParserFloat(t *testing.T) { var desc string for i := range names { name := names[i] - metric := metrics[i] + metric := metrics[name] desc = name + ` GetFloat("percent", false)` exp = 0.11 @@ -229,7 +229,7 @@ func TestParserString(t *testing.T) { var desc string for i := range names { name := names[i] - metric := metrics[i] + metric := metrics[name] desc = name + ` GetString("channel", false)` exp = "escaped_\"ws" @@ -252,15 +252,14 @@ func TestParserString(t *testing.T) { require.Nil(t, err, desc) require.Nil(t, v, desc) - // Verify parsers treat type mismatch as error. + // Verify everything can be converted to string. desc = name + ` GetString("its", false)` - _, err = metric.GetString("its", false) - switch name { - case "csv": - require.Nil(t, err, desc) - default: - require.NotNil(t, err, desc) - } + exp = "1536813227" + v, err = metric.GetString("its", false) + require.Nil(t, err, desc) + require.IsType(t, exp, v, desc) + act, _ = v.(string) + require.Equal(t, exp, act, desc) } } @@ -274,7 +273,7 @@ func TestParserDateTime(t *testing.T) { var desc string for i := range names { name := names[i] - metric := metrics[i] + metric := metrics[name] desc = name + ` GetDate("date1", false)` exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) @@ -333,7 +332,7 @@ func TestParserDateTime64(t *testing.T) { var desc string for i := range names { name := names[i] - metric := metrics[i] + metric := metrics[name] desc = name + ` GetDateTime64("time_ms_rfc3339_1", false)` exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) @@ -379,7 +378,7 @@ func TestParserElasticDateTime(t *testing.T) { var desc string for i := range names { name := names[i] - metric := metrics[i] + metric := metrics[name] desc = name + ` GetElasticDateTime("time_sec_rfc3339_1", false)` exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC).Unix() @@ -413,7 +412,7 @@ func TestParserArray(t *testing.T) { var desc string for i := range names { name := names[i] - metric := metrics[i] + metric := metrics[name] desc = name + ` GetArray("array_int", "int")` expI := []int64{1, 2, 3} From ae3cd51733c4b84f25670ea2527c0f46691be421 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 27 Apr 2021 16:07:44 +0800 Subject: [PATCH 086/404] improved parser test --- parser/fastjson.go | 2 +- parser/gjson.go | 2 +- parser/parser.go | 6 +- parser/parser_test.go | 409 +++++++++++++----------------------------- task/task.go | 2 +- 5 files changed, 129 insertions(+), 292 deletions(-) diff --git a/parser/fastjson.go b/parser/fastjson.go index 95ecb4d2..07987742 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -124,7 +124,7 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} if f, err = v.Float64(); err != nil { return } - val = time.Unix(int64(f), int64(f*1e9)%1e9) + val = time.Unix(int64(f), int64(f*1e9)%1e9).In(time.UTC) case fastjson.TypeString: var b []byte if b, err = v.StringBytes(); err != nil { diff --git a/parser/gjson.go b/parser/gjson.go index 24530bfb..63c7f94e 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -106,7 +106,7 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, e } switch r.Type { case gjson.Number: - val = time.Unix(int64(r.Num), int64(r.Num*1e9)%1e9) + val = time.Unix(int64(r.Num), int64(r.Num*1e9)%1e9).In(time.UTC) case gjson.String: val, err = c.pp.ParseDateTime(key, r.Str) default: diff --git a/parser/parser.go b/parser/parser.go index ddd8d3da..56a83052 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -137,6 +137,7 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time, err error) { if lay, ok = pp.knownLayouts.Load(key); !ok { for _, layout = range Layouts { if t, err = time.ParseInLocation(layout, val, pp.timeZone); err == nil { + t = t.In(time.UTC) pp.knownLayouts.Store(key, layout) return } @@ -148,7 +149,10 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time, err error) { return } layout, _ = lay.(string) - t, _ = time.ParseInLocation(layout, val, pp.timeZone) + if t, err = time.ParseInLocation(layout, val, pp.timeZone); err != nil { + return + } + t = t.In(time.UTC) return } diff --git a/parser/parser_test.go b/parser/parser_test.go index ef9ecf78..f4d69619 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -16,6 +16,7 @@ package parser import ( "encoding/json" + "fmt" "sync" "testing" "time" @@ -37,7 +38,7 @@ var jsonSample = []byte(`{ "version":"5.8.3", "success":0, "percent":0.11, - "mp": {"i" : [1,2,3], "f": [1.1,2.2,3.3], "s": ["aa","bb","cc"], "e": []}, + "mp": {"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}, "date1": "2019-12-16", "time_sec_rfc3339_1": "2019-12-16T12:10:30Z", "time_sec_rfc3339_2": "2019-12-16T12:10:30+08:00", @@ -80,7 +81,7 @@ var jsonSchema = map[string]string{ "bool_false": "false", } -var csvSample = []byte(`1536813227,"0.11","escaped_""ws",{""i"" : [1,2,3], ""f"": [1.1,2.2,3.3], ""s"": [""aa"",""bb"",""cc""], ""e"": []},2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[aa,bb,cc]","[]", "true", "false"`) +var csvSample = []byte(`1536813227,"0.11","escaped_""ws","{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[""aa"",""bb"",""cc""]","[]","true","false"`) var csvSchema = []string{ "its", @@ -109,6 +110,29 @@ var pools map[string]*Pool var parsers map[string]Parser var metrics map[string]model.Metric +type SimpleCase struct { + Field string + Nullable bool + ExpVal interface{} + ExpErr error +} + +type ArrayCase struct { + Field string + Type string + ExpVal interface{} + ExpErr error +} + +var ErrParse = fmt.Errorf("generic parsing error") + +func Bool2Str(b bool) string { + if b { + return "true" + } + return "false" +} + func initMetrics() { var pp *Pool var parser Parser @@ -139,328 +163,137 @@ func initMetrics() { } } -func TestParserInt(t *testing.T) { +func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { + t.Helper() initialize.Do(initMetrics) require.Nil(t, initErr) - - var err error - var v interface{} - var exp, act int64 - var desc string for i := range names { name := names[i] metric := metrics[name] + for j := range testCases { + var err error + var v interface{} + desc := fmt.Sprintf(`%s %s("%s", %s)`, name, method, testCases[j].Field, Bool2Str(testCases[j].Nullable)) + switch method { + case "GetInt": + v, err = metric.GetInt(testCases[j].Field, testCases[j].Nullable) + case "GetFloat": + v, err = metric.GetFloat(testCases[j].Field, testCases[j].Nullable) + case "GetString": + v, err = metric.GetString(testCases[j].Field, testCases[j].Nullable) + case "GetDate": + v, err = metric.GetDate(testCases[j].Field, testCases[j].Nullable) + case "GetDateTime": + v, err = metric.GetDateTime(testCases[j].Field, testCases[j].Nullable) + case "GetDateTime64": + v, err = metric.GetDateTime64(testCases[j].Field, testCases[j].Nullable) + case "GetElasticDateTime": + v, err = metric.GetElasticDateTime(testCases[j].Field, testCases[j].Nullable) + default: + panic("error!") + } + if testCases[j].ExpErr == nil { + require.Nil(t, err, desc) + require.Equal(t, testCases[j].ExpVal, v, desc) + } else { + require.NotNil(t, err, desc) + } + } + } +} - desc = name + ` GetInt("its", false)` - exp = 1536813227 - v, err = metric.GetInt("its", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act, _ = v.(int64) - require.Equal(t, exp, act, desc) - - desc = name + ` GetInt("not_exist", false)` - exp = int64(0) - v, err = metric.GetInt("not_exist", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act, _ = v.(int64) - require.Equal(t, exp, act, desc) - - desc = name + ` GetInt("not_exist", true)` - v, err = metric.GetInt("not_exist", true) - require.Nil(t, err, desc) - require.Nil(t, v, desc) - - // Verify parsers treat type mismatch as error. - desc = name + ` GetInt("channel", false)` - _, err = metric.GetInt("channel", false) - require.NotNil(t, err, desc) +func TestParserInt(t *testing.T) { + testCases := []SimpleCase{ + {"its", false, int64(1536813227), nil}, + {"not_exist", false, int64(0), nil}, + {"not_exist", true, nil, nil}, + {"channel", false, nil, ErrParse}, } + doTestSimple(t, "GetInt", testCases) } func TestParserFloat(t *testing.T) { - initialize.Do(initMetrics) - require.Nil(t, initErr) - - var err error - var v interface{} - var exp, act float64 - var desc string - for i := range names { - name := names[i] - metric := metrics[name] - - desc = name + ` GetFloat("percent", false)` - exp = 0.11 - v, err = metric.GetFloat("percent", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act, _ = v.(float64) - require.Equal(t, exp, act, desc) - - desc = name + ` GetFloat("not_exist", false)` - exp = 0.0 - v, err = metric.GetFloat("not_exist", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act, _ = v.(float64) - require.Equal(t, exp, act, desc) - - desc = name + ` GetFloat("not_exist", true)` - v, err = metric.GetFloat("not_exist", true) - require.Nil(t, err, desc) - require.Nil(t, v, desc) - - // Verify parsers treat type mismatch as error. - desc = name + ` GetFloat("channel", false)` - _, err = metric.GetFloat("channel", false) - require.NotNil(t, err, desc) + testCases := []SimpleCase{ + {"percent", false, 0.11, nil}, + {"not_exist", false, 0.0, nil}, + {"not_exist", true, nil, nil}, + {"channel", false, nil, ErrParse}, } + doTestSimple(t, "GetFloat", testCases) } func TestParserString(t *testing.T) { - initialize.Do(initMetrics) - require.Nil(t, initErr) - - var err error - var v interface{} - var exp, act string - var desc string - for i := range names { - name := names[i] - metric := metrics[name] - - desc = name + ` GetString("channel", false)` - exp = "escaped_\"ws" - v, err = metric.GetString("channel", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act, _ = v.(string) - require.Equal(t, exp, act, desc) - - desc = name + ` GetString("not_exist", false)` - exp = "" - v, err = metric.GetString("not_exist", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act, _ = v.(string) - require.Equal(t, exp, act, desc) - - desc = name + ` GetString("not_exist", true)` - v, err = metric.GetString("not_exist", true) - require.Nil(t, err, desc) - require.Nil(t, v, desc) - - // Verify everything can be converted to string. - desc = name + ` GetString("its", false)` - exp = "1536813227" - v, err = metric.GetString("its", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act, _ = v.(string) - require.Equal(t, exp, act, desc) + testCases := []SimpleCase{ + {"channel", false, "escaped_\"ws", nil}, + {"not_exist", false, "", nil}, + {"not_exist", true, nil, nil}, + {"its", false, "1536813227", nil}, + {"array_int", false, "[1,2,3]", nil}, + {"array_string", false, `["aa","bb","cc"]`, nil}, + {"mp", false, `{"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}`, nil}, } + doTestSimple(t, "GetString", testCases) } func TestParserDateTime(t *testing.T) { - initialize.Do(initMetrics) - require.Nil(t, initErr) - - var err error - var v interface{} - var exp, act time.Time - var desc string - for i := range names { - name := names[i] - metric := metrics[name] - - desc = name + ` GetDate("date1", false)` - exp = time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local) - v, err = metric.GetDate("date1", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act = v.(time.Time).In(time.Local) - require.Equal(t, exp, act, desc) - - desc = name + ` GetDateTime("time_sec_rfc3339_1", false)` - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC) - v, err = metric.GetDateTime("time_sec_rfc3339_1", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act = v.(time.Time).In(time.UTC) - require.Equal(t, exp, act, desc) - - desc = name + ` GetDateTime("time_sec_rfc3339_2", false)` - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC) - v, err = metric.GetDateTime("time_sec_rfc3339_2", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act = v.(time.Time).In(time.UTC) - require.Equal(t, exp, act, desc) - - desc = name + ` GetDateTime("time_sec_clickhouse_1", false)` - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC) - v, err = metric.GetDateTime("time_sec_clickhouse_1", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act = v.(time.Time).In(time.UTC) - require.Equal(t, exp, act, desc) - - desc = name + ` GetDateTime("not_exist", false)` - exp = Epoch - v, err = metric.GetDateTime("not_exist", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act = v.(time.Time).In(time.UTC) - require.Equal(t, exp, act, desc) - - // Verify parsers treat type mismatch as error. - desc = name + ` GetDateTime("array_int", false)` - _, err = metric.GetDateTime("array_int", false) - require.NotNil(t, err, desc) + testCases := []SimpleCase{ + {"date1", false, time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local).In(time.UTC), nil}, + {"time_sec_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC), nil}, + {"time_sec_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC), nil}, + {"time_sec_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC), nil}, + {"not_exist", false, Epoch, nil}, + {"array_int", false, nil, ErrParse}, } + doTestSimple(t, "GetDateTime", testCases) } func TestParserDateTime64(t *testing.T) { - initialize.Do(initMetrics) - require.Nil(t, initErr) - - var err error - var v interface{} - var exp, act time.Time - var desc string - for i := range names { - name := names[i] - metric := metrics[name] - - desc = name + ` GetDateTime64("time_ms_rfc3339_1", false)` - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC) - v, err = metric.GetDateTime64("time_ms_rfc3339_1", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act = v.(time.Time).In(time.UTC) - require.Equal(t, exp, act, desc) - - desc = name + ` GetDateTime64("time_ms_rfc3339_2", false)` - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC) - v, err = metric.GetDateTime64("time_ms_rfc3339_2", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act = v.(time.Time).In(time.UTC) - require.Equal(t, exp, act, desc) - - desc = name + ` GetDateTime64("time_ms_clickhouse_1", false)` - exp = time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC) - v, err = metric.GetDateTime64("time_ms_clickhouse_1", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act = v.(time.Time).In(time.UTC) - require.Equal(t, exp, act, desc) - - desc = name + ` GetDateTime64("not_exist", false)` - exp = Epoch - v, err = metric.GetDateTime64("not_exist", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act = v.(time.Time).In(time.UTC) - require.Equal(t, exp, act, desc) + testCases := []SimpleCase{ + {"time_ms_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC), nil}, + {"time_ms_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC), nil}, + {"time_ms_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC), nil}, + {"not_exist", false, Epoch, nil}, + {"array_int", false, nil, ErrParse}, } + doTestSimple(t, "GetDateTime64", testCases) } func TestParserElasticDateTime(t *testing.T) { - initialize.Do(initMetrics) - require.Nil(t, initErr) - - var err error - var v interface{} - var exp, act int64 - var desc string - for i := range names { - name := names[i] - metric := metrics[name] - - desc = name + ` GetElasticDateTime("time_sec_rfc3339_1", false)` - exp = time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC).Unix() - v, err = metric.GetElasticDateTime("time_sec_rfc3339_1", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act, _ = v.(int64) - require.Equal(t, exp, act, desc) - - desc = name + ` GetElasticDateTime("not_exist", false)` - exp = 0 - v, err = metric.GetElasticDateTime("not_exist", false) - require.Nil(t, err, desc) - require.IsType(t, exp, v, desc) - act, _ = v.(int64) - require.Equal(t, exp, act, desc) - - desc = name + ` GetElasticDateTime("not_exist", true)` - v, err = metric.GetElasticDateTime("not_exist", true) - require.Nil(t, err, desc) - require.Nil(t, v, desc) + testCases := []SimpleCase{ + {"time_ms_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC).Unix(), nil}, + {"not_exist", false, int64(0), nil}, + {"not_exist", true, nil, nil}, } + doTestSimple(t, "GetElasticDateTime", testCases) } func TestParserArray(t *testing.T) { initialize.Do(initMetrics) require.Nil(t, initErr) + testCases := []ArrayCase{ + {"array_int", "int", []int64{1, 2, 3}, nil}, + {"array_float", "float", []float64{1.1, 2.2, 3.3}, nil}, + {"array_string", "string", []string{"aa", "bb", "cc"}, nil}, + {"array_empty", "int", []int64{}, nil}, + {"array_empty", "float", []float64{}, nil}, + } - var err error - var v interface{} - var desc string for i := range names { name := names[i] metric := metrics[name] - - desc = name + ` GetArray("array_int", "int")` - expI := []int64{1, 2, 3} - v, err = metric.GetArray("array_int", "int") - require.Nil(t, err, desc) - require.IsType(t, expI, v, desc) - actI, _ := v.([]int64) - require.Equal(t, expI, actI, desc) - - desc = name + ` GetArray("array_float", "float")` - expF := []float64{1.1, 2.2, 3.3} - v, err = metric.GetArray("array_float", "float") - require.Nil(t, err, desc) - require.IsType(t, expF, v, desc) - actF, _ := v.([]float64) - require.Equal(t, expF, actF, desc) - - desc = name + ` GetArray("array_string", "string")` - expS := []string{"aa", "bb", "cc"} - v, err = metric.GetArray("array_string", "string") - require.Nil(t, err, desc) - require.IsType(t, expS, v, desc) - actS, _ := v.([]string) - require.Equal(t, expS, actS, desc) - - desc = name + ` GetArray("array_empty", "int")` - expIE := []int64{} - v, err = metric.GetArray("array_empty", "int") - require.Nil(t, err, desc) - require.IsType(t, expIE, v, desc) - actIE, _ := v.([]int64) - require.Equal(t, expIE, actIE, desc) - - desc = name + ` GetArray("array_empty", "float")` - expFE := []float64{} - v, err = metric.GetArray("array_empty", "float") - require.Nil(t, err, desc) - require.IsType(t, expFE, v, desc) - actFE, _ := v.([]float64) - require.Equal(t, expFE, actFE, desc) - - desc = name + ` GetArray("array_empty", "string")` - expSE := []string{} - v, err = metric.GetArray("array_empty", "string") - require.Nil(t, err, desc) - require.IsType(t, expSE, v, desc) - actSE, _ := v.([]string) - require.Equal(t, expSE, actSE, desc) + for j := range testCases { + var err error + var v interface{} + desc := fmt.Sprintf(`%s GetArray("%s", "%s")`, name, testCases[j].Field, testCases[j].Type) + v, err = metric.GetArray(testCases[j].Field, testCases[j].Type) + if testCases[j].ExpErr == nil { + require.Nil(t, err, desc) + require.Equal(t, testCases[j].ExpVal, v, desc) + } else { + require.NotNil(t, err, desc) + require.Nil(t, v, desc) + } + } } } diff --git a/task/task.go b/task/task.go index d3a64e41..7ed2bcf0 100644 --- a/task/task.go +++ b/task/task.go @@ -235,7 +235,7 @@ func (service *Service) put(msg model.InputMessage) { if err == nil { row, err = model.MetricToRow(metric, msg, service.dims) } - // WARNNING: Always PutElem even if there's parsing error, so that this message can be acked to Kafka and skipped writting to ClickHouse. + // WARNNING: Always PutElem even if there's parsing error, so that this message can be acked to Kafka and skipped writing to ClickHouse. if err != nil { statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter1.Allow() { From 264ce1ff2f81ff815769b1f91cd915d6f66ffb49 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 27 Apr 2021 18:26:47 +0800 Subject: [PATCH 087/404] fix add column --- output/clickhouse.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 413172c1..0d1abdde 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -255,7 +255,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { err = errors.Errorf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) return false } - query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS %s %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) + query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) queries = append(queries, query) return true }) From cd6deec65cc037b577599a3cddc5e9ffbe2a5fc4 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 29 Apr 2021 08:46:02 +0800 Subject: [PATCH 088/404] fix ParsingPoolBacklog --- task/task.go | 1 + 1 file changed, 1 insertion(+) diff --git a/task/task.go b/task/task.go index 7ed2bcf0..8d94830a 100644 --- a/task/task.go +++ b/task/task.go @@ -230,6 +230,7 @@ func (service *Service) put(msg model.InputMessage) { var row *model.Row var foundNewKeys bool var metric model.Metric + defer statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Dec() p := service.pp.Get() metric, err = p.Parse(msg.Value) if err == nil { From c064790ba3839087c07eb1576ee584f377ee0ebf Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 29 Apr 2021 16:20:28 +0800 Subject: [PATCH 089/404] log stacktrace for errors --- util/common.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/common.go b/util/common.go index b8dfafe2..95f61925 100644 --- a/util/common.go +++ b/util/common.go @@ -247,5 +247,5 @@ func InitLogger(logLevel string, logPaths []string) { zapcore.NewMultiWriteSyncer(syncers...), lvl, ) - Logger = zap.New(core) + Logger = zap.New(core, zap.AddStacktrace(zap.ErrorLevel)) } From db03ed21671e5644fa50cf6934d3a8f56b24342e Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 29 Apr 2021 17:49:12 +0800 Subject: [PATCH 090/404] ignore all parse errors of date and return Epoch --- parser/csv.go | 2 +- parser/fastjson.go | 8 ++++++-- parser/gjson.go | 4 ++-- parser/parser.go | 10 +++++++--- parser/parser_test.go | 4 ++-- 5 files changed, 18 insertions(+), 10 deletions(-) diff --git a/parser/csv.go b/parser/csv.go index c6fb37d3..5512ea9d 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -115,7 +115,7 @@ func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}, err val = Epoch return } - val, err = c.pp.ParseDateTime(key, c.values[idx]) + val = c.pp.ParseDateTime(key, c.values[idx]) return } diff --git a/parser/fastjson.go b/parser/fastjson.go index 07987742..aaff5b63 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -122,17 +122,21 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} case fastjson.TypeNumber: var f float64 if f, err = v.Float64(); err != nil { + val = Epoch + err = nil return } val = time.Unix(int64(f), int64(f*1e9)%1e9).In(time.UTC) case fastjson.TypeString: var b []byte if b, err = v.StringBytes(); err != nil { + val = Epoch + err = nil return } - val, err = c.pp.ParseDateTime(key, string(b)) + val = c.pp.ParseDateTime(key, string(b)) default: - err = errors.Errorf("value doesn't contain number nor string, it contains %s", v.Type().String()) + val = Epoch } return } diff --git a/parser/gjson.go b/parser/gjson.go index 63c7f94e..1824d2a2 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -108,9 +108,9 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, e case gjson.Number: val = time.Unix(int64(r.Num), int64(r.Num*1e9)%1e9).In(time.UTC) case gjson.String: - val, err = c.pp.ParseDateTime(key, r.Str) + val = c.pp.ParseDateTime(key, r.Str) default: - err = errors.Errorf("value doesn't contain string, it contains %s", r.Type.String()) + val = Epoch } return } diff --git a/parser/parser.go b/parser/parser.go index 56a83052..0e229a25 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -16,7 +16,6 @@ package parser import ( "encoding/json" - "strconv" "sync" "time" @@ -130,7 +129,11 @@ func (pp *Pool) Put(p Parser) { pp.pool.Put(p) } -func (pp *Pool) ParseDateTime(key string, val string) (t time.Time, err error) { +// Detect date format for each key at the first message. +// Return time in UTC. +// Return Epoch if parsing fail. +func (pp *Pool) ParseDateTime(key string, val string) (t time.Time) { + var err error var layout string var lay interface{} var ok bool @@ -145,11 +148,12 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time, err error) { pp.knownLayouts.Store(key, nil) } if lay == nil { - err = errors.Errorf("cannot parse time %s at field %s", strconv.Quote(val), key) + t = Epoch return } layout, _ = lay.(string) if t, err = time.ParseInLocation(layout, val, pp.timeZone); err != nil { + t = Epoch return } t = t.In(time.UTC) diff --git a/parser/parser_test.go b/parser/parser_test.go index f4d69619..680182c7 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -242,7 +242,7 @@ func TestParserDateTime(t *testing.T) { {"time_sec_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC), nil}, {"time_sec_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC), nil}, {"not_exist", false, Epoch, nil}, - {"array_int", false, nil, ErrParse}, + {"array_int", false, Epoch, nil}, } doTestSimple(t, "GetDateTime", testCases) } @@ -253,7 +253,7 @@ func TestParserDateTime64(t *testing.T) { {"time_ms_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC), nil}, {"time_ms_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC), nil}, {"not_exist", false, Epoch, nil}, - {"array_int", false, nil, ErrParse}, + {"array_int", false, Epoch, nil}, } doTestSimple(t, "GetDateTime64", testCases) } From 1423c29c6fddd24190b5ded119dc3269f7594bd9 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 2 May 2021 23:19:36 +0800 Subject: [PATCH 091/404] tiny goreleaser change --- .goreleaser.yml | 4 ++-- Dockerfile_goreleaser | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.goreleaser.yml b/.goreleaser.yml index d300df17..1bc26690 100644 --- a/.goreleaser.yml +++ b/.goreleaser.yml @@ -13,14 +13,14 @@ builds: goos: - linux main: cmd/clickhouse_sinker/main.go - binary: clickhouse_sinker + binary: dist/clickhouse_sinker - id: nacos_publish_config env: - CGO_ENABLED=0 goos: - linux main: cmd/nacos_publish_config/main.go - binary: nacos_publish_config + binary: dist/nacos_publish_config dockers: - binaries: diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index d566960a..1c5f72d7 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -1,8 +1,8 @@ FROM alpine:latest RUN apk --no-cache add ca-certificates tzdata RUN echo "UTC" > /etc/timezone -ADD ./clickhouse_sinker /usr/local/bin/clickhouse_sinker -ADD ./nacos_publish_config /usr/local/bin/nacos_publish_config +ADD dist/clickhouse_sinker /usr/local/bin/clickhouse_sinker +ADD dist/nacos_publish_config /usr/local/bin/nacos_publish_config # clickhouse_sinker gets config from local directory "/etc/clickhouse_sinker" by default. # Customize behavior with following env variables: From 0b9cd3f4a743cca967b76d9a8cb1d7ae26e5566e Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 7 May 2021 14:52:37 +0800 Subject: [PATCH 092/404] documented parse error behavior --- docs/dev/introduction.md | 12 +++++ model/message.go | 9 +--- model/metric.go | 16 +++--- model/value.go | 27 ++++------ parser/csv.go | 62 +++++++++-------------- parser/fastjson.go | 66 +++++++----------------- parser/gjson.go | 56 +++++++-------------- parser/parser.go | 16 ++++-- parser/parser_test.go | 105 ++++++++++++++++----------------------- task/task.go | 3 +- 10 files changed, 151 insertions(+), 221 deletions(-) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index a9f41236..5b518e43 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -33,6 +33,18 @@ Refers to [design](./design.md) for how it works. - [x] Nullable(T), where T is one of above basic types - [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) +Note: +- A message is ignored if it's invalid json, or CSV value doesn't match with the format. This is counted by `ParseMsgsErrorTotal`. +- If a message field value doesn't match with the type `T` declared in ClickHouse, the default value of `T` (see the following table) is filled. + +| ClickHouse data type | default value | +| -------------------- |:-------------:| +| Int8, Int16, ... | 0 | +| Float32, Float64 | 0.0 | +| String, ... | "" | +| Date, DateTime, ... | EPOCH | +| Nullable(T) | NULL | +| Array(T) | [] | ## Configuration diff --git a/model/message.go b/model/message.go index 16e8d141..cd10eaf7 100644 --- a/model/message.go +++ b/model/message.go @@ -170,9 +170,8 @@ func PutRow(r *Row) { rowPool.Put(r) } -func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType) (row *Row, err error) { +func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType) (row *Row) { row = GetRow() - var val interface{} for _, dim := range dims { if strings.HasPrefix(dim.Name, "__kafka") { if strings.HasSuffix(dim.Name, "_topic") { @@ -183,11 +182,7 @@ func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType) (row * *row = append(*row, msg.Offset) } } else { - if val, err = GetValueByType(metric, dim); err != nil { - PutRow(row) - row = nil - return - } + val := GetValueByType(metric, dim) *row = append(*row, val) } } diff --git a/model/metric.go b/model/metric.go index 42d5c232..71b13707 100644 --- a/model/metric.go +++ b/model/metric.go @@ -21,14 +21,14 @@ import ( // Metric interface for metric collection type Metric interface { - GetString(key string, nullable bool) (val interface{}, err error) - GetFloat(key string, nullable bool) (val interface{}, err error) - GetInt(key string, nullable bool) (val interface{}, err error) - GetDate(key string, nullable bool) (val interface{}, err error) - GetDateTime(key string, nullable bool) (val interface{}, err error) - GetDateTime64(key string, nullable bool) (val interface{}, err error) - GetElasticDateTime(key string, nullable bool) (val interface{}, err error) - GetArray(key string, t string) (val interface{}, err error) + GetString(key string, nullable bool) (val interface{}) + GetFloat(key string, nullable bool) (val interface{}) + GetInt(key string, nullable bool) (val interface{}) + GetDate(key string, nullable bool) (val interface{}) + GetDateTime(key string, nullable bool) (val interface{}) + GetDateTime64(key string, nullable bool) (val interface{}) + GetElasticDateTime(key string, nullable bool) (val interface{}) + GetArray(key string, t string) (val interface{}) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool } diff --git a/model/value.go b/model/value.go index 3658b60d..4605adfb 100644 --- a/model/value.go +++ b/model/value.go @@ -16,8 +16,6 @@ package model import ( "strings" - - "github.com/pkg/errors" ) const ( @@ -43,35 +41,32 @@ var ( ) // There are only three cases for the value type of metric, (float64, string, map [string] interface {}) -func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}, err error) { +func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { name := cwt.SourceName switch cwt.Type { case Int: - val, err = metric.GetInt(name, cwt.Nullable) + val = metric.GetInt(name, cwt.Nullable) case Float: - val, err = metric.GetFloat(name, cwt.Nullable) + val = metric.GetFloat(name, cwt.Nullable) case String: - val, err = metric.GetString(name, cwt.Nullable) + val = metric.GetString(name, cwt.Nullable) case IntArray: - val, err = metric.GetArray(name, "int") + val = metric.GetArray(name, "int") case FloatArray: - val, err = metric.GetArray(name, "float") + val = metric.GetArray(name, "float") case StringArray: - val, err = metric.GetArray(name, "string") + val = metric.GetArray(name, "string") case Date: - val, err = metric.GetDate(name, cwt.Nullable) + val = metric.GetDate(name, cwt.Nullable) case DateTime: - val, err = metric.GetDateTime(name, cwt.Nullable) + val = metric.GetDateTime(name, cwt.Nullable) case DateTime64: - val, err = metric.GetDateTime64(name, cwt.Nullable) + val = metric.GetDateTime64(name, cwt.Nullable) case ElasticDateTime: - val, err = metric.GetElasticDateTime(name, cwt.Nullable) + val = metric.GetElasticDateTime(name, cwt.Nullable) default: panic("BUG: reached switch default condition") } - if err != nil { - err = errors.Errorf(name + " " + err.Error()) - } return } diff --git a/parser/csv.go b/parser/csv.go index 5512ea9d..7048a028 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -17,7 +17,6 @@ package parser import ( "bytes" "encoding/csv" - "io" "strconv" "strings" "sync" @@ -46,6 +45,10 @@ func (p *CsvParser) Parse(bs []byte) (metric model.Metric, err error) { err = errors.Wrap(err, "") return } + if len(value) != len(p.pp.csvFormat) { + err = errors.Errorf("csv value doesn't match the format") + return + } metric = &CsvMetric{p.pp, value} return } @@ -57,7 +60,7 @@ type CsvMetric struct { } // GetString get the value as string -func (c *CsvMetric) GetString(key string, nullable bool) (val interface{}, err error) { +func (c *CsvMetric) GetString(key string, nullable bool) (val interface{}) { var idx int var ok bool if idx, ok = c.pp.csvFormat[key]; !ok { @@ -72,7 +75,7 @@ func (c *CsvMetric) GetString(key string, nullable bool) (val interface{}, err e } // GetFloat returns the value as float -func (c *CsvMetric) GetFloat(key string, nullable bool) (val interface{}, err error) { +func (c *CsvMetric) GetFloat(key string, nullable bool) (val interface{}) { var idx int var ok bool if idx, ok = c.pp.csvFormat[key]; !ok { @@ -82,12 +85,12 @@ func (c *CsvMetric) GetFloat(key string, nullable bool) (val interface{}, err er val = float64(0.0) return } - val, err = strconv.ParseFloat(c.values[idx], 64) + val, _ = strconv.ParseFloat(c.values[idx], 64) return } // GetInt returns int -func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}, err error) { +func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}) { var idx int var ok bool if idx, ok = c.pp.csvFormat[key]; !ok { @@ -97,15 +100,15 @@ func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}, err erro val = int64(0) return } - val, err = strconv.ParseInt(c.values[idx], 10, 64) + val, _ = strconv.ParseInt(c.values[idx], 10, 64) return } -func (c *CsvMetric) GetDate(key string, nullable bool) (val interface{}, err error) { +func (c *CsvMetric) GetDate(key string, nullable bool) (val interface{}) { return c.GetDateTime(key, nullable) } -func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}, err error) { +func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}) { var idx int var ok bool if idx, ok = c.pp.csvFormat[key]; !ok { @@ -119,15 +122,12 @@ func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}, err return } -func (c *CsvMetric) GetDateTime64(key string, nullable bool) (val interface{}, err error) { +func (c *CsvMetric) GetDateTime64(key string, nullable bool) (val interface{}) { return c.GetDateTime(key, nullable) } -func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) (val interface{}, err error) { - var t interface{} - if t, err = c.GetDateTime(key, nullable); err != nil { - return - } +func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) (val interface{}) { + t := c.GetDateTime(key, nullable) if t != nil { val = t.(time.Time).Unix() } @@ -135,34 +135,20 @@ func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) (val interface } // GetArray parse an CSV encoded array -func (c *CsvMetric) GetArray(key string, t string) (val interface{}, err error) { - var s interface{} +func (c *CsvMetric) GetArray(key string, t string) (val interface{}) { + var err error var array []string var r *csv.Reader - if s, err = c.GetString(key, false); err != nil { - return - } + s := c.GetString(key, false) str, _ := s.(string) strLen := len(str) if str == "" || str[0] != '[' || str[strLen-1] != ']' { - err = errors.Errorf("GetArray %s got unexpected value %s", key, str) + val = makeArray(t) return } r = csv.NewReader(strings.NewReader(str[1 : strLen-1])) if array, err = r.Read(); err != nil { - if errors.Is(err, io.EOF) { - err = nil - switch t { - case "int": - val = []int64{} - case "float": - val = []float64{} - case "string": - val = []string{} - default: - panic("LOGIC ERROR: not supported array type " + t) - } - } + val = makeArray(t) return } switch t { @@ -170,20 +156,18 @@ func (c *CsvMetric) GetArray(key string, t string) (val interface{}, err error) results := make([]int64, 0, len(array)) var v int64 for _, e := range array { - if v, err = strconv.ParseInt(e, 10, 64); err != nil { - return + if v, err = strconv.ParseInt(e, 10, 64); err == nil { + results = append(results, v) } - results = append(results, v) } val = results case "float": results := make([]float64, 0, len(array)) var v float64 for _, e := range array { - if v, err = strconv.ParseFloat(e, 64); err != nil { - return + if v, err = strconv.ParseFloat(e, 64); err == nil { + results = append(results, v) } - results = append(results, v) } val = results case "string": diff --git a/parser/fastjson.go b/parser/fastjson.go index aaff5b63..4e50cda0 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -49,7 +49,7 @@ type FastjsonMetric struct { value *fastjson.Value } -func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}, err error) { +func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}) { v := c.value.Get(key) if v == nil || v.Type() == fastjson.TypeNull { if nullable { @@ -58,13 +58,9 @@ func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}, val = "" return } - // Everything can be converted to string. switch v.Type() { case fastjson.TypeString: - var b []byte - if b, err = v.StringBytes(); err != nil { - return - } + b, _ := v.StringBytes() val = string(b) default: val = v.String() @@ -72,7 +68,7 @@ func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}, return } -func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}, err error) { +func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { v := c.value.Get(key) if v == nil || v.Type() == fastjson.TypeNull { if nullable { @@ -81,11 +77,11 @@ func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}, e val = float64(0.0) return } - val, err = v.Float64() + val, _ = v.Float64() return } -func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}, err error) { +func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}) { v := c.value.Get(key) if v == nil || v.Type() == fastjson.TypeNull { if nullable { @@ -100,16 +96,16 @@ func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}, err case fastjson.TypeFalse: val = int64(0) default: - val, err = v.Int64() + val, _ = v.Int64() } return } -func (c *FastjsonMetric) GetDate(key string, nullable bool) (val interface{}, err error) { +func (c *FastjsonMetric) GetDate(key string, nullable bool) (val interface{}) { return c.GetDateTime(key, nullable) } -func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, err error) { +func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { v := c.value.Get(key) if v == nil || v.Type() == fastjson.TypeNull { if nullable { @@ -118,12 +114,12 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} val = Epoch return } + var err error switch v.Type() { case fastjson.TypeNumber: var f float64 if f, err = v.Float64(); err != nil { val = Epoch - err = nil return } val = time.Unix(int64(f), int64(f*1e9)%1e9).In(time.UTC) @@ -131,7 +127,6 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} var b []byte if b, err = v.StringBytes(); err != nil { val = Epoch - err = nil return } val = c.pp.ParseDateTime(key, string(b)) @@ -141,40 +136,25 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} return } -func (c *FastjsonMetric) GetDateTime64(key string, nullable bool) (val interface{}, err error) { +func (c *FastjsonMetric) GetDateTime64(key string, nullable bool) (val interface{}) { return c.GetDateTime(key, nullable) } -func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}, err error) { - var t interface{} - if t, err = c.GetDateTime(key, nullable); err != nil { - return - } +func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}) { + t := c.GetDateTime(key, nullable) if t != nil { val = t.(time.Time).Unix() } return } -func (c *FastjsonMetric) GetArray(key string, t string) (val interface{}, err error) { +func (c *FastjsonMetric) GetArray(key string, t string) (val interface{}) { v := c.value.Get(key) - if v == nil || v.Type() == fastjson.TypeNull { - switch t { - case "int": - val = []int64{} - case "float": - val = []float64{} - case "string": - val = []string{} - default: - panic("LOGIC ERROR: not supported array type " + t) - } - return - } - var array []*fastjson.Value - if array, err = v.Array(); err != nil { + if v == nil || v.Type() != fastjson.TypeArray { + val = makeArray(t) return } + array, _ := v.Array() switch t { case "int": results := make([]int64, 0, len(array)) @@ -183,9 +163,7 @@ func (c *FastjsonMetric) GetArray(key string, t string) (val interface{}, err er if e.Type() == fastjson.TypeTrue { v = 1 } else { - if v, err = e.Int64(); err != nil { - return - } + v, _ = e.Int64() } results = append(results, v) } @@ -193,20 +171,14 @@ func (c *FastjsonMetric) GetArray(key string, t string) (val interface{}, err er case "float": results := make([]float64, 0, len(array)) for _, e := range array { - var v float64 - if v, err = e.Float64(); err != nil { - return - } + v, _ := e.Float64() results = append(results, v) } val = results case "string": results := make([]string, 0, len(array)) for _, e := range array { - var v []byte - if v, err = e.StringBytes(); err != nil { - return - } + v, _ := e.StringBytes() results = append(results, string(v)) } val = results diff --git a/parser/gjson.go b/parser/gjson.go index 1824d2a2..58a676de 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -19,7 +19,6 @@ import ( "sync" "time" - "github.com/pkg/errors" "github.com/tidwall/gjson" "github.com/housepower/clickhouse_sinker/model" @@ -41,7 +40,7 @@ type GjsonMetric struct { raw string } -func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}, err error) { +func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) if !r.Exists() || r.Type == gjson.Null { if nullable { @@ -50,12 +49,11 @@ func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}, err val = "" return } - // Everything can be converted to string. val = r.String() return } -func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}, err error) { +func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) if !r.Exists() || r.Type == gjson.Null { if nullable { @@ -68,12 +66,12 @@ func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}, err case gjson.Number: val = r.Num default: - err = errors.Errorf("value doesn't contain number, it contains %s", r.Type.String()) + val = float64(0.0) } return } -func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}, err error) { +func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) if !r.Exists() || r.Type == gjson.Null { if nullable { @@ -86,16 +84,16 @@ func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}, err er case gjson.Number: val = int64(r.Num) default: - err = errors.Errorf("value doesn't contain number, it contains %s", r.Type.String()) + val = int64(0) } return } -func (c *GjsonMetric) GetDate(key string, nullable bool) (val interface{}, err error) { +func (c *GjsonMetric) GetDate(key string, nullable bool) (val interface{}) { return c.GetDateTime(key, nullable) } -func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, err error) { +func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) if !r.Exists() || r.Type == gjson.Null { if nullable { @@ -115,46 +113,30 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}, e return } -func (c *GjsonMetric) GetDateTime64(key string, nullable bool) (val interface{}, err error) { +func (c *GjsonMetric) GetDateTime64(key string, nullable bool) (val interface{}) { return c.GetDateTime(key, nullable) } -func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}, err error) { - var t interface{} - if t, err = c.GetDateTime(key, nullable); err != nil { - return - } +func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}) { + t := c.GetDateTime(key, nullable) if t != nil { val = t.(time.Time).Unix() } return } -func (c *GjsonMetric) GetArray(key string, t string) (val interface{}, err error) { +func (c *GjsonMetric) GetArray(key string, t string) (val interface{}) { r := gjson.Get(c.raw, key) - if !r.Exists() || r.Type == gjson.Null { - switch t { - case "int": - val = []int64{} - case "float": - val = []float64{} - case "string": - val = []string{} - default: - panic("LOGIC ERROR: not supported array type " + t) - } - return - } - if r.Type != gjson.JSON { - err = errors.Errorf("value doesn't contain json, it contains %s", r.Type.String()) + if !r.Exists() || r.Type != gjson.JSON { + val = makeArray(t) return } array := r.Array() switch t { - case "string": - results := make([]string, 0, len(array)) + case "int": + results := make([]int64, 0, len(array)) for _, s := range array { - results = append(results, s.String()) + results = append(results, s.Int()) } val = results case "float": @@ -164,10 +146,10 @@ func (c *GjsonMetric) GetArray(key string, t string) (val interface{}, err error results = append(results, s.Float()) } val = results - case "int": - results := make([]int64, 0, len(array)) + case "string": + results := make([]string, 0, len(array)) for _, s := range array { - results = append(results, s.Int()) + results = append(results, s.String()) } val = results default: diff --git a/parser/parser.go b/parser/parser.go index 0e229a25..c0b22d02 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -15,7 +15,6 @@ limitations under the License. package parser import ( - "encoding/json" "sync" "time" @@ -160,7 +159,16 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time) { return } -func GetJSONShortStr(v interface{}) string { - bs, _ := json.Marshal(v) - return string(bs) +func makeArray(typ string) (val interface{}) { + switch typ { + case "int": + val = []int64{} + case "float": + val = []float64{} + case "string": + val = []string{} + default: + panic("LOGIC ERROR: not supported array type " + typ) + } + return } diff --git a/parser/parser_test.go b/parser/parser_test.go index 680182c7..82c172b1 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -114,18 +114,14 @@ type SimpleCase struct { Field string Nullable bool ExpVal interface{} - ExpErr error } type ArrayCase struct { Field string Type string ExpVal interface{} - ExpErr error } -var ErrParse = fmt.Errorf("generic parsing error") - func Bool2Str(b bool) string { if b { return "true" @@ -171,98 +167,92 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { name := names[i] metric := metrics[name] for j := range testCases { - var err error var v interface{} desc := fmt.Sprintf(`%s %s("%s", %s)`, name, method, testCases[j].Field, Bool2Str(testCases[j].Nullable)) switch method { case "GetInt": - v, err = metric.GetInt(testCases[j].Field, testCases[j].Nullable) + v = metric.GetInt(testCases[j].Field, testCases[j].Nullable) case "GetFloat": - v, err = metric.GetFloat(testCases[j].Field, testCases[j].Nullable) + v = metric.GetFloat(testCases[j].Field, testCases[j].Nullable) case "GetString": - v, err = metric.GetString(testCases[j].Field, testCases[j].Nullable) + v = metric.GetString(testCases[j].Field, testCases[j].Nullable) case "GetDate": - v, err = metric.GetDate(testCases[j].Field, testCases[j].Nullable) + v = metric.GetDate(testCases[j].Field, testCases[j].Nullable) case "GetDateTime": - v, err = metric.GetDateTime(testCases[j].Field, testCases[j].Nullable) + v = metric.GetDateTime(testCases[j].Field, testCases[j].Nullable) case "GetDateTime64": - v, err = metric.GetDateTime64(testCases[j].Field, testCases[j].Nullable) + v = metric.GetDateTime64(testCases[j].Field, testCases[j].Nullable) case "GetElasticDateTime": - v, err = metric.GetElasticDateTime(testCases[j].Field, testCases[j].Nullable) + v = metric.GetElasticDateTime(testCases[j].Field, testCases[j].Nullable) default: panic("error!") } - if testCases[j].ExpErr == nil { - require.Nil(t, err, desc) - require.Equal(t, testCases[j].ExpVal, v, desc) - } else { - require.NotNil(t, err, desc) - } + require.Equal(t, testCases[j].ExpVal, v, desc) } } } func TestParserInt(t *testing.T) { testCases := []SimpleCase{ - {"its", false, int64(1536813227), nil}, - {"not_exist", false, int64(0), nil}, - {"not_exist", true, nil, nil}, - {"channel", false, nil, ErrParse}, + {"its", false, int64(1536813227)}, + {"not_exist", false, int64(0)}, + {"not_exist", true, nil}, + {"channel", false, int64(0)}, } doTestSimple(t, "GetInt", testCases) } func TestParserFloat(t *testing.T) { testCases := []SimpleCase{ - {"percent", false, 0.11, nil}, - {"not_exist", false, 0.0, nil}, - {"not_exist", true, nil, nil}, - {"channel", false, nil, ErrParse}, + {"percent", false, 0.11}, + {"not_exist", false, 0.0}, + {"not_exist", true, nil}, + {"channel", false, 0.0}, } doTestSimple(t, "GetFloat", testCases) } func TestParserString(t *testing.T) { testCases := []SimpleCase{ - {"channel", false, "escaped_\"ws", nil}, - {"not_exist", false, "", nil}, - {"not_exist", true, nil, nil}, - {"its", false, "1536813227", nil}, - {"array_int", false, "[1,2,3]", nil}, - {"array_string", false, `["aa","bb","cc"]`, nil}, - {"mp", false, `{"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}`, nil}, + {"channel", false, "escaped_\"ws"}, + {"not_exist", false, ""}, + {"not_exist", true, nil}, + {"its", false, "1536813227"}, + {"array_int", false, "[1,2,3]"}, + {"array_string", false, `["aa","bb","cc"]`}, + {"mp", false, `{"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}`}, } doTestSimple(t, "GetString", testCases) } func TestParserDateTime(t *testing.T) { testCases := []SimpleCase{ - {"date1", false, time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local).In(time.UTC), nil}, - {"time_sec_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC), nil}, - {"time_sec_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC), nil}, - {"time_sec_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC), nil}, - {"not_exist", false, Epoch, nil}, - {"array_int", false, Epoch, nil}, + {"date1", false, time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local).In(time.UTC)}, + {"time_sec_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC)}, + {"time_sec_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC)}, + {"time_sec_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC)}, + {"not_exist", false, Epoch}, + {"array_int", false, Epoch}, } doTestSimple(t, "GetDateTime", testCases) } func TestParserDateTime64(t *testing.T) { testCases := []SimpleCase{ - {"time_ms_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC), nil}, - {"time_ms_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC), nil}, - {"time_ms_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC), nil}, - {"not_exist", false, Epoch, nil}, - {"array_int", false, Epoch, nil}, + {"time_ms_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC)}, + {"time_ms_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC)}, + {"time_ms_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC)}, + {"not_exist", false, Epoch}, + {"array_int", false, Epoch}, } doTestSimple(t, "GetDateTime64", testCases) } func TestParserElasticDateTime(t *testing.T) { testCases := []SimpleCase{ - {"time_ms_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC).Unix(), nil}, - {"not_exist", false, int64(0), nil}, - {"not_exist", true, nil, nil}, + {"time_ms_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC).Unix()}, + {"not_exist", false, int64(0)}, + {"not_exist", true, nil}, } doTestSimple(t, "GetElasticDateTime", testCases) } @@ -271,28 +261,21 @@ func TestParserArray(t *testing.T) { initialize.Do(initMetrics) require.Nil(t, initErr) testCases := []ArrayCase{ - {"array_int", "int", []int64{1, 2, 3}, nil}, - {"array_float", "float", []float64{1.1, 2.2, 3.3}, nil}, - {"array_string", "string", []string{"aa", "bb", "cc"}, nil}, - {"array_empty", "int", []int64{}, nil}, - {"array_empty", "float", []float64{}, nil}, + {"array_int", "int", []int64{1, 2, 3}}, + {"array_float", "float", []float64{1.1, 2.2, 3.3}}, + {"array_string", "string", []string{"aa", "bb", "cc"}}, + {"array_empty", "int", []int64{}}, + {"array_empty", "float", []float64{}}, } for i := range names { name := names[i] metric := metrics[name] for j := range testCases { - var err error var v interface{} desc := fmt.Sprintf(`%s GetArray("%s", "%s")`, name, testCases[j].Field, testCases[j].Type) - v, err = metric.GetArray(testCases[j].Field, testCases[j].Type) - if testCases[j].ExpErr == nil { - require.Nil(t, err, desc) - require.Equal(t, testCases[j].ExpVal, v, desc) - } else { - require.NotNil(t, err, desc) - require.Nil(t, v, desc) - } + v = metric.GetArray(testCases[j].Field, testCases[j].Type) + require.Equal(t, testCases[j].ExpVal, v, desc) } } } diff --git a/task/task.go b/task/task.go index 8d94830a..4fe5e525 100644 --- a/task/task.go +++ b/task/task.go @@ -80,7 +80,6 @@ func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *pa } // Init initializes the kafak and clickhouse task associated with this service - func (service *Service) Init() (err error) { if err = service.clickhouse.Init(); err != nil { return @@ -234,7 +233,7 @@ func (service *Service) put(msg model.InputMessage) { p := service.pp.Get() metric, err = p.Parse(msg.Value) if err == nil { - row, err = model.MetricToRow(metric, msg, service.dims) + row = model.MetricToRow(metric, msg, service.dims) } // WARNNING: Always PutElem even if there's parsing error, so that this message can be acked to Kafka and skipped writing to ClickHouse. if err != nil { From 09021c20e51a088f8ba106eaf80ebae05ec9fd8d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 7 May 2021 18:15:31 +0800 Subject: [PATCH 093/404] added Array(DateTime) and dynamically detect array --- go.test.sh | 6 +-- model/metric.go | 8 ++-- model/value.go | 88 ++++++++++++++++++++----------------------- output/clickhouse.go | 22 ++++++++--- parser/csv.go | 33 ++++++++-------- parser/fastjson.go | 73 +++++++++++++++++++++++------------ parser/gjson.go | 43 ++++++++++----------- parser/parser.go | 37 ++++++++++++------ parser/parser_test.go | 41 ++++++++++---------- task/task.go | 17 +-------- util/workerpool.go | 4 +- 11 files changed, 200 insertions(+), 172 deletions(-) diff --git a/go.test.sh b/go.test.sh index 574247a9..aca05161 100755 --- a/go.test.sh +++ b/go.test.sh @@ -31,10 +31,10 @@ for i in `seq 10001 30000`;do echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey1\" : $i }" done >> a.json for i in `seq 30001 50000`;do - echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey2\" : $i.123 }" + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey2\" : $i.123, \"newkey3\" : \"name$i\", \"newkey4\" : \"${now}\" }" done >> a.json for i in `seq 50001 70000`;do - echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey3\" : \"name$i\" }" + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey5\" : [$i], \"newkey6\" : [$i.123], \"newkey7\" : [\"name$i\"], \"newkey8\" : [\"${now}\"] }" done >> a.json for i in `seq 70001 100000`;do echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i }" @@ -62,7 +62,7 @@ echo "Got test_auto_schema count => $count" schema=`curl "localhost:8123" -d 'DESC test_dynamic_schema' 2>/dev/null | grep newkey | sort | tr -d '\t' | tr '\n' ','` echo "Got test_dynamic_schema schema => $schema" -[ $schema = "newkey1Nullable(Int64),newkey2Nullable(Float64),newkey3Nullable(String)," ] || exit 1 +[ $schema = "newkey1Nullable(Int64),newkey2Nullable(Float64),newkey3Nullable(String),newkey4Nullable(DateTime),newkey5Array(Int64),newkey6Array(Float64),newkey7Array(String),newkey8Array(DateTime)," ] || exit 1 count=`curl "localhost:8123" -d 'SELECT count() FROM test_dynamic_schema'` echo "Got test_dynamic_schema count => $count" [ $count -eq 100000 ] || exit 1 diff --git a/model/metric.go b/model/metric.go index 71b13707..0db781d4 100644 --- a/model/metric.go +++ b/model/metric.go @@ -21,14 +21,12 @@ import ( // Metric interface for metric collection type Metric interface { - GetString(key string, nullable bool) (val interface{}) - GetFloat(key string, nullable bool) (val interface{}) GetInt(key string, nullable bool) (val interface{}) - GetDate(key string, nullable bool) (val interface{}) + GetFloat(key string, nullable bool) (val interface{}) + GetString(key string, nullable bool) (val interface{}) GetDateTime(key string, nullable bool) (val interface{}) - GetDateTime64(key string, nullable bool) (val interface{}) GetElasticDateTime(key string, nullable bool) (val interface{}) - GetArray(key string, t string) (val interface{}) + GetArray(key string, t int) (val interface{}) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool } diff --git a/model/value.go b/model/value.go index 4605adfb..e4b4888d 100644 --- a/model/value.go +++ b/model/value.go @@ -15,20 +15,23 @@ limitations under the License. package model import ( + "fmt" "strings" + + "github.com/housepower/clickhouse_sinker/util" ) const ( - Int = iota + TypeUnknown = iota + Int Float String + DateTime + ElasticDateTime IntArray FloatArray StringArray - Date - DateTime - DateTime64 - ElasticDateTime + DateTimeArray ) type TypeInfo struct { @@ -50,22 +53,20 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { val = metric.GetFloat(name, cwt.Nullable) case String: val = metric.GetString(name, cwt.Nullable) - case IntArray: - val = metric.GetArray(name, "int") - case FloatArray: - val = metric.GetArray(name, "float") - case StringArray: - val = metric.GetArray(name, "string") - case Date: - val = metric.GetDate(name, cwt.Nullable) case DateTime: val = metric.GetDateTime(name, cwt.Nullable) - case DateTime64: - val = metric.GetDateTime64(name, cwt.Nullable) case ElasticDateTime: val = metric.GetElasticDateTime(name, cwt.Nullable) + case IntArray: + val = metric.GetArray(name, Int) + case FloatArray: + val = metric.GetArray(name, Float) + case StringArray: + val = metric.GetArray(name, String) + case DateTimeArray: + val = metric.GetArray(name, DateTime) default: - panic("BUG: reached switch default condition") + util.Logger.Fatal("LOGIC ERROR: reached switch default condition") } return } @@ -81,54 +82,45 @@ func WhichType(typ string) (dataType int, nullable bool) { typ = typ[len("Nullable(") : len(typ)-1] } if strings.HasPrefix(typ, "DateTime64") { - dataType = DateTime64 - } else if strings.HasPrefix(typ, "DateTime") { dataType = DateTime } else { - panic("unsupported ClickHouse data type " + typ) + util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported ClickHouse data type %v", typ)) } typeInfo[typ] = TypeInfo{Type: dataType, Nullable: nullable} return } func init() { + primTypeInfo := make(map[string]TypeInfo) typeInfo = make(map[string]TypeInfo) for _, t := range []string{"UInt8", "UInt16", "UInt32", "UInt64", "Int8", "Int16", "Int32", "Int64"} { - typeInfo[t] = TypeInfo{Type: Int, Nullable: false} - } - for _, t := range []string{"Nullable(UInt8)", "Nullable(UInt16)", "Nullable(UInt32)", "Nullable(UInt64)", - "Nullable(Int8)", "Nullable(Int16)", "Nullable(Int32)", "Nullable(Int64)"} { - typeInfo[t] = TypeInfo{Type: Int, Nullable: true} + primTypeInfo[t] = TypeInfo{Type: Int, Nullable: false} } for _, t := range []string{"Float32", "Float64"} { - typeInfo[t] = TypeInfo{Type: Float, Nullable: false} - } - for _, t := range []string{"Nullable(Float32)", "Nullable(Float64)"} { - typeInfo[t] = TypeInfo{Type: Float, Nullable: true} + primTypeInfo[t] = TypeInfo{Type: Float, Nullable: false} } for _, t := range []string{"String", "FixedString"} { - typeInfo[t] = TypeInfo{Type: String, Nullable: false} - } - for _, t := range []string{"Nullable(String)", "Nullable(FixedString)"} { - typeInfo[t] = TypeInfo{Type: String, Nullable: true} - } - for _, t := range []string{"Array(UInt8)", "Array(UInt16)", "Array(UInt32)", - "Array(UInt64)", "Array(Int8)", "Array(Int16)", "Array(Int32)", "Array(Int64)"} { - typeInfo[t] = TypeInfo{Type: IntArray, Nullable: false} + primTypeInfo[t] = TypeInfo{Type: String, Nullable: false} } - for _, t := range []string{"Array(Float32)", "Array(Float64)"} { - typeInfo[t] = TypeInfo{Type: FloatArray, Nullable: false} + for _, t := range []string{"Date", "DateTime", "DateTime64"} { + primTypeInfo[t] = TypeInfo{Type: DateTime, Nullable: false} } - for _, t := range []string{"Array(String)", "Array(FixedString)"} { - typeInfo[t] = TypeInfo{Type: StringArray, Nullable: false} + primTypeInfo["ElasticDateTime"] = TypeInfo{Type: ElasticDateTime, Nullable: false} + for k, v := range primTypeInfo { + typeInfo[k] = v + nullK := fmt.Sprintf("Nullable(%s)", k) + typeInfo[nullK] = TypeInfo{Type: v.Type, Nullable: true} + arrK := fmt.Sprintf("Array(%s)", k) + switch v.Type { + case Int: + typeInfo[arrK] = TypeInfo{Type: IntArray, Nullable: false} + case Float: + typeInfo[arrK] = TypeInfo{Type: FloatArray, Nullable: false} + case String: + typeInfo[arrK] = TypeInfo{Type: StringArray, Nullable: false} + case DateTime: + typeInfo[arrK] = TypeInfo{Type: DateTimeArray, Nullable: false} + } } - typeInfo["Date"] = TypeInfo{Type: Date, Nullable: false} - typeInfo["Nullable(Date)"] = TypeInfo{Type: Date, Nullable: true} - typeInfo["DateTime"] = TypeInfo{Type: DateTime, Nullable: false} - typeInfo["Nullable(DateTime)"] = TypeInfo{Type: DateTime, Nullable: true} - typeInfo["DateTime64"] = TypeInfo{Type: DateTime64, Nullable: false} - typeInfo["Nullable(DateTime64)"] = TypeInfo{Type: DateTime64, Nullable: true} - typeInfo["ElasticDateTime"] = TypeInfo{Type: ElasticDateTime, Nullable: false} - typeInfo["Nullable(ElasticDateTime)"] = TypeInfo{Type: ElasticDateTime, Nullable: true} } diff --git a/output/clickhouse.go b/output/clickhouse.go index 0d1abdde..5420af25 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -23,6 +23,7 @@ import ( "io" "math" "regexp" + "sort" "strings" "sync" "time" @@ -243,14 +244,24 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return false } strKey, _ := key.(string) - strVal, _ := value.(string) - switch strVal { - case "int": + var strVal string + switch value.(int) { + case model.Int: strVal = "Nullable(Int64)" - case "float": + case model.Float: strVal = "Nullable(Float64)" - case "string": + case model.String: strVal = "Nullable(String)" + case model.DateTime: + strVal = "Nullable(DateTime)" + case model.IntArray: + strVal = "Array(Int64)" + case model.FloatArray: + strVal = "Array(Float64)" + case model.StringArray: + strVal = "Array(String)" + case model.DateTimeArray: + strVal = "Array(DateTime)" default: err = errors.Errorf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) return false @@ -262,6 +273,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { if err != nil { return } + sort.Strings(queries) if chCfg.Cluster != "" { var distTbls []string if distTbls, err = c.getDistTbls(); err != nil { diff --git a/parser/csv.go b/parser/csv.go index 7048a028..e92b4d1a 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -17,12 +17,14 @@ package parser import ( "bytes" "encoding/csv" + "fmt" "strconv" "strings" "sync" "time" "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" ) @@ -104,10 +106,6 @@ func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}) { return } -func (c *CsvMetric) GetDate(key string, nullable bool) (val interface{}) { - return c.GetDateTime(key, nullable) -} - func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}) { var idx int var ok bool @@ -122,10 +120,6 @@ func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}) { return } -func (c *CsvMetric) GetDateTime64(key string, nullable bool) (val interface{}) { - return c.GetDateTime(key, nullable) -} - func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) (val interface{}) { t := c.GetDateTime(key, nullable) if t != nil { @@ -135,7 +129,7 @@ func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) (val interface } // GetArray parse an CSV encoded array -func (c *CsvMetric) GetArray(key string, t string) (val interface{}) { +func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { var err error var array []string var r *csv.Reader @@ -143,16 +137,16 @@ func (c *CsvMetric) GetArray(key string, t string) (val interface{}) { str, _ := s.(string) strLen := len(str) if str == "" || str[0] != '[' || str[strLen-1] != ']' { - val = makeArray(t) + val = makeArray(typ) return } r = csv.NewReader(strings.NewReader(str[1 : strLen-1])) if array, err = r.Read(); err != nil { - val = makeArray(t) + val = makeArray(typ) return } - switch t { - case "int": + switch typ { + case model.Int: results := make([]int64, 0, len(array)) var v int64 for _, e := range array { @@ -161,7 +155,7 @@ func (c *CsvMetric) GetArray(key string, t string) (val interface{}) { } } val = results - case "float": + case model.Float: results := make([]float64, 0, len(array)) var v float64 for _, e := range array { @@ -170,10 +164,17 @@ func (c *CsvMetric) GetArray(key string, t string) (val interface{}) { } } val = results - case "string": + case model.String: val = array + case model.DateTime: + results := make([]time.Time, 0, len(array)) + for _, e := range array { + v := c.pp.ParseDateTime(key, e) + results = append(results, v) + } + val = results default: - panic("LOGIC ERROR: not supported array type " + t) + util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) } return } diff --git a/parser/fastjson.go b/parser/fastjson.go index 4e50cda0..07d00846 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -16,6 +16,7 @@ limitations under the License. package parser import ( + "fmt" "sync" "time" @@ -101,10 +102,6 @@ func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}) { return } -func (c *FastjsonMetric) GetDate(key string, nullable bool) (val interface{}) { - return c.GetDateTime(key, nullable) -} - func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { v := c.value.Get(key) if v == nil || v.Type() == fastjson.TypeNull { @@ -136,10 +133,6 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} return } -func (c *FastjsonMetric) GetDateTime64(key string, nullable bool) (val interface{}) { - return c.GetDateTime(key, nullable) -} - func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}) { t := c.GetDateTime(key, nullable) if t != nil { @@ -148,15 +141,15 @@ func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) (val inte return } -func (c *FastjsonMetric) GetArray(key string, t string) (val interface{}) { +func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { v := c.value.Get(key) if v == nil || v.Type() != fastjson.TypeArray { - val = makeArray(t) + val = makeArray(typ) return } array, _ := v.Array() - switch t { - case "int": + switch typ { + case model.Int: results := make([]int64, 0, len(array)) for _, e := range array { var v int64 @@ -168,22 +161,30 @@ func (c *FastjsonMetric) GetArray(key string, t string) (val interface{}) { results = append(results, v) } val = results - case "float": + case model.Float: results := make([]float64, 0, len(array)) for _, e := range array { v, _ := e.Float64() results = append(results, v) } val = results - case "string": + case model.String: results := make([]string, 0, len(array)) for _, e := range array { v, _ := e.StringBytes() results = append(results, string(v)) } val = results + case model.DateTime: + results := make([]time.Time, 0, len(array)) + for _, e := range array { + v, _ := e.StringBytes() + t := c.pp.ParseDateTime(key, string(v)) + results = append(results, t) + } + val = results default: - panic("LOGIC ERROR: not supported array type " + t) + util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) } return } @@ -197,19 +198,43 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (fou obj.Visit(func(key []byte, v *fastjson.Value) { strKey := string(key) if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { - if _, err = v.Int64(); err == nil { - newKeys.Store(strKey, "int") - foundNew = true - } else if _, err = v.Float64(); err == nil { - newKeys.Store(strKey, "float") - foundNew = true - } else if _, err = v.StringBytes(); err == nil { - newKeys.Store(strKey, "string") + if typ := fjDetectType(v); typ != model.TypeUnknown { + newKeys.Store(strKey, typ) foundNew = true } else { - util.Logger.Warn("FastjsonMetric.GetNewKeys found a kv not be int/float/string", zap.String("key", strKey), zap.String("value", v.String())) + util.Logger.Warn("FastjsonMetric.GetNewKeys failed to detect field type", zap.String("key", strKey), zap.String("value", v.String())) } } }) return } + +func fjDetectType(v *fastjson.Value) (typ int) { + if vt := v.Type(); vt == fastjson.TypeNull { + } else if vt == fastjson.TypeTrue || vt == fastjson.TypeFalse { + typ = model.Int + } else if _, err := v.Int64(); err == nil { + typ = model.Int + } else if _, err := v.Float64(); err == nil { + typ = model.Float + } else if val, err := v.StringBytes(); err == nil { + if _, layout := parseInLocation(string(val), time.Local); layout != "" { + typ = model.DateTime + } else { + typ = model.String + } + } else if arr, err := v.Array(); err == nil && len(arr) > 0 { + typ2 := fjDetectType(arr[0]) + switch typ2 { + case model.Int: + typ = model.IntArray + case model.Float: + typ = model.FloatArray + case model.String: + typ = model.StringArray + case model.DateTime: + typ = model.DateTimeArray + } + } + return +} diff --git a/parser/gjson.go b/parser/gjson.go index 58a676de..5028d723 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -16,12 +16,14 @@ limitations under the License. package parser import ( + "fmt" "sync" "time" "github.com/tidwall/gjson" "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" ) var _ Parser = (*GjsonParser)(nil) @@ -89,10 +91,6 @@ func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}) { return } -func (c *GjsonMetric) GetDate(key string, nullable bool) (val interface{}) { - return c.GetDateTime(key, nullable) -} - func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) if !r.Exists() || r.Type == gjson.Null { @@ -113,10 +111,6 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { return } -func (c *GjsonMetric) GetDateTime64(key string, nullable bool) (val interface{}) { - return c.GetDateTime(key, nullable) -} - func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}) { t := c.GetDateTime(key, nullable) if t != nil { @@ -125,35 +119,42 @@ func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) (val interfa return } -func (c *GjsonMetric) GetArray(key string, t string) (val interface{}) { +func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { r := gjson.Get(c.raw, key) if !r.Exists() || r.Type != gjson.JSON { - val = makeArray(t) + val = makeArray(typ) return } array := r.Array() - switch t { - case "int": + switch typ { + case model.Int: results := make([]int64, 0, len(array)) - for _, s := range array { - results = append(results, s.Int()) + for _, e := range array { + results = append(results, e.Int()) } val = results - case "float": + case model.Float: results := make([]float64, 0, len(array)) - for _, s := range array { - results = append(results, s.Float()) + for _, e := range array { + results = append(results, e.Float()) } val = results - case "string": + case model.String: results := make([]string, 0, len(array)) - for _, s := range array { - results = append(results, s.String()) + for _, e := range array { + results = append(results, e.String()) + } + val = results + case model.DateTime: + results := make([]time.Time, 0, len(array)) + for _, e := range array { + t := c.pp.ParseDateTime(key, e.String()) + results = append(results, t) } val = results default: - panic("LOGIC ERROR: not supported array type " + t) + util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) } return } diff --git a/parser/parser.go b/parser/parser.go index c0b22d02..015bfb59 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -15,10 +15,12 @@ limitations under the License. package parser import ( + "fmt" "sync" "time" "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" ) @@ -137,12 +139,10 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time) { var lay interface{} var ok bool if lay, ok = pp.knownLayouts.Load(key); !ok { - for _, layout = range Layouts { - if t, err = time.ParseInLocation(layout, val, pp.timeZone); err == nil { - t = t.In(time.UTC) - pp.knownLayouts.Store(key, layout) - return - } + t, layout = parseInLocation(val, pp.timeZone) + if layout != "" { + pp.knownLayouts.Store(key, layout) + return } pp.knownLayouts.Store(key, nil) } @@ -159,16 +159,31 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time) { return } -func makeArray(typ string) (val interface{}) { +func makeArray(typ int) (val interface{}) { switch typ { - case "int": + case model.Int: val = []int64{} - case "float": + case model.Float: val = []float64{} - case "string": + case model.String: val = []string{} + case model.DateTime: + val = []time.Time{} default: - panic("LOGIC ERROR: not supported array type " + typ) + util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) + } + return +} + +func parseInLocation(val string, loc *time.Location) (t time.Time, layout string) { + var err error + var lay string + for _, lay = range Layouts { + if t, err = time.ParseInLocation(lay, val, loc); err == nil { + t = t.In(time.UTC) + layout = lay + return + } } return } diff --git a/parser/parser_test.go b/parser/parser_test.go index 82c172b1..f03447fc 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -49,6 +49,7 @@ var jsonSample = []byte(`{ "array_int": [1,2,3], "array_float": [1.1,2.2,3.3], "array_string": ["aa","bb","cc"], + "array_date": ["2000-01-01","2000-01-02","2000-01-03"], "array_empty": [], "bool_true": true, "bool_false": false @@ -76,12 +77,13 @@ var jsonSchema = map[string]string{ "array_int": "array", "array_float": "array", "array_string": "array", + "array_date": "array", "array_empty": "array", "bool_true": "true", "bool_false": "false", } -var csvSample = []byte(`1536813227,"0.11","escaped_""ws","{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[""aa"",""bb"",""cc""]","[]","true","false"`) +var csvSample = []byte(`1536813227,"0.11","escaped_""ws","{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[""aa"",""bb"",""cc""]","[""2000-01-01"",""2000-01-02"",""2000-01-03""]","[]","true","false"`) var csvSchema = []string{ "its", @@ -98,6 +100,7 @@ var csvSchema = []string{ "array_int", "array_float", "array_string", + "array_date", "array_empty", "bool_true", "bool_false", @@ -118,7 +121,7 @@ type SimpleCase struct { type ArrayCase struct { Field string - Type string + Type int ExpVal interface{} } @@ -176,12 +179,8 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { v = metric.GetFloat(testCases[j].Field, testCases[j].Nullable) case "GetString": v = metric.GetString(testCases[j].Field, testCases[j].Nullable) - case "GetDate": - v = metric.GetDate(testCases[j].Field, testCases[j].Nullable) case "GetDateTime": v = metric.GetDateTime(testCases[j].Field, testCases[j].Nullable) - case "GetDateTime64": - v = metric.GetDateTime64(testCases[j].Field, testCases[j].Nullable) case "GetElasticDateTime": v = metric.GetElasticDateTime(testCases[j].Field, testCases[j].Nullable) default: @@ -231,21 +230,13 @@ func TestParserDateTime(t *testing.T) { {"time_sec_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC)}, {"time_sec_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC)}, {"time_sec_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC)}, - {"not_exist", false, Epoch}, - {"array_int", false, Epoch}, - } - doTestSimple(t, "GetDateTime", testCases) -} - -func TestParserDateTime64(t *testing.T) { - testCases := []SimpleCase{ {"time_ms_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC)}, {"time_ms_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC)}, {"time_ms_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC)}, {"not_exist", false, Epoch}, {"array_int", false, Epoch}, } - doTestSimple(t, "GetDateTime64", testCases) + doTestSimple(t, "GetDateTime", testCases) } func TestParserElasticDateTime(t *testing.T) { @@ -260,12 +251,20 @@ func TestParserElasticDateTime(t *testing.T) { func TestParserArray(t *testing.T) { initialize.Do(initMetrics) require.Nil(t, initErr) + var ts []time.Time + for _, e := range []string{"2000-01-01", "2000-01-02", "2000-01-03"} { + t, _ := parseInLocation(e, time.Local) + ts = append(ts, t) + } testCases := []ArrayCase{ - {"array_int", "int", []int64{1, 2, 3}}, - {"array_float", "float", []float64{1.1, 2.2, 3.3}}, - {"array_string", "string", []string{"aa", "bb", "cc"}}, - {"array_empty", "int", []int64{}}, - {"array_empty", "float", []float64{}}, + {"array_int", model.Int, []int64{1, 2, 3}}, + {"array_float", model.Float, []float64{1.1, 2.2, 3.3}}, + {"array_string", model.String, []string{"aa", "bb", "cc"}}, + {"array_date", model.DateTime, ts}, + {"array_empty", model.Int, []int64{}}, + {"array_empty", model.Float, []float64{}}, + {"array_empty", model.String, []string{}}, + {"array_empty", model.DateTime, []time.Time{}}, } for i := range names { @@ -273,7 +272,7 @@ func TestParserArray(t *testing.T) { metric := metrics[name] for j := range testCases { var v interface{} - desc := fmt.Sprintf(`%s GetArray("%s", "%s")`, name, testCases[j].Field, testCases[j].Type) + desc := fmt.Sprintf(`%s GetArray("%s", %d)`, name, testCases[j].Field, testCases[j].Type) v = metric.GetArray(testCases[j].Field, testCases[j].Type) require.Equal(t, testCases[j].ExpVal, v, desc) } diff --git a/task/task.go b/task/task.go index 4fe5e525..7ae7093f 100644 --- a/task/task.go +++ b/task/task.go @@ -19,9 +19,6 @@ import ( "context" "fmt" "math" - "runtime" - "strconv" - "strings" "sync" "sync/atomic" "time" @@ -306,7 +303,7 @@ func (service *Service) Stop() { util.Logger.Info("stopping task service...", zap.String("task", taskCfg.Name)) service.cancel() if err := service.inputer.Stop(); err != nil { - panic(err) + util.Logger.Fatal("service.inputer.Stop failed", zap.Error(err)) } util.Logger.Info("stopped input", zap.String("task", taskCfg.Name)) @@ -329,15 +326,3 @@ func (service *Service) Stop() { } util.Logger.Info("stopped", zap.String("task", taskCfg.Name)) } - -// GoID returns goroutine id -func GoID() int { - var buf [64]byte - n := runtime.Stack(buf[:], false) - idField := strings.Fields(strings.TrimPrefix(string(buf[:n]), "goroutine "))[0] - id, err := strconv.Atoi(idField) - if err != nil { - panic(fmt.Sprintf("cannot get goroutine id: %v", err)) - } - return id -} diff --git a/util/workerpool.go b/util/workerpool.go index 58b060a6..13d03135 100644 --- a/util/workerpool.go +++ b/util/workerpool.go @@ -28,10 +28,10 @@ type WorkerPool struct { // New creates and starts a pool of worker goroutines. func NewWorkerPool(maxWorkers int, queueSize int) *WorkerPool { if maxWorkers <= 0 { - panic("WorkerNum must be greater than zero") + Logger.Fatal("WorkerNum must be greater than zero") } if queueSize <= 0 { - panic("queueSize must be greater than zero") + Logger.Fatal("queueSize must be greater than zero") } w := &WorkerPool{ From ef11a8b438ac5f23411fdcfe04b0b2ce37a7faac Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 7 May 2021 23:00:33 +0800 Subject: [PATCH 094/404] fix goreleaser --- .goreleaser.yml | 4 ++-- Dockerfile | 4 ++-- Dockerfile_goreleaser | 4 ++-- Makefile | 8 ++++---- go.sum | 27 +++++++++++++++++++++++++++ go.test.sh | 18 +++++++++--------- 6 files changed, 46 insertions(+), 19 deletions(-) diff --git a/.goreleaser.yml b/.goreleaser.yml index 1bc26690..d300df17 100644 --- a/.goreleaser.yml +++ b/.goreleaser.yml @@ -13,14 +13,14 @@ builds: goos: - linux main: cmd/clickhouse_sinker/main.go - binary: dist/clickhouse_sinker + binary: clickhouse_sinker - id: nacos_publish_config env: - CGO_ENABLED=0 goos: - linux main: cmd/nacos_publish_config/main.go - binary: dist/nacos_publish_config + binary: nacos_publish_config dockers: - binaries: diff --git a/Dockerfile b/Dockerfile index 4309edbe..24c80d52 100644 --- a/Dockerfile +++ b/Dockerfile @@ -8,8 +8,8 @@ RUN make build FROM alpine:latest RUN apk --no-cache add ca-certificates tzdata RUN echo "UTC" > /etc/timezone -COPY --from=builder /app/dist/clickhouse_sinker /usr/local/bin/clickhouse_sinker -COPY --from=builder /app/dist/nacos_publish_config /usr/local/bin/nacos_publish_config +COPY --from=builder /app/clickhouse_sinker /usr/local/bin/clickhouse_sinker +COPY --from=builder /app/nacos_publish_config /usr/local/bin/nacos_publish_config # clickhouse_sinker gets config from local directory "/etc/clickhouse_sinker" by default. # Customize behavior with following env variables: diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index 1c5f72d7..9b8a7730 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -1,8 +1,8 @@ FROM alpine:latest RUN apk --no-cache add ca-certificates tzdata RUN echo "UTC" > /etc/timezone -ADD dist/clickhouse_sinker /usr/local/bin/clickhouse_sinker -ADD dist/nacos_publish_config /usr/local/bin/nacos_publish_config +ADD clickhouse_sinker /usr/local/bin/clickhouse_sinker +ADD nacos_publish_config /usr/local/bin/nacos_publish_config # clickhouse_sinker gets config from local directory "/etc/clickhouse_sinker" by default. # Customize behavior with following env variables: diff --git a/Makefile b/Makefile index 6f6e8df9..bcf852e6 100644 --- a/Makefile +++ b/Makefile @@ -9,11 +9,11 @@ GOBUILD := $(GO) build $(BUILD_FLAG) pre: go mod tidy build: pre - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o dist/clickhouse_sinker cmd/clickhouse_sinker/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o dist/nacos_publish_config cmd/nacos_publish_config/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o clickhouse_sinker cmd/clickhouse_sinker/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o nacos_publish_config cmd/nacos_publish_config/main.go debug: pre - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o dist/clickhouse_sinker cmd/clickhouse_sinker/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o dist/nacos_publish_config cmd/nacos_publish_config/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o clickhouse_sinker cmd/clickhouse_sinker/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o nacos_publish_config cmd/nacos_publish_config/main.go unittest: pre go test -v ./... benchtest: pre diff --git a/go.sum b/go.sum index 0e80d535..406990ec 100644 --- a/go.sum +++ b/go.sum @@ -11,8 +11,10 @@ github.com/Shopify/sarama v1.27.2/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4 h1:Hs82Z41s6SdL1CELW+XaDYmOH4hkBN4/N9og/AsOv7E= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 h1:zOVTBdCKFd9JbCKz9/nt+FovbjPFmb7mUnp8nH9fQBA= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= @@ -30,6 +32,7 @@ github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+ github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= +github.com/creack/pty v1.1.9 h1:uDmaGzcdjhF4i/plgjmEsriH11Y0o7RKapEf/LDaM3w= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -51,12 +54,18 @@ github.com/frankban/quicktest v1.10.2/go.mod h1:K+q6oSqb0W0Ininfk863uOk1lMy69l/P github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0 h1:wDJmvq38kDhkVxi50ni9ykkdUr1PKgqKOoi01fa0Mdk= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0 h1:MP4Eh7ZCb31lleYCFuwm0oe4/YGak+5l1vA2NOE80nA= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-sql-driver/mysql v1.4.0 h1:7LxgVwFb2hIQtMm87NdgAVfXjnt4OePseqT1tKx+opk= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= +github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gogo/protobuf v1.1.1 h1:72R+M5VuhED/KujmZVcIquuo8mBgX4oVda//DQb3PXo= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d h1:lBXNCxVENCipq4D1Is42JVOP4eQjlB8TQ6H69Yx5J9Q= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= @@ -77,7 +86,9 @@ github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMyw github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/renameio v0.1.0 h1:GOZbcHa3HfsPKPlmyPyN2KEohoMXOhdMbHrvbpl2QaA= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= @@ -89,6 +100,7 @@ github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uc github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869/go.mod h1:cJ6Cj7dQo+O6GJNiMx+Pa94qKj+TG8ONdKHgMNIyyag= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= +github.com/jmoiron/sqlx v1.2.0 h1:41Ip0zITnmWNR/vHV+S4m+VoUivnWY5E4OJfLZjCJMA= github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= @@ -98,16 +110,21 @@ github.com/json-iterator/go v1.1.8 h1:QiWkFLKq0T7mpzwOTu6BzNDbfTE8OLrYhVKYMLF46O github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/julienschmidt/httprouter v1.2.0 h1:TDTW5Yz1mjftljbcKqRcrYhd4XeOoI98t+9HbQbYf7g= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.11.0 h1:wJbzvpYMVGG9iTI9VxpnNZfd4DzMPoCWze3GgSqz8yg= github.com/klauspost/compress v1.11.0/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pty v1.1.1 h1:VkoXIwSboBpnk99O/KFauAEILuNHv5DVFKZMBN/gUgw= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -118,7 +135,9 @@ github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f h1:sgU github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f/go.mod h1:UGmTpUd3rjbtfIpwAPrcfmGf/Z1HS95TATB+m57TPB8= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 h1:Bvq8AziQ5jFF4BHGAEDSqwPW1NJS3XshxbRCxtjFAZc= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042/go.mod h1:TPpsiPUEh0zFL1Snz4crhMlBe60PYxRHr5oFF3rRYg0= +github.com/lib/pq v1.0.0 h1:X5PMW56eZitiTeO7tKzZxFCSpbFZJtkMMooicw2us9A= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/mattn/go-sqlite3 v1.9.0 h1:pDRiWfl+++eC2FEFRy6jXmQlvp4Yh3z1MJKg4UeYM/4= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -128,6 +147,7 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223 h1:F9x/1yl3T2AeKLr2AMdilSD8+f9bvMnNN8VS5iDtovc= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nacos-group/nacos-sdk-go v1.0.1 h1:VNmXGlSS28xOmkO5Nxk5WRp6f1HMosAmG9pDtcnUFcw= github.com/nacos-group/nacos-sdk-go v1.0.1/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= @@ -160,10 +180,12 @@ github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLk github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 h1:MkV+77GLUNo5oJ0jf870itWm3D0Sjh7+Za9gazKc5LQ= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rogpeppe/go-internal v1.3.0 h1:RR9dF3JtopPvtkroDZuVD7qquD0bnHlKSqaQhgwt8yk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= @@ -175,6 +197,7 @@ github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasO github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5NYJe/zRYDwOu9ku6YHy+Iw7l5DM= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= @@ -215,6 +238,7 @@ golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rB golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e h1:JgcxKXxCjrA2tyDP/aNU9K0Ck5Czfk6C7e2tMw7+bSI= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -226,6 +250,7 @@ golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -260,11 +285,13 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/errgo.v2 v2.1.0 h1:0vLT13EuvQ0hNvakwLuFZ/jYrLp5F3kcWHXdRggjCE8= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/ini.v1 v1.42.0 h1:7N3gPTt50s8GuLortA00n8AqRTk75qOP98+mTPpgzRk= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= diff --git a/go.test.sh b/go.test.sh index aca05161..5d4bf6b1 100755 --- a/go.test.sh +++ b/go.test.sh @@ -47,9 +47,9 @@ sudo docker cp send.sh kafka:/tmp/ sudo docker exec kafka sh /tmp/send.sh echo "start clickhouse_sinker to consume" -timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/test_fixed_schema.json -timeout 30 ./dist/clickhouse_sinker --local-cfg-file docker/test_auto_schema.json -timeout 60 ./dist/clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.json +timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_fixed_schema.json +timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_auto_schema.json +timeout 60 ./clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.json echo "check result" count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` @@ -79,14 +79,14 @@ curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' curl "localhost:8123" -d 'TRUNCATE TABLE test_dynamic_schema' echo "publish clickhouse_sinker config" -./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema --local-cfg-file docker/test_fixed_schema.json -./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema --local-cfg-file docker/test_auto_schema.json -./dist/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.json +./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema --local-cfg-file docker/test_fixed_schema.json +./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema --local-cfg-file docker/test_auto_schema.json +./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.json echo "start clickhouse_sinker to consume" -timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema -timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema -timeout 30 ./dist/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema +timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema +timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema +timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema echo "check result" count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` From b541f81aac5f0f2b6397e626fa475abc5c1ea9c6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 10 May 2021 17:48:28 +0800 Subject: [PATCH 095/404] fix json object array parsing. use DateTime64(3) for detected date field. --- go.test.sh | 8 ++--- model/value.go | 3 ++ output/clickhouse.go | 4 +-- parser/fastjson.go | 79 ++++++++++++++++++++++++++++++------------- parser/gjson.go | 10 +++++- parser/parser_test.go | 13 +++++-- task/task.go | 11 +++--- 7 files changed, 89 insertions(+), 39 deletions(-) diff --git a/go.test.sh b/go.test.sh index 5d4bf6b1..36981bc7 100755 --- a/go.test.sh +++ b/go.test.sh @@ -28,13 +28,13 @@ for i in `seq 1 10000`;do echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i }" done > a.json for i in `seq 10001 30000`;do - echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey1\" : $i }" + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey01\" : $i }" done >> a.json for i in `seq 30001 50000`;do - echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey2\" : $i.123, \"newkey3\" : \"name$i\", \"newkey4\" : \"${now}\" }" + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey02\" : $i.123, \"newkey03\" : \"name$i\", \"newkey04\" : \"${now}\", \"newkey05\" : {\"k1\": 1, \"k2\": 2} }" done >> a.json for i in `seq 50001 70000`;do - echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey5\" : [$i], \"newkey6\" : [$i.123], \"newkey7\" : [\"name$i\"], \"newkey8\" : [\"${now}\"] }" + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey06\" : [$i], \"newkey07\" : [$i.123], \"newkey08\" : [\"name$i\"], \"newkey09\" : [\"${now}\"], \"newkey10\" : [{\"k1\": 1, \"k2\": 2}, {\"k3\": 3, \"k4\": 4}] }" done >> a.json for i in `seq 70001 100000`;do echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i }" @@ -62,7 +62,7 @@ echo "Got test_auto_schema count => $count" schema=`curl "localhost:8123" -d 'DESC test_dynamic_schema' 2>/dev/null | grep newkey | sort | tr -d '\t' | tr '\n' ','` echo "Got test_dynamic_schema schema => $schema" -[ $schema = "newkey1Nullable(Int64),newkey2Nullable(Float64),newkey3Nullable(String),newkey4Nullable(DateTime),newkey5Array(Int64),newkey6Array(Float64),newkey7Array(String),newkey8Array(DateTime)," ] || exit 1 +[ $schema = "newkey01Nullable(Int64),newkey02Nullable(Float64),newkey03Nullable(String),newkey04Nullable(DateTime64(3)),newkey05Nullable(String),newkey06Array(Int64),newkey07Array(Float64),newkey08Array(String),newkey09Array(DateTime64(3)),newkey10Array(String)," ] || exit 1 count=`curl "localhost:8123" -d 'SELECT count() FROM test_dynamic_schema'` echo "Got test_dynamic_schema count => $count" [ $count -eq 100000 ] || exit 1 diff --git a/model/value.go b/model/value.go index e4b4888d..9a9356c2 100644 --- a/model/value.go +++ b/model/value.go @@ -83,6 +83,9 @@ func WhichType(typ string) (dataType int, nullable bool) { } if strings.HasPrefix(typ, "DateTime64") { dataType = DateTime + } else if strings.HasPrefix(typ, "Array(DateTime64") { + dataType = DateTimeArray + nullable = false } else { util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported ClickHouse data type %v", typ)) } diff --git a/output/clickhouse.go b/output/clickhouse.go index 5420af25..bb40335f 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -253,7 +253,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { case model.String: strVal = "Nullable(String)" case model.DateTime: - strVal = "Nullable(DateTime)" + strVal = "Nullable(DateTime64(3))" case model.IntArray: strVal = "Array(Int64)" case model.FloatArray: @@ -261,7 +261,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { case model.StringArray: strVal = "Array(String)" case model.DateTimeArray: - strVal = "Array(DateTime)" + strVal = "Array(DateTime64(3))" default: err = errors.Errorf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) return false diff --git a/parser/fastjson.go b/parser/fastjson.go index 07d00846..34b061a8 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -171,15 +171,37 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { case model.String: results := make([]string, 0, len(array)) for _, e := range array { - v, _ := e.StringBytes() - results = append(results, string(v)) + if e.Type() == fastjson.TypeString { + v, _ := e.StringBytes() + results = append(results, string(v)) + } else { + results = append(results, e.String()) + } } val = results case model.DateTime: results := make([]time.Time, 0, len(array)) + var err error for _, e := range array { - v, _ := e.StringBytes() - t := c.pp.ParseDateTime(key, string(v)) + var t time.Time + switch e.Type() { + case fastjson.TypeNumber: + var f float64 + if f, err = e.Float64(); err != nil { + t = Epoch + } else { + t = time.Unix(int64(f), int64(f*1e9)%1e9).In(time.UTC) + } + case fastjson.TypeString: + var b []byte + if b, err = e.StringBytes(); err != nil { + t = Epoch + } else { + t = c.pp.ParseDateTime(key, string(b)) + } + default: + t = Epoch + } results = append(results, t) } val = results @@ -210,31 +232,40 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (fou } func fjDetectType(v *fastjson.Value) (typ int) { - if vt := v.Type(); vt == fastjson.TypeNull { - } else if vt == fastjson.TypeTrue || vt == fastjson.TypeFalse { + switch v.Type() { + case fastjson.TypeNull: + case fastjson.TypeTrue: typ = model.Int - } else if _, err := v.Int64(); err == nil { + case fastjson.TypeFalse: typ = model.Int - } else if _, err := v.Float64(); err == nil { + case fastjson.TypeNumber: typ = model.Float - } else if val, err := v.StringBytes(); err == nil { - if _, layout := parseInLocation(string(val), time.Local); layout != "" { - typ = model.DateTime - } else { - typ = model.String + if _, err := v.Int64(); err == nil { + typ = model.Int + } + case fastjson.TypeString: + typ = model.String + if val, err := v.StringBytes(); err == nil { + if _, layout := parseInLocation(string(val), time.Local); layout != "" { + typ = model.DateTime + } } - } else if arr, err := v.Array(); err == nil && len(arr) > 0 { - typ2 := fjDetectType(arr[0]) - switch typ2 { - case model.Int: - typ = model.IntArray - case model.Float: - typ = model.FloatArray - case model.String: - typ = model.StringArray - case model.DateTime: - typ = model.DateTimeArray + case fastjson.TypeArray: + if arr, err := v.Array(); err == nil && len(arr) > 0 { + typ2 := fjDetectType(arr[0]) + switch typ2 { + case model.Int: + typ = model.IntArray + case model.Float: + typ = model.FloatArray + case model.String: + typ = model.StringArray + case model.DateTime: + typ = model.DateTimeArray + } } + default: + typ = model.String } return } diff --git a/parser/gjson.go b/parser/gjson.go index 5028d723..2ca04118 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -149,7 +149,15 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { case model.DateTime: results := make([]time.Time, 0, len(array)) for _, e := range array { - t := c.pp.ParseDateTime(key, e.String()) + var t time.Time + switch e.Type { + case gjson.Number: + t = time.Unix(int64(e.Num), int64(r.Num*1e9)%1e9).In(time.UTC) + case gjson.String: + t = c.pp.ParseDateTime(key, e.Str) + default: + t = Epoch + } results = append(results, t) } val = results diff --git a/parser/parser_test.go b/parser/parser_test.go index f03447fc..5eccb8a9 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -50,6 +50,7 @@ var jsonSample = []byte(`{ "array_float": [1.1,2.2,3.3], "array_string": ["aa","bb","cc"], "array_date": ["2000-01-01","2000-01-02","2000-01-03"], + "array_object": [{"i":[1,2,3],"f":[1.1,2.2,3.3]},{"s":["aa","bb","cc"],"e":[]}], "array_empty": [], "bool_true": true, "bool_false": false @@ -78,12 +79,13 @@ var jsonSchema = map[string]string{ "array_float": "array", "array_string": "array", "array_date": "array", + "array_object": "array", "array_empty": "array", "bool_true": "true", "bool_false": "false", } -var csvSample = []byte(`1536813227,"0.11","escaped_""ws","{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[""aa"",""bb"",""cc""]","[""2000-01-01"",""2000-01-02"",""2000-01-03""]","[]","true","false"`) +var csvSample = []byte(`1536813227,"0.11","escaped_""ws","{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[""aa"",""bb"",""cc""]","[""2000-01-01"",""2000-01-02"",""2000-01-03""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]","[]","true","false"`) var csvSchema = []string{ "its", @@ -101,6 +103,7 @@ var csvSchema = []string{ "array_float", "array_string", "array_date", + "array_object", "array_empty", "bool_true", "bool_false", @@ -154,7 +157,8 @@ func initMetrics() { } parser = pp.Get() if metric, initErr = parser.Parse(sample); initErr != nil { - return + msg := fmt.Sprintf("%+v", initErr) + panic(msg) } pools[name] = pp parsers[name] = parser @@ -261,6 +265,7 @@ func TestParserArray(t *testing.T) { {"array_float", model.Float, []float64{1.1, 2.2, 3.3}}, {"array_string", model.String, []string{"aa", "bb", "cc"}}, {"array_date", model.DateTime, ts}, + {"array_object", model.String, []string{`{"i":[1,2,3],"f":[1.1,2.2,3.3]}`, `{"s":["aa","bb","cc"],"e":[]}`}}, {"array_empty", model.Int, []int64{}}, {"array_empty", model.Float, []float64{}}, {"array_empty", model.String, []string{}}, @@ -271,6 +276,10 @@ func TestParserArray(t *testing.T) { name := names[i] metric := metrics[name] for j := range testCases { + if name == "csv" && testCases[j].Field == "array_object" { + // csv parser doesn't support object array yet. + continue + } var v interface{} desc := fmt.Sprintf(`%s GetArray("%s", %d)`, name, testCases[j].Field, testCases[j].Type) v = metric.GetArray(testCases[j].Field, testCases[j].Type) diff --git a/task/task.go b/task/task.go index 7ae7093f..cddcbf66 100644 --- a/task/task.go +++ b/task/task.go @@ -229,9 +229,6 @@ func (service *Service) put(msg model.InputMessage) { defer statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Dec() p := service.pp.Get() metric, err = p.Parse(msg.Value) - if err == nil { - row = model.MetricToRow(metric, msg, service.dims) - } // WARNNING: Always PutElem even if there's parsing error, so that this message can be acked to Kafka and skipped writing to ClickHouse. if err != nil { statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() @@ -239,13 +236,15 @@ func (service *Service) put(msg model.InputMessage) { util.Logger.Error(fmt.Sprintf("failed to parse message(topic %v, partition %d, offset %v)", msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", service.cfg.Task.Name), zap.Error(err)) } + } else { + row = model.MetricToRow(metric, msg, service.dims) + if taskCfg.DynamicSchema.Enable { + foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys) + } } // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. service.pp.Put(p) - if taskCfg.DynamicSchema.Enable { - foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys) - } if foundNewKeys { cntNewKeys := atomic.AddInt32(&service.cntNewKeys, 1) if cntNewKeys == 1 { From e5b466ebe2ab07230dbc09bf693d6282316755fc Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 27 Apr 2021 09:08:19 +0800 Subject: [PATCH 096/404] added tcp_port_secure support --- config/config.go | 19 ++++++++++++------- docs/configuration/config.md | 4 ++++ output/clickhouse.go | 2 +- pool/conn.go | 6 +++++- 4 files changed, 22 insertions(+), 9 deletions(-) diff --git a/config/config.go b/config/config.go index 5721bc88..3438b08c 100644 --- a/config/config.go +++ b/config/config.go @@ -88,14 +88,19 @@ type KafkaConfig struct { // ClickHouseConfig configuration parameters type ClickHouseConfig struct { - Cluster string - DB string - Hosts [][]string - Port int + Cluster string + DB string + Hosts [][]string + Port int + Username string + Password string + DsnParams string + + // Whether enable TLS encryption with clickhouse-server + Secure bool + // Whether skip verify clickhouse-server cert + InsecureSkipVerify bool - Username string - Password string - DsnParams string RetryTimes int //<=0 means retry infinitely } diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 9b0efa37..5237a641 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -19,6 +19,10 @@ "username": "default" "password": "", "db": "default", // database name + // Whether enable TLS encryption with clickhouse-server + secure bool + // Whether skip verify clickhouse-server cert if secure=true. + insecureSkipVerify bool // retryTimes when error occurs in inserting datas "retryTimes": 0, }, diff --git a/output/clickhouse.go b/output/clickhouse.go index bb40335f..da633678 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -60,7 +60,7 @@ func NewClickHouse(cfg *config.Config) *ClickHouse { // Init the clickhouse intance func (c *ClickHouse) Init() (err error) { chCfg := &c.cfg.Clickhouse - if err = pool.InitConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams); err != nil { + if err = pool.InitConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify); err != nil { return } return c.initSchema() diff --git a/pool/conn.go b/pool/conn.go index 6ac64ace..5958f2c7 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -22,6 +22,7 @@ import ( "database/sql" "fmt" "net/url" + "strconv" "strings" "sync" "time" @@ -61,7 +62,7 @@ func (c *Connection) ReConnect() error { return nil } -func InitConn(hosts [][]string, port int, db, username, password, dsnParams string) (err error) { +func InitConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool) (err error) { var sqlDB *sql.DB lock.Lock() defer lock.Unlock() @@ -84,6 +85,9 @@ func InitConn(hosts [][]string, port int, db, username, password, dsnParams stri if dsnParams != "" { dsn += "&" + dsnParams } + if secure { + dsn += "&secure=true&skip_verify=" + strconv.FormatBool(skipVerify) + } util.Logger.Debug("sql.Open", zap.String("dsn", dsn)) if sqlDB, err = sql.Open("clickhouse", dsn); err != nil { err = errors.Wrapf(err, "") From 533bbb0710a67935cd88986b6479fb56d5f2762a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 11 May 2021 16:40:06 +0800 Subject: [PATCH 097/404] test MinInt64, MaxInt64 --- parser/csv.go | 18 +++++++----------- parser/gjson.go | 7 +------ parser/parser_test.go | 9 +++++---- 3 files changed, 13 insertions(+), 21 deletions(-) diff --git a/parser/csv.go b/parser/csv.go index e92b4d1a..5f75311e 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -18,7 +18,6 @@ import ( "bytes" "encoding/csv" "fmt" - "strconv" "strings" "sync" "time" @@ -26,6 +25,7 @@ import ( "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" + "github.com/valyala/fastjson/fastfloat" ) var _ Parser = (*CsvParser)(nil) @@ -87,7 +87,7 @@ func (c *CsvMetric) GetFloat(key string, nullable bool) (val interface{}) { val = float64(0.0) return } - val, _ = strconv.ParseFloat(c.values[idx], 64) + val = fastfloat.ParseBestEffort(c.values[idx]) return } @@ -102,7 +102,7 @@ func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}) { val = int64(0) return } - val, _ = strconv.ParseInt(c.values[idx], 10, 64) + val = fastfloat.ParseInt64BestEffort(c.values[idx]) return } @@ -148,20 +148,16 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { switch typ { case model.Int: results := make([]int64, 0, len(array)) - var v int64 for _, e := range array { - if v, err = strconv.ParseInt(e, 10, 64); err == nil { - results = append(results, v) - } + v := fastfloat.ParseInt64BestEffort(e) + results = append(results, v) } val = results case model.Float: results := make([]float64, 0, len(array)) - var v float64 for _, e := range array { - if v, err = strconv.ParseFloat(e, 64); err == nil { - results = append(results, v) - } + v := fastfloat.ParseBestEffort(e) + results = append(results, v) } val = results case model.String: diff --git a/parser/gjson.go b/parser/gjson.go index 2ca04118..eb91396b 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -82,12 +82,7 @@ func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}) { val = int64(0) return } - switch r.Type { - case gjson.Number: - val = int64(r.Num) - default: - val = int64(0) - } + val = r.Int() return } diff --git a/parser/parser_test.go b/parser/parser_test.go index 5eccb8a9..3bedd578 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -46,7 +46,7 @@ var jsonSample = []byte(`{ "time_ms_rfc3339_1": "2019-12-16T12:10:30.123Z", "time_ms_rfc3339_2": "2019-12-16T12:10:30.123+08:00", "time_ms_clickhouse_1": "2019-12-16 12:10:30.123", - "array_int": [1,2,3], + "array_int": [-9223372036854775808,9223372036854775807], "array_float": [1.1,2.2,3.3], "array_string": ["aa","bb","cc"], "array_date": ["2000-01-01","2000-01-02","2000-01-03"], @@ -85,7 +85,7 @@ var jsonSchema = map[string]string{ "bool_false": "false", } -var csvSample = []byte(`1536813227,"0.11","escaped_""ws","{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[1,2,3]","[1.1,2.2,3.3]","[""aa"",""bb"",""cc""]","[""2000-01-01"",""2000-01-02"",""2000-01-03""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]","[]","true","false"`) +var csvSample = []byte(`1536813227,"0.11","escaped_""ws","{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[-9223372036854775808,9223372036854775807]","[1.1,2.2,3.3]","[""aa"",""bb"",""cc""]","[""2000-01-01"",""2000-01-02"",""2000-01-03""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]","[]","true","false"`) var csvSchema = []string{ "its", @@ -221,7 +221,7 @@ func TestParserString(t *testing.T) { {"not_exist", false, ""}, {"not_exist", true, nil}, {"its", false, "1536813227"}, - {"array_int", false, "[1,2,3]"}, + {"array_int", false, "[-9223372036854775808,9223372036854775807]"}, {"array_string", false, `["aa","bb","cc"]`}, {"mp", false, `{"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}`}, } @@ -261,7 +261,8 @@ func TestParserArray(t *testing.T) { ts = append(ts, t) } testCases := []ArrayCase{ - {"array_int", model.Int, []int64{1, 2, 3}}, + {"array_int", model.Float, []float64{-9223372036854775808, 9223372036854775807}}, + {"array_int", model.Int, []int64{-9223372036854775808, 9223372036854775807}}, {"array_float", model.Float, []float64{1.1, 2.2, 3.3}}, {"array_string", model.String, []string{"aa", "bb", "cc"}}, {"array_date", model.DateTime, ts}, From 52e488a7e3635b3c30153a19ce580d907d02bef3 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 12 May 2021 16:39:19 +0800 Subject: [PATCH 098/404] flexable connection switch among replicas --- cmd/clickhouse_sinker/main.go | 4 +- go.sum | 27 -------- output/clickhouse.go | 90 ++++++++++++++------------ pool/conn.go | 115 +++++++++++++++++++--------------- task/sharding.go | 2 +- 5 files changed, 117 insertions(+), 121 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 55a89181..a48e8d09 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -334,7 +334,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { concurrentParsers = 1 } util.InitGlobalParsingPool(concurrentParsers) - totalConn := pool.GetTotalConn() + totalConn := pool.NumShard() util.InitGlobalWritingPool(totalConn) go s.task.Run(s.ctx) @@ -364,7 +364,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 3. Restart goroutine pools. util.InitGlobalTimerWheel() util.GlobalParsingPool.Restart() - totalConn := pool.GetTotalConn() + totalConn := pool.NumShard() util.GlobalWritingPool.Resize(totalConn) util.GlobalWritingPool.Restart() diff --git a/go.sum b/go.sum index 406990ec..0e80d535 100644 --- a/go.sum +++ b/go.sum @@ -11,10 +11,8 @@ github.com/Shopify/sarama v1.27.2/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4 h1:Hs82Z41s6SdL1CELW+XaDYmOH4hkBN4/N9og/AsOv7E= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 h1:zOVTBdCKFd9JbCKz9/nt+FovbjPFmb7mUnp8nH9fQBA= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= @@ -32,7 +30,6 @@ github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+ github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= -github.com/creack/pty v1.1.9 h1:uDmaGzcdjhF4i/plgjmEsriH11Y0o7RKapEf/LDaM3w= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -54,18 +51,12 @@ github.com/frankban/quicktest v1.10.2/go.mod h1:K+q6oSqb0W0Ininfk863uOk1lMy69l/P github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/kit v0.9.0 h1:wDJmvq38kDhkVxi50ni9ykkdUr1PKgqKOoi01fa0Mdk= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= -github.com/go-logfmt/logfmt v0.4.0 h1:MP4Eh7ZCb31lleYCFuwm0oe4/YGak+5l1vA2NOE80nA= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-sql-driver/mysql v1.4.0 h1:7LxgVwFb2hIQtMm87NdgAVfXjnt4OePseqT1tKx+opk= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= -github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/gogo/protobuf v1.1.1 h1:72R+M5VuhED/KujmZVcIquuo8mBgX4oVda//DQb3PXo= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d h1:lBXNCxVENCipq4D1Is42JVOP4eQjlB8TQ6H69Yx5J9Q= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= @@ -86,9 +77,7 @@ github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMyw github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/renameio v0.1.0 h1:GOZbcHa3HfsPKPlmyPyN2KEohoMXOhdMbHrvbpl2QaA= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= @@ -100,7 +89,6 @@ github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uc github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869/go.mod h1:cJ6Cj7dQo+O6GJNiMx+Pa94qKj+TG8ONdKHgMNIyyag= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= -github.com/jmoiron/sqlx v1.2.0 h1:41Ip0zITnmWNR/vHV+S4m+VoUivnWY5E4OJfLZjCJMA= github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= @@ -110,21 +98,16 @@ github.com/json-iterator/go v1.1.8 h1:QiWkFLKq0T7mpzwOTu6BzNDbfTE8OLrYhVKYMLF46O github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/julienschmidt/httprouter v1.2.0 h1:TDTW5Yz1mjftljbcKqRcrYhd4XeOoI98t+9HbQbYf7g= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.11.0 h1:wJbzvpYMVGG9iTI9VxpnNZfd4DzMPoCWze3GgSqz8yg= github.com/klauspost/compress v1.11.0/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= -github.com/kr/pty v1.1.1 h1:VkoXIwSboBpnk99O/KFauAEILuNHv5DVFKZMBN/gUgw= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -135,9 +118,7 @@ github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f h1:sgU github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f/go.mod h1:UGmTpUd3rjbtfIpwAPrcfmGf/Z1HS95TATB+m57TPB8= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 h1:Bvq8AziQ5jFF4BHGAEDSqwPW1NJS3XshxbRCxtjFAZc= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042/go.mod h1:TPpsiPUEh0zFL1Snz4crhMlBe60PYxRHr5oFF3rRYg0= -github.com/lib/pq v1.0.0 h1:X5PMW56eZitiTeO7tKzZxFCSpbFZJtkMMooicw2us9A= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= -github.com/mattn/go-sqlite3 v1.9.0 h1:pDRiWfl+++eC2FEFRy6jXmQlvp4Yh3z1MJKg4UeYM/4= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -147,7 +128,6 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223 h1:F9x/1yl3T2AeKLr2AMdilSD8+f9bvMnNN8VS5iDtovc= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nacos-group/nacos-sdk-go v1.0.1 h1:VNmXGlSS28xOmkO5Nxk5WRp6f1HMosAmG9pDtcnUFcw= github.com/nacos-group/nacos-sdk-go v1.0.1/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= @@ -180,12 +160,10 @@ github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLk github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 h1:MkV+77GLUNo5oJ0jf870itWm3D0Sjh7+Za9gazKc5LQ= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= -github.com/rogpeppe/go-internal v1.3.0 h1:RR9dF3JtopPvtkroDZuVD7qquD0bnHlKSqaQhgwt8yk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= @@ -197,7 +175,6 @@ github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasO github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5NYJe/zRYDwOu9ku6YHy+Iw7l5DM= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= @@ -238,7 +215,6 @@ golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rB golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e h1:JgcxKXxCjrA2tyDP/aNU9K0Ck5Czfk6C7e2tMw7+bSI= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -250,7 +226,6 @@ golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -285,13 +260,11 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/errgo.v2 v2.1.0 h1:0vLT13EuvQ0hNvakwLuFZ/jYrLp5F3kcWHXdRggjCE8= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/ini.v1 v1.42.0 h1:7N3gPTt50s8GuLortA00n8AqRTk75qOP98+mTPpgzRk= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= diff --git a/output/clickhouse.go b/output/clickhouse.go index da633678..9cc94c66 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -18,7 +18,6 @@ package output import ( "context" "database/sql" - std_errors "errors" "fmt" "io" "math" @@ -28,6 +27,7 @@ import ( "sync" "time" + "github.com/ClickHouse/clickhouse-go" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" @@ -40,6 +40,9 @@ import ( var ( selectSQLTemplate = `select name, type, default_kind from system.columns where database = '%s' and table = '%s'` lowCardinalityRegexp = regexp.MustCompile(`LowCardinality\((.+)\)`) + + // refers to src/Common/ErrorCodes.cpp, https://github.com/ClickHouse/ClickHouse/issues/24036 + replicaSpecificErrorCodes = []int32{1000} ) // ClickHouse is an output service consumers from kafka messages @@ -60,7 +63,7 @@ func NewClickHouse(cfg *config.Config) *ClickHouse { // Init the clickhouse intance func (c *ClickHouse) Init() (err error) { chCfg := &c.cfg.Clickhouse - if err = pool.InitConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify); err != nil { + if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify); err != nil { return } return c.initSchema() @@ -76,83 +79,88 @@ func (c *ClickHouse) Send(batch *model.Batch) { } // Write kvs to clickhouse -func (c *ClickHouse) write(batch *model.Batch) error { - var numErr int - var err, tmpErr error +func (c *ClickHouse) write(batch *model.Batch, reconnect bool) (err error) { var stmt *sql.Stmt var tx *sql.Tx if len(*batch.Rows) == 0 { - return nil + return } - conn := pool.GetConn(batch.BatchIdx) + conn := pool.GetShardConn(batch.BatchIdx) + if reconnect { + if err = conn.NextGoodReplica(); err != nil { + return + } + } if tx, err = conn.Begin(); err != nil { - goto ERR + err = errors.Wrapf(err, "conn.Begin") + return } if stmt, err = tx.Prepare(c.prepareSQL); err != nil { - goto ERR + err = errors.Wrapf(err, "tx.Prepare %s", c.prepareSQL) + return } defer stmt.Close() for _, row := range *batch.Rows { - if _, tmpErr = stmt.Exec(*row...); tmpErr != nil { - numErr++ - err = tmpErr + if _, err = stmt.Exec(*row...); err != nil { + err = errors.Wrapf(err, "stmt.Exec") + break } } if err != nil { - util.Logger.Error("stmt.Exec failed", zap.String("task", c.cfg.Task.Name), zap.Int("times", numErr), zap.Error(err)) - goto ERR + _ = tx.Rollback() + return err } - if err = tx.Commit(); err != nil { - goto ERR + err = errors.Wrapf(err, "tx.Commit") + return } statistics.FlushMsgsTotal.WithLabelValues(c.cfg.Task.Name).Add(float64(batch.RealSize)) - return err -ERR: - if shouldReconnect(err) { - _ = conn.ReConnect() - statistics.ClickhouseReconnectTotal.WithLabelValues(c.cfg.Task.Name).Inc() - } - return err + return } func shouldReconnect(err error) bool { - if err == nil { - return false - } - if strings.Contains(err.Error(), "connection refused") || strings.Contains(err.Error(), "bad connection") { - return true + var exp *clickhouse.Exception + if errors.As(err, &exp) { + util.Logger.Error("this is an exception from clickhouse-server", zap.Reflect("exception", exp)) + var replicaSpecific bool + for _, ec := range replicaSpecificErrorCodes { + if ec == exp.Code { + replicaSpecific = true + break + } + } + return replicaSpecific } - util.Logger.Info("this is a permanent error", zap.Error(err)) - return false + return true } // LoopWrite will dead loop to write the records func (c *ClickHouse) loopWrite(batch *model.Batch) { var err error var times int + var reconnect bool for { - if err = c.write(batch); err == nil { + if err = c.write(batch, reconnect); err == nil { if err = batch.Commit(); err == nil { return } - // TODO: kafka_go and sarama commit give different error when context is cancceled. - // How to unify them? - if std_errors.Is(err, context.Canceled) || std_errors.Is(err, io.ErrClosedPipe) { + // Note: kafka_go and sarama commit give different error when context is cancceled. + if errors.Is(err, context.Canceled) || errors.Is(err, io.ErrClosedPipe) { util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.cfg.Task.Name)) return } util.Logger.Fatal("committing offset failed with permanent error %+v", zap.String("task", c.cfg.Task.Name), zap.Error(err)) } - if std_errors.Is(err, context.Canceled) { + if errors.Is(err, context.Canceled) { util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.cfg.Task.Name)) return } - util.Logger.Error("flush batch failed", zap.String("task", c.cfg.Task.Name), zap.Int("try", c.cfg.Clickhouse.RetryTimes-times), zap.Error(err)) + util.Logger.Error("flush batch failed", zap.String("task", c.cfg.Task.Name), zap.Int("try", times), zap.Error(err)) statistics.FlushMsgsErrorTotal.WithLabelValues(c.cfg.Task.Name).Add(float64(batch.RealSize)) times++ - if shouldReconnect(err) && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { + reconnect = shouldReconnect(err) + if reconnect && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { time.Sleep(10 * time.Second) } else { util.Logger.Fatal("ClickHouse.loopWrite failed", zap.String("task", c.cfg.Task.Name)) @@ -162,13 +170,13 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { // Stop free clickhouse connections func (c *ClickHouse) Stop() error { - pool.FreeConn() + pool.FreeClusterConn() return nil } func (c *ClickHouse) initSchema() (err error) { if c.cfg.Task.AutoSchema { - conn := pool.GetConn(0) + conn := pool.GetShardConn(0) var rs *sql.Rows if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, c.cfg.Clickhouse.DB, c.cfg.Task.TableName)); err != nil { err = errors.Wrapf(err, "") @@ -286,7 +294,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { chCfg.Cluster, chCfg.DB, taskCfg.TableName)) } } - conn := pool.GetConn(0) + conn := pool.GetShardConn(0) for _, query := range queries { util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) if _, err = conn.Exec(query); err != nil { @@ -300,7 +308,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { taskCfg := &c.cfg.Task chCfg := &c.cfg.Clickhouse - conn := pool.GetConn(0) + conn := pool.GetShardConn(0) query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, chCfg.DB, chCfg.DB, taskCfg.TableName) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) diff --git a/pool/conn.go b/pool/conn.go index 5958f2c7..79a03a38 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -23,7 +23,6 @@ import ( "fmt" "net/url" "strconv" - "strings" "sync" "time" @@ -40,34 +39,63 @@ const ( var ( lock sync.Mutex - connections []*Connection + clusterConn []*ShardConn + dsnTmpl string ) -// Connection a datastructure for storing the clickhouse connection -type Connection struct { +// ShardConn a datastructure for storing the clickhouse connection +type ShardConn struct { *sql.DB - dsn string + Dsn string + Replicas []string //ip:port list of replicas + NextRep int //index of next replica } -// ReConnect used for restablishing connection with server -func (c *Connection) ReConnect() error { - sqlDB, err := sql.Open("clickhouse", c.dsn) - if err != nil { - util.Logger.Debug("sql.Open failed", zap.String("dsn", c.dsn), zap.Error(err)) - return err +// NextGoodReplica connects to next good replica +func (c *ShardConn) NextGoodReplica() error { + if c.DB != nil { + if err := health.Health.RemoveReadinessCheck(c.Dsn); err != nil { + util.Logger.Warn("health.Health.RemoveReadinessCheck failed", zap.String("dsn", c.Dsn), zap.Error(err)) + } + c.DB.Close() + c.DB = nil } - setDBParams(sqlDB) - util.Logger.Debug("sql.Open succeeded", zap.String("dsn", c.dsn)) - c.DB = sqlDB - return nil + savedNextRep := c.NextRep + // try all replicas, including the current one + for i := 0; i < len(c.Replicas); i++ { + c.Dsn = fmt.Sprintf(dsnTmpl, c.Replicas[c.NextRep]) + c.NextRep = (c.NextRep + 1) % len(c.Replicas) + sqlDB, err := sql.Open("clickhouse", c.Dsn) + if err != nil { + util.Logger.Warn("sql.Open failed", zap.String("dsn", c.Dsn), zap.Error(err)) + continue + } + setDBParams(sqlDB) + util.Logger.Info("sql.Open succeeded", zap.String("dsn", c.Dsn)) + if err = health.Health.AddReadinessCheck(c.Dsn, healthcheck.DatabasePingCheck(sqlDB, 30*time.Second)); err != nil { + util.Logger.Warn("health.Health.RemoveReadinessCheck failed", zap.String("dsn", c.Dsn), zap.Error(err)) + } + c.DB = sqlDB + return nil + } + err := errors.Errorf("no good replica among replicas %v since %d", c.Replicas, savedNextRep) + return err } -func InitConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool) (err error) { - var sqlDB *sql.DB +func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool) (err error) { lock.Lock() defer lock.Unlock() - // Each shard has a *sql.DB which connects to all replicas inside the shard. - // "alt_hosts" tolerates replica single-point-failure. + // Each shard has a *sql.DB which connects to one replica inside the shard. + // "alt_hosts" tolerates replica single-point-failure. However more flexable switching is needed for some cases for example https://github.com/ClickHouse/ClickHouse/issues/24036. + dsnTmpl = "tcp://%s" + fmt.Sprintf("?database=%s&username=%s&password=%s&block_size=%d", + url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password), BlockSize) + if dsnParams != "" { + dsnTmpl += "&" + dsnParams + } + if secure { + dsnTmpl += "&secure=true&skip_verify=" + strconv.FormatBool(skipVerify) + } + for _, replicas := range hosts { numReplicas := len(replicas) replicaAddrs := make([]string, numReplicas) @@ -77,28 +105,13 @@ func InitConn(hosts [][]string, port int, db, username, password, dsnParams stri } replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) } - dsn := fmt.Sprintf("tcp://%s?database=%s&username=%s&password=%s&block_size=%d", - replicaAddrs[0], url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password), BlockSize) - if numReplicas > 1 { - dsn += "&connection_open_strategy=in_order&alt_hosts=" + strings.Join(replicaAddrs[1:numReplicas], ",") - } - if dsnParams != "" { - dsn += "&" + dsnParams + sc := &ShardConn{ + Replicas: replicaAddrs, } - if secure { - dsn += "&secure=true&skip_verify=" + strconv.FormatBool(skipVerify) - } - util.Logger.Debug("sql.Open", zap.String("dsn", dsn)) - if sqlDB, err = sql.Open("clickhouse", dsn); err != nil { - err = errors.Wrapf(err, "") + if err = sc.NextGoodReplica(); err != nil { return } - setDBParams(sqlDB) - if err = health.Health.AddReadinessCheck(dsn, healthcheck.DatabasePingCheck(sqlDB, 30*time.Second)); err != nil { - err = errors.Wrapf(err, "") - return - } - connections = append(connections, &Connection{sqlDB, dsn}) + clusterConn = append(clusterConn, sc) } return } @@ -111,33 +124,35 @@ func setDBParams(sqlDB *sql.DB) { sqlDB.SetConnMaxIdleTime(10 * time.Second) } -func FreeConn() { +func FreeClusterConn() { lock.Lock() defer lock.Unlock() - for _, conn := range connections { - if err := health.Health.RemoveReadinessCheck(conn.dsn); err != nil { - util.Logger.Error("health.Health.RemoveReadinessCheck failed", zap.Error(err)) + for _, sc := range clusterConn { + if sc.DB != nil { + if err := health.Health.RemoveReadinessCheck(sc.Dsn); err != nil { + util.Logger.Error("health.Health.RemoveReadinessCheck failed", zap.String("dsn", sc.Dsn), zap.Error(err)) + } + sc.DB.Close() } - conn.DB.Close() } - connections = []*Connection{} + clusterConn = []*ShardConn{} } -func GetTotalConn() (cnt int) { +func NumShard() (cnt int) { lock.Lock() defer lock.Unlock() - return len(connections) + return len(clusterConn) } -// GetConn select a clickhouse node from the cluster based on batchNum -func GetConn(batchNum int64) (con *Connection) { +// GetShardConn select a clickhouse shard based on batchNum +func GetShardConn(batchNum int64) (con *ShardConn) { lock.Lock() defer lock.Unlock() - con = connections[batchNum%int64(len(connections))] + con = clusterConn[batchNum%int64(len(clusterConn))] return } // CloseAll closed all connection and destroys the pool func CloseAll() { - FreeConn() + FreeClusterConn() } diff --git a/task/sharding.go b/task/sharding.go index 6fb40849..5de656ca 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -113,7 +113,7 @@ type Sharder struct { func NewSharder(service *Service) (sh *Sharder, err error) { var policy *ShardingPolicy - ckNum := pool.GetTotalConn() + ckNum := pool.NumShard() taskCfg := &service.cfg.Task if policy, err = NewShardingPolicy(taskCfg.ShardingKey, taskCfg.ShardingPolicy, service.clickhouse.Dms, ckNum); err != nil { return From 1851066b263ae119faf55d626949755ae9ac7a4f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 13 May 2021 16:05:39 +0800 Subject: [PATCH 099/404] added border cases for parser test --- .github/workflows/lint.yml | 4 +- Makefile | 2 +- docs/dev/introduction.md | 49 ++-- parser/csv.go | 81 ++++-- parser/fastjson.go | 42 ++-- parser/gjson.go | 60 ++++- parser/parser.go | 22 +- parser/parser_test.go | 500 +++++++++++++++++++++++++------------ 8 files changed, 526 insertions(+), 234 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index dde41c61..6f96e84e 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -14,13 +14,13 @@ jobs: uses: golangci/golangci-lint-action@v2 with: # Required: the version of golangci-lint is required and must be specified without patch version: we always use the latest patch version. - version: v1.39 + version: v1.40 # Optional: working directory, useful for monorepos # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,revive # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true diff --git a/Makefile b/Makefile index bcf852e6..7597b935 100644 --- a/Makefile +++ b/Makefile @@ -21,6 +21,6 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,revive run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.json diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 5b518e43..f9edc5a2 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -34,18 +34,19 @@ Refers to [design](./design.md) for how it works. - [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) Note: -- A message is ignored if it's invalid json, or CSV value doesn't match with the format. This is counted by `ParseMsgsErrorTotal`. -- If a message field value doesn't match with the type `T` declared in ClickHouse, the default value of `T` (see the following table) is filled. - -| ClickHouse data type | default value | -| -------------------- |:-------------:| -| Int8, Int16, ... | 0 | -| Float32, Float64 | 0.0 | -| String, ... | "" | -| Date, DateTime, ... | EPOCH | -| Nullable(T) | NULL | -| Array(T) | [] | +- A message is ignored if it's invalid json, or CSV value doesn't match with the format. This is counted by `ParseMsgsErrorTotal`. +- If a message field type is imcompatible with the type `T` declared in ClickHouse, or field value is invalid to parse, the default value of `T` (see the following table) is filled. +- If a message field type is compatible with the type `T` declared in ClickHouse, but field value is overflow, the nearer border of `T` is filled. + +| ClickHouse data type | default value | compatible Json data type | valid range | +|:--------------------:|:-------------:|:-----------------------------------:|:-------------------------------------:| +| Int8, Int16, ... | 0 | Bool, Number | Int8 [-128,127], ... | +| Float32, Float64 | 0.0 | Number | Float32 [-MaxFloat32,MaxFloat32], ... | +| String, ... | "" | Bool, Number, String, Object, Array | N/A | +| Date, DateTime, ... | EPOCH | Number, String | [EPOCH,MaxUint32_seconds_since_epoch) | +| Nullable(T) | NULL | (The same as T) | (The same as T) | +| Array(T) | [] | (The same as T) | (The same as T) | ## Configuration @@ -70,6 +71,7 @@ An example kafka config: - Encryption using SSL An example kafka config: + ```json "kafka": { "brokers": "192.168.31.64:9093", @@ -87,6 +89,7 @@ An example kafka config: ``` Or if you have extracted certificates from JKS, use the following config: + ```json "kafka": { "brokers": "192.168.31.64:9093", @@ -134,6 +137,7 @@ An example kafka config: - SASL/PLAIN An example kafka config: + ```json "kafka": { "brokers": "192.168.31.64:9094", @@ -164,6 +168,7 @@ $ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic su - SASL/SCRAM An example kafka config: + ```json "kafka": { "brokers": "192.168.31.64:9094", @@ -195,6 +200,7 @@ $ bin/kafka-console-consumer.sh --bootstrap-server 192.168.31.64:9094 --topic su - SASL/GSSAPI(Kerberos) An example kafka config: + ```json "kafka": { "brokers": "192.168.31.64:9094", @@ -280,31 +286,29 @@ Sinker registers with Nacos if CLI `--consul-cfg-enable` or env `CONSUL_REGISTER If CLI `--metric-push-gateway-addrs` or env `METRIC_PUSH_GATEWAY_ADDRS` (a list of comma-separated urls) is present, metrics are pushed to one of given URLs regualarly. - ## Extending There are several abstract interfaces which you can implement to support more message format, message queue and config management mechanism. ```go type Parser interface { - Parse(bs []byte) model.Metric + Parse(bs []byte) model.Metric } type Inputer interface { - Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) error - Run(ctx context.Context) - Stop() error - CommitMessages(ctx context.Context, message *model.InputMessage) error + Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) error + Run(ctx context.Context) + Stop() error + CommitMessages(ctx context.Context, message *model.InputMessage) error } // RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... type RemoteConfManager interface { - Init(properties map[string]interface{}) error - GetConfig() (conf *Config, err error) - // PublishConfig publishs the config. The manager shall not reference the passed Config object after call. - PublishConfig(conf *Config) (err error) + Init(properties map[string]interface{}) error + GetConfig() (conf *Config, err error) + // PublishConfig publishs the config. The manager shall not reference the passed Config object after call. + PublishConfig(conf *Config) (err error) } - ``` ## Why not [`Kafka Engine`](https://clickhouse.tech/docs/en/engines/table-engines/integrations/kafka/) built in ClickHouse? @@ -314,7 +318,6 @@ type RemoteConfManager interface { - `Kafka Engine` doesn't support custom sharding policy. - Neither `Kafka Engine` nor clickhouse_sinker support exactly-once. - ## Kafka Compatibility Kafka broker [exposes versions of various APIs it supports since 0.10.0.0](https://kafka.apache.org/protocol#api_versions). diff --git a/parser/csv.go b/parser/csv.go index 5f75311e..2519d2aa 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -18,13 +18,14 @@ import ( "bytes" "encoding/csv" "fmt" - "strings" + "strconv" "sync" "time" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" + "github.com/tidwall/gjson" "github.com/valyala/fastjson/fastfloat" ) @@ -65,7 +66,7 @@ type CsvMetric struct { func (c *CsvMetric) GetString(key string, nullable bool) (val interface{}) { var idx int var ok bool - if idx, ok = c.pp.csvFormat[key]; !ok { + if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { if nullable { return } @@ -80,7 +81,7 @@ func (c *CsvMetric) GetString(key string, nullable bool) (val interface{}) { func (c *CsvMetric) GetFloat(key string, nullable bool) (val interface{}) { var idx int var ok bool - if idx, ok = c.pp.csvFormat[key]; !ok { + if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { if nullable { return } @@ -95,28 +96,37 @@ func (c *CsvMetric) GetFloat(key string, nullable bool) (val interface{}) { func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}) { var idx int var ok bool - if idx, ok = c.pp.csvFormat[key]; !ok { + if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { if nullable { return } val = int64(0) return } - val = fastfloat.ParseInt64BestEffort(c.values[idx]) + if s := c.values[idx]; s == "true" { + val = int64(1) + } else { + val = fastfloat.ParseInt64BestEffort(s) + } return } func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}) { var idx int var ok bool - if idx, ok = c.pp.csvFormat[key]; !ok { + if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { if nullable { return } val = Epoch return } - val = c.pp.ParseDateTime(key, c.values[idx]) + s := c.values[idx] + if dd, err := strconv.ParseFloat(s, 64); err != nil { + val = c.pp.ParseDateTime(key, s) + } else { + val = UnixFloat(dd) + } return } @@ -130,18 +140,14 @@ func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) (val interface // GetArray parse an CSV encoded array func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { - var err error - var array []string - var r *csv.Reader s := c.GetString(key, false) str, _ := s.(string) - strLen := len(str) - if str == "" || str[0] != '[' || str[strLen-1] != ']' { + if str == "" || str[0] != '[' { val = makeArray(typ) return } - r = csv.NewReader(strings.NewReader(str[1 : strLen-1])) - if array, err = r.Read(); err != nil { + array := gjson.Parse(str).Array() + if len(array) == 0 { val = makeArray(typ) return } @@ -149,24 +155,61 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { case model.Int: results := make([]int64, 0, len(array)) for _, e := range array { - v := fastfloat.ParseInt64BestEffort(e) + var v int64 + switch e.Type { + case gjson.True: + v = int64(1) + case gjson.Number: + if v = e.Int(); float64(v) != e.Num { + v = int64(0) + } + default: + v = int64(0) + } results = append(results, v) } val = results case model.Float: results := make([]float64, 0, len(array)) for _, e := range array { - v := fastfloat.ParseBestEffort(e) + var v float64 + switch e.Type { + case gjson.Number: + v = e.Num + default: + v = float64(0.0) + } results = append(results, v) } val = results case model.String: - val = array + results := make([]string, 0, len(array)) + for _, e := range array { + var v string + switch e.Type { + case gjson.Null: + v = "" + case gjson.String: + v = e.Str + default: + v = e.Raw + } + results = append(results, v) + } + val = results case model.DateTime: results := make([]time.Time, 0, len(array)) for _, e := range array { - v := c.pp.ParseDateTime(key, e) - results = append(results, v) + var t time.Time + switch e.Type { + case gjson.Number: + t = UnixFloat(e.Num) + case gjson.String: + t = c.pp.ParseDateTime(key, e.Str) + default: + t = Epoch + } + results = append(results, t) } val = results default: diff --git a/parser/fastjson.go b/parser/fastjson.go index 34b061a8..2fc4bc88 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -119,7 +119,7 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} val = Epoch return } - val = time.Unix(int64(f), int64(f*1e9)%1e9).In(time.UTC) + val = UnixFloat(f) case fastjson.TypeString: var b []byte if b, err = v.StringBytes(); err != nil { @@ -143,14 +143,13 @@ func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) (val inte func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { v := c.value.Get(key) + val = makeArray(typ) if v == nil || v.Type() != fastjson.TypeArray { - val = makeArray(typ) return } array, _ := v.Array() switch typ { case model.Int: - results := make([]int64, 0, len(array)) for _, e := range array { var v int64 if e.Type() == fastjson.TypeTrue { @@ -158,43 +157,39 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { } else { v, _ = e.Int64() } - results = append(results, v) + val = append(val.([]int64), v) } - val = results case model.Float: - results := make([]float64, 0, len(array)) for _, e := range array { v, _ := e.Float64() - results = append(results, v) + val = append(val.([]float64), v) } - val = results case model.String: - results := make([]string, 0, len(array)) for _, e := range array { - if e.Type() == fastjson.TypeString { - v, _ := e.StringBytes() - results = append(results, string(v)) - } else { - results = append(results, e.String()) + var s string + switch e.Type() { + case fastjson.TypeNull: + s = "" + case fastjson.TypeString: + b, _ := e.StringBytes() + s = string(b) + default: + s = e.String() } + val = append(val.([]string), s) } - val = results case model.DateTime: - results := make([]time.Time, 0, len(array)) - var err error for _, e := range array { var t time.Time switch e.Type() { case fastjson.TypeNumber: - var f float64 - if f, err = e.Float64(); err != nil { + if f, err := e.Float64(); err != nil { t = Epoch } else { - t = time.Unix(int64(f), int64(f*1e9)%1e9).In(time.UTC) + t = UnixFloat(f) } case fastjson.TypeString: - var b []byte - if b, err = e.StringBytes(); err != nil { + if b, err := e.StringBytes(); err != nil { t = Epoch } else { t = c.pp.ParseDateTime(key, string(b)) @@ -202,9 +197,8 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { default: t = Epoch } - results = append(results, t) + val = append(val.([]time.Time), t) } - val = results default: util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) } diff --git a/parser/gjson.go b/parser/gjson.go index eb91396b..ac4a39a9 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -51,7 +51,14 @@ func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}) { val = "" return } - val = r.String() + switch r.Type { + case gjson.Null: + val = "" + case gjson.String: + val = r.Str + default: + val = r.Raw + } return } @@ -82,7 +89,18 @@ func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}) { val = int64(0) return } - val = r.Int() + switch r.Type { + case gjson.True: + val = int64(1) + case gjson.Number: + if v := r.Int(); float64(v) != r.Num { + val = int64(0) + } else { + val = v + } + default: + val = int64(0) + } return } @@ -97,7 +115,7 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { } switch r.Type { case gjson.Number: - val = time.Unix(int64(r.Num), int64(r.Num*1e9)%1e9).In(time.UTC) + val = UnixFloat(r.Num) case gjson.String: val = c.pp.ParseDateTime(key, r.Str) default: @@ -125,20 +143,46 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { case model.Int: results := make([]int64, 0, len(array)) for _, e := range array { - results = append(results, e.Int()) + var v int64 + switch e.Type { + case gjson.True: + v = int64(1) + case gjson.Number: + if v = e.Int(); float64(v) != e.Num { + v = int64(0) + } + default: + v = int64(0) + } + results = append(results, v) } val = results case model.Float: results := make([]float64, 0, len(array)) - for _, e := range array { - results = append(results, e.Float()) + var f float64 + switch e.Type { + case gjson.Number: + f = e.Num + default: + f = float64(0.0) + } + results = append(results, f) } val = results case model.String: results := make([]string, 0, len(array)) for _, e := range array { - results = append(results, e.String()) + var s string + switch e.Type { + case gjson.Null: + s = "" + case gjson.String: + s = e.Str + default: + s = e.Raw + } + results = append(results, s) } val = results case model.DateTime: @@ -147,7 +191,7 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { var t time.Time switch e.Type { case gjson.Number: - t = time.Unix(int64(e.Num), int64(r.Num*1e9)%1e9).In(time.UTC) + t = UnixFloat(e.Num) case gjson.String: t = c.pp.ParseDateTime(key, e.Str) default: diff --git a/parser/parser.go b/parser/parser.go index 015bfb59..71226319 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -16,6 +16,7 @@ package parser import ( "fmt" + "math" "sync" "time" @@ -155,7 +156,7 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time) { t = Epoch return } - t = t.In(time.UTC) + t = t.UTC() return } @@ -180,10 +181,27 @@ func parseInLocation(val string, loc *time.Location) (t time.Time, layout string var lay string for _, lay = range Layouts { if t, err = time.ParseInLocation(lay, val, loc); err == nil { - t = t.In(time.UTC) + t = t.UTC() layout = lay return } } return } + +func UnixInt(sec int64) (t time.Time) { + //2^32 seconds since epoch: 2106-02-07T06:28:16Z + if sec < 0 || sec >= 4294967296 { + return Epoch + } + return time.Unix(sec, 0).UTC() +} + +func UnixFloat(sec float64) (t time.Time) { + //2^32 seconds since epoch: 2106-02-07T06:28:16Z + if sec < 0 || sec >= 4294967296.0 { + return Epoch + } + i, f := math.Modf(sec) + return time.Unix(int64(i), int64(f*1e9)).UTC() +} diff --git a/parser/parser_test.go b/parser/parser_test.go index 3bedd578..50fd86fb 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -17,6 +17,8 @@ package parser import ( "encoding/json" "fmt" + "log" + "strconv" "sync" "testing" "time" @@ -27,93 +29,123 @@ import ( "github.com/valyala/fastjson" ) +// https://golang.org/pkg/math/, Mathematical constants var jsonSample = []byte(`{ - "its":1536813227, - "_ip":"112.96.65.228", - "cgi":"/commui/queryhttpdns", - "channel":"escaped_\"ws", - "platform":"adr", - "experiment":"default", - "ip":"36.248.20.69", - "version":"5.8.3", - "success":0, - "percent":0.11, - "mp": {"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}, - "date1": "2019-12-16", - "time_sec_rfc3339_1": "2019-12-16T12:10:30Z", - "time_sec_rfc3339_2": "2019-12-16T12:10:30+08:00", - "time_sec_clickhouse_1": "2019-12-16 12:10:30", - "time_ms_rfc3339_1": "2019-12-16T12:10:30.123Z", - "time_ms_rfc3339_2": "2019-12-16T12:10:30.123+08:00", - "time_ms_clickhouse_1": "2019-12-16 12:10:30.123", - "array_int": [-9223372036854775808,9223372036854775807], - "array_float": [1.1,2.2,3.3], - "array_string": ["aa","bb","cc"], - "array_date": ["2000-01-01","2000-01-02","2000-01-03"], - "array_object": [{"i":[1,2,3],"f":[1.1,2.2,3.3]},{"s":["aa","bb","cc"],"e":[]}], - "array_empty": [], + "null": null, "bool_true": true, - "bool_false": false + "bool_false": false, + "num_int": 123, + "num_float": 123.321, + "str": "escaped_\"ws", + "str_int": "123", + "str_float": "123.321", + "str_date_1": "2009-07-13", + "str_date_2": "13/07/2009", + "str_time_rfc3339_1": "2009-07-13T09:07:13Z", + "str_time_rfc3339_2": "2009-07-13T09:07:13.123+08:00", + "str_time_clickhouse_1": "2009-07-13 09:07:13", + "str_time_clickhouse_2": "2009-07-13 09:07:13.123", + "obj": {"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}, + "array_empty": [], + "array_null": [null], + "array_bool": [true,false], + "array_num_int_1": [0, 255, 256, 65535, 65536, 4294967295, 4294967296, 18446744073709551615, 18446744073709551616], + "array_num_int_2": [-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807], + "array_num_float": [4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308], + "array_str": ["aa","bb","cc"], + "array_str_int_1": ["0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"], + "array_str_int_2": ["-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"], + "array_str_float": ["4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308"], + "array_str_date_1": ["2009-07-13","2009-07-14","2009-07-15"], + "array_str_date_2": ["13/07/2009","14/07/2009","15/07/2009"], + "array_str_time_rfc3339": ["2009-07-13T09:07:13Z", "2009-07-13T09:07:13+08:00", "2009-07-13T09:07:13.123Z", "2009-07-13T09:07:13.123+08:00"], + "array_str_time_clickhouse": ["2009-07-13 09:07:13", "2009-07-13 09:07:13.123"], + "array_obj": [{"i":[1,2,3],"f":[1.1,2.2,3.3]},{"s":["aa","bb","cc"],"e":[]}] }`) var jsonSchema = map[string]string{ - "its": "number", - "_ip": "string", - "cgi": "string", - "channel": "string", - "platform": "string", - "experiment": "string", - "ip": "string", - "version": "string", - "success": "number", - "percent": "number", - "mp": "object", - "date1": "string", - "time_sec_rfc3339_1": "string", - "time_sec_rfc3339_2": "string", - "time_sec_clickhouse_1": "string", - "time_ms_rfc3339_1": "string", - "time_ms_rfc3339_2": "string", - "time_ms_clickhouse_1": "string", - "array_int": "array", - "array_float": "array", - "array_string": "array", - "array_date": "array", - "array_object": "array", - "array_empty": "array", - "bool_true": "true", - "bool_false": "false", + "null": "null", + "bool_true": "true", + "bool_false": "false", + "num_int": "number", + "num_float": "number", + "str": "string", + "str_int": "string", + "str_float": "string", + "str_date_1": "string", + "str_date_2": "string", + "str_time_rfc3339_1": "string", + "str_time_rfc3339_2": "string", + "str_time_clickhouse_1": "string", + "str_time_clickhouse_2": "string", + "obj": "object", + "array_empty": "array", + "array_null": "array", + "array_bool": "array", + "array_num_int_1": "array", + "array_num_int_2": "array", + "array_num_float": "array", + "array_str": "array", + "array_str_int_1": "array", + "array_str_int_2": "array", + "array_str_float": "array", + "array_str_date_1": "array", + "array_str_date_2": "array", + "array_str_time_rfc3339": "array", + "array_str_time_clickhouse": "array", + "array_obj": "array", } -var csvSample = []byte(`1536813227,"0.11","escaped_""ws","{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",2019-12-16,2019-12-16T12:10:30Z,2019-12-16T12:10:30+08:00,2019-12-16 12:10:30,2019-12-16T12:10:30.123Z,2019-12-16T12:10:30.123+08:00,2019-12-16 12:10:30.123,"[-9223372036854775808,9223372036854775807]","[1.1,2.2,3.3]","[""aa"",""bb"",""cc""]","[""2000-01-01"",""2000-01-02"",""2000-01-03""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]","[]","true","false"`) +var csvSample = []byte(`null,true,false,123,123.321,"escaped_""ws",123,123.321,2009-07-13,13/07/2009,2009-07-13T09:07:13Z,2009-07-13T09:07:13.123+08:00,2009-07-13 09:07:13,2009-07-13 09:07:13.123,"{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",[],[null],"[true,false]","[0,255,256,65535,65536,4294967295,4294967296,18446744073709551615,18446744073709551616]","[-9223372036854775808,-2147483649,-2147483648,-32769,-32768,-129,-128,0,127,128,32767,32768,2147483647,2147483648,9223372036854775807]","[4.940656458412465441765687928682213723651e-324,1.401298464324817070923729583289916131280e-45,0.0,3.40282346638528859811704183484516925440e+38,1.797693134862315708145274237317043567981e+308]","[""aa"",""bb"",""cc""]","[""0"",""255"",""256"",""65535"",""65536"",""4294967295"",""4294967296"",""18446744073709551615"",""18446744073709551616""]","[""-9223372036854775808"",""-2147483649"",""-2147483648"",""-32769"",""-32768"",""-129"",""-128"",""0"",""127"",""128"",""32767"",""32768"",""2147483647"",""2147483648"",""9223372036854775807""]","[""4.940656458412465441765687928682213723651e-324"",""1.401298464324817070923729583289916131280e-45"",""0.0"",""3.40282346638528859811704183484516925440e+38"",""1.797693134862315708145274237317043567981e+308""]","[""2009-07-13"",""2009-07-14"",""2009-07-15""]","[""13/07/2009"",""14/07/2009"",""15/07/2009""]","[""2009-07-13T09:07:13Z"",""2009-07-13T09:07:13+08:00"",""2009-07-13T09:07:13.123Z"",""2009-07-13T09:07:13.123+08:00""]","[""2009-07-13 09:07:13"",""2009-07-13 09:07:13.123""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]"`) var csvSchema = []string{ - "its", - "percent", - "channel", - "mp", - "date1", - "time_sec_rfc3339_1", - "time_sec_rfc3339_2", - "time_sec_clickhouse_1", - "time_ms_rfc3339_1", - "time_ms_rfc3339_2", - "time_ms_clickhouse_1", - "array_int", - "array_float", - "array_string", - "array_date", - "array_object", - "array_empty", + "null", "bool_true", "bool_false", + "num_int", + "num_float", + "str", + "str_int", + "str_float", + "str_date_1", + "str_date_2", + "str_time_rfc3339_1", + "str_time_rfc3339_2", + "str_time_clickhouse_1", + "str_time_clickhouse_2", + "obj", + "array_empty", + "array_null", + "array_bool", + "array_num_int_1", + "array_num_int_2", + "array_num_float", + "array_str", + "array_str_int_1", + "array_str_int_2", + "array_str_float", + "array_str_date_1", + "array_str_date_2", + "array_str_time_rfc3339", + "array_str_time_clickhouse", + "array_obj", } +var ( + bdUtcNs = time.Date(2009, 7, 13, 9, 7, 13, 123000000, time.UTC) + bdUtcSec = bdUtcNs.Truncate(1 * time.Second) + bdShNsOrig = time.Date(2009, 7, 13, 9, 7, 13, 123000000, time.FixedZone("CST", 8*60*60)) + bdShNs = bdShNsOrig.UTC() + bdShSec = bdShNsOrig.Truncate(1 * time.Second).UTC() + bdLocalNsOrig = time.Date(2009, 7, 13, 9, 7, 13, 123000000, time.Local) + bdLocalNs = bdLocalNsOrig.UTC() + bdLocalSec = bdLocalNsOrig.Truncate(1 * time.Second).UTC() + bdLocalDate = time.Date(2009, 7, 13, 0, 0, 0, 0, time.Local).UTC() +) + var initialize sync.Once var initErr error -var names = []string{"csv", "fastjson", "gjson"} -var pools map[string]*Pool -var parsers map[string]Parser +var names = []string{"fastjson", "gjson", "csv"} var metrics map[string]model.Metric type SimpleCase struct { @@ -128,25 +160,16 @@ type ArrayCase struct { ExpVal interface{} } -func Bool2Str(b bool) string { - if b { - return "true" - } - return "false" -} - func initMetrics() { var pp *Pool var parser Parser var metric model.Metric var sample []byte - pools = make(map[string]*Pool) - parsers = make(map[string]Parser) metrics = make(map[string]model.Metric) for _, name := range names { switch name { case "csv": - pp, _ = NewParserPool("csv", csvSchema, "", "") + pp, _ = NewParserPool("csv", csvSchema, ",", "") sample = csvSample case "fastjson": pp, _ = NewParserPool("fastjson", nil, "", "") @@ -157,15 +180,22 @@ func initMetrics() { } parser = pp.Get() if metric, initErr = parser.Parse(sample); initErr != nil { - msg := fmt.Sprintf("%+v", initErr) + msg := fmt.Sprintf("parser.Parse failed: %+v\n", initErr) panic(msg) } - pools[name] = pp - parsers[name] = parser metrics[name] = metric } } +func sliceContains(list []string, target string) bool { + for _, s := range list { + if s == target { + return true + } + } + return false +} + func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { t.Helper() initialize.Do(initMetrics) @@ -175,7 +205,11 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { metric := metrics[name] for j := range testCases { var v interface{} - desc := fmt.Sprintf(`%s %s("%s", %s)`, name, method, testCases[j].Field, Bool2Str(testCases[j].Nullable)) + desc := fmt.Sprintf(`%s %s("%s", %s)`, name, method, testCases[j].Field, strconv.FormatBool(testCases[j].Nullable)) + if name == "csv" && sliceContains([]string{"GetInt", "GetFloat", "GetDateTime", "GetElasticDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) { + log.Printf("%s is known to not compatible with fastjson parser, skipping", desc) + continue + } switch method { case "GetInt": v = metric.GetInt(testCases[j].Field, testCases[j].Nullable) @@ -197,57 +231,182 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { func TestParserInt(t *testing.T) { testCases := []SimpleCase{ - {"its", false, int64(1536813227)}, + // nullable: false {"not_exist", false, int64(0)}, + {"null", false, int64(0)}, + {"bool_true", false, int64(1)}, + {"bool_false", false, int64(0)}, + {"num_int", false, int64(123)}, + {"num_float", false, int64(0)}, + {"str", false, int64(0)}, + {"str_int", false, int64(0)}, + {"str_float", false, int64(0)}, + {"str_date_1", false, int64(0)}, + {"obj", false, int64(0)}, + {"array_empty", false, int64(0)}, + // nullable: true {"not_exist", true, nil}, - {"channel", false, int64(0)}, + {"null", true, nil}, + {"bool_true", true, int64(1)}, + {"bool_false", true, int64(0)}, + {"num_int", true, int64(123)}, + {"num_float", true, int64(0)}, + {"str", true, int64(0)}, + {"str_int", true, int64(0)}, + {"str_float", true, int64(0)}, + {"str_date_1", true, int64(0)}, + {"obj", true, int64(0)}, + {"array_empty", true, int64(0)}, } doTestSimple(t, "GetInt", testCases) } func TestParserFloat(t *testing.T) { testCases := []SimpleCase{ - {"percent", false, 0.11}, + // nullable: false {"not_exist", false, 0.0}, + {"null", false, 0.0}, + {"bool_true", false, 0.0}, + {"bool_false", false, 0.0}, + {"num_int", false, 123.0}, + {"num_float", false, 123.321}, + {"str", false, 0.0}, + {"str_int", false, 0.0}, + {"str_float", false, 0.0}, + {"str_date_1", false, 0.0}, + {"obj", false, 0.0}, + {"array_empty", false, 0.0}, + // nullable: true {"not_exist", true, nil}, - {"channel", false, 0.0}, + {"null", true, nil}, + {"bool_true", true, 0.0}, + {"bool_false", true, 0.0}, + {"num_int", true, 123.0}, + {"num_float", true, 123.321}, + {"str", true, 0.0}, + {"str_int", true, 0.0}, + {"str_float", true, 0.0}, + {"str_date_1", true, 0.0}, + {"obj", true, 0.0}, + {"array_empty", true, 0.0}, } doTestSimple(t, "GetFloat", testCases) } func TestParserString(t *testing.T) { testCases := []SimpleCase{ - {"channel", false, "escaped_\"ws"}, + // nullable: false {"not_exist", false, ""}, + {"null", false, ""}, + {"bool_true", false, "true"}, + {"bool_false", false, "false"}, + {"num_int", false, "123"}, + {"num_float", false, "123.321"}, + {"str", false, `escaped_"ws`}, + {"str_int", false, "123"}, + {"str_float", false, "123.321"}, + {"str_date_1", false, "2009-07-13"}, + {"obj", false, `{"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}`}, + {"array_empty", false, "[]"}, + {"array_null", false, "[null]"}, + {"array_bool", false, "[true,false]"}, + {"array_str", false, `["aa","bb","cc"]`}, + // nullable: true {"not_exist", true, nil}, - {"its", false, "1536813227"}, - {"array_int", false, "[-9223372036854775808,9223372036854775807]"}, - {"array_string", false, `["aa","bb","cc"]`}, - {"mp", false, `{"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}`}, + {"null", true, nil}, + {"bool_true", true, "true"}, + {"bool_false", true, "false"}, + {"num_int", true, "123"}, + {"num_float", true, "123.321"}, + {"str", true, `escaped_"ws`}, + {"str_int", true, "123"}, + {"str_float", true, "123.321"}, + {"str_date_1", true, "2009-07-13"}, + {"obj", true, `{"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}`}, + {"array_empty", true, "[]"}, + {"array_null", true, "[null]"}, + {"array_bool", true, "[true,false]"}, + {"array_str", true, `["aa","bb","cc"]`}, } doTestSimple(t, "GetString", testCases) } func TestParserDateTime(t *testing.T) { testCases := []SimpleCase{ - {"date1", false, time.Date(2019, 12, 16, 0, 0, 0, 0, time.Local).In(time.UTC)}, - {"time_sec_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC)}, - {"time_sec_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.FixedZone("CST", 8*60*60)).In(time.UTC)}, - {"time_sec_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.Local).In(time.UTC)}, - {"time_ms_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.UTC)}, - {"time_ms_rfc3339_2", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.FixedZone("CST", 8*60*60)).In(time.UTC)}, - {"time_ms_clickhouse_1", false, time.Date(2019, 12, 16, 12, 10, 30, 123000000, time.Local).In(time.UTC)}, + // nullable: false {"not_exist", false, Epoch}, - {"array_int", false, Epoch}, + {"null", false, Epoch}, + {"bool_true", false, Epoch}, + {"bool_false", false, Epoch}, + {"num_int", false, UnixInt(123)}, + {"num_float", false, UnixFloat(123.321)}, + {"str", false, Epoch}, + {"str_int", false, Epoch}, + {"str_float", false, Epoch}, + {"str_date_1", false, bdLocalDate}, + {"str_time_rfc3339_1", false, bdUtcSec}, + {"str_time_rfc3339_2", false, bdShNs}, + {"str_time_clickhouse_1", false, bdLocalSec}, + {"str_time_clickhouse_2", false, bdLocalNs}, + {"obj", false, Epoch}, + {"array_empty", false, Epoch}, + // nullable: true + {"not_exist", true, nil}, + {"null", true, nil}, + {"bool_true", true, Epoch}, + {"bool_false", true, Epoch}, + {"num_int", true, UnixInt(123)}, + {"num_float", true, UnixFloat(123.321)}, + {"str", true, Epoch}, + {"str_int", true, Epoch}, + {"str_float", true, Epoch}, + {"str_date_1", true, bdLocalDate}, + {"str_time_rfc3339_1", true, bdUtcSec}, + {"str_time_rfc3339_2", true, bdShNs}, + {"str_time_clickhouse_1", true, bdLocalSec}, + {"str_time_clickhouse_2", true, bdLocalNs}, + {"obj", true, Epoch}, + {"array_empty", true, Epoch}, } doTestSimple(t, "GetDateTime", testCases) } func TestParserElasticDateTime(t *testing.T) { testCases := []SimpleCase{ - {"time_ms_rfc3339_1", false, time.Date(2019, 12, 16, 12, 10, 30, 0, time.UTC).Unix()}, - {"not_exist", false, int64(0)}, + // nullable: false + {"not_exist", false, Epoch.Unix()}, + {"null", false, Epoch.Unix()}, + {"bool_true", false, Epoch.Unix()}, + {"bool_false", false, Epoch.Unix()}, + {"num_int", false, UnixInt(123).Unix()}, + {"num_float", false, UnixFloat(123.321).Unix()}, + {"str", false, Epoch.Unix()}, + {"str_int", false, Epoch.Unix()}, + {"str_float", false, Epoch.Unix()}, + {"str_date_1", false, bdLocalDate.Unix()}, + {"str_time_rfc3339_1", false, bdUtcSec.Unix()}, + {"str_time_rfc3339_2", false, bdShNs.Unix()}, + {"str_time_clickhouse_1", false, bdLocalSec.Unix()}, + {"str_time_clickhouse_2", false, bdLocalNs.Unix()}, + {"obj", false, Epoch.Unix()}, + {"array_empty", false, Epoch.Unix()}, + // nullable: true {"not_exist", true, nil}, + {"null", true, nil}, + {"bool_true", true, Epoch.Unix()}, + {"bool_false", true, Epoch.Unix()}, + {"num_int", true, UnixInt(123).Unix()}, + {"num_float", true, UnixFloat(123.321).Unix()}, + {"str", true, Epoch.Unix()}, + {"str_int", true, Epoch.Unix()}, + {"str_float", true, Epoch.Unix()}, + {"str_date_1", true, bdLocalDate.Unix()}, + {"str_time_rfc3339_1", true, bdUtcSec.Unix()}, + {"str_time_rfc3339_2", true, bdShNs.Unix()}, + {"str_time_clickhouse_1", true, bdLocalSec.Unix()}, + {"str_time_clickhouse_2", true, bdLocalNs.Unix()}, + {"obj", true, Epoch.Unix()}, + {"array_empty", true, Epoch.Unix()}, } doTestSimple(t, "GetElasticDateTime", testCases) } @@ -255,29 +414,78 @@ func TestParserElasticDateTime(t *testing.T) { func TestParserArray(t *testing.T) { initialize.Do(initMetrics) require.Nil(t, initErr) - var ts []time.Time - for _, e := range []string{"2000-01-01", "2000-01-02", "2000-01-03"} { - t, _ := parseInLocation(e, time.Local) - ts = append(ts, t) - } + testCases := []ArrayCase{ - {"array_int", model.Float, []float64{-9223372036854775808, 9223372036854775807}}, - {"array_int", model.Int, []int64{-9223372036854775808, 9223372036854775807}}, - {"array_float", model.Float, []float64{1.1, 2.2, 3.3}}, - {"array_string", model.String, []string{"aa", "bb", "cc"}}, - {"array_date", model.DateTime, ts}, - {"array_object", model.String, []string{`{"i":[1,2,3],"f":[1.1,2.2,3.3]}`, `{"s":["aa","bb","cc"],"e":[]}`}}, + {"not_exist", model.Float, []float64{}}, + {"null", model.Float, []float64{}}, + {"num_int", model.Int, []int64{}}, + {"num_float", model.Float, []float64{}}, + {"str", model.String, []string{}}, + {"str_int", model.String, []string{}}, + {"str_date_1", model.DateTime, []time.Time{}}, + {"obj", model.String, []string{}}, + {"array_empty", model.Int, []int64{}}, {"array_empty", model.Float, []float64{}}, {"array_empty", model.String, []string{}}, {"array_empty", model.DateTime, []time.Time{}}, + + {"array_null", model.Int, []int64{0}}, + {"array_null", model.Float, []float64{0.0}}, + {"array_null", model.String, []string{""}}, + {"array_null", model.DateTime, []time.Time{Epoch}}, + + {"array_bool", model.Int, []int64{1, 0}}, + {"array_bool", model.Float, []float64{0.0, 0.0}}, + {"array_bool", model.String, []string{"true", "false"}}, + {"array_bool", model.DateTime, []time.Time{Epoch, Epoch}}, + + {"array_num_int_1", model.Int, []int64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 0, 0}}, + {"array_num_int_1", model.Float, []float64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 18446744073709551615, 18446744073709551616}}, + {"array_num_int_1", model.String, []string{"0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"}}, + {"array_num_int_1", model.DateTime, []time.Time{Epoch, UnixInt(255), UnixInt(256), UnixInt(65535), UnixInt(65536), UnixInt(4294967295), UnixInt(4294967296), Epoch, Epoch}}, + + {"array_num_int_2", model.Int, []int64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, + {"array_num_int_2", model.Float, []float64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, + {"array_num_int_2", model.String, []string{"-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"}}, + {"array_num_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, UnixInt(127), UnixInt(128), UnixInt(32767), UnixInt(32768), UnixInt(2147483647), UnixInt(2147483648), UnixInt(9223372036854775807)}}, + + {"array_num_float", model.Int, []int64{0, 0, 0, 0, 0}}, + {"array_num_float", model.Float, []float64{4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308}}, + {"array_num_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308"}}, + {"array_num_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, UnixFloat(3.40282346638528859811704183484516925440e+38), UnixFloat(1.797693134862315708145274237317043567981e+308)}}, + + {"array_str", model.Int, []int64{0, 0, 0}}, + {"array_str", model.Float, []float64{0.0, 0.0, 0.0}}, + {"array_str", model.String, []string{"aa", "bb", "cc"}}, + {"array_str", model.DateTime, []time.Time{Epoch, Epoch, Epoch}}, + + {"array_str_int_1", model.Int, []int64{0, 0, 0, 0, 0, 0, 0, 0, 0}}, + {"array_str_int_1", model.Float, []float64{0, 0, 0, 0, 0, 0, 0, 0, 0}}, + {"array_str_int_1", model.String, []string{"0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"}}, + {"array_str_int_1", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch}}, + + {"array_str_int_2", model.Int, []int64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}}, + {"array_str_int_2", model.Float, []float64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}}, + {"array_str_int_2", model.String, []string{"-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"}}, + {"array_str_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch}}, + + {"array_str_float", model.Int, []int64{0, 0, 0, 0, 0}}, + {"array_str_float", model.Float, []float64{0, 0, 0, 0, 0}}, + {"array_str_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308"}}, + {"array_str_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch}}, + + {"array_str_date_1", model.DateTime, []time.Time{bdLocalDate, bdLocalDate.Add(24 * time.Hour), bdLocalDate.Add(48 * time.Hour)}}, + {"array_str_date_2", model.DateTime, []time.Time{bdLocalDate, bdLocalDate.Add(24 * time.Hour), bdLocalDate.Add(48 * time.Hour)}}, + {"array_str_time_rfc3339", model.DateTime, []time.Time{bdUtcSec, bdShSec, bdUtcNs, bdShNs}}, + {"array_str_time_clickhouse", model.DateTime, []time.Time{bdLocalSec, bdLocalNs}}, } for i := range names { name := names[i] metric := metrics[name] for j := range testCases { - if name == "csv" && testCases[j].Field == "array_object" { + if name == "csv" && testCases[j].Field == "array_obj" { // csv parser doesn't support object array yet. continue } @@ -290,15 +498,13 @@ func TestParserArray(t *testing.T) { } func BenchmarkUnmarshalljson(b *testing.B) { - mp := map[string]interface{}{} + object := map[string]interface{}{} for i := 0; i < b.N; i++ { - _ = json.Unmarshal(jsonSample, &mp) + _ = json.Unmarshal(jsonSample, &object) } } func BenchmarkUnmarshallFastJson(b *testing.B) { - // mp := map[string]interface{}{} - // var p fastjson.Parser str := string(jsonSample) var p fastjson.Parser for i := 0; i < b.N; i++ { @@ -306,53 +512,37 @@ func BenchmarkUnmarshallFastJson(b *testing.B) { if err != nil { panic(err) } - v.GetInt("its") - v.GetStringBytes("_ip") - v.GetStringBytes("cgi") - v.GetStringBytes("channel") - v.GetStringBytes("platform") - v.GetStringBytes("experiment") - v.GetStringBytes("ip") - v.GetStringBytes("version") - v.GetInt("success") - v.GetInt("trycount") + v.GetInt("null") + v.GetInt("bool_true") + v.GetInt("num_int") + v.GetFloat64("num_float") + v.GetStringBytes("str") + v.GetStringBytes("str_float") } } // 字段个数较少的情况下,直接Get性能更好 func BenchmarkUnmarshallGjson(b *testing.B) { - // mp := map[string]interface{}{} - // var p fastjson.Parser str := string(jsonSample) for i := 0; i < b.N; i++ { - _ = gjson.Get(str, "its").Int() - _ = gjson.Get(str, "_ip").String() - _ = gjson.Get(str, "cgi").String() - _ = gjson.Get(str, "channel").String() - _ = gjson.Get(str, "platform").String() - _ = gjson.Get(str, "experiment").String() - _ = gjson.Get(str, "ip").String() - _ = gjson.Get(str, "version").String() - _ = gjson.Get(str, "success").Int() - _ = gjson.Get(str, "trycount").Int() + _ = gjson.Get(str, "null").Int() + _ = gjson.Get(str, "bool_true").Int() + _ = gjson.Get(str, "num_int").Int() + _ = gjson.Get(str, "num_float").Float() + _ = gjson.Get(str, "str").String() + _ = gjson.Get(str, "str_float").String() } } func BenchmarkUnmarshalGabon2(b *testing.B) { - // mp := map[string]interface{}{} - // var p fastjson.Parser str := string(jsonSample) for i := 0; i < b.N; i++ { result := gjson.Parse(str).Map() - _ = result["its"].Int() - _ = result["_ip"].String() - _ = result["cgi"].String() - _ = result["channel"].String() - _ = result["platform"].String() - _ = result["experiment"].String() - _ = result["ip"].String() - _ = result["version"].String() - _ = result["success"].Int() - _ = result["trycount"].Int() + _ = result["null"].Int() + _ = result["bool_true"].Int() + _ = result["num_int"].Int() + _ = result["num_float"].Float() + _ = result["str"].String() + _ = result["str_float"].String() } } From 666635d577e8915aea3942a2e8ce476364349f2b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 14 May 2021 17:57:50 +0800 Subject: [PATCH 100/404] fixed applyConfig loop --- cmd/clickhouse_sinker/main.go | 8 ++++---- pool/conn.go | 12 +++++++++--- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index a48e8d09..5ea8bdf8 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -269,18 +269,18 @@ func (s *Sinker) Run() { select { case <-s.ctx.Done(): return - case <-time.After(5 * time.Second): + case <-time.After(10 * time.Second): if newCfg, err = s.rcm.GetConfig(); err != nil { util.Logger.Error("s.rcm.GetConfig failed", zap.Error(err)) - return + continue } if err = newCfg.Normallize(); err != nil { util.Logger.Error("newCfg.Normallize failed", zap.Error(err)) - return + continue } if err = s.applyConfig(newCfg); err != nil { util.Logger.Error("s.applyConfig failed", zap.Error(err)) - return + continue } } } diff --git a/pool/conn.go b/pool/conn.go index 79a03a38..692aadf4 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -65,6 +65,7 @@ func (c *ShardConn) NextGoodReplica() error { for i := 0; i < len(c.Replicas); i++ { c.Dsn = fmt.Sprintf(dsnTmpl, c.Replicas[c.NextRep]) c.NextRep = (c.NextRep + 1) % len(c.Replicas) + util.Logger.Info("sql.Open", zap.String("dsn", c.Dsn)) sqlDB, err := sql.Open("clickhouse", c.Dsn) if err != nil { util.Logger.Warn("sql.Open failed", zap.String("dsn", c.Dsn), zap.Error(err)) @@ -85,6 +86,7 @@ func (c *ShardConn) NextGoodReplica() error { func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool) (err error) { lock.Lock() defer lock.Unlock() + freeClusterConn() // Each shard has a *sql.DB which connects to one replica inside the shard. // "alt_hosts" tolerates replica single-point-failure. However more flexable switching is needed for some cases for example https://github.com/ClickHouse/ClickHouse/issues/24036. dsnTmpl = "tcp://%s" + fmt.Sprintf("?database=%s&username=%s&password=%s&block_size=%d", @@ -124,9 +126,7 @@ func setDBParams(sqlDB *sql.DB) { sqlDB.SetConnMaxIdleTime(10 * time.Second) } -func FreeClusterConn() { - lock.Lock() - defer lock.Unlock() +func freeClusterConn() { for _, sc := range clusterConn { if sc.DB != nil { if err := health.Health.RemoveReadinessCheck(sc.Dsn); err != nil { @@ -138,6 +138,12 @@ func FreeClusterConn() { clusterConn = []*ShardConn{} } +func FreeClusterConn() { + lock.Lock() + defer lock.Unlock() + freeClusterConn() +} + func NumShard() (cnt int) { lock.Lock() defer lock.Unlock() From 878540b486b0e91b12916e9d39ce20bca5c0915b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 14 May 2021 22:12:40 +0800 Subject: [PATCH 101/404] changed log time format --- parser/parser.go | 10 +++++++--- util/common.go | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/parser/parser.go b/parser/parser.go index 71226319..a9b50b97 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -29,15 +29,19 @@ var ( Layouts = []string{ //DateTime "2006-01-02T15:04:05.999999999Z07:00", //time.RFC3339Nano, `date --iso-8601=ns` output format - "2006-01-02T15:04:05Z07:00", //time.RFC3339, `date --iso-8601=s` on Ubuntu 20.04 - "2006-01-02T15:04:05-0700", //`date --iso-8601=s` on CentOS 7.6 + "2006-01-02T15:04:05.999999999Z0700", + "2006-01-02T15:04:05Z07:00", //time.RFC3339, `date --iso-8601=s` on Ubuntu 20.04 + "2006-01-02T15:04:05Z0700", //`date --iso-8601=s` on CentOS 7.6 "2006-01-02T15:04:05.999999999", "2006-01-02T15:04:05", "2006-01-02 15:04:05.999999999Z07:00", //`date --rfc-3339=ns` output format - "2006-01-02 15:04:05Z07:00", //`date --rfc-3339=s` output format + "2006-01-02 15:04:05.999999999Z0700", + "2006-01-02 15:04:05Z07:00", //`date --rfc-3339=s` output format + "2006-01-02 15:04:05Z0700", "2006-01-02 15:04:05.999999999", "2006-01-02 15:04:05", "Jan 02, 2006 15:04:05.999999999Z07:00", + "Jan 02, 2006 15:04:05.999999999Z0700", "Jan 02, 2006 15:04:05.999999999", "Jan 02, 2006 15:04:05", "Jan 02, 2006 03:04:05 PM", diff --git a/util/common.go b/util/common.go index 95f61925..91ace37e 100644 --- a/util/common.go +++ b/util/common.go @@ -241,7 +241,7 @@ func InitLogger(logLevel string, logPaths []string) { } cfg := zap.NewProductionEncoderConfig() - cfg.EncodeTime = zapcore.RFC3339NanoTimeEncoder + cfg.EncodeTime = zapcore.ISO8601TimeEncoder core := zapcore.NewCore( zapcore.NewJSONEncoder(cfg), zapcore.NewMultiWriteSyncer(syncers...), From 658f476dbf3aca0ef9dd370ad4e36942c5a4e92a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 14 May 2021 23:34:08 +0800 Subject: [PATCH 102/404] let OS choose the listen port, and record the exact metrics URL to log --- cmd/clickhouse_sinker/main.go | 18 ++++++++++++++---- util/common.go | 11 +++++++++++ 2 files changed, 25 insertions(+), 4 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 5ea8bdf8..933d3296 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -75,7 +75,7 @@ func initCmdOptions() { // 1. Set options to default value. cmdOps = CmdOptions{ ShowVer: false, - HTTPPort: 2112, + HTTPPort: 0, // 0 menas a randomly OS chosen port PushGatewayAddrs: "", PushInterval: 10, LocalCfgFile: "/etc/clickhouse_sinker.json", @@ -134,7 +134,6 @@ func init() { log.Fatal("unable to determine self ip", err) } selfIP = ip.String() - cmdOps.HTTPPort = util.GetSpareTCPPort(cmdOps.HTTPPort) } // GenTask generate a task via config @@ -198,8 +197,19 @@ func main() { mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) mux.HandleFunc("/debug/pprof/trace", pprof.Trace) - util.Logger.Info(fmt.Sprintf("Run http server http://%s:%d", selfIP, cmdOps.HTTPPort)) - if err := http.ListenAndServe(fmt.Sprintf(":%d", cmdOps.HTTPPort), mux); err != nil { + // cmdOps.HTTPPort=0: let OS choose the listen port, and record the exact metrics URL to log. + httpPort := cmdOps.HTTPPort + if httpPort != 0 { + httpPort = util.GetSpareTCPPort(httpPort) + } + httpAddr := fmt.Sprintf(":%d", httpPort) + listener, err := net.Listen("tcp", httpAddr) + if err != nil { + util.Logger.Fatal("net.Listen failed", zap.String("httpAddr", httpAddr), zap.Error(err)) + } + httpPort = util.GetNetAddrPort(listener.Addr()) + util.Logger.Info(fmt.Sprintf("Run metrics server at http://%s:%d/metrics", selfIP, httpPort)) + if err := http.Serve(listener, mux); err != nil { util.Logger.Error("http.ListenAndServe failed", zap.Error(err)) } }() diff --git a/util/common.go b/util/common.go index 91ace37e..c5f001c1 100644 --- a/util/common.go +++ b/util/common.go @@ -115,6 +115,17 @@ LOOP: return } +// https://stackoverflow.com/questions/50428176/how-to-get-ip-and-port-from-net-addr-when-it-could-be-a-net-udpaddr-or-net-tcpad +func GetNetAddrPort(addr net.Addr) (port int) { + switch addr := addr.(type) { + case *net.UDPAddr: + port = addr.Port + case *net.TCPAddr: + port = addr.Port + } + return +} + // Refers to: // https://medium.com/processone/using-tls-authentication-for-your-go-kafka-client-3c5841f2a625 // https://github.com/denji/golang-tls From 09f7021f4818b464fe0a1ec1efaa43be22ea0999 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 15 May 2021 00:19:46 +0800 Subject: [PATCH 103/404] handle applyFirstConfig failure properly --- cmd/clickhouse_sinker/main.go | 81 ++++++++++++++++------------------- go.mod | 2 +- go.sum | 4 +- task/task.go | 5 +++ util/common.go | 31 +++++++++++++- 5 files changed, 74 insertions(+), 49 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 933d3296..0379559f 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -25,7 +25,6 @@ import ( "net/http/pprof" "os" "reflect" - "runtime" "strings" "time" @@ -34,7 +33,6 @@ import ( "github.com/housepower/clickhouse_sinker/input" "github.com/housepower/clickhouse_sinker/output" "github.com/housepower/clickhouse_sinker/parser" - "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/task" "github.com/housepower/clickhouse_sinker/util" @@ -184,6 +182,7 @@ func main() {

Ready Full

Live

Live Full

+

pprof

`)) }) @@ -208,7 +207,8 @@ func main() { util.Logger.Fatal("net.Listen failed", zap.String("httpAddr", httpAddr), zap.Error(err)) } httpPort = util.GetNetAddrPort(listener.Addr()) - util.Logger.Info(fmt.Sprintf("Run metrics server at http://%s:%d/metrics", selfIP, httpPort)) + util.Logger.Info(fmt.Sprintf("Run http server at http://%s:%d/", selfIP, httpPort)) + if err := http.Serve(listener, mux); err != nil { util.Logger.Error("http.ListenAndServe failed", zap.Error(err)) } @@ -300,55 +300,50 @@ func (s *Sinker) Run() { // Close shutdown task func (s *Sinker) Close() { // Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). - util.Logger.Info("stopping parsing pool", zap.String("task", s.curCfg.Task.Name)) + util.Logger.Info("stopping parsing pool") util.GlobalParsingPool.StopWait() - util.Logger.Info("stopping writing pool", zap.String("task", s.curCfg.Task.Name)) + util.Logger.Info("stopping writing pool") util.GlobalWritingPool.StopWait() - util.Logger.Info("stopping timer wheel", zap.String("task", s.curCfg.Task.Name)) + util.Logger.Info("stopping timer wheel") util.GlobalTimerWheel.Stop() - - s.task.Stop() + if s.task != nil { + s.task.Stop() + s.task = nil + } if s.pusher != nil { s.pusher.Stop() + s.pusher = nil } s.cancel() } func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { - if s.curCfg == nil || newCfg.LogLevel != s.curCfg.LogLevel || !reflect.DeepEqual(newCfg.LogPaths, s.curCfg.LogPaths) { - util.InitLogger(newCfg.LogLevel, newCfg.LogPaths) - } + util.InitLogger(newCfg.LogLevel, newCfg.LogPaths) if s.curCfg == nil { // The first time invoking of applyConfig err = s.applyFirstConfig(newCfg) } else if !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { err = s.applyAnotherConfig(newCfg) + } else { + util.Logger.Debug("got the same config", zap.Reflect("config", newCfg)) } return } func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { util.Logger.Info("going to apply the first config", zap.Reflect("config", newCfg)) + // 1. Start goroutine pools. util.InitGlobalTimerWheel() - t := GenTask(newCfg) - if err = t.Init(); err != nil { + util.InitGlobalParsingPool() + util.InitGlobalWritingPool(len(newCfg.Clickhouse.Hosts)) + + // 2. Generate, initialize and run task + s.task = GenTask(newCfg) + if err = s.task.Init(); err != nil { return } - s.task = t - concurrentParsers := 10 - if runtime.NumCPU() >= 2 { - if concurrentParsers > runtime.NumCPU()/2 { - concurrentParsers = runtime.NumCPU() / 2 - } - } else { - concurrentParsers = 1 - } - util.InitGlobalParsingPool(concurrentParsers) - totalConn := pool.NumShard() - util.InitGlobalWritingPool(totalConn) - - go s.task.Run(s.ctx) s.curCfg = newCfg + go s.task.Run(s.ctx) return } @@ -357,32 +352,30 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { // 1. Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). - util.Logger.Info("stopping parsing pool", zap.String("task", s.curCfg.Task.Name)) + util.Logger.Info("stopping parsing pool") util.GlobalParsingPool.StopWait() - util.Logger.Info("stopping writing pool", zap.String("task", s.curCfg.Task.Name)) + util.Logger.Info("stopping writing pool") util.GlobalWritingPool.StopWait() - util.Logger.Info("stopping timer wheel", zap.String("task", s.curCfg.Task.Name)) + util.Logger.Info("stopping timer wheel") util.GlobalTimerWheel.Stop() s.task.Stop() - // 2. Generate and initialize task - t := GenTask(newCfg) - if err = t.Init(); err != nil { - return - } - - // 3. Restart goroutine pools. + // 2. Restart goroutine pools. + util.Logger.Info("restarting parsing, writing and timer pool") util.InitGlobalTimerWheel() util.GlobalParsingPool.Restart() - totalConn := pool.NumShard() - util.GlobalWritingPool.Resize(totalConn) + util.GlobalWritingPool.Resize(len(newCfg.Clickhouse.Hosts)) util.GlobalWritingPool.Restart() + util.Logger.Info("resized parsing pool", zap.Int("maxWorkers", len(newCfg.Clickhouse.Hosts))) - // 4. Start task - go t.Run(s.ctx) - s.task = t + // 3. Generate, initialize and run task + s.task = GenTask(newCfg) + if err = s.task.Init(); err != nil { + return + } + // Record the new config + s.curCfg = newCfg + go s.task.Run(s.ctx) } - // Record the new config - s.curCfg = newCfg return } diff --git a/go.mod b/go.mod index d0994484..4551baa3 100644 --- a/go.mod +++ b/go.mod @@ -21,7 +21,7 @@ require ( github.com/valyala/fastjson v1.4.1 github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c go.uber.org/zap v1.15.0 - golang.org/x/sys v0.0.0-20200917061948-648f2a039071 // indirect + golang.org/x/sys v0.0.0-20210217105451-b926d437f341 // indirect golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) diff --git a/go.sum b/go.sum index 0e80d535..3db139c7 100644 --- a/go.sum +++ b/go.sum @@ -234,8 +234,8 @@ golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200917061948-648f2a039071 h1:t0H7WMwCt9t0LnLSYz5zdZ/OiAtROxc5cHb5iHt3Xyw= -golang.org/x/sys v0.0.0-20200917061948-648f2a039071/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210217105451-b926d437f341 h1:2/QtM1mL37YmcsT8HaDNHDgTqqFVw+zr8UzMiBVLzYU= +golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= diff --git a/task/task.go b/task/task.go index cddcbf66..8810eaef 100644 --- a/task/task.go +++ b/task/task.go @@ -78,6 +78,7 @@ func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *pa // Init initializes the kafak and clickhouse task associated with this service func (service *Service) Init() (err error) { + util.Logger.Info("task initializing", zap.String("task", service.cfg.Task.Name)) if err = service.clickhouse.Init(); err != nil { return } @@ -299,6 +300,10 @@ func (service *Service) changeSchema(arg interface{}) { // Stop stop kafka and clickhouse client. This is blocking. func (service *Service) Stop() { taskCfg := &service.cfg.Task + if !service.started { + util.Logger.Info("stopped a already stopped task service", zap.String("task", taskCfg.Name)) + return + } util.Logger.Info("stopping task service...", zap.String("task", taskCfg.Name)) service.cancel() if err := service.inputer.Stop(); err != nil { diff --git a/util/common.go b/util/common.go index c5f001c1..421a47e8 100644 --- a/util/common.go +++ b/util/common.go @@ -25,6 +25,7 @@ import ( "os" "os/exec" "path/filepath" + "reflect" "runtime" "strconv" "strings" @@ -43,25 +44,46 @@ var ( GlobalParsingPool *WorkerPool //for all tasks' parsing, cpu intensive GlobalWritingPool *WorkerPool //the all tasks' writing ClickHouse, cpu-net balance Logger *zap.Logger + logLevel string + logPaths []string ) // InitGlobalTimerWheel initialize the global timer wheel func InitGlobalTimerWheel() { + if GlobalTimerWheel != nil { + return + } GlobalTimerWheel = goetty.NewTimeoutWheel(goetty.WithTickInterval(time.Second)) } // InitGlobalParsingPool initialize GlobalParsingPool -func InitGlobalParsingPool(maxWorkers int) { +func InitGlobalParsingPool() { + if GlobalParsingPool != nil { + return + } + maxWorkers := 10 + if runtime.NumCPU() >= 2 { + if maxWorkers > runtime.NumCPU()/2 { + maxWorkers = runtime.NumCPU() / 2 + } + } else { + maxWorkers = 1 + } GlobalParsingPool = NewWorkerPool(maxWorkers, 100*runtime.NumCPU()) + Logger.Info("initialized parsing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", 100*runtime.NumCPU())) } // InitGlobalWritingPool initialize GlobalWritingPool func InitGlobalWritingPool(maxWorkers int) { + if GlobalWritingPool != nil { + return + } queueSize := runtime.NumCPU() / 4 if queueSize < 3 { queueSize = 3 } GlobalWritingPool = NewWorkerPool(maxWorkers, queueSize) + Logger.Info("initialized writing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", queueSize)) } // StringContains check if contains string in array @@ -229,7 +251,12 @@ func JksToPem(jksPath, jksPassword string, overwrite bool) (certPemPath, keyPemP return } -func InitLogger(logLevel string, logPaths []string) { +func InitLogger(newLogLevel string, newLogPaths []string) { + if logLevel == newLogLevel && reflect.DeepEqual(logPaths, newLogPaths) { + return + } + logLevel = newLogLevel + logPaths = newLogPaths var lvl zapcore.Level if err := lvl.Set(logLevel); err != nil { lvl = zap.InfoLevel From cdaccae51d9e459f72b0403f72dc791fc6db3c52 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 15 May 2021 20:25:55 +0800 Subject: [PATCH 104/404] sql.Open may just validate its arguments without creating a connection to the database. --- cmd/clickhouse_sinker/main.go | 2 -- pool/conn.go | 16 +++++++++++++--- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 0379559f..d2fabf05 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -324,8 +324,6 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { err = s.applyFirstConfig(newCfg) } else if !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { err = s.applyAnotherConfig(newCfg) - } else { - util.Logger.Debug("got the same config", zap.Reflect("config", newCfg)) } return } diff --git a/pool/conn.go b/pool/conn.go index 692aadf4..d818cc13 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -51,6 +51,11 @@ type ShardConn struct { NextRep int //index of next replica } +// NumReplica returns number of replicas of this shard +func (c *ShardConn) NumReplica() int { + return len(c.Replicas) +} + // NextGoodReplica connects to next good replica func (c *ShardConn) NextGoodReplica() error { if c.DB != nil { @@ -65,16 +70,21 @@ func (c *ShardConn) NextGoodReplica() error { for i := 0; i < len(c.Replicas); i++ { c.Dsn = fmt.Sprintf(dsnTmpl, c.Replicas[c.NextRep]) c.NextRep = (c.NextRep + 1) % len(c.Replicas) - util.Logger.Info("sql.Open", zap.String("dsn", c.Dsn)) sqlDB, err := sql.Open("clickhouse", c.Dsn) if err != nil { util.Logger.Warn("sql.Open failed", zap.String("dsn", c.Dsn), zap.Error(err)) continue } + // According to sql.Open doc, "Open may just validate its arguments without creating a connection + // to the database. To verify that the data source name is valid, call Ping." + if err := sqlDB.Ping(); err != nil { + util.Logger.Warn("sqlDB.Ping failed", zap.String("dsn", c.Dsn), zap.Error(err)) + continue + } setDBParams(sqlDB) - util.Logger.Info("sql.Open succeeded", zap.String("dsn", c.Dsn)) + util.Logger.Info("sql.Open and sqlDB.Ping succeeded", zap.String("dsn", c.Dsn)) if err = health.Health.AddReadinessCheck(c.Dsn, healthcheck.DatabasePingCheck(sqlDB, 30*time.Second)); err != nil { - util.Logger.Warn("health.Health.RemoveReadinessCheck failed", zap.String("dsn", c.Dsn), zap.Error(err)) + util.Logger.Warn("health.Health.AddReadinessCheck failed", zap.String("dsn", c.Dsn), zap.Error(err)) } c.DB = sqlDB return nil From 132985fa82cca650a464f022a4a3293d518a34ae Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 15 May 2021 21:15:37 +0800 Subject: [PATCH 105/404] ClickHouse.initSchema check if table exist --- output/clickhouse.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/output/clickhouse.go b/output/clickhouse.go index 9cc94c66..a0116578 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -197,6 +197,10 @@ func (c *ClickHouse) initSchema() (err error) { c.Dims = append(c.Dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, SourceName: util.GetSourceName(name)}) } } + if len(c.Dims) == 0 { + err = errors.Errorf("Table %s.%s doesn't exist", c.cfg.Clickhouse.DB, c.cfg.Task.TableName) + return + } } else { c.Dims = make([]*model.ColumnWithType, 0) for _, dim := range c.cfg.Task.Dims { From 8949345aa3894878d5ead63bba46e7d752e10983 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 15 May 2021 23:40:49 +0800 Subject: [PATCH 106/404] added TestParseDateTime --- parser/parser.go | 31 ++++++-------- parser/parser_test.go | 94 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 107 insertions(+), 18 deletions(-) diff --git a/parser/parser.go b/parser/parser.go index a9b50b97..d401daf1 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -27,28 +27,15 @@ import ( var ( Layouts = []string{ - //DateTime - "2006-01-02T15:04:05.999999999Z07:00", //time.RFC3339Nano, `date --iso-8601=ns` output format - "2006-01-02T15:04:05.999999999Z0700", + //DateTime, RFC3339 "2006-01-02T15:04:05Z07:00", //time.RFC3339, `date --iso-8601=s` on Ubuntu 20.04 "2006-01-02T15:04:05Z0700", //`date --iso-8601=s` on CentOS 7.6 - "2006-01-02T15:04:05.999999999", "2006-01-02T15:04:05", - "2006-01-02 15:04:05.999999999Z07:00", //`date --rfc-3339=ns` output format - "2006-01-02 15:04:05.999999999Z0700", + //DateTime, ISO8601 "2006-01-02 15:04:05Z07:00", //`date --rfc-3339=s` output format "2006-01-02 15:04:05Z0700", - "2006-01-02 15:04:05.999999999", "2006-01-02 15:04:05", - "Jan 02, 2006 15:04:05.999999999Z07:00", - "Jan 02, 2006 15:04:05.999999999Z0700", - "Jan 02, 2006 15:04:05.999999999", - "Jan 02, 2006 15:04:05", - "Jan 02, 2006 03:04:05 PM", - "02/01/2006 15:04:05.999999999", - "02/01/06 15:04:05.999999999", - "02/Jan/2006 15:04:05 Z07:00", - "02/Jan/2006 15:04:05 -0700", + //DateTime, other layouts supported by golang "Mon Jan _2 15:04:05 2006", //time.ANSIC "Mon Jan _2 15:04:05 MST 2006", //time.UnixDate "Mon Jan 02 15:04:05 -0700 2006", //time.RubyDate @@ -57,8 +44,16 @@ var ( "Monday, 02-Jan-06 15:04:05 MST", //time.RFC850 "Mon, 02 Jan 2006 15:04:05 MST", //time.RFC1123 "Mon, 02 Jan 2006 15:04:05 -0700", //time.RFC1123Z - "Mon Jan 02 15:04:05 MST 2006", - "Mon 02 Jan 2006 03:04:05 PM MST", // `date` default output format + //DateTime, linux utils + "Mon Jan 02 15:04:05 MST 2006", // `date` on CentOS 7.6 default output format + "Mon 02 Jan 2006 03:04:05 PM MST", // `date` on Ubuntu 20.4 default output format + //DateTime, home-brewed + "Jan 02, 2006 15:04:05Z07:00", + "Jan 02, 2006 15:04:05Z0700", + "Jan 02, 2006 15:04:05", + "02/Jan/2006 15:04:05 Z07:00", + "02/Jan/2006 15:04:05 Z0700", + "02/Jan/2006 15:04:05", //Date "2006-01-02", "02/01/2006", diff --git a/parser/parser_test.go b/parser/parser_test.go index 50fd86fb..de7ed92c 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -19,6 +19,7 @@ import ( "fmt" "log" "strconv" + "strings" "sync" "testing" "time" @@ -137,6 +138,7 @@ var ( bdShNsOrig = time.Date(2009, 7, 13, 9, 7, 13, 123000000, time.FixedZone("CST", 8*60*60)) bdShNs = bdShNsOrig.UTC() bdShSec = bdShNsOrig.Truncate(1 * time.Second).UTC() + bdShMin = bdShNsOrig.Truncate(1 * time.Minute).UTC() bdLocalNsOrig = time.Date(2009, 7, 13, 9, 7, 13, 123000000, time.Local) bdLocalNs = bdLocalNsOrig.UTC() bdLocalSec = bdLocalNsOrig.Truncate(1 * time.Second).UTC() @@ -160,6 +162,11 @@ type ArrayCase struct { ExpVal interface{} } +type DateTimeCase struct { + TS string + ExpVal time.Time +} + func initMetrics() { var pp *Pool var parser Parser @@ -497,6 +504,93 @@ func TestParserArray(t *testing.T) { } } +func TestParseDateTime(t *testing.T) { + // https://en.wikipedia.org/wiki/List_of_tz_database_time_zones + // https://en.wikipedia.org/wiki/List_of_time_zone_abbreviations, "not part of the international time and date standard ISO 8601 and their use as sole designator for a time zone is discouraged." + savedLocal := time.Local + defer func() { + time.Local = savedLocal + }() + locations := []string{"UTC", "Asia/Shanghai", "Europe/Moscow", "America/Los_Angeles"} + for _, location := range locations { + // change timezone settings programmatically + loc, err := time.LoadLocation(location) + require.Nil(t, err, fmt.Sprintf("time.LoadLocation(%s)", location)) + time.Local = loc + + bdLocalNsOrig = time.Date(2009, 7, 13, 9, 7, 13, 123000000, time.Local) + bdLocalNs = bdLocalNsOrig.UTC() + bdLocalSec = bdLocalNsOrig.Truncate(1 * time.Second).UTC() + bdLocalDate = time.Date(2009, 7, 13, 0, 0, 0, 0, time.Local).UTC() + + testCases := []DateTimeCase{ + //DateTime, RFC3339 + {"2009-07-13T09:07:13.123+08:00", bdShNs}, + {"2009-07-13T09:07:13.123+0800", bdShNs}, + {"2009-07-13T09:07:13+08:00", bdShSec}, + {"2009-07-13T09:07:13+0800", bdShSec}, + {"2009-07-13T09:07:13.123Z", bdUtcNs}, + {"2009-07-13T09:07:13Z", bdUtcSec}, + {"2009-07-13T09:07:13.123", bdLocalNs}, + {"2009-07-13T09:07:13", bdLocalSec}, + //DateTime, ISO8601 + {"2009-07-13 09:07:13.123+08:00", bdShNs}, + {"2009-07-13 09:07:13.123+0800", bdShNs}, + {"2009-07-13 09:07:13+08:00", bdShSec}, + {"2009-07-13 09:07:13+0800", bdShSec}, + {"2009-07-13 09:07:13.123Z", bdUtcNs}, + {"2009-07-13 09:07:13Z", bdUtcSec}, + {"2009-07-13 09:07:13.123", bdLocalNs}, + {"2009-07-13 09:07:13", bdLocalSec}, + //DateTime, other layouts supported by golang + {"Mon Jul 13 09:07:13 2009", bdLocalSec}, + {"Mon Jul 13 09:07:13 CST 2009", bdShSec}, + {"Mon Jul 13 09:07:13 +0800 2009", bdShSec}, + {"13 Jul 09 09:07 CST", bdShMin}, + {"13 Jul 09 09:07 +0800", bdShMin}, + {"Monday, 13-Jul-09 09:07:13 CST", bdShSec}, + {"Mon, 13 Jul 2009 09:07:13 CST", bdShSec}, + {"Mon, 13 Jul 2009 09:07:13 +0800", bdShSec}, + //DateTime, linux utils + {"Mon 13 Jul 2009 09:07:13 AM CST", bdShSec}, + {"Mon Jul 13 09:07:13 CST 2009", bdShSec}, + //DateTime, home-brewed + {"Jul 13, 2009 09:07:13.123+08:00", bdShNs}, + {"Jul 13, 2009 09:07:13.123+0800", bdShNs}, + {"Jul 13, 2009 09:07:13+08:00", bdShSec}, + {"Jul 13, 2009 09:07:13+0800", bdShSec}, + {"Jul 13, 2009 09:07:13.123Z", bdUtcNs}, + {"Jul 13, 2009 09:07:13Z", bdUtcSec}, + {"Jul 13, 2009 09:07:13.123", bdLocalNs}, + {"Jul 13, 2009 09:07:13", bdLocalSec}, + {"13/Jul/2009 09:07:13.123 +08:00", bdShNs}, + {"13/Jul/2009 09:07:13.123 +0800", bdShNs}, + {"13/Jul/2009 09:07:13 +08:00", bdShSec}, + {"13/Jul/2009 09:07:13 +0800", bdShSec}, + {"13/Jul/2009 09:07:13.123 Z", bdUtcNs}, + {"13/Jul/2009 09:07:13 Z", bdUtcSec}, + {"13/Jul/2009 09:07:13.123", bdLocalNs}, + {"13/Jul/2009 09:07:13", bdLocalSec}, + //Date + {"2009-07-13", bdLocalDate}, + {"13/07/2009", bdLocalDate}, + {"13/Jul/2009", bdLocalDate}, + {"Jul 13, 2009", bdLocalDate}, + {"Mon Jul 13, 2009", bdLocalDate}, + } + + for _, tc := range testCases { + v, layout := parseInLocation(tc.TS, time.Local) + desc := fmt.Sprintf(`parseInLocation("%s", "%s") = %s(layout: %s), expect %s`, tc.TS, location, v.Format(time.RFC3339Nano), layout, tc.ExpVal.Format(time.RFC3339Nano)) + if strings.Contains(tc.TS, "CST") && v != tc.ExpVal { + log.Printf(desc + "(CST is ambiguous)") + } else { + require.Equal(t, tc.ExpVal, v, desc) + } + } + } +} + func BenchmarkUnmarshalljson(b *testing.B) { object := map[string]interface{}{} for i := 0; i < b.N; i++ { From 5b110eeea8e8660eb32a4dc5fc2b90f94cc64602 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 16 May 2021 10:53:55 +0800 Subject: [PATCH 107/404] replicaSpecificErrorCodes for ReplicatedMergeTree --- output/clickhouse.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index a0116578..b12c0e7a 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -41,8 +41,10 @@ var ( selectSQLTemplate = `select name, type, default_kind from system.columns where database = '%s' and table = '%s'` lowCardinalityRegexp = regexp.MustCompile(`LowCardinality\((.+)\)`) - // refers to src/Common/ErrorCodes.cpp, https://github.com/ClickHouse/ClickHouse/issues/24036 - replicaSpecificErrorCodes = []int32{1000} + // https://github.com/ClickHouse/ClickHouse/issues/24036 + // src/Common/ErrorCodes.cpp + // src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp + replicaSpecificErrorCodes = []int32{164, 225, 319, 1000} //READONLY, NO_ZOOKEEPER, UNKNOWN_STATUS_OF_INSERT, POCO_EXCEPTION ) // ClickHouse is an output service consumers from kafka messages From 237fbabd7cc03ab2024b2656ecf74f223e63b8ae Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 16 May 2021 17:55:59 +0800 Subject: [PATCH 108/404] added ParseInt --- parser/parser_test.go | 57 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/parser/parser_test.go b/parser/parser_test.go index de7ed92c..6fd0a0c3 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -16,6 +16,7 @@ package parser import ( "encoding/json" + "errors" "fmt" "log" "strconv" @@ -591,6 +592,62 @@ func TestParseDateTime(t *testing.T) { } } +func TestParseInt(t *testing.T) { + arrayStrInt := []string{"invalid", "-9223372036854775809", "-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "255", "256", "32767", "32768", "65535", "65536", "2147483647", "2147483648", "4294967295", "4294967296", "9223372036854775807", "18446744073709551615", "18446744073709551616"} + i8Exp := []int8{0, -128, -128, -128, -128, -128, -128, -128, -128, 0, 127, 127, 127, 127, 127, 127, 127, 127, 127, 127, 127, 127, 127, 127, 127} + i16Exp := []int16{0, -32768, -32768, -32768, -32768, -32768, -32768, -129, -128, 0, 127, 128, 255, 256, 32767, 32767, 32767, 32767, 32767, 32767, 32767, 32767, 32767, 32767, 32767} + i32Exp := []int32{0, -2147483648, -2147483648, -2147483648, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 255, 256, 32767, 32768, 65535, 65536, 2147483647, 2147483647, 2147483647, 2147483647, 2147483647, 2147483647, 2147483647} + i64Exp := []int64{0, -9223372036854775808, -9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 255, 256, 32767, 32768, 65535, 65536, 2147483647, 2147483648, 4294967295, 4294967296, 9223372036854775807, 9223372036854775807, 9223372036854775807} + u8Exp := []uint8{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 127, 128, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255} + u16Exp := []uint16{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 127, 128, 255, 256, 32767, 32768, 65535, 65535, 65535, 65535, 65535, 65535, 65535, 65535, 65535} + u32Exp := []uint32{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 127, 128, 255, 256, 32767, 32768, 65535, 65536, 2147483647, 2147483648, 4294967295, 4294967295, 4294967295, 4294967295, 4294967295} + u64Exp := []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 127, 128, 255, 256, 32767, 32768, 65535, 65536, 2147483647, 2147483648, 4294967295, 4294967296, 9223372036854775807, 18446744073709551615, 18446744073709551615} + + for _, bitSize := range []int{8, 16, 32, 64} { + for i, s := range arrayStrInt { + var iv int64 + var uv uint64 + var ivErr, uvErr error + var desc string + iv, ivErr = strconv.ParseInt(s, 10, bitSize) + uv, uvErr = strconv.ParseUint(s, 10, bitSize) + var ivExp, uvExp, ivAct, uvAct interface{} + switch bitSize { + case 8: + ivExp = i8Exp[i] + ivAct = int8(iv) + uvExp = u8Exp[i] + uvAct = uint8(uv) + case 16: + ivExp = i16Exp[i] + ivAct = int16(iv) + uvExp = u16Exp[i] + uvAct = uint16(uv) + case 32: + ivExp = i32Exp[i] + ivAct = int32(iv) + uvExp = u32Exp[i] + uvAct = uint32(uv) + case 64: + ivExp = i64Exp[i] + ivAct = int64(iv) + uvExp = u64Exp[i] + uvAct = uint64(uv) + } + desc = fmt.Sprintf(`ParseInt("%s", 10, %d)=%d(%v)`, s, bitSize, iv, errors.Unwrap(ivErr)) + require.Equal(t, ivExp, ivAct, desc) + desc = fmt.Sprintf(`ParseUint("%s", 10, %d)=%d(%v)`, s, bitSize, uv, errors.Unwrap(uvErr)) + require.Equal(t, uvExp, uvAct, desc) + if strings.Contains(s, "invalid") { + require.True(t, errors.Is(ivErr, strconv.ErrSyntax)) + require.True(t, errors.Is(uvErr, strconv.ErrSyntax)) + } else if strings.Contains(s, "-") { + require.True(t, errors.Is(uvErr, strconv.ErrSyntax)) + } + } + } +} + func BenchmarkUnmarshalljson(b *testing.B) { object := map[string]interface{}{} for i := 0; i < b.N; i++ { From 825c6292bf16282201d3cc653de229d51b89a600 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 16 May 2021 20:46:52 +0800 Subject: [PATCH 109/404] updated dependencies --- .goreleaser.yml | 6 + go.mod | 30 ++-- go.sum | 414 +++++++++++++++++++++++++++++++++++++++-------- task/sharding.go | 2 +- 4 files changed, 365 insertions(+), 87 deletions(-) diff --git a/.goreleaser.yml b/.goreleaser.yml index d300df17..35c5acf7 100644 --- a/.goreleaser.yml +++ b/.goreleaser.yml @@ -12,6 +12,9 @@ builds: - CGO_ENABLED=0 goos: - linux + goarch: + - amd64 + - arm64 main: cmd/clickhouse_sinker/main.go binary: clickhouse_sinker - id: nacos_publish_config @@ -19,6 +22,9 @@ builds: - CGO_ENABLED=0 goos: - linux + goarch: + - amd64 + - arm64 main: cmd/nacos_publish_config/main.go binary: nacos_publish_config dockers: diff --git a/go.mod b/go.mod index 4551baa3..4ade9c67 100644 --- a/go.mod +++ b/go.mod @@ -3,25 +3,21 @@ module github.com/housepower/clickhouse_sinker go 1.14 require ( - github.com/ClickHouse/clickhouse-go v1.4.3 - github.com/Shopify/sarama v1.27.2 - github.com/cespare/xxhash v1.1.0 - github.com/fagongzi/goetty v1.6.0 - github.com/golang/protobuf v1.4.2 // indirect - github.com/nacos-group/nacos-sdk-go v1.0.1 + github.com/ClickHouse/clickhouse-go v1.4.5 + github.com/Shopify/sarama v1.29.0 + github.com/cespare/xxhash/v2 v2.1.1 + github.com/fagongzi/goetty v1.7.0 + github.com/nacos-group/nacos-sdk-go v1.0.7 github.com/pkg/errors v0.9.1 - github.com/prometheus/client_golang v1.3.0 - github.com/prometheus/common v0.8.0 + github.com/prometheus/client_golang v1.10.0 + github.com/prometheus/common v0.24.0 github.com/segmentio/kafka-go v0.4.8 - github.com/smartystreets/goconvey v1.6.4 // indirect - github.com/stretchr/testify v1.6.1 - github.com/tidwall/gjson v1.1.3 - github.com/tidwall/match v1.0.0 // indirect + github.com/stretchr/testify v1.7.0 + github.com/tidwall/gjson v1.8.0 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/valyala/fastjson v1.4.1 - github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c - go.uber.org/zap v1.15.0 - golang.org/x/sys v0.0.0-20210217105451-b926d437f341 // indirect - golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e + github.com/valyala/fastjson v1.6.3 + github.com/xdg/scram v1.0.3 + go.uber.org/zap v1.16.0 + golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) diff --git a/go.sum b/go.sum index 3db139c7..615cc565 100644 --- a/go.sum +++ b/go.sum @@ -1,63 +1,109 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/clickhouse-go v1.4.3 h1:iAFMa2UrQdR5bHJ2/yaSLffZkxpcOYQMCUuKeNXGdqc= -github.com/ClickHouse/clickhouse-go v1.4.3/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= +github.com/ClickHouse/clickhouse-go v1.4.5 h1:FfhyEnv6/BaWldyjgT2k4gDDmeNwJ9C4NbY/MXxJlXk= +github.com/ClickHouse/clickhouse-go v1.4.5/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= -github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= -github.com/Shopify/sarama v1.27.2 h1:1EyY1dsxNDUQEv0O/4TsjosHI2CgB1uo9H/v56xzTxc= -github.com/Shopify/sarama v1.27.2/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt/Mc93II= +github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0= +github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= +github.com/Shopify/sarama v1.29.0 h1:ARid8o8oieau9XrHI55f/L3EoRAhm9px6sonbD7yuUE= +github.com/Shopify/sarama v1.29.0/go.mod h1:2QpgD79wpdAESqNQMxNc0KYMkycd4slxGdV3TWSVqrU= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= +github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= +github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 h1:zOVTBdCKFd9JbCKz9/nt+FovbjPFmb7mUnp8nH9fQBA= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= +github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= +github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= +github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= +github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= +github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= +github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a/go.mod h1:DAHtR1m6lCRdSC2Tm3DSWRPvIPr6xNKyeHdqDQSQT+A= +github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU= +github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= +github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAKk= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= -github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= -github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= +github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= +github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= +github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= +github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= +github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= +github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= +github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= +github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= github.com/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q= github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= -github.com/fagongzi/goetty v1.6.0 h1:LBj+N14o9g3A1w5Mp+TJYEBXWfp07mFdfUH2nvSPFkk= -github.com/fagongzi/goetty v1.6.0/go.mod h1:lLUyHhtjlOqatxVXgyLocwoI2o359JzLE7EWRGZiGw4= +github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M= +github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4safvEdbitLhGGK48rN6g= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/fagongzi/goetty v1.7.0 h1:Z0uoEVqP4uQSQW+HR3bg5GGwmisZpJQ1sK/ab9HK7q0= +github.com/fagongzi/goetty v1.7.0/go.mod h1:lLUyHhtjlOqatxVXgyLocwoI2o359JzLE7EWRGZiGw4= github.com/fastly/go-utils v0.0.0-20180712184237-d95a45783239 h1:Ghm4eQYC0nEPnSJdVkTrXpu9KtoVCSo1hg7mtI7G9KU= github.com/fastly/go-utils v0.0.0-20180712184237-d95a45783239/go.mod h1:Gdwt2ce0yfBxPvZrHkprdPPTTS3N5rwmLE8T22KBXlw= +github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= -github.com/frankban/quicktest v1.10.2 h1:19ARM85nVi4xH7xPXuc5eM/udya5ieh7b/Sv+d844Tk= -github.com/frankban/quicktest v1.10.2/go.mod h1:K+q6oSqb0W0Ininfk863uOk1lMy69l/P6txr3mVT54s= +github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= +github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= +github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= +github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= +github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -68,23 +114,75 @@ github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:x github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= +github.com/golang/protobuf v1.4.3 h1:JjCZWpVbqXDqFVmTfYWEVTMIYrL/NPdPSCHPJ0T/raM= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA= +github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= -github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.4 h1:L8R9j+yAqZuZjsqh/z+F1NCffTKKLShY6zXTItVIZ8M= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= +github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= +github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= +github.com/gorilla/securecookie v1.1.1 h1:miw7JPhV+b/lAHSXz4qd/nN9jRiAFV5FwjeKyCS8BvQ= +github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= +github.com/gorilla/sessions v1.2.1 h1:DHd3rPN5lE3Ts3D8rKkQ8x/0kqfeNmBAaiSi+o7FsgI= +github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= +github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= +github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= +github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= +github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE= +github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= +github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= +github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= +github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= +github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= +github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= +github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= +github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= +github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg= +github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= +github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= +github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= +github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= +github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM= github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8= github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= +github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= +github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= +github.com/jcmturner/gokrb5/v8 v8.4.2 h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJzodkA= +github.com/jcmturner/gokrb5/v8 v8.4.2/go.mod h1:sb+Xq/fTY5yktf/VxLsE3wlfPqQjp0aWNYyvBVK62bc= +github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= +github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uczJe5YQdrYB16oTJlGSC/OyZDqUk9xX4= github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869/go.mod h1:cJ6Cj7dQo+O6GJNiMx+Pa94qKj+TG8ONdKHgMNIyyag= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= @@ -92,18 +190,24 @@ github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.8 h1:QiWkFLKq0T7mpzwOTu6BzNDbfTE8OLrYhVKYMLF46Ok= +github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.10 h1:Kz6Cvnvv2wGdaG/V8yMvfkmNiXq9Ya2KUv4rouJJr68= +github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.11.0 h1:wJbzvpYMVGG9iTI9VxpnNZfd4DzMPoCWze3GgSqz8yg= -github.com/klauspost/compress v1.11.0/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/klauspost/compress v1.12.2 h1:2KCfW3I9M7nSc5wOqXAlW2v2U6v+w6cbjvbfp+OykW8= +github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= @@ -119,9 +223,24 @@ github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f/go.mod github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 h1:Bvq8AziQ5jFF4BHGAEDSqwPW1NJS3XshxbRCxtjFAZc= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042/go.mod h1:TPpsiPUEh0zFL1Snz4crhMlBe60PYxRHr5oFF3rRYg0= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743/go.mod h1:qklhhLq1aX+mtWk9cPHPzaBjWImj5ULL6C7HFJtXQMM= +github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4= +github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= +github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= +github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= +github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= +github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= +github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= +github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= +github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= +github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= +github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -129,130 +248,282 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nacos-group/nacos-sdk-go v1.0.1 h1:VNmXGlSS28xOmkO5Nxk5WRp6f1HMosAmG9pDtcnUFcw= -github.com/nacos-group/nacos-sdk-go v1.0.1/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= -github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= -github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/nacos-group/nacos-sdk-go v1.0.7 h1:Am1tJFe7GUTNCREKsZ5ok0H2OspHDRmRcsxn7DiSwhA= +github.com/nacos-group/nacos-sdk-go v1.0.7/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= +github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= +github.com/nats-io/jwt v0.3.2/go.mod h1:/euKqTS1ZD+zzjYrY7pseZrTtWQSjujC7xjPc8wL6eU= +github.com/nats-io/nats-server/v2 v2.1.2/go.mod h1:Afk+wRZqkMQs/p45uXdrVLuab3gwv3Z8C4HTBu8GD/k= +github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= +github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= +github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= +github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= +github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= +github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= +github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= +github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= +github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= +github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= +github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= +github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/openzipkin-contrib/zipkin-go-opentracing v0.4.5/go.mod h1:/wsWhb9smxSfWAKL3wpBW7V8scJMt8N8gnaMCS9E/cA= +github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw= +github.com/openzipkin/zipkin-go v0.2.1/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= +github.com/openzipkin/zipkin-go v0.2.2/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= +github.com/pact-foundation/pact-go v1.0.4/go.mod h1:uExwJY4kCzNPcHRj+hCR/HBbOOIwwtUjcrb0b5/5kLM= +github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= +github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= +github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac= +github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= -github.com/pierrec/lz4 v2.5.2+incompatible h1:WCjObylUIOlKy/+7Abdn34TLIkXiA4UWUMhxq9m9ZXI= -github.com/pierrec/lz4 v2.5.2+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= +github.com/pierrec/lz4 v2.6.0+incompatible h1:Ix9yFKn1nSPBLFl/yZknTp8TU5G4Ps0JDmguYK6iH1A= +github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= +github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= -github.com/prometheus/client_golang v1.3.0 h1:miYCvYqFXtl/J9FIy8eNpBfYthAEFg+Ys0XyUVEcDsc= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= +github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.10.0 h1:/o0BDeWzLWXNZ+4q5gXltUvaMpJqckTa+jTNoB+z4cg= +github.com/prometheus/client_golang v1.10.0/go.mod h1:WJM3cc3yu7XKBKa/I8WeZm+V3eltZnBwfENSU7mdogU= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= +github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.1.0 h1:ElTg5tNp4DqfV7UQjDqv2+RJlNzsDtvNAWccbItceIE= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= +github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= -github.com/prometheus/common v0.8.0 h1:bLkjvFe2ZRX1DpcgZcdf7j/+MnusEps5hktST/FHA34= -github.com/prometheus/common v0.8.0/go.mod h1:PC/OgXc+UN7B4ALwvn1yzVZmVwvhXp5JsbBv6wSv6i0= +github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= +github.com/prometheus/common v0.18.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= +github.com/prometheus/common v0.24.0 h1:aIycr3wRFxPUq8XlLQlGQ9aNXV3dFi5y62pe/SB262k= +github.com/prometheus/common v0.24.0/go.mod h1:H6QK/N6XVT42whUeIdI3dp36w49c+/iMDk7UAI2qm7Q= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= +github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLkt8= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 h1:MkV+77GLUNo5oJ0jf870itWm3D0Sjh7+Za9gazKc5LQ= -github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= +github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= +github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= +github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= +github.com/segmentio/kafka-go v0.4.16 h1:9dt78ehM9qzAkekA60D6A96RlqDzC3hnYYa8y5Szd+U= +github.com/segmentio/kafka-go v0.4.16/go.mod h1:19+Eg7KwrNKy/PFhiIthEPkO8k+ac7/ZYXwYM9Df10w= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= -github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= +github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= +github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= +github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= +github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5NYJe/zRYDwOu9ku6YHy+Iw7l5DM= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= +github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= -github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/tebeka/strftime v0.1.3 h1:5HQXOqWKYRFfNyBMNVc9z5+QzuBtIXy03psIhtdJYto= github.com/tebeka/strftime v0.1.3/go.mod h1:7wJm3dZlpr4l/oVK0t1HYIc4rMzQ2XJlOMIUJUJH6XQ= -github.com/tidwall/gjson v1.1.3 h1:u4mspaByxY+Qk4U1QYYVzGFI8qxN/3jtEV0ZDb2vRic= -github.com/tidwall/gjson v1.1.3/go.mod h1:c/nTNbUr0E0OrXEhq1pwa8iEgc2DOt4ZZqAt1HtCkPA= -github.com/tidwall/match v1.0.0 h1:Ym1EcFkp+UQ4ptxfWlW+iMdq5cPH5nEuGzdf/Pb7VmI= -github.com/tidwall/match v1.0.0/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= +github.com/tidwall/gjson v1.8.0 h1:Qt+orfosKn0rbNTZqHYDqBrmm3UDA4KRkv70fDzG+PQ= +github.com/tidwall/gjson v1.8.0/go.mod h1:5/xDoumyyDNerp2U36lyolv46b3uF/9Bu6OfyQ9GImk= +github.com/tidwall/match v1.0.3 h1:FQUVvBImDutD8wJLN6c5eMzWtjgONK9MwIBCOrUJKeE= +github.com/tidwall/match v1.0.3/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= +github.com/tidwall/pretty v1.1.0 h1:K3hMW5epkdAVwibsQEfR/7Zj0Qgt4DxtNumTq/VloO8= +github.com/tidwall/pretty v1.1.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= +github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 h1:kF/7m/ZU+0D4Jj5eZ41Zm3IH/J8OElK1Qtd7tVKAwLk= github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3/go.mod h1:QDlpd3qS71vYtakd2hmdpqhJ9nwv6mD6A30bQ1BPBFE= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= -github.com/valyala/fastjson v1.4.1 h1:hrltpHpIpkaxll8QltMU8c3QZ5+qIiCL8yKqPFJI/yE= -github.com/valyala/fastjson v1.4.1/go.mod h1:nV6MsjxL2IMJQUoHDIrjEI7oLyeqK6aBD7EFWPsvP8o= -github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c h1:u40Z8hqBAAQyv+vATcGgV0YCnDjqSL7/q/JyPhhJSPk= +github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= +github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= +github.com/valyala/fastjson v1.6.3/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= -github.com/xdg/stringprep v1.0.0 h1:d9X0esnoa3dFsV0FG35rAT0RIhYFlPq7MiP+DW89La0= +github.com/xdg/scram v1.0.3 h1:nTadYh2Fs4BK2xdldEa2g5bbaZp0/+1nJMMPtPxS/to= +github.com/xdg/scram v1.0.3/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= +github.com/xdg/stringprep v1.0.3 h1:cmL5Enob4W83ti/ZHuZLuKD/xqJfus4fVPwE+/BDm+4= +github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= +github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= +go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= +go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= +go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= +go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= +go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= +go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= -go.uber.org/zap v1.15.0 h1:ZZCA22JRF2gQE5FoNmhmrf7jeJJ2uhqDUNRYKm8dvmM= +go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= +go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= +go.uber.org/zap v1.16.0 h1:uFRZXykJGK9lLY4HtgSw44DnIcAM+kRBP7x5m+NpAOM= +go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190506204251-e1dfcc566284/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rBCcS0QyQY66Mpf/7BZbInM= -golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b h1:7mWr3k41Qtv8XlltBkDkl8LoP3mpSgBW8BUoxtEdbXg= +golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190125091013-d26f9f9a57f3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200904194848-62affa334b73 h1:MXfv8rhZWmFeqX3GNZRsd6vOLoaCHjYEX3qkRo3YBUA= -golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a h1:njMmldwFTyDLqonHMagNXKBWptTBeDZOdblgaDsNEGQ= +golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181122145206-62eef0e2fa9b/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210217105451-b926d437f341 h1:2/QtM1mL37YmcsT8HaDNHDgTqqFVw+zr8UzMiBVLzYU= -golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da h1:b3NXsE2LusjYGGjL5bxEVZZORm/YEFFrWFjR8eFrw/c= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e h1:EHBhcS0mlXEAVwNyO2dLfjToGsyY4j24pTs2ScHnX7s= -golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba h1:O8mE0/t419eoIwhTFpKVkHiTs/Igowgfkj25AcZrtiE= +golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5 h1:hKsoRgsbwY1NafxrwTs+k64bikrLBkAgPir1TNCj3Zs= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200103221440-774c71fcf114 h1:DnSr2mCsxyCE6ZgIkmcWUQY2R5cH/6wL7eIxEmQOMSE= +golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190530194941-fb225487d101/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.0/go.mod h1:chYK+tFQF0nDUGJgXMSgLCQk3phJEuONr2DCgLDdAQM= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -263,29 +534,34 @@ google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2 gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= -gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= +gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o= gopkg.in/ini.v1 v1.42.0 h1:7N3gPTt50s8GuLortA00n8AqRTk75qOP98+mTPpgzRk= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/jcmturner/aescts.v1 v1.0.1 h1:cVVZBK2b1zY26haWB4vbBiZrfFQnfbTVrE3xZq6hrEw= -gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo= -gopkg.in/jcmturner/dnsutils.v1 v1.0.1 h1:cIuC1OLRGZrld+16ZJvvZxVJeKPsvd5eUIvxfoN5hSM= -gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q= -gopkg.in/jcmturner/goidentity.v3 v3.0.0 h1:1duIyWiTaYvVx3YX2CYtpJbUFd7/UuPYCfgXtQ3VTbI= -gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4= -gopkg.in/jcmturner/gokrb5.v7 v7.5.0 h1:a9tsXlIDD9SKxotJMK3niV7rPZAJeX2aD/0yg3qlIrg= -gopkg.in/jcmturner/gokrb5.v7 v7.5.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM= -gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU= -gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= +gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= +gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI= +gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.4 h1:/eiJrUcujPVeJ3xlSWaiNi3uSVmDGBK1pDHUHAnao1I= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU= +gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= -gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= +sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= diff --git a/task/sharding.go b/task/sharding.go index 5de656ca..4aab46b3 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -7,7 +7,7 @@ import ( "sync" "time" - "github.com/cespare/xxhash" + "github.com/cespare/xxhash/v2" "github.com/fagongzi/goetty" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" From 33d0c445939f4434a634483334fd3934e1b7d0de Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 17 May 2021 11:08:18 +0800 Subject: [PATCH 110/404] ShardConn mutex --- output/clickhouse.go | 25 +++++++----- pool/conn.go | 94 +++++++++++++++++++++++++++----------------- 2 files changed, 72 insertions(+), 47 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index b12c0e7a..5772a874 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -81,19 +81,18 @@ func (c *ClickHouse) Send(batch *model.Batch) { } // Write kvs to clickhouse -func (c *ClickHouse) write(batch *model.Batch, reconnect bool) (err error) { +func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, failAt time.Time) (err error) { var stmt *sql.Stmt var tx *sql.Tx if len(*batch.Rows) == 0 { return } - conn := pool.GetShardConn(batch.BatchIdx) - if reconnect { - if err = conn.NextGoodReplica(); err != nil { - return - } + var conn *sql.DB + if conn, err = sc.NextGoodReplica(failAt); err != nil { + return } + if tx, err = conn.Begin(); err != nil { err = errors.Wrapf(err, "conn.Begin") return @@ -142,8 +141,10 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { var err error var times int var reconnect bool + var failAt time.Time + sc := pool.GetShardConn(batch.BatchIdx) for { - if err = c.write(batch, reconnect); err == nil { + if err = c.write(batch, sc, failAt); err == nil { if err = batch.Commit(); err == nil { return } @@ -163,6 +164,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { times++ reconnect = shouldReconnect(err) if reconnect && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { + failAt = time.Now() time.Sleep(10 * time.Second) } else { util.Logger.Fatal("ClickHouse.loopWrite failed", zap.String("task", c.cfg.Task.Name)) @@ -178,7 +180,8 @@ func (c *ClickHouse) Stop() error { func (c *ClickHouse) initSchema() (err error) { if c.cfg.Task.AutoSchema { - conn := pool.GetShardConn(0) + sc := pool.GetShardConn(0) + conn := sc.GetCurReplica() var rs *sql.Rows if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, c.cfg.Clickhouse.DB, c.cfg.Task.TableName)); err != nil { err = errors.Wrapf(err, "") @@ -300,7 +303,8 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { chCfg.Cluster, chCfg.DB, taskCfg.TableName)) } } - conn := pool.GetShardConn(0) + sc := pool.GetShardConn(0) + conn := sc.GetCurReplica() for _, query := range queries { util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) if _, err = conn.Exec(query); err != nil { @@ -314,7 +318,8 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { taskCfg := &c.cfg.Task chCfg := &c.cfg.Clickhouse - conn := pool.GetShardConn(0) + sc := pool.GetShardConn(0) + conn := sc.GetCurReplica() query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, chCfg.DB, chCfg.DB, taskCfg.TableName) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) diff --git a/pool/conn.go b/pool/conn.go index d818cc13..8f699bda 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -45,52 +45,77 @@ var ( // ShardConn a datastructure for storing the clickhouse connection type ShardConn struct { - *sql.DB - Dsn string - Replicas []string //ip:port list of replicas - NextRep int //index of next replica + lock sync.Mutex + db *sql.DB + connAt time.Time + dsn string + replicas []string //ip:port list of replicas + nextRep int //index of next replica } -// NumReplica returns number of replicas of this shard -func (c *ShardConn) NumReplica() int { - return len(c.Replicas) +// GetCurReplica returns the current replica connection +func (sc *ShardConn) GetCurReplica() (db *sql.DB) { + sc.lock.Lock() + db = sc.db + sc.lock.Unlock() + return +} + +// Close closes the current replica connection +func (sc *ShardConn) Close() { + sc.lock.Lock() + defer sc.lock.Unlock() + if sc.db != nil { + sc.db.Close() + sc.db = nil + if err := health.Health.RemoveReadinessCheck(sc.dsn); err != nil { + util.Logger.Error("health.Health.RemoveReadinessCheck failed", zap.String("dsn", sc.dsn), zap.Error(err)) + } + } } // NextGoodReplica connects to next good replica -func (c *ShardConn) NextGoodReplica() error { - if c.DB != nil { - if err := health.Health.RemoveReadinessCheck(c.Dsn); err != nil { - util.Logger.Warn("health.Health.RemoveReadinessCheck failed", zap.String("dsn", c.Dsn), zap.Error(err)) +func (sc *ShardConn) NextGoodReplica(failAt time.Time) (db *sql.DB, err error) { + sc.lock.Lock() + defer sc.lock.Unlock() + if sc.db != nil { + if sc.connAt.After(failAt) { + // Another goroutine has already done connection. + return sc.db, nil + } + if err := health.Health.RemoveReadinessCheck(sc.dsn); err != nil { + util.Logger.Warn("health.Health.RemoveReadinessCheck failed", zap.String("dsn", sc.dsn), zap.Error(err)) } - c.DB.Close() - c.DB = nil + sc.db.Close() + sc.db = nil } - savedNextRep := c.NextRep + savedNextRep := sc.nextRep // try all replicas, including the current one - for i := 0; i < len(c.Replicas); i++ { - c.Dsn = fmt.Sprintf(dsnTmpl, c.Replicas[c.NextRep]) - c.NextRep = (c.NextRep + 1) % len(c.Replicas) - sqlDB, err := sql.Open("clickhouse", c.Dsn) + for i := 0; i < len(sc.replicas); i++ { + sc.dsn = fmt.Sprintf(dsnTmpl, sc.replicas[sc.nextRep]) + sc.nextRep = (sc.nextRep + 1) % len(sc.replicas) + sqlDB, err := sql.Open("clickhouse", sc.dsn) if err != nil { - util.Logger.Warn("sql.Open failed", zap.String("dsn", c.Dsn), zap.Error(err)) + util.Logger.Warn("sql.Open failed", zap.String("dsn", sc.dsn), zap.Error(err)) continue } // According to sql.Open doc, "Open may just validate its arguments without creating a connection // to the database. To verify that the data source name is valid, call Ping." if err := sqlDB.Ping(); err != nil { - util.Logger.Warn("sqlDB.Ping failed", zap.String("dsn", c.Dsn), zap.Error(err)) + util.Logger.Warn("sqlDB.Ping failed", zap.String("dsn", sc.dsn), zap.Error(err)) continue } setDBParams(sqlDB) - util.Logger.Info("sql.Open and sqlDB.Ping succeeded", zap.String("dsn", c.Dsn)) - if err = health.Health.AddReadinessCheck(c.Dsn, healthcheck.DatabasePingCheck(sqlDB, 30*time.Second)); err != nil { - util.Logger.Warn("health.Health.AddReadinessCheck failed", zap.String("dsn", c.Dsn), zap.Error(err)) + util.Logger.Info("sql.Open and sqlDB.Ping succeeded", zap.String("dsn", sc.dsn)) + if err = health.Health.AddReadinessCheck(sc.dsn, healthcheck.DatabasePingCheck(sqlDB, 30*time.Second)); err != nil { + util.Logger.Warn("health.Health.AddReadinessCheck failed", zap.String("dsn", sc.dsn), zap.Error(err)) } - c.DB = sqlDB - return nil + sc.db = sqlDB + sc.connAt = time.Now() + return sc.db, nil } - err := errors.Errorf("no good replica among replicas %v since %d", c.Replicas, savedNextRep) - return err + err = errors.Errorf("no good replica among replicas %v since %d", sc.replicas, savedNextRep) + return nil, err } func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool) (err error) { @@ -118,9 +143,9 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) } sc := &ShardConn{ - Replicas: replicaAddrs, + replicas: replicaAddrs, } - if err = sc.NextGoodReplica(); err != nil { + if _, err = sc.NextGoodReplica(time.Now()); err != nil { return } clusterConn = append(clusterConn, sc) @@ -138,12 +163,7 @@ func setDBParams(sqlDB *sql.DB) { func freeClusterConn() { for _, sc := range clusterConn { - if sc.DB != nil { - if err := health.Health.RemoveReadinessCheck(sc.Dsn); err != nil { - util.Logger.Error("health.Health.RemoveReadinessCheck failed", zap.String("dsn", sc.Dsn), zap.Error(err)) - } - sc.DB.Close() - } + sc.Close() } clusterConn = []*ShardConn{} } @@ -161,10 +181,10 @@ func NumShard() (cnt int) { } // GetShardConn select a clickhouse shard based on batchNum -func GetShardConn(batchNum int64) (con *ShardConn) { +func GetShardConn(batchNum int64) (sc *ShardConn) { lock.Lock() defer lock.Unlock() - con = clusterConn[batchNum%int64(len(clusterConn))] + sc = clusterConn[batchNum%int64(len(clusterConn))] return } From 1208a4b4e6a7a1aa817f54ba1f90036233a7cf90 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 17 May 2021 10:12:53 +0800 Subject: [PATCH 111/404] removed MinBufferSize config --- config/config.go | 17 ++++++++--------- docs/configuration/config.md | 12 +++++------- go.test.sh | 4 ++-- input/kafka_go.go | 4 ++-- input/kafka_sarama.go | 2 +- parser/parser_test.go | 4 ++-- pool/conn.go | 7 ++----- 7 files changed, 22 insertions(+), 28 deletions(-) diff --git a/config/config.go b/config/config.go index 3438b08c..8be5bd7c 100644 --- a/config/config.go +++ b/config/config.go @@ -142,15 +142,15 @@ type TaskConfig struct { FlushInterval int `json:"flushInterval,omitempty"` BufferSize int `json:"bufferSize,omitempty"` - MinBufferSize int `json:"minBufferSize,omitempty"` MsgSizeHint int `json:"msgSizeHint,omitempty"` TimeZone string `json:"timezone"` } const ( - defaultFlushInterval = 3 - defaultBufferSize = 1 << 20 //1048576 - defaultMinBufferSize = 1 << 14 // 16384 + maxFlushInterval = 10 + defaultFlushInterval = 5 + MaxBufferSize = 1 << 20 //1048576 + defaultBufferSize = 1 << 18 //262144 defaultMsgSizeHint = 1000 defaultTimeZone = "Local" defaultLogLevel = "info" @@ -214,17 +214,16 @@ func (cfg *Config) Normallize() (err error) { if cfg.Task.FlushInterval <= 0 { cfg.Task.FlushInterval = defaultFlushInterval + } else if cfg.Task.FlushInterval > maxFlushInterval { + cfg.Task.FlushInterval = maxFlushInterval } if cfg.Task.BufferSize <= 0 { cfg.Task.BufferSize = defaultBufferSize + } else if cfg.Task.BufferSize > MaxBufferSize { + cfg.Task.BufferSize = MaxBufferSize } else { cfg.Task.BufferSize = 1 << util.GetShift(cfg.Task.BufferSize) } - if cfg.Task.MinBufferSize <= 0 { - cfg.Task.MinBufferSize = defaultMinBufferSize - } else { - cfg.Task.MinBufferSize = 1 << util.GetShift(cfg.Task.MinBufferSize) - } if cfg.Task.MsgSizeHint <= 0 { cfg.Task.MsgSizeHint = defaultMsgSizeHint } diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 5237a641..a4fb4463 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -121,23 +121,21 @@ // shardingPolicy is `stripe,`(requires ShardingKey be numerical) or `hash`(requires ShardingKey be string) "shardingPolicy": "", - // interval of flushing the batch + // interval of flushing the batch. Default to 5, max to 10. "flushInterval": 5, - // batch size to insert into clickhouse. sinker will round upward it to the the nearest 2^n. + // batch size to insert into clickhouse. sinker will round upward it to the the nearest 2^n. Default to 262114, max to 1048576. "bufferSize": 90000, - // min batch size to insert into clickhouse. sinker will round upward it to the the nearest 2^n. - "minBufferSize": 1, - // estimated avg message size. kafka-go needs this to determize receive buffer size. default to 1000. + // estimated avg message size. kafka-go needs this to determize receive buffer size. Default to 1000. "msgSizeHint": 1000, // In the absence of time zone information, interprets the time as in the given location. Default to "Local" (aka /etc/localtime of the machine on which sinker runs) "timezone": "" }, - // log level, possible value: "debug", "info", "warn", "error", "dpanic", "panic", "fatal" + // log level, possible value: "debug", "info", "warn", "error", "dpanic", "panic", "fatal". Default to "info". "logLevel": "debug", - // log output paths, possible value: "stdout", "stderr", relative file path, absoute file path. Log files will be rotated every 100MB, keep 10 old ones. + // log output paths, possible value: "stdout", "stderr", relative file path, absoute file path. Log files will be rotated every 100MB, keep 10 old ones. Default to ["stdout"]. "logPaths": ["stdout", "test_dynamic_schema.log"] } ``` diff --git a/go.test.sh b/go.test.sh index 36981bc7..9f352f53 100755 --- a/go.test.sh +++ b/go.test.sh @@ -51,7 +51,7 @@ timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_fixed_schema.json timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_auto_schema.json timeout 60 ./clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.json -echo "check result" +echo "check result 1" count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` echo "Got test_fixed_schema count => $count" [ $count -eq 100000 ] || exit 1 @@ -88,7 +88,7 @@ timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username naco timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema -echo "check result" +echo "check result 2" count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` echo "Got test_fixed_schema count => $count" [ $count -eq 100000 ] || exit 1 diff --git a/input/kafka_go.go b/input/kafka_go.go index efdcac8b..c1e0c8b9 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -64,9 +64,9 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model GroupID: k.cfg.Task.ConsumerGroup, Topic: k.cfg.Task.Topic, StartOffset: offset, - MinBytes: k.cfg.Task.MinBufferSize * k.cfg.Task.MsgSizeHint, + MinBytes: (k.cfg.Task.BufferSize / 2) * k.cfg.Task.MsgSizeHint, MaxBytes: k.cfg.Task.BufferSize * k.cfg.Task.MsgSizeHint, - MaxWait: time.Duration(k.cfg.Task.FlushInterval) * time.Second, + MaxWait: time.Duration(3) * time.Second, CommitInterval: time.Second, // flushes commits to Kafka every second } if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 805f7722..a8cf2089 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -127,7 +127,7 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg m if taskCfg.Earliest { config.Consumer.Offsets.Initial = sarama.OffsetOldest } - config.ChannelBufferSize = taskCfg.MinBufferSize + config.ChannelBufferSize = 1024 cg, err := sarama.NewConsumerGroup(strings.Split(kfkCfg.Brokers, ","), taskCfg.ConsumerGroup, config) if err != nil { return err diff --git a/parser/parser_test.go b/parser/parser_test.go index 6fd0a0c3..60e5a88a 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -630,9 +630,9 @@ func TestParseInt(t *testing.T) { uvAct = uint32(uv) case 64: ivExp = i64Exp[i] - ivAct = int64(iv) + ivAct = iv uvExp = u64Exp[i] - uvAct = uint64(uv) + uvAct = uv } desc = fmt.Sprintf(`ParseInt("%s", 10, %d)=%d(%v)`, s, bitSize, iv, errors.Unwrap(ivErr)) require.Equal(t, ivExp, ivAct, desc) diff --git a/pool/conn.go b/pool/conn.go index 8f699bda..0143201f 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -26,6 +26,7 @@ import ( "sync" "time" + "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/health" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" @@ -33,10 +34,6 @@ import ( "go.uber.org/zap" ) -const ( - BlockSize = 1 << 21 //2097152, two times of the default value -) - var ( lock sync.Mutex clusterConn []*ShardConn @@ -125,7 +122,7 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara // Each shard has a *sql.DB which connects to one replica inside the shard. // "alt_hosts" tolerates replica single-point-failure. However more flexable switching is needed for some cases for example https://github.com/ClickHouse/ClickHouse/issues/24036. dsnTmpl = "tcp://%s" + fmt.Sprintf("?database=%s&username=%s&password=%s&block_size=%d", - url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password), BlockSize) + url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password), 2*config.MaxBufferSize) if dsnParams != "" { dsnTmpl += "&" + dsnParams } From 6f7124f335b824131371348aa78e7e10c7011783 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 18 May 2021 09:20:57 +0800 Subject: [PATCH 112/404] optimized ShardConn --- output/clickhouse.go | 24 ++++++++++++++++-------- pool/conn.go | 33 +++++++++++++++------------------ 2 files changed, 31 insertions(+), 26 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 5772a874..8ea6518d 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -81,7 +81,7 @@ func (c *ClickHouse) Send(batch *model.Batch) { } // Write kvs to clickhouse -func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, failAt time.Time) (err error) { +func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) (err error) { var stmt *sql.Stmt var tx *sql.Tx if len(*batch.Rows) == 0 { @@ -89,7 +89,7 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, failAt time.T } var conn *sql.DB - if conn, err = sc.NextGoodReplica(failAt); err != nil { + if conn, *dbVer, err = sc.NextGoodReplica(*dbVer); err != nil { return } @@ -141,10 +141,10 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { var err error var times int var reconnect bool - var failAt time.Time + var dbVer int sc := pool.GetShardConn(batch.BatchIdx) for { - if err = c.write(batch, sc, failAt); err == nil { + if err = c.write(batch, sc, &dbVer); err == nil { if err = batch.Commit(); err == nil { return } @@ -164,7 +164,6 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { times++ reconnect = shouldReconnect(err) if reconnect && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { - failAt = time.Now() time.Sleep(10 * time.Second) } else { util.Logger.Fatal("ClickHouse.loopWrite failed", zap.String("task", c.cfg.Task.Name)) @@ -181,7 +180,10 @@ func (c *ClickHouse) Stop() error { func (c *ClickHouse) initSchema() (err error) { if c.cfg.Task.AutoSchema { sc := pool.GetShardConn(0) - conn := sc.GetCurReplica() + var conn *sql.DB + if conn, _, err = sc.NextGoodReplica(0); err != nil { + return + } var rs *sql.Rows if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, c.cfg.Clickhouse.DB, c.cfg.Task.TableName)); err != nil { err = errors.Wrapf(err, "") @@ -304,7 +306,10 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } } sc := pool.GetShardConn(0) - conn := sc.GetCurReplica() + var conn *sql.DB + if conn, _, err = sc.NextGoodReplica(0); err != nil { + return + } for _, query := range queries { util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) if _, err = conn.Exec(query); err != nil { @@ -319,7 +324,10 @@ func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { taskCfg := &c.cfg.Task chCfg := &c.cfg.Clickhouse sc := pool.GetShardConn(0) - conn := sc.GetCurReplica() + var conn *sql.DB + if conn, _, err = sc.NextGoodReplica(0); err != nil { + return + } query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, chCfg.DB, chCfg.DB, taskCfg.TableName) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) diff --git a/pool/conn.go b/pool/conn.go index 0143201f..5699f403 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -44,20 +44,12 @@ var ( type ShardConn struct { lock sync.Mutex db *sql.DB - connAt time.Time + dbVer int dsn string replicas []string //ip:port list of replicas nextRep int //index of next replica } -// GetCurReplica returns the current replica connection -func (sc *ShardConn) GetCurReplica() (db *sql.DB) { - sc.lock.Lock() - db = sc.db - sc.lock.Unlock() - return -} - // Close closes the current replica connection func (sc *ShardConn) Close() { sc.lock.Lock() @@ -72,13 +64,18 @@ func (sc *ShardConn) Close() { } // NextGoodReplica connects to next good replica -func (sc *ShardConn) NextGoodReplica(failAt time.Time) (db *sql.DB, err error) { +func (sc *ShardConn) NextGoodReplica(failedVer int) (db *sql.DB, dbVer int, err error) { sc.lock.Lock() defer sc.lock.Unlock() if sc.db != nil { - if sc.connAt.After(failAt) { + if sc.dbVer > failedVer { // Another goroutine has already done connection. - return sc.db, nil + // Notice: Why recording failure version instead timestamp? + // Consider following scenario: + // conn1 = NextGood(0); conn2 = NexGood(0); conn1.Exec failed at ts1; + // conn3 = NextGood(ts1); conn2.Exec failed at ts2; + // conn4 = NextGood(ts2) will close the good connection and break users. + return sc.db, sc.dbVer, nil } if err := health.Health.RemoveReadinessCheck(sc.dsn); err != nil { util.Logger.Warn("health.Health.RemoveReadinessCheck failed", zap.String("dsn", sc.dsn), zap.Error(err)) @@ -103,16 +100,16 @@ func (sc *ShardConn) NextGoodReplica(failAt time.Time) (db *sql.DB, err error) { continue } setDBParams(sqlDB) - util.Logger.Info("sql.Open and sqlDB.Ping succeeded", zap.String("dsn", sc.dsn)) + sc.db = sqlDB + sc.dbVer++ + util.Logger.Info("sql.Open and sqlDB.Ping succeeded", zap.Int("dbVer", sc.dbVer), zap.String("dsn", sc.dsn)) if err = health.Health.AddReadinessCheck(sc.dsn, healthcheck.DatabasePingCheck(sqlDB, 30*time.Second)); err != nil { util.Logger.Warn("health.Health.AddReadinessCheck failed", zap.String("dsn", sc.dsn), zap.Error(err)) } - sc.db = sqlDB - sc.connAt = time.Now() - return sc.db, nil + return sc.db, sc.dbVer, nil } err = errors.Errorf("no good replica among replicas %v since %d", sc.replicas, savedNextRep) - return nil, err + return nil, sc.dbVer, err } func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool) (err error) { @@ -142,7 +139,7 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara sc := &ShardConn{ replicas: replicaAddrs, } - if _, err = sc.NextGoodReplica(time.Now()); err != nil { + if _, _, err = sc.NextGoodReplica(0); err != nil { return } clusterConn = append(clusterConn, sc) From 2549fe7dc2de5d952eb6043b98cd09aac0a3698e Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 18 May 2021 17:25:04 +0800 Subject: [PATCH 113/404] detect DateTime format till success --- docs/dev/introduction.md | 2 +- go.sum | 3 --- parser/fastjson.go | 4 ++-- parser/parser.go | 11 +++++++---- task/task.go | 3 +++ 5 files changed, 13 insertions(+), 10 deletions(-) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index f9edc5a2..8a046ff1 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -28,7 +28,7 @@ Refers to [design](./design.md) for how it works. - [x] UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 - [x] Float32, Float64 - [x] String, FixedString, LowCardinality(String) -- [x] Date, DateTime, DateTime64. Automatically detect [these date formats](https://github.com/housepower/clickhouse_sinker/blob/master/parser/parser.go). +- [x] Date, DateTime, DateTime64. Assuming that all values of a field of kafka message has the same layout, and layouts of each field are unrelated. Automatically detect the layout from [these date layouts](https://github.com/housepower/clickhouse_sinker/blob/master/parser/parser.go) till the first successful detection and reuse that layout forever. - [x] Array(T), where T is one of above basic types - [x] Nullable(T), where T is one of above basic types - [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) diff --git a/go.sum b/go.sum index 615cc565..4acd052a 100644 --- a/go.sum +++ b/go.sum @@ -328,8 +328,6 @@ github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0 github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= -github.com/segmentio/kafka-go v0.4.16 h1:9dt78ehM9qzAkekA60D6A96RlqDzC3hnYYa8y5Szd+U= -github.com/segmentio/kafka-go v0.4.16/go.mod h1:19+Eg7KwrNKy/PFhiIthEPkO8k+ac7/ZYXwYM9Df10w= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= @@ -349,7 +347,6 @@ github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5 github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= diff --git a/parser/fastjson.go b/parser/fastjson.go index 2fc4bc88..94cb0e5d 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -122,7 +122,7 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} val = UnixFloat(f) case fastjson.TypeString: var b []byte - if b, err = v.StringBytes(); err != nil { + if b, err = v.StringBytes(); err != nil || len(b) == 0 { val = Epoch return } @@ -189,7 +189,7 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { t = UnixFloat(f) } case fastjson.TypeString: - if b, err := e.StringBytes(); err != nil { + if b, err := e.StringBytes(); err != nil || len(b) == 0 { t = Epoch } else { t = c.pp.ParseDateTime(key, string(b)) diff --git a/parser/parser.go b/parser/parser.go index d401daf1..68f21af7 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -130,7 +130,8 @@ func (pp *Pool) Put(p Parser) { pp.pool.Put(p) } -// Detect date format for each key at the first message. +// Assuming that all values of a field of kafka message has the same layout, and layouts of each field are unrelated. +// Automatically detect the layout from till the first successful detection and reuse that layout forever. // Return time in UTC. // Return Epoch if parsing fail. func (pp *Pool) ParseDateTime(key string, val string) (t time.Time) { @@ -138,19 +139,21 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time) { var layout string var lay interface{} var ok bool + if val == "" { + t = Epoch + return + } if lay, ok = pp.knownLayouts.Load(key); !ok { t, layout = parseInLocation(val, pp.timeZone) if layout != "" { pp.knownLayouts.Store(key, layout) return } - pp.knownLayouts.Store(key, nil) } - if lay == nil { + if layout, ok = lay.(string); !ok { t = Epoch return } - layout, _ = lay.(string) if t, err = time.ParseInLocation(layout, val, pp.timeZone); err != nil { t = Epoch return diff --git a/task/task.go b/task/task.go index 8810eaef..ca45c8ef 100644 --- a/task/task.go +++ b/task/task.go @@ -242,6 +242,9 @@ func (service *Service) put(msg model.InputMessage) { if taskCfg.DynamicSchema.Enable { foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys) } + // Dumping message and result + //util.Logger.Debug("parsed kafka message", zap.Int("partition", msg.Partition), zap.Int64("offset", msg.Offset), + // zap.String("message value", string(msg.Value)), zap.String("row(spew)", spew.Sdump(row))) } // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. service.pp.Put(p) From 119947e374a191d539b8717d04f09aa2334ed1e8 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 19 May 2021 15:22:50 +0800 Subject: [PATCH 114/404] fix DM issues via http://www.hero-go.com/ --- go.mod | 2 +- go.sum | 7 +++++++ input/kafka_sarama.go | 2 +- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 4ade9c67..72b8d81c 100644 --- a/go.mod +++ b/go.mod @@ -16,7 +16,7 @@ require ( github.com/tidwall/gjson v1.8.0 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/valyala/fastjson v1.6.3 - github.com/xdg/scram v1.0.3 + github.com/xdg-go/scram v1.0.2 go.uber.org/zap v1.16.0 golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba gopkg.in/natefinch/lumberjack.v2 v2.0.0 diff --git a/go.sum b/go.sum index 4acd052a..f131e0b6 100644 --- a/go.sum +++ b/go.sum @@ -372,6 +372,12 @@ github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijb github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= github.com/valyala/fastjson v1.6.3/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= +github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= +github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.0.2 h1:akYIkZ28e6A96dkWNJQu3nmCzH3YfwMPQExUYDaRv7w= +github.com/xdg-go/scram v1.0.2/go.mod h1:1WAq6h33pAW+iRreB34OORO2Nf7qel3VV3fjBj+hCSs= +github.com/xdg-go/stringprep v1.0.2 h1:6iq84/ryjjeRmMJwxutI51F2GIPlP5BfTvXHeYjyhBc= +github.com/xdg-go/stringprep v1.0.2/go.mod h1:8F9zXuvzgwmyT5DUm4GUfZGDdT3W+LCvS6+da4O5kxM= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= github.com/xdg/scram v1.0.3 h1:nTadYh2Fs4BK2xdldEa2g5bbaZp0/+1nJMMPtPxS/to= github.com/xdg/scram v1.0.3/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= @@ -477,6 +483,7 @@ golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9sn golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index a8cf2089..b448520e 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -25,7 +25,7 @@ import ( "github.com/Shopify/sarama" "github.com/pkg/errors" - "github.com/xdg/scram" + "github.com/xdg-go/scram" "go.uber.org/zap" "github.com/housepower/clickhouse_sinker/config" From 97d2697ffe6b8e35c8cd71c3d9f2b79c0fa260c1 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 20 May 2021 14:48:28 +0800 Subject: [PATCH 115/404] fix doc typo --- docs/dev/introduction.md | 4 ++-- docs/guide/run.md | 27 +++++++++++++++------------ 2 files changed, 17 insertions(+), 14 deletions(-) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 8a046ff1..bc407999 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -264,9 +264,9 @@ Controled by: - CLI parameters: `nacos-addr, nacos-username, nacos-password, nacos-namespace-id, nacos-group, nacos-dataid` - env variables: `NACOS_ADDR, NACOS_USERNAME, NACOS_PASSWORD, NACOS_NAMESPACE_ID, NACOS_GROUP, NACOS_DATAID` -### Local Files +### Local Config File -Currently sinker is able to parse local config files at startup, but unable to detect file changes. +Currently sinker is able to parse local config file at startup, but unable to detect file changes. Controled by: - CLI parameters: `local-cfg-file` diff --git a/docs/guide/run.md b/docs/guide/run.md index 48798b2b..1b440356 100644 --- a/docs/guide/run.md +++ b/docs/guide/run.md @@ -16,7 +16,7 @@ Note: Ensure `clickhouse-server` and `kafka` work before running clickhouse_sink `clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema` -> Read more detail descriptions of config in [here](docs/config.md) +> Read more detail descriptions of config in [here](../configuration/config.html) ## Example @@ -25,13 +25,15 @@ Let's follow up a piece of the systest script. * Prepare - let's checkout `clickhouse_sinker` - ``` + + ```bash $ git clone https://github.com/housepower/clickhouse_sinker.git $ cd clickhouse_sinker ``` - let's start standalone clickhouse-server and kafka in container: - ``` + + ```bash $ docker-compose up -d ``` @@ -39,7 +41,7 @@ Let's follow up a piece of the systest script. > It's not the duty for clickhouse_sinker to auto create table, so we should do that manually. - ``` + ```sql CREATE TABLE IF NOT EXISTS test_auto_schema ( `day` Date DEFAULT toDate(time), @@ -52,12 +54,12 @@ Let's follow up a piece of the systest script. ORDER BY (time, name); ``` -* Enable topic is created in kafka +* Create a topic in kafka > I use [kaf](https://github.com/birdayz/kaf) tool to create topics. - ``` - kaf topic create topic1 -p 1 -r 1 + ```bash + $ kaf topic create topic1 -p 1 -r 1 ✅ Created topic! Topic Name: topic1 Partitions: 1 @@ -68,21 +70,22 @@ Let's follow up a piece of the systest script. * Run clickhouse_sinker - ``` - clickhouse_sinker --local-cfg-file docker/test_auto_schema.json + ```bash + $ ./clickhouse_sinker --local-cfg-file docker/test_auto_schema.json ``` * Send messages to the topic - ``` + ```bash echo '{"time" : "2020-12-18T03:38:39.000Z", "name" : "name1", "value" : 1}' | kaf -b '127.0.0.1:9092' produce topic1 echo '{"time" : "2020-12-18T03:38:39.000Z", "name" : "name2", "value" : 2}' | kaf -b '127.0.0.1:9092' produce topic1 echo '{"time" : "2020-12-18T03:38:39.000Z", "name" : "name3", "value" : 3}' | kaf -b '127.0.0.1:9092' produce topic1 ``` - - Check the data in clickhouse - ``` +* Check the data in clickhouse + + ```sql SELECT count() FROM test_auto_schema; 3 rows in set. Elapsed: 0.016 sec. From 7dc460983cd890eea4e51280deb0878ce1fd99cf Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 20 May 2021 16:42:14 +0800 Subject: [PATCH 116/404] fix password escaping bug --- pool/conn.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pool/conn.go b/pool/conn.go index 5699f403..bb1cd6bf 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -37,7 +37,7 @@ import ( var ( lock sync.Mutex clusterConn []*ShardConn - dsnTmpl string + dsnSuffix string ) // ShardConn a datastructure for storing the clickhouse connection @@ -86,7 +86,7 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db *sql.DB, dbVer int, err savedNextRep := sc.nextRep // try all replicas, including the current one for i := 0; i < len(sc.replicas); i++ { - sc.dsn = fmt.Sprintf(dsnTmpl, sc.replicas[sc.nextRep]) + sc.dsn = fmt.Sprintf("tcp://%s", sc.replicas[sc.nextRep]) + dsnSuffix sc.nextRep = (sc.nextRep + 1) % len(sc.replicas) sqlDB, err := sql.Open("clickhouse", sc.dsn) if err != nil { @@ -118,13 +118,13 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara freeClusterConn() // Each shard has a *sql.DB which connects to one replica inside the shard. // "alt_hosts" tolerates replica single-point-failure. However more flexable switching is needed for some cases for example https://github.com/ClickHouse/ClickHouse/issues/24036. - dsnTmpl = "tcp://%s" + fmt.Sprintf("?database=%s&username=%s&password=%s&block_size=%d", + dsnSuffix = fmt.Sprintf("?database=%s&username=%s&password=%s&block_size=%d", url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password), 2*config.MaxBufferSize) if dsnParams != "" { - dsnTmpl += "&" + dsnParams + dsnSuffix += "&" + dsnParams } if secure { - dsnTmpl += "&secure=true&skip_verify=" + strconv.FormatBool(skipVerify) + dsnSuffix += "&secure=true&skip_verify=" + strconv.FormatBool(skipVerify) } for _, replicas := range hosts { From 981cee9fbac63c83d83f050a5f99371e55e5d15f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 21 May 2021 10:01:35 +0800 Subject: [PATCH 117/404] removed MsgSizeHint config --- config/config.go | 5 ----- docs/configuration/config.md | 4 +--- input/kafka_go.go | 8 ++++---- 3 files changed, 5 insertions(+), 12 deletions(-) diff --git a/config/config.go b/config/config.go index 8be5bd7c..ffc42d43 100644 --- a/config/config.go +++ b/config/config.go @@ -142,7 +142,6 @@ type TaskConfig struct { FlushInterval int `json:"flushInterval,omitempty"` BufferSize int `json:"bufferSize,omitempty"` - MsgSizeHint int `json:"msgSizeHint,omitempty"` TimeZone string `json:"timezone"` } @@ -151,7 +150,6 @@ const ( defaultFlushInterval = 5 MaxBufferSize = 1 << 20 //1048576 defaultBufferSize = 1 << 18 //262144 - defaultMsgSizeHint = 1000 defaultTimeZone = "Local" defaultLogLevel = "info" defaultKerberosConfigPath = "/etc/krb5.conf" @@ -224,9 +222,6 @@ func (cfg *Config) Normallize() (err error) { } else { cfg.Task.BufferSize = 1 << util.GetShift(cfg.Task.BufferSize) } - if cfg.Task.MsgSizeHint <= 0 { - cfg.Task.MsgSizeHint = defaultMsgSizeHint - } if cfg.Task.TimeZone == "" { cfg.Task.TimeZone = defaultTimeZone } diff --git a/docs/configuration/config.md b/docs/configuration/config.md index a4fb4463..7231a91f 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -124,9 +124,7 @@ // interval of flushing the batch. Default to 5, max to 10. "flushInterval": 5, // batch size to insert into clickhouse. sinker will round upward it to the the nearest 2^n. Default to 262114, max to 1048576. - "bufferSize": 90000, - // estimated avg message size. kafka-go needs this to determize receive buffer size. Default to 1000. - "msgSizeHint": 1000, + "bufferSize": 262114, // In the absence of time zone information, interprets the time as in the given location. Default to "Local" (aka /etc/localtime of the machine on which sinker runs) "timezone": "" diff --git a/input/kafka_go.go b/input/kafka_go.go index c1e0c8b9..38b85db5 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -64,10 +64,10 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model GroupID: k.cfg.Task.ConsumerGroup, Topic: k.cfg.Task.Topic, StartOffset: offset, - MinBytes: (k.cfg.Task.BufferSize / 2) * k.cfg.Task.MsgSizeHint, - MaxBytes: k.cfg.Task.BufferSize * k.cfg.Task.MsgSizeHint, - MaxWait: time.Duration(3) * time.Second, - CommitInterval: time.Second, // flushes commits to Kafka every second + MinBytes: 1, // sarama.Consumer.Fetch.Min + MaxBytes: 100 * 1024 * 1024, // sarama.MaxResponseSize + MaxWait: time.Duration(250) * time.Millisecond, // sarama.Consumer.MaxWaitTime + CommitInterval: time.Second, // flushes commits to Kafka every second } if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { if kfkCfg.TLS.CaCertFiles, _, err = util.JksToPem(kfkCfg.TLS.TrustStoreLocation, kfkCfg.TLS.TrustStorePassword, false); err != nil { From 93d15ad0f9e572d30bac0d07f17c681b282ef5e8 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 21 May 2021 11:22:09 +0800 Subject: [PATCH 118/404] handle goetty.ErrSystemStopped --- task/ring.go | 15 +++++++++++---- task/sharding.go | 8 ++++++-- task/task.go | 23 ++++++++++++++++++----- 3 files changed, 35 insertions(+), 11 deletions(-) diff --git a/task/ring.go b/task/ring.go index 29ae21d9..5c7170d3 100644 --- a/task/ring.go +++ b/task/ring.go @@ -66,8 +66,11 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { // reschedule the delayed ForceBatchOrShard ring.tid.Stop() if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { - err = errors.Wrap(err, "") - util.Logger.Fatal("sheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) + if errors.Is(err, goetty.ErrSystemStopped) { + util.Logger.Info("Ring.PutElem scheduling timer to a stopped timer wheel", zap.String("task", taskCfg.Name), zap.Error(err)) + } else { + util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) + } } } } @@ -127,8 +130,12 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { ring.tid.Stop() var err error if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { - err = errors.Wrap(err, "") - util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) + if errors.Is(err, goetty.ErrSystemStopped) { + util.Logger.Info("Ring.ForceBatchOrShard scheduling timer to a stopped timer wheel", zap.String("task", taskCfg.Name), zap.Error(err)) + } else { + err = errors.Wrap(err, "") + util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) + } } } diff --git a/task/sharding.go b/task/sharding.go index 4aab46b3..bcb8ffe0 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -232,7 +232,11 @@ func (sh *Sharder) doFlush(_ interface{}) { // reschedule the delayed ForceFlush sh.tid.Stop() if sh.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, sh.ForceFlush, nil); err != nil { - err = errors.Wrap(err, "") - util.Logger.Fatal("scheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) + if errors.Is(err, goetty.ErrSystemStopped) { + util.Logger.Info("Sharder.doFlush scheduling timer to a stopped timer wheel") + } else { + err = errors.Wrap(err, "") + util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) + } } } diff --git a/task/task.go b/task/task.go index ca45c8ef..23c5cc86 100644 --- a/task/task.go +++ b/task/task.go @@ -134,8 +134,12 @@ func (service *Service) Run(ctx context.Context) { if service.sharder != nil { // schedule a delayed ForceFlush if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(service.cfg.Task.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { - err = errors.Wrap(err, "") - util.Logger.Fatal("scheduling timer failed", zap.String("task", service.cfg.Task.Name), zap.Error(err)) + if errors.Is(err, goetty.ErrSystemStopped) { + util.Logger.Info("Service.Run scheduling timer to a stopped timer wheel") + } else { + err = errors.Wrap(err, "") + util.Logger.Fatal("scheduling timer filed", zap.String("task", service.cfg.Task.Name), zap.Error(err)) + } } } @@ -191,8 +195,12 @@ func (service *Service) put(msg model.InputMessage) { } // schedule a delayed ForceBatchOrShard if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { - err = errors.Wrap(err, "") - util.Logger.Fatal("scheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) + if errors.Is(err, goetty.ErrSystemStopped) { + util.Logger.Info("Service.put scheduling timer to a stopped timer wheel") + } else { + err = errors.Wrap(err, "") + util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) + } } service.rings[msg.Partition] = ring service.Unlock() @@ -263,7 +271,12 @@ func (service *Service) put(msg model.InputMessage) { service.sharder.ForceFlush(nil) } if service.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.changeSchema, nil); err != nil { - util.Logger.Fatal("scheduling timer failed", zap.String("task", service.cfg.Task.Name), zap.Error(err)) + if errors.Is(err, goetty.ErrSystemStopped) { + util.Logger.Info("Service.put scheduling timer to a stopped timer wheel") + } else { + err = errors.Wrap(err, "") + util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) + } } } } From 167b0b42abcde49b3b9025e866f35fc42d736844 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 21 May 2021 11:37:45 +0800 Subject: [PATCH 119/404] use httpAddr as instance in metrics --- cmd/clickhouse_sinker/main.go | 92 ++++++++++++++++++----------------- statistics/statistics.go | 41 +--------------- statistics/statistics_test.go | 23 +++------ 3 files changed, 55 insertions(+), 101 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index d2fabf05..f9d7a4ff 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -65,6 +65,7 @@ var ( cmdOps CmdOptions selfIP string + httpAddr string httpMetrics = promhttp.Handler() runner *Sinker ) @@ -146,6 +147,51 @@ func GenTask(cfg *config.Config) (taskImpl *task.Service) { func main() { util.Run("clickhouse_sinker", func() error { + // Initialize http server for metrics and debug + mux := http.NewServeMux() + mux.HandleFunc("/", func(w http.ResponseWriter, r *http.Request) { + _, _ = w.Write([]byte(` + ClickHouse Sinker + +

ClickHouse Sinker

+

Metrics

+

Ready

+

Ready Full

+

Live

+

Live Full

+

pprof

+ `)) + }) + + mux.Handle("/metrics", httpMetrics) + mux.HandleFunc("/ready", health.Health.ReadyEndpoint) // GET /ready?full=1 + mux.HandleFunc("/live", health.Health.LiveEndpoint) // GET /live?full=1 + mux.HandleFunc("/debug/pprof/", pprof.Index) + mux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) + mux.HandleFunc("/debug/pprof/profile", pprof.Profile) + mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) + mux.HandleFunc("/debug/pprof/trace", pprof.Trace) + + // cmdOps.HTTPPort=0: let OS choose the listen port, and record the exact metrics URL to log. + httpPort := cmdOps.HTTPPort + if httpPort != 0 { + httpPort = util.GetSpareTCPPort(httpPort) + } + httpAddr = fmt.Sprintf(":%d", httpPort) + listener, err := net.Listen("tcp", httpAddr) + if err != nil { + util.Logger.Fatal("net.Listen failed", zap.String("httpAddr", httpAddr), zap.Error(err)) + } + httpPort = util.GetNetAddrPort(listener.Addr()) + httpAddr = fmt.Sprintf("%s:%d", selfIP, httpPort) + util.Logger.Info(fmt.Sprintf("Run http server at http://%s/", httpAddr)) + + go func() { + if err := http.Serve(listener, mux); err != nil { + util.Logger.Error("http.ListenAndServe failed", zap.Error(err)) + } + }() + var rcm config.RemoteConfManager var properties map[string]interface{} if cmdOps.NacosDataID != "" { @@ -170,50 +216,6 @@ func main() { runner = NewSinker(rcm) return runner.Init() }, func() error { - go func() { - mux := http.NewServeMux() - mux.HandleFunc("/", func(w http.ResponseWriter, r *http.Request) { - _, _ = w.Write([]byte(` - ClickHouse Sinker - -

ClickHouse Sinker

-

Metrics

-

Ready

-

Ready Full

-

Live

-

Live Full

-

pprof

- `)) - }) - - mux.Handle("/metrics", httpMetrics) - mux.HandleFunc("/ready", health.Health.ReadyEndpoint) // GET /ready?full=1 - mux.HandleFunc("/live", health.Health.LiveEndpoint) // GET /live?full=1 - - mux.HandleFunc("/debug/pprof/", pprof.Index) - mux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) - mux.HandleFunc("/debug/pprof/profile", pprof.Profile) - mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) - mux.HandleFunc("/debug/pprof/trace", pprof.Trace) - - // cmdOps.HTTPPort=0: let OS choose the listen port, and record the exact metrics URL to log. - httpPort := cmdOps.HTTPPort - if httpPort != 0 { - httpPort = util.GetSpareTCPPort(httpPort) - } - httpAddr := fmt.Sprintf(":%d", httpPort) - listener, err := net.Listen("tcp", httpAddr) - if err != nil { - util.Logger.Fatal("net.Listen failed", zap.String("httpAddr", httpAddr), zap.Error(err)) - } - httpPort = util.GetNetAddrPort(listener.Addr()) - util.Logger.Info(fmt.Sprintf("Run http server at http://%s:%d/", selfIP, httpPort)) - - if err := http.Serve(listener, mux); err != nil { - util.Logger.Error("http.ListenAndServe failed", zap.Error(err)) - } - }() - runner.Run() return nil }, func() error { @@ -249,7 +251,7 @@ func (s *Sinker) Run() { var newCfg *config.Config if cmdOps.PushGatewayAddrs != "" { addrs := strings.Split(cmdOps.PushGatewayAddrs, ",") - s.pusher = statistics.NewPusher(addrs, cmdOps.PushInterval) + s.pusher = statistics.NewPusher(addrs, cmdOps.PushInterval, httpAddr) if err = s.pusher.Init(); err != nil { return } diff --git a/statistics/statistics.go b/statistics/statistics.go index 7e052796..cb21897e 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -17,7 +17,6 @@ package statistics import ( "context" "math/rand" - "net" "time" "github.com/housepower/clickhouse_sinker/util" @@ -185,11 +184,12 @@ type Pusher struct { cancel context.CancelFunc } -func NewPusher(addrs []string, interval int) *Pusher { +func NewPusher(addrs []string, interval int, selfAddr string) *Pusher { return &Pusher{ pgwAddrs: addrs, pushInterval: interval, inUseAddr: -1, + instance: selfAddr, } } @@ -201,7 +201,6 @@ func (p *Pusher) Init() error { if len(p.pgwAddrs) == 0 || p.pushInterval <= 0 { return errPgwEmpty } - p.instance = p.getInstance() p.reconnect() return nil } @@ -258,39 +257,3 @@ func (p *Pusher) reconnect() { Grouping("instance", p.instance).Format(expfmt.FmtText) p.inUseAddr = nextAddr } - -func (p *Pusher) getInstance() string { - addrs, err := net.InterfaceAddrs() - if err != nil { - return "unknown" - } - - for _, addr := range addrs { - if ipnet, ok := addr.(*net.IPNet); ok { - if p.IsExternalIP(ipnet.IP) { - return ipnet.IP.String() - } - } - } - return "unknown" -} - -func (p *Pusher) IsExternalIP(ip net.IP) bool { - if ip.IsLoopback() || ip.IsLinkLocalMulticast() || ip.IsLinkLocalUnicast() { - return false - } - - if ip4 := ip.To4(); ip4 != nil { - switch { - case ip4[0] == 10: - return false - case ip4[0] == 172 && ip4[1] >= 16 && ip4[1] <= 31: - return false - case ip4[0] == 192 && ip4[1] == 168: - return false - default: - return true - } - } - return false -} diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 00a31382..c8b6ee96 100755 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -16,32 +16,21 @@ package statistics import ( "context" - "net" + "fmt" "testing" "time" + "github.com/housepower/clickhouse_sinker/util" "github.com/stretchr/testify/require" ) -func TestPusher_IsExternalIP(t *testing.T) { - addrs := []string{"172.24.25.1:9091"} - interval := 30 - pusher := NewPusher(addrs, interval) - - external := pusher.IsExternalIP(net.ParseIP("192.168.154.134")) - require.Equal(t, false, external, "192.168.154.134 should not be external ip") - - external = pusher.IsExternalIP(net.ParseIP("127.0.0.1")) - require.Equal(t, false, external, "127.0.0.1 should not be external ip") - - external = pusher.IsExternalIP(net.ParseIP("43.230.88.7")) - require.Equal(t, true, external, "43.230.88.7 should be external ip") -} - func TestPusher(t *testing.T) { addrs := []string{"172.24.25.1:9091", "172.24.25.2:9091"} interval := 1 - pusher := NewPusher(addrs, interval) + selfIP, _ := util.GetOutboundIP() + selfPort := util.GetSpareTCPPort(1024) + selfAddr := fmt.Sprintf("%s:%d", selfIP, selfPort) + pusher := NewPusher(addrs, interval, selfAddr) err := pusher.Init() require.Nilf(t, err, "pusher init failed") From 62b07ad8526b0c2d9a7dd30e9c18d982765e2e78 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 31 May 2021 15:57:14 +0800 Subject: [PATCH 120/404] handle type mismatch for nullable fields --- parser/csv.go | 10 +++- parser/fastjson.go | 111 +++++++++++++++++++++++++++++++++--------- parser/gjson.go | 83 +++++++++++++++++++++++-------- parser/parser.go | 27 +++++----- parser/parser_test.go | 60 +++++++++++------------ 5 files changed, 204 insertions(+), 87 deletions(-) diff --git a/parser/csv.go b/parser/csv.go index 2519d2aa..d37372bf 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -123,7 +123,10 @@ func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}) { } s := c.values[idx] if dd, err := strconv.ParseFloat(s, 64); err != nil { - val = c.pp.ParseDateTime(key, s) + var err error + if val, err = c.pp.ParseDateTime(key, s); err != nil { + val = Epoch + } } else { val = UnixFloat(dd) } @@ -205,7 +208,10 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { case gjson.Number: t = UnixFloat(e.Num) case gjson.String: - t = c.pp.ParseDateTime(key, e.Str) + var err error + if t, err = c.pp.ParseDateTime(key, e.Str); err != nil { + t = Epoch + } default: t = Epoch } diff --git a/parser/fastjson.go b/parser/fastjson.go index 94cb0e5d..c94d22e1 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -71,24 +71,22 @@ func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}) func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { v := c.value.Get(key) - if v == nil || v.Type() == fastjson.TypeNull { - if nullable { - return - } - val = float64(0.0) + if !fjCompatibleFloat(v) { + val = getDefaultFloat(nullable) return } - val, _ = v.Float64() + if val2, err := v.Float64(); err != nil { + val = getDefaultFloat(nullable) + } else { + val = val2 + } return } func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}) { v := c.value.Get(key) - if v == nil || v.Type() == fastjson.TypeNull { - if nullable { - return - } - val = int64(0) + if !fjCompatibleInt(v) { + val = getDefaultInt(nullable) return } switch v.Type() { @@ -97,18 +95,19 @@ func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}) { case fastjson.TypeFalse: val = int64(0) default: - val, _ = v.Int64() + if val2, err := v.Int64(); err != nil { + val = getDefaultInt(nullable) + } else { + val = val2 + } } return } func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { v := c.value.Get(key) - if v == nil || v.Type() == fastjson.TypeNull { - if nullable { - return - } - val = Epoch + if !fjCompatibleDateTime(v) { + val = getDefaultDateTime(nullable) return } var err error @@ -116,19 +115,21 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} case fastjson.TypeNumber: var f float64 if f, err = v.Float64(); err != nil { - val = Epoch + val = getDefaultDateTime(nullable) return } val = UnixFloat(f) case fastjson.TypeString: var b []byte if b, err = v.StringBytes(); err != nil || len(b) == 0 { - val = Epoch + val = getDefaultDateTime(nullable) return } - val = c.pp.ParseDateTime(key, string(b)) + if val, err = c.pp.ParseDateTime(key, string(b)); err != nil { + val = getDefaultDateTime(nullable) + } default: - val = Epoch + val = getDefaultDateTime(nullable) } return } @@ -192,7 +193,10 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { if b, err := e.StringBytes(); err != nil || len(b) == 0 { t = Epoch } else { - t = c.pp.ParseDateTime(key, string(b)) + var err error + if t, err = c.pp.ParseDateTime(key, string(b)); err != nil { + t = Epoch + } } default: t = Epoch @@ -225,6 +229,69 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (fou return } +func fjCompatibleInt(v *fastjson.Value) (ok bool) { + if v == nil { + return + } + switch v.Type() { + case fastjson.TypeTrue: + ok = true + case fastjson.TypeFalse: + ok = true + case fastjson.TypeNumber: + ok = true + } + return +} + +func fjCompatibleFloat(v *fastjson.Value) (ok bool) { + if v == nil { + return + } + switch v.Type() { + case fastjson.TypeNumber: + ok = true + } + return +} + +func fjCompatibleDateTime(v *fastjson.Value) (ok bool) { + if v == nil { + return + } + switch v.Type() { + case fastjson.TypeNumber: + ok = true + case fastjson.TypeString: + ok = true + } + return +} + +func getDefaultInt(nullable bool) (val interface{}) { + if nullable { + return + } + val = int64(0) + return +} + +func getDefaultFloat(nullable bool) (val interface{}) { + if nullable { + return + } + val = float64(0.0) + return +} + +func getDefaultDateTime(nullable bool) (val interface{}) { + if nullable { + return + } + val = Epoch + return +} + func fjDetectType(v *fastjson.Value) (typ int) { switch v.Type() { case fastjson.TypeNull: diff --git a/parser/gjson.go b/parser/gjson.go index ac4a39a9..38d0cac2 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -64,62 +64,58 @@ func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}) { func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) - if !r.Exists() || r.Type == gjson.Null { - if nullable { - return - } - val = float64(0.0) + if !gjCompatibleFloat(r) { + val = getDefaultFloat(nullable) return } switch r.Type { case gjson.Number: val = r.Num default: - val = float64(0.0) + val = getDefaultFloat(nullable) } return } func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) - if !r.Exists() || r.Type == gjson.Null { - if nullable { - return - } - val = int64(0) + if !gjCompatibleInt(r) { + val = getDefaultInt(nullable) return } switch r.Type { case gjson.True: val = int64(1) + case gjson.False: + val = int64(0) case gjson.Number: if v := r.Int(); float64(v) != r.Num { - val = int64(0) + val = getDefaultInt(nullable) } else { val = v } default: - val = int64(0) + val = getDefaultInt(nullable) } return } func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) - if !r.Exists() || r.Type == gjson.Null { - if nullable { - return - } - val = Epoch + if !gjCompatibleDateTime(r) { + val = getDefaultDateTime(nullable) return } switch r.Type { case gjson.Number: val = UnixFloat(r.Num) case gjson.String: - val = c.pp.ParseDateTime(key, r.Str) + var err error + if val, err = c.pp.ParseDateTime(key, r.Str); err != nil { + val = getDefaultDateTime(nullable) + } default: - val = Epoch + val = getDefaultDateTime(nullable) } return } @@ -193,7 +189,10 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { case gjson.Number: t = UnixFloat(e.Num) case gjson.String: - t = c.pp.ParseDateTime(key, e.Str) + var err error + if t, err = c.pp.ParseDateTime(key, e.Str); err != nil { + t = Epoch + } default: t = Epoch } @@ -209,3 +208,45 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { func (c *GjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool { return false } + +func gjCompatibleInt(r gjson.Result) (ok bool) { + if !r.Exists() { + return + } + switch r.Type { + case gjson.True: + ok = true + case gjson.False: + ok = true + case gjson.Number: + ok = true + default: + } + return +} + +func gjCompatibleFloat(r gjson.Result) (ok bool) { + if !r.Exists() { + return + } + switch r.Type { + case gjson.Number: + ok = true + default: + } + return +} + +func gjCompatibleDateTime(r gjson.Result) (ok bool) { + if !r.Exists() { + return + } + switch r.Type { + case gjson.Number: + ok = true + case gjson.String: + ok = true + default: + } + return +} diff --git a/parser/parser.go b/parser/parser.go index 68f21af7..56aee971 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -61,7 +61,8 @@ var ( "Jan 02, 2006", "Mon Jan 02, 2006", } - Epoch = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC) + Epoch = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC) + ErrParseDateTime = errors.Errorf("value doesn't contain DateTime") ) // Parse is the Parser interface @@ -133,32 +134,34 @@ func (pp *Pool) Put(p Parser) { // Assuming that all values of a field of kafka message has the same layout, and layouts of each field are unrelated. // Automatically detect the layout from till the first successful detection and reuse that layout forever. // Return time in UTC. -// Return Epoch if parsing fail. -func (pp *Pool) ParseDateTime(key string, val string) (t time.Time) { - var err error +func (pp *Pool) ParseDateTime(key string, val string) (t time.Time, err error) { var layout string var lay interface{} var ok bool + var t2 time.Time if val == "" { - t = Epoch + err = ErrParseDateTime return } if lay, ok = pp.knownLayouts.Load(key); !ok { - t, layout = parseInLocation(val, pp.timeZone) - if layout != "" { - pp.knownLayouts.Store(key, layout) + t2, layout = parseInLocation(val, pp.timeZone) + if layout == "" { + err = ErrParseDateTime return } + t = t2 + pp.knownLayouts.Store(key, layout) + return } if layout, ok = lay.(string); !ok { - t = Epoch + err = ErrParseDateTime return } - if t, err = time.ParseInLocation(layout, val, pp.timeZone); err != nil { - t = Epoch + if t2, err = time.ParseInLocation(layout, val, pp.timeZone); err != nil { + err = ErrParseDateTime return } - t = t.UTC() + t = t2.UTC() return } diff --git a/parser/parser_test.go b/parser/parser_test.go index 60e5a88a..5ff34eae 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -214,7 +214,7 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { for j := range testCases { var v interface{} desc := fmt.Sprintf(`%s %s("%s", %s)`, name, method, testCases[j].Field, strconv.FormatBool(testCases[j].Nullable)) - if name == "csv" && sliceContains([]string{"GetInt", "GetFloat", "GetDateTime", "GetElasticDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) { + if name == "csv" && (sliceContains([]string{"GetInt", "GetFloat", "GetDateTime", "GetElasticDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || testCases[j].Nullable) { log.Printf("%s is known to not compatible with fastjson parser, skipping", desc) continue } @@ -258,13 +258,13 @@ func TestParserInt(t *testing.T) { {"bool_true", true, int64(1)}, {"bool_false", true, int64(0)}, {"num_int", true, int64(123)}, - {"num_float", true, int64(0)}, - {"str", true, int64(0)}, - {"str_int", true, int64(0)}, - {"str_float", true, int64(0)}, - {"str_date_1", true, int64(0)}, - {"obj", true, int64(0)}, - {"array_empty", true, int64(0)}, + {"num_float", true, nil}, + {"str", true, nil}, + {"str_int", true, nil}, + {"str_float", true, nil}, + {"str_date_1", true, nil}, + {"obj", true, nil}, + {"array_empty", true, nil}, } doTestSimple(t, "GetInt", testCases) } @@ -287,16 +287,16 @@ func TestParserFloat(t *testing.T) { // nullable: true {"not_exist", true, nil}, {"null", true, nil}, - {"bool_true", true, 0.0}, - {"bool_false", true, 0.0}, + {"bool_true", true, nil}, + {"bool_false", true, nil}, {"num_int", true, 123.0}, {"num_float", true, 123.321}, - {"str", true, 0.0}, - {"str_int", true, 0.0}, - {"str_float", true, 0.0}, - {"str_date_1", true, 0.0}, - {"obj", true, 0.0}, - {"array_empty", true, 0.0}, + {"str", true, nil}, + {"str_int", true, nil}, + {"str_float", true, nil}, + {"str_date_1", true, nil}, + {"obj", true, nil}, + {"array_empty", true, nil}, } doTestSimple(t, "GetFloat", testCases) } @@ -361,20 +361,20 @@ func TestParserDateTime(t *testing.T) { // nullable: true {"not_exist", true, nil}, {"null", true, nil}, - {"bool_true", true, Epoch}, - {"bool_false", true, Epoch}, + {"bool_true", true, nil}, + {"bool_false", true, nil}, {"num_int", true, UnixInt(123)}, {"num_float", true, UnixFloat(123.321)}, - {"str", true, Epoch}, - {"str_int", true, Epoch}, - {"str_float", true, Epoch}, + {"str", true, nil}, + {"str_int", true, nil}, + {"str_float", true, nil}, {"str_date_1", true, bdLocalDate}, {"str_time_rfc3339_1", true, bdUtcSec}, {"str_time_rfc3339_2", true, bdShNs}, {"str_time_clickhouse_1", true, bdLocalSec}, {"str_time_clickhouse_2", true, bdLocalNs}, - {"obj", true, Epoch}, - {"array_empty", true, Epoch}, + {"obj", true, nil}, + {"array_empty", true, nil}, } doTestSimple(t, "GetDateTime", testCases) } @@ -401,20 +401,20 @@ func TestParserElasticDateTime(t *testing.T) { // nullable: true {"not_exist", true, nil}, {"null", true, nil}, - {"bool_true", true, Epoch.Unix()}, - {"bool_false", true, Epoch.Unix()}, + {"bool_true", true, nil}, + {"bool_false", true, nil}, {"num_int", true, UnixInt(123).Unix()}, {"num_float", true, UnixFloat(123.321).Unix()}, - {"str", true, Epoch.Unix()}, - {"str_int", true, Epoch.Unix()}, - {"str_float", true, Epoch.Unix()}, + {"str", true, nil}, + {"str_int", true, nil}, + {"str_float", true, nil}, {"str_date_1", true, bdLocalDate.Unix()}, {"str_time_rfc3339_1", true, bdUtcSec.Unix()}, {"str_time_rfc3339_2", true, bdShNs.Unix()}, {"str_time_clickhouse_1", true, bdLocalSec.Unix()}, {"str_time_clickhouse_2", true, bdLocalNs.Unix()}, - {"obj", true, Epoch.Unix()}, - {"array_empty", true, Epoch.Unix()}, + {"obj", true, nil}, + {"array_empty", true, nil}, } doTestSimple(t, "GetElasticDateTime", testCases) } From 688772c4fa4bc266d279af697dd0d727676b9d86 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 2 Jun 2021 13:42:58 +0800 Subject: [PATCH 121/404] fix RingMsgs --- task/ring.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/task/ring.go b/task/ring.go index 5c7170d3..0f8af519 100644 --- a/task/ring.go +++ b/task/ring.go @@ -200,7 +200,7 @@ func (ring *Ring) genBatchOrShard(expNewGroundOff int64) { statistics.RingForceBatchAllGapTotal.WithLabelValues(taskCfg.Name).Inc() } } - statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(batch.RealSize)) + statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } ring.ringGroundOff = endOff From 7e4026568493e5ed74e7120174b087479f707a5d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 2 Jun 2021 14:23:15 +0800 Subject: [PATCH 122/404] delete metric group on quit --- statistics/statistics.go | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/statistics/statistics.go b/statistics/statistics.go index cb21897e..a8ea83b2 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -212,21 +212,27 @@ FOR: for { select { case <-ticker.C: - err := p.pusher.Push() - if err != nil { + if err := p.pusher.Push(); err != nil { err = errors.Wrapf(err, "") util.Logger.Error("pushing metrics failed", zap.Error(err)) p.reconnect() } - case <-ctx.Done(): - util.Logger.Warn("metric pusher quit due to context has been canceled") + case <-p.ctx.Done(): + util.Logger.Info("Pusher.Run quit due to context has been canceled") break FOR } } } func (p *Pusher) Stop() { + // https://stackoverflow.com/questions/63540280/how-to-set-a-retention-time-for-pushgateway-for-metrics-to-expire + // https://github.com/prometheus/pushgateway/issues/19 + if err := p.pusher.Delete(); err != nil { + err = errors.Wrapf(err, "") + util.Logger.Error("failed to delete metric group", zap.String("pushgateway", p.pgwAddrs[p.inUseAddr]), zap.String("job", "clickhouse_sinker"), zap.String("instance", p.instance), zap.Error(err)) + } p.cancel() + util.Logger.Info("stopped metric pusher") } func (p *Pusher) reconnect() { From 8417dcd51d73b1aafedce641fe58ae333a23c431 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 2 Jun 2021 22:13:45 +0800 Subject: [PATCH 123/404] fix TestPusher --- parser/parser_test.go | 2 ++ statistics/statistics.go | 3 ++- statistics/statistics_test.go | 1 + 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/parser/parser_test.go b/parser/parser_test.go index 5ff34eae..79d172ea 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -26,6 +26,7 @@ import ( "time" "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" "github.com/stretchr/testify/require" "github.com/tidwall/gjson" "github.com/valyala/fastjson" @@ -193,6 +194,7 @@ func initMetrics() { } metrics[name] = metric } + util.InitLogger("info", []string{"stdout"}) } func sliceContains(list []string, target string) bool { diff --git a/statistics/statistics.go b/statistics/statistics.go index a8ea83b2..40bd370f 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -229,7 +229,8 @@ func (p *Pusher) Stop() { // https://github.com/prometheus/pushgateway/issues/19 if err := p.pusher.Delete(); err != nil { err = errors.Wrapf(err, "") - util.Logger.Error("failed to delete metric group", zap.String("pushgateway", p.pgwAddrs[p.inUseAddr]), zap.String("job", "clickhouse_sinker"), zap.String("instance", p.instance), zap.Error(err)) + util.Logger.Error("failed to delete metric group", zap.String("pushgateway", p.pgwAddrs[p.inUseAddr]), + zap.String("job", "clickhouse_sinker"), zap.String("instance", p.instance), zap.Error(err)) } p.cancel() util.Logger.Info("stopped metric pusher") diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index c8b6ee96..63aa5420 100755 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -25,6 +25,7 @@ import ( ) func TestPusher(t *testing.T) { + util.InitLogger("debug", []string{"stdout"}) addrs := []string{"172.24.25.1:9091", "172.24.25.2:9091"} interval := 1 selfIP, _ := util.GetOutboundIP() From 0720e5fb5d8970982cacfc466432cf55b1d31952 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 3 Jun 2021 10:21:34 +0800 Subject: [PATCH 124/404] enlarge parsing pool channel size --- util/common.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/common.go b/util/common.go index 421a47e8..55e4ed68 100644 --- a/util/common.go +++ b/util/common.go @@ -69,7 +69,7 @@ func InitGlobalParsingPool() { } else { maxWorkers = 1 } - GlobalParsingPool = NewWorkerPool(maxWorkers, 100*runtime.NumCPU()) + GlobalParsingPool = NewWorkerPool(maxWorkers, 1<<16) Logger.Info("initialized parsing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", 100*runtime.NumCPU())) } From e2aeeef5234cc972d7631c37d4a13560f0691cf3 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 3 Jun 2021 14:36:32 +0800 Subject: [PATCH 125/404] added kafka_gen_log and kafka_gen_metric --- .github/workflows/lint.yml | 2 +- Dockerfile | 2 + Dockerfile_goreleaser | 2 + Makefile | 6 +- cmd/kafka_gen_log/main.go | 379 +++++++++++++++++++++++++++++++++++ cmd/kafka_gen_metric/main.go | 202 +++++++++++++++++++ go.mod | 1 + go.sum | 2 + 8 files changed, 594 insertions(+), 2 deletions(-) create mode 100644 cmd/kafka_gen_log/main.go create mode 100644 cmd/kafka_gen_metric/main.go diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 6f96e84e..fdb6b28b 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -20,7 +20,7 @@ jobs: # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,revive + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,revive,tagliatelle # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true diff --git a/Dockerfile b/Dockerfile index 24c80d52..153f058b 100644 --- a/Dockerfile +++ b/Dockerfile @@ -10,6 +10,8 @@ RUN apk --no-cache add ca-certificates tzdata RUN echo "UTC" > /etc/timezone COPY --from=builder /app/clickhouse_sinker /usr/local/bin/clickhouse_sinker COPY --from=builder /app/nacos_publish_config /usr/local/bin/nacos_publish_config +COPY --from=builder /app/kafka_gen_log /usr/local/bin/kafka_gen_log +COPY --from=builder /app/kafka_gen_metric /usr/local/bin/kafka_gen_metric # clickhouse_sinker gets config from local directory "/etc/clickhouse_sinker" by default. # Customize behavior with following env variables: diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index 9b8a7730..43846abc 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -3,6 +3,8 @@ RUN apk --no-cache add ca-certificates tzdata RUN echo "UTC" > /etc/timezone ADD clickhouse_sinker /usr/local/bin/clickhouse_sinker ADD nacos_publish_config /usr/local/bin/nacos_publish_config +ADD kafka_gen_log /usr/local/bin/kafka_gen_log +ADD kafka_gen_metric /usr/local/bin/kafka_gen_metric # clickhouse_sinker gets config from local directory "/etc/clickhouse_sinker" by default. # Customize behavior with following env variables: diff --git a/Makefile b/Makefile index 7597b935..9a5612ed 100644 --- a/Makefile +++ b/Makefile @@ -11,9 +11,13 @@ pre: build: pre $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o clickhouse_sinker cmd/clickhouse_sinker/main.go $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o nacos_publish_config cmd/nacos_publish_config/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o kafka_gen_log cmd/kafka_gen_log/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o kafka_gen_metric cmd/kafka_gen_metric/main.go debug: pre $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o clickhouse_sinker cmd/clickhouse_sinker/main.go $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o nacos_publish_config cmd/nacos_publish_config/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o kafka_gen_log cmd/kafka_gen_log/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o kafka_gen_metric cmd/kafka_gen_metric/main.go unittest: pre go test -v ./... benchtest: pre @@ -21,6 +25,6 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,revive + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,revive,tagliatelle run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.json diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go new file mode 100644 index 00000000..ad3143b3 --- /dev/null +++ b/cmd/kafka_gen_log/main.go @@ -0,0 +1,379 @@ +package main + +/* +CREATE TABLE apache_access_log ON CLUSTER abc ( + `@collectiontime` DateTime, + `@hostname` LowCardinality(String), + `@ip` LowCardinality(String), + `@path` String, + `@lineno` Int64, + `@message` String, + agent String, + auth String, + bytes Int64, + clientIp String, + device_family LowCardinality(String), + httpversion LowCardinality(String), + ident String, + os_family LowCardinality(String), + os_major LowCardinality(String), + os_minor LowCardinality(String), + referrer String, + request String, + requesttime Float64, + response LowCardinality(String), + timestamp DateTime64(3), + userAgent_family LowCardinality(String), + userAgent_major LowCardinality(String), + userAgent_minor LowCardinality(String), + verb LowCardinality(String), + xforwardfor LowCardinality(String) +) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/default/apache_access_log', '{replica}') +PARTITION BY toYYYYMMDD(timestamp) +ORDER BY (timestamp, `@hostname`, `@path`, `@lineno`); + +CREATE TABLE dist_apache_access_log ON CLUSTER abc AS apache_access_log ENGINE = Distributed(abc, default, apache_access_log); + +*/ + +import ( + "bufio" + "context" + "encoding/json" + "flag" + "fmt" + "math/rand" + "os" + "os/signal" + "path/filepath" + "regexp" + "sort" + "strings" + "sync" + "sync/atomic" + "syscall" + "time" + + "github.com/pkg/errors" + kafka "github.com/segmentio/kafka-go" + log "github.com/sirupsen/logrus" +) + +const ( + NumGenerators = 1 +) + +var ( + KafkaBrokers string + KafkaTopic string + LogfileDir string + LogfilePattern string + + ListHostname = []string{"vm101101", "vm101102", "vm101103", "vm101104", "vm101105", "vm101106", "vm101107", "vm101108", "vm101109", "vm101110"} + ListIP = []string{"192.168.101.101", + "192.168.101.102", + "192.168.101.103", + "192.168.101.104", + "192.168.101.105", + "192.168.101.106", + "192.168.101.107", + "192.168.101.108", + "192.168.101.109", + "192.168.101.110"} + ListAgent = []string{"Mozilla/5.0(Windows NT 6.1; Win64; x64)AppleWebKit/537.36(KHTML,like Gecko)Chrome/69.0.3497.100Safari/537.36"} + ListAuth = []string{"RFC1413身份"} + ListClientIP = []string{"192.168.1.1", "192.168.1.2", "192.168.1.3", "192.168.1.4", "192.168.1.5"} + ListDeviceFamily = []string{"Hawei", "Xiaomi", "OPPO", "Apple", "Other"} + ListHttpversion = []string{"1.0", "1.1", "2.0", "3.0"} + ListOsFamily = []string{"Android", "Mac OS X", "HMS"} + ListOsMajor = []string{"6", "7", "8", "9", "10"} + ListOsMinor = []string{"0", "1", "2", "3"} + ListResponse = []string{"200", "301", "400", "404", "503"} + ListUserAgentFamily = []string{"Chrome", "Firefox", "AppleWebKit"} + ListUserAgentMajor = []string{"75", "76", "77", "78", "79", "80", "81"} + ListUserAgentMinor = []string{"0", "1", "2", "3"} + ListVerb = []string{"GET", "POST", "HEAD"} +) + +// generated by https://mholt.github.io/json-to-go/, https://transform.tools/json-to-go +type Log struct { + Collectiontime time.Time `json:"@collectiontime"` + Hostname string `json:"@hostname"` + IP string `json:"@ip"` + Path string `json:"@path"` + LineNo int `json:"@lineno"` + Message string `json:"@message"` + Agent string `json:"agent"` + Auth string `json:"auth"` + Bytes int `json:"bytes"` + ClientIP string `json:"clientIp"` + DeviceFamily string `json:"device_family"` + Httpversion string `json:"httpversion"` + Ident string `json:"ident"` + OsFamily string `json:"os_family"` + OsMajor string `json:"os_major"` + OsMinor string `json:"os_minor"` + Referrer string `json:"referrer"` + Request string `json:"request"` + Requesttime int `json:"requesttime"` + Response string `json:"response"` + Timestamp time.Time `json:"timestamp"` + UserAgentFamily string `json:"userAgent_family"` + UserAgentMajor string `json:"userAgent_major"` + UserAgentMinor string `json:"userAgent_minor"` + Verb string `json:"verb"` + Xforwardfor string `json:"xforwardfor"` +} + +func randElement(list []string) string { + off := rand.Intn(len(list)) + return list[off] +} + +type LogGenerator struct { + logfiles []string + off int + fp string + lineno int + reader *os.File + scanner *bufio.Scanner + lines int64 + size int64 +} + +func (g *LogGenerator) Stat() (l, s int64) { + l = atomic.LoadInt64(&g.lines) + s = atomic.LoadInt64(&g.size) + return +} + +//reset logfiles +func (g *LogGenerator) Init() error { + g.logfiles = nil + g.off = -1 + g.fp = "" + g.lineno = 0 + fnPatt := regexp.MustCompile(LogfilePattern) + d, err := os.Open(LogfileDir) + defer func() { + d.Close() + }() + if err != nil { + err = errors.Wrapf(err, "") + return err + } + fis, err := d.Readdir(0) + if err != nil { + err = errors.Wrapf(err, "") + return err + } + for _, fi := range fis { + if !fi.IsDir() && fnPatt.MatchString(fi.Name()) { + fp, err := filepath.Abs(filepath.Join(LogfileDir, fi.Name())) + if err != nil { + err = errors.Wrapf(err, "") + return err + } + g.logfiles = append(g.logfiles, fp) + } + } + if g.logfiles == nil || len(g.logfiles) == 0 { + err := errors.Errorf("There is no files under %v match pattern %v", LogfileDir, LogfilePattern) + return err + } + sort.Strings(g.logfiles) + log.Infof("Following files under %v match pattern %v: %+v", LogfileDir, LogfilePattern, g.logfiles) + + if err := g.next(); err != nil { + return err + } + return nil +} + +//switch to next log file +func (g *LogGenerator) next() (err error) { + g.scanner = nil + if g.reader != nil { + log.Debugf("closed %+v", g.fp) + g.reader.Close() + g.reader = nil + } + g.lineno = 0 + for i := 0; i < len(g.logfiles); i++ { + // a log file may disappear, retry another log file + g.off = (g.off + 1) % len(g.logfiles) + g.fp = g.logfiles[g.off] + var reader *os.File + if reader, err = os.Open(g.fp); err == nil { + g.reader = reader + g.scanner = bufio.NewScanner(g.reader) + log.Debugf("scanning %+v", g.fp) + return nil + } + err = errors.Wrapf(err, "") + log.Infof("failed to open %+v, %+v", g.fp, err) + time.Sleep(6000 * time.Second) + } + err = errors.Errorf("no readable file") + return +} + +func (g *LogGenerator) getLine() (fp string, lineno int, line string) { + for { + if g.scanner.Scan() { + g.lineno++ + return g.fp, g.lineno, g.scanner.Text() + } + if g.scanner.Err() != nil { + log.Warnf("Scan %+v", g.scanner.Err()) + } + if err := g.next(); err != nil { + log.Fatalf("got error %+v", err) + } + } +} + +func (g *LogGenerator) Run(ctx context.Context) { + toRound := time.Now() + // refers to time.Time.Truncate + rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) + + // make a writer that produces to topic-A, using the least-bytes distribution + w := kafka.NewWriter(kafka.WriterConfig{ + Brokers: strings.Split(KafkaBrokers, ","), + Topic: KafkaTopic, + Balancer: kafka.CRC32Balancer{}, + BatchSize: 1000, + Async: true, + }) + defer w.Close() + var err error + var b []byte + for day := 0; ; day++ { + tsDay := rounded.Add(time.Duration(-24*day) * time.Hour) + for step := 0; step < 24*60*60*1000; step++ { + timestamp := tsDay.Add(time.Duration(step) * time.Millisecond) + fp, lineno, line := g.getLine() + logObj := Log{ + Collectiontime: timestamp, + Hostname: randElement(ListHostname), + IP: randElement(ListIP), + Path: fp, + LineNo: lineno, + Message: line, + Agent: randElement(ListAgent), + Auth: randElement(ListAuth), + Bytes: len(line), + ClientIP: randElement(ListClientIP), + DeviceFamily: randElement(ListDeviceFamily), + Httpversion: randElement(ListHttpversion), + Ident: "", + OsFamily: randElement(ListOsFamily), + OsMajor: randElement(ListOsMajor), + OsMinor: randElement(ListOsMinor), + Referrer: "", + Request: "", + Requesttime: rand.Intn(1000), + Response: randElement(ListResponse), + Timestamp: timestamp, + UserAgentFamily: randElement(ListUserAgentFamily), + UserAgentMajor: randElement(ListUserAgentMajor), + UserAgentMinor: randElement(ListUserAgentMinor), + Verb: randElement(ListVerb), + Xforwardfor: "", + } + if b, err = json.Marshal(&logObj); err != nil { + err = errors.Wrapf(err, "") + log.Fatalf("got error %+v", err) + } + if err = w.WriteMessages(ctx, kafka.Message{Key: []byte(logObj.Hostname), Value: b}); err != nil { + err = errors.Wrapf(err, "") + log.Fatalf("got error %+v", err) + } + atomic.AddInt64(&g.lines, int64(1)) + atomic.AddInt64(&g.size, int64(len(b))) + } + } +} + +func main() { + flag.Usage = func() { + usage := fmt.Sprintf(`Usage of %s + %s kakfa_brokers topic log_file_dir log_file_pattern +This util read log from given paths, fill some fields with random content, serialize and send to kafka. +kakfa_brokers: for example, 192.168.102.114:9092,192.168.102.115:9092 +topic: for example, apache_access_log +log_file_dir: log file directory, for example, /var/log +log_file_pattern: file name pattern, for example, '^secure.*$'`, os.Args[0], os.Args[0]) + log.Infof(usage) + } + flag.Parse() + args := flag.Args() + if len(args) != 4 { + flag.Usage() + log.Fatal("Invalid CLI arguments!") + } + KafkaBrokers = args[0] + KafkaTopic = args[1] + LogfileDir = args[2] + LogfilePattern = args[3] + log.Infof("KafkaBrokers: %v\nKafkaTopic: %v\nLogfileDir: %v\nLogFilePattern: %v\n\n\n", KafkaBrokers, KafkaTopic, LogfileDir, LogfilePattern) + + ctx, cancel := context.WithCancel(context.Background()) + go func() { + sc := make(chan os.Signal, 1) + signal.Notify(sc, syscall.SIGUSR1) + for { + sig := <-sc + switch sig { + case syscall.SIGTERM: + log.Infof("got signal=<%d>. quit", sig) + cancel() + } + } + }() + + wg := sync.WaitGroup{} + wg.Add(NumGenerators) + var generators []*LogGenerator + for i := 0; i < NumGenerators; i++ { + g := &LogGenerator{} + if err := g.Init(); err != nil { + log.Fatalf("got error %+v", err) + } + go func() { + g.Run(ctx) + wg.Done() + }() + generators = append(generators, g) + } + + var prevLines, prevSize int64 + ticker := time.NewTicker(10 * time.Second) +LOOP: + for { + select { + case <-ctx.Done(): + break LOOP + case <-ticker.C: + listL := make([]int64, NumGenerators) + listS := make([]int64, NumGenerators) + var lines, size int64 + var speedLine, speedSize int64 + for i := 0; i < NumGenerators; i++ { + g := generators[i] + listL[i], listS[i] = g.Stat() + lines += listL[i] + size += listS[i] + } + if lines != 0 { + speedLine = (lines - prevLines) / int64(10) + speedSize = (size - prevSize) / int64(10) + } + prevLines = lines + prevSize = size + log.Infof("generated %+v lines, %+v Bytes, speedLine: %v lines/s, speedSize: %v B/s", lines, size, speedLine, speedSize) + } + } + wg.Wait() +} diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go new file mode 100644 index 00000000..db911e4f --- /dev/null +++ b/cmd/kafka_gen_metric/main.go @@ -0,0 +1,202 @@ +package main + +/* +CREATE TABLE sensor_dt_result_online ON CLUSTER abc ( + `@time` DateTime, + `@ItemGUID` String, + `@MetricName` LowCardinality(String), + `@AlgName` LowCardinality(String), + value Float64, + upper Float64, + lower Float64, + yhat_upper Float64, + yhat_lower Float64, + yhat_flag Int32, + total_anomaly Int64, + anomaly Float32, + abnormal_type Int16, + abnormality Int16, + container_id Int64, + hard_upper Float64, + hard_lower Float64, + hard_anomaly Int64, + shift_tag Int32, + season_tag Int32, + spike_tag Int32, + is_missing Int32 +) ENGINE=ReplicatedReplacingMergeTree('/clickhouse/tables/{cluster}/{shard}/default/sensor_dt_result_online', '{replica}') +PARTITION BY toYYYYMMDD(`@time`) +ORDER BY (`@time`, `@ItemGUID`, `@MetricName`); + +CREATE TABLE dist_sensor_dt_result_online ON CLUSTER abc AS sensor_dt_result_online ENGINE = Distributed(abc, default, sensor_dt_result_online); + +*/ + +import ( + "context" + "encoding/json" + "flag" + "fmt" + "math/rand" + "os" + "strings" + "time" + + "github.com/pkg/errors" + kafka "github.com/segmentio/kafka-go" + log "github.com/sirupsen/logrus" +) + +const ( + BusinessNum = 10 + InstanceNum = 100 +) + +var ( + KafkaBrokers string + KafkaTopic string + + ListMetricName = []string{"CPU", "RAM", "IOPS"} + ListArgName = []string{ + "DecisionTrees", + "NaiveBayesClassification", + "OrdinaryLeastSquaresRegression", + "LogisticRegression", + "SupportVectorMachines", + "EnsembleMethods", + "ClusteringAlgorithms", + "PrincipalComponentAnalysis", + "SingularValueDecomposition", + "IndependentComponentAnalysis"} +) + +type Metric struct { + Time time.Time `json:"@time"` //seconds since epoch + ItemGUID string `json:"@item_guid"` + MetricName string `json:"@metric_name"` + AlgName string `json:"@alg_name"` + Value float64 `json:"value"` + Upper float64 `json:"upper"` + Lower float64 `json:"lower"` + YhatUpper float64 `json:"yhat_upper"` + YhatLower float64 `json:"yhat_lower"` + YhatFlag int32 `json:"yhat_flag"` + TotalAnomaly int64 `json:"total_anomaly"` + Anomaly float64 `json:"anomaly"` + AbnormalType int16 `json:"abnormal_type"` + Abnormality int16 `json:"abnormality"` + ContainerID int64 `json:"container_id"` + HardUpper float64 `json:"hard_upper"` + HardLower float64 `json:"hard_lower"` + HardAnomaly int64 `json:"hard_anomaly"` + ShiftTag int32 `json:"shift_tag"` + SeasonTag int32 `json:"season_tag"` + SpikeTag int32 `json:"spike_tag"` + IsMissing int32 `json:"is_missing"` +} + +func randElement(list []string) string { + off := rand.Intn(len(list)) + return list[off] +} + +func generate() (err error) { + toRound := time.Now().Add(time.Duration(-30*24) * time.Hour) + // refers to time.Time.Truncate + rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) + + // make a writer that produces to topic-A, using the least-bytes distribution + w := kafka.NewWriter(kafka.WriterConfig{ + Brokers: strings.Split(KafkaBrokers, ","), + Topic: KafkaTopic, + Balancer: kafka.CRC32Balancer{}, + Async: true, + //Logger: log.StandardLogger(), + }) + defer w.Close() + ctx := context.Background() + var b []byte + var lines, size int64 + var prevLines, prevSize int64 + var speedLine, speedSize int64 + var ts, prevTS int64 + + for day := 0; ; day++ { + tsDay := rounded.Add(time.Duration(24*day) * time.Hour) + for step := 0; step < 24*60*60; step++ { + timestamp := tsDay.Add(time.Duration(step) * time.Second) + for bus := 0; bus < BusinessNum; bus++ { + for ins := 0; ins < InstanceNum; ins++ { + if lines%1000000 == 0 { + ts = time.Now().Unix() + if lines != 0 { + speedLine = (lines - prevLines) / (ts - prevTS) + speedSize = (size - prevSize) / (ts - prevTS) + } + prevLines = lines + prevSize = size + prevTS = ts + log.Infof("generated %+v lines %+v Bytes, speedLine: %v lines/s, speedSize: %v B/s", lines, size, speedLine, speedSize) + } + metric := Metric{ + Time: timestamp, + ItemGUID: fmt.Sprintf("bus%03d_ins%03d", bus, ins), + MetricName: randElement(ListMetricName), + AlgName: randElement(ListArgName), + Value: float64(rand.Intn(100)), + Upper: float64(100.0), + Lower: float64(60.0), + YhatUpper: float64(100.0), + YhatLower: float64(60.0), + YhatFlag: rand.Int31n(65535), + TotalAnomaly: rand.Int63n(65535), + Anomaly: float64(rand.Intn(100)) / float64(100), + AbnormalType: int16(rand.Intn(1000)), + Abnormality: int16(rand.Intn(1000)), + ContainerID: rand.Int63n(65535), + HardUpper: float64(100), + HardLower: float64(60), + HardAnomaly: int64(rand.Intn(65535)), + ShiftTag: int32(rand.Intn(65535)), + SeasonTag: int32(rand.Intn(65535)), + SpikeTag: int32(rand.Intn(65535)), + IsMissing: int32(rand.Intn(1)), + } + if b, err = json.Marshal(&metric); err != nil { + err = errors.Wrapf(err, "got error") + return + } + if err = w.WriteMessages(ctx, kafka.Message{Key: []byte(metric.ItemGUID), Value: b}); err != nil { + err = errors.Wrapf(err, "got error") + return + } + lines++ + size += int64(len(b)) + } + } + } + } +} + +func main() { + flag.Usage = func() { + usage := fmt.Sprintf(`Usage of %s + %s kakfa_brokers topic +This util fill some fields with random content, serialize and send to kafka. +kakfa_brokers: for example, 192.168.102.114:9092,192.168.102.115:9092 +topic: for example, apache_access_log`, os.Args[0], os.Args[0]) + log.Infof(usage) + } + flag.Parse() + args := flag.Args() + if len(args) != 2 { + flag.Usage() + log.Fatal("Invalid CLI arguments!") + } + KafkaBrokers = args[0] + KafkaTopic = args[1] + log.Infof("KafkaBrokers: %v\nKafkaTopic: %v\nBusinessNum: %v\nInstanceNum: %v\n", KafkaBrokers, KafkaTopic, BusinessNum, InstanceNum) + if err := generate(); err != nil { + log.Fatalf("got error %+v", err) + } +} diff --git a/go.mod b/go.mod index 72b8d81c..6e6c966b 100644 --- a/go.mod +++ b/go.mod @@ -12,6 +12,7 @@ require ( github.com/prometheus/client_golang v1.10.0 github.com/prometheus/common v0.24.0 github.com/segmentio/kafka-go v0.4.8 + github.com/sirupsen/logrus v1.6.0 github.com/stretchr/testify v1.7.0 github.com/tidwall/gjson v1.8.0 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 diff --git a/go.sum b/go.sum index f131e0b6..faa3e226 100644 --- a/go.sum +++ b/go.sum @@ -207,6 +207,7 @@ github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0 github.com/klauspost/compress v1.12.2 h1:2KCfW3I9M7nSc5wOqXAlW2v2U6v+w6cbjvbfp+OykW8= github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -331,6 +332,7 @@ github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9 github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= From c1138d3215fc2f6067b4aeed85c9b6a6c5df9287 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 3 Jun 2021 16:02:12 +0800 Subject: [PATCH 126/404] added benchmark doc --- docs/configuration/config.md | 38 +++++++++++++++++++++++------------- docs/dev/introduction.md | 18 +++++++++++++++++ 2 files changed, 42 insertions(+), 14 deletions(-) diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 7231a91f..8c90d663 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -1,9 +1,10 @@ # Config Items + > Here we use json with comments for documentation -``` +```json { - // clickhouse configs, it's map[string]ClickHouse for multiple clickhouse + // ClickHouse config "clickhouse": { // cluster the ClickHouse node belongs "cluster": "test", @@ -12,32 +13,41 @@ // it helps data deduplication for ReplicateMergeTree when driver error occurs "hosts": [ [ - "127.0.0.1" + "192.168.101.106", + "192.168.101.108" + ], + [ + "192.168.102.114", + "192.168.101.110" + ], + [ + "192.168.102.115" ] ], "port": 9000, - "username": "default" + "username": "default", "password": "", - "db": "default", // database name + // database name + "db": "default", // Whether enable TLS encryption with clickhouse-server - secure bool + "secure": false, // Whether skip verify clickhouse-server cert if secure=true. - insecureSkipVerify bool + "insecureSkipVerify": false, // retryTimes when error occurs in inserting datas "retryTimes": 0, }, - // kafka configs + // Kafka config "kafka": { "brokers": "127.0.0.1:9093", - // jave client style security authentication + // jave client style security authentication "security":{ "security.protocol": "SASL_PLAINTEXT", "sasl.kerberos.service.name": "kafka", "sasl.mechanism":"GSSAPI", - "sasl.jaas.config":"com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab=\"/etc/security/mmmtest.keytab\" principal=\"mmm@ALANWANG.COM\";" - } + "sasl.jaas.config":"com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab=\"/etc/security/mmmtest.keytab\" principal=\"mmm@ALANWANG.COM\";" + }, // SSL "tls": { @@ -45,10 +55,10 @@ // Required. It's the CA certificate with which Kafka brokers certs be signed. "caCertFiles": "/etc/security/ca-cert", // Required if Kafka brokers require client authentication. - clientCertFile: "", + "clientCertFile": "", // Required if and only if ClientCertFile is present. - clientKeyFile: "", - } + "clientKeyFile": "", + }, // SASL "sasl": { diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index bc407999..6a034a93 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -48,6 +48,24 @@ Note: | Nullable(T) | NULL | (The same as T) | (The same as T) | | Array(T) | [] | (The same as T) | (The same as T) | +## Benchmark + +- ClickHouse cluster: 3 shards, 2 physical hosts in each shard. Each host contains 48 cpu, 256 GB RAM, 12TB HDD RAID5. +- ZooKeeper cluster: on three hosts of ClickHouse cluster. +- Kafka cluster: 2 nodes on three hosts of ClickHouse cluster. Share the same zookeeper cluster wich ClickHouse. +- Kafka topic apache_access_log1: partition 1, replicator factor: 1 +- Kafka topic apache_access_log2: partition 2, replicator factor: 1 +- Kafka topic apache_access_log4: partition 4, replicator factor: 1 +- Generate json messages via kafka_gen_log(https://github.com/housepower/clickhouse_sinker/blob/master/cmd/kafka_gen_log). Messages avg lenght is 754 bytes. + +| config | thoughput(rows/s) | writer total cost | clickhouse cost per node | +|-----------------------------|-------------------|---------------|---------------| +| 1 kafka partition, 1 sinker | 142 K | 11.0 cpu, 8 GB | 0.3 cpu | +| 2 kafka partition, 1 sinker | 159 K | 14.0 cpu, 14 GB | 0.7 cpu | +| 4 kafka partition, 1 sinker | 25~127 K | 2~22 cpu, 16 GB | 1 cpu | +| 2 kafka partition, 2 sinker | 275 K | 22 cpu, 8 GB | 1.3 cpu | +| 4 kafka partition, 2 sinker | 301 K | 25 cpu, 18 GB | 1.5 cpu | + ## Configuration Refers to how [integration test](https://github.com/housepower/clickhouse_sinker/blob/master/go.test.sh) use the example config. Also refers to [code](https://github.com/housepower/clickhouse_sinker/blob/master/config/config.go) for all config items. From a696cc2609e789fe36cfba23f2accaaa558a6162 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 11 Jun 2021 02:25:41 +0000 Subject: [PATCH 127/404] optimized a bit --- input/kafka_go.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/input/kafka_go.go b/input/kafka_go.go index 38b85db5..2ec1adbe 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -64,9 +64,9 @@ func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model GroupID: k.cfg.Task.ConsumerGroup, Topic: k.cfg.Task.Topic, StartOffset: offset, - MinBytes: 1, // sarama.Consumer.Fetch.Min + MinBytes: 1024 * 1024, // sarama.Consumer.Fetch.Min MaxBytes: 100 * 1024 * 1024, // sarama.MaxResponseSize - MaxWait: time.Duration(250) * time.Millisecond, // sarama.Consumer.MaxWaitTime + MaxWait: time.Duration(100) * time.Millisecond, // sarama.Consumer.MaxWaitTime CommitInterval: time.Second, // flushes commits to Kafka every second } if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { From 253b6b4f8627fe9ed475267b404ffc9f50b598d8 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 11 Jun 2021 03:11:25 +0000 Subject: [PATCH 128/404] optimize kafka_gen_log and kafka_gen_metric --- cmd/kafka_gen_log/main.go | 97 ++++++++++++--------------------- cmd/kafka_gen_metric/main.go | 101 +++++++++++++++++++++-------------- go.mod | 1 + go.sum | 11 ++++ 4 files changed, 107 insertions(+), 103 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index ad3143b3..bdbc8850 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -49,20 +49,17 @@ import ( "regexp" "sort" "strings" - "sync" "sync/atomic" "syscall" "time" + "github.com/Shopify/sarama" + "github.com/google/gops/agent" + "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" - kafka "github.com/segmentio/kafka-go" log "github.com/sirupsen/logrus" ) -const ( - NumGenerators = 1 -) - var ( KafkaBrokers string KafkaTopic string @@ -238,16 +235,15 @@ func (g *LogGenerator) Run(ctx context.Context) { // refers to time.Time.Truncate rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) - // make a writer that produces to topic-A, using the least-bytes distribution - w := kafka.NewWriter(kafka.WriterConfig{ - Brokers: strings.Split(KafkaBrokers, ","), - Topic: KafkaTopic, - Balancer: kafka.CRC32Balancer{}, - BatchSize: 1000, - Async: true, - }) + wp := util.NewWorkerPool(10, 10000) + config := sarama.NewConfig() + config.Version = sarama.V2_1_0_0 + w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) + if err != nil { + log.Fatal("sarama.NewAsyncProducer failed %+v", err) + } defer w.Close() - var err error + var b []byte for day := 0; ; day++ { tsDay := rounded.Add(time.Duration(-24*day) * time.Hour) @@ -282,16 +278,19 @@ func (g *LogGenerator) Run(ctx context.Context) { Verb: randElement(ListVerb), Xforwardfor: "", } - if b, err = json.Marshal(&logObj); err != nil { - err = errors.Wrapf(err, "") - log.Fatalf("got error %+v", err) - } - if err = w.WriteMessages(ctx, kafka.Message{Key: []byte(logObj.Hostname), Value: b}); err != nil { - err = errors.Wrapf(err, "") - log.Fatalf("got error %+v", err) - } - atomic.AddInt64(&g.lines, int64(1)) - atomic.AddInt64(&g.size, int64(len(b))) + wp.Submit(func() { + if b, err = json.Marshal(&logObj); err != nil { + err = errors.Wrapf(err, "") + log.Fatalf("got error %+v", err) + } + w.Input() <- &sarama.ProducerMessage{ + Topic: KafkaTopic, + Key: sarama.StringEncoder(logObj.Hostname), + Value: sarama.ByteEncoder(b), + } + atomic.AddInt64(&g.lines, int64(1)) + atomic.AddInt64(&g.size, int64(len(b))) + }) } } } @@ -317,36 +316,18 @@ log_file_pattern: file name pattern, for example, '^secure.*$'`, os.Args[0], os. KafkaTopic = args[1] LogfileDir = args[2] LogfilePattern = args[3] - log.Infof("KafkaBrokers: %v\nKafkaTopic: %v\nLogfileDir: %v\nLogFilePattern: %v\n\n\n", KafkaBrokers, KafkaTopic, LogfileDir, LogfilePattern) + log.Infof("KafkaBrokers: %v\nKafkaTopic: %v\nLogfileDir: %v\nLogFilePattern: %v\n", KafkaBrokers, KafkaTopic, LogfileDir, LogfilePattern) - ctx, cancel := context.WithCancel(context.Background()) - go func() { - sc := make(chan os.Signal, 1) - signal.Notify(sc, syscall.SIGUSR1) - for { - sig := <-sc - switch sig { - case syscall.SIGTERM: - log.Infof("got signal=<%d>. quit", sig) - cancel() - } - } - }() + if err := agent.Listen(agent.Options{}); err != nil { + log.Fatal(err) + } - wg := sync.WaitGroup{} - wg.Add(NumGenerators) - var generators []*LogGenerator - for i := 0; i < NumGenerators; i++ { - g := &LogGenerator{} - if err := g.Init(); err != nil { - log.Fatalf("got error %+v", err) - } - go func() { - g.Run(ctx) - wg.Done() - }() - generators = append(generators, g) + ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) + g := &LogGenerator{} + if err := g.Init(); err != nil { + log.Fatalf("got error %+v", err) } + go g.Run(ctx) var prevLines, prevSize int64 ticker := time.NewTicker(10 * time.Second) @@ -354,18 +335,11 @@ LOOP: for { select { case <-ctx.Done(): + log.Infof("quit due to context been canceled") break LOOP case <-ticker.C: - listL := make([]int64, NumGenerators) - listS := make([]int64, NumGenerators) - var lines, size int64 var speedLine, speedSize int64 - for i := 0; i < NumGenerators; i++ { - g := generators[i] - listL[i], listS[i] = g.Stat() - lines += listL[i] - size += listS[i] - } + lines, size := g.Stat() if lines != 0 { speedLine = (lines - prevLines) / int64(10) speedSize = (size - prevSize) / int64(10) @@ -375,5 +349,4 @@ LOOP: log.Infof("generated %+v lines, %+v Bytes, speedLine: %v lines/s, speedSize: %v B/s", lines, size, speedLine, speedSize) } } - wg.Wait() } diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index db911e4f..be01472f 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -39,11 +39,16 @@ import ( "fmt" "math/rand" "os" + "os/signal" "strings" + "sync/atomic" + "syscall" "time" + "github.com/Shopify/sarama" + "github.com/google/gops/agent" + "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" - kafka "github.com/segmentio/kafka-go" log "github.com/sirupsen/logrus" ) @@ -68,6 +73,9 @@ var ( "PrincipalComponentAnalysis", "SingularValueDecomposition", "IndependentComponentAnalysis"} + + gLines int64 + gSize int64 ) type Metric struct { @@ -100,44 +108,26 @@ func randElement(list []string) string { return list[off] } -func generate() (err error) { +func generate(ctx context.Context) { toRound := time.Now().Add(time.Duration(-30*24) * time.Hour) // refers to time.Time.Truncate rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) - // make a writer that produces to topic-A, using the least-bytes distribution - w := kafka.NewWriter(kafka.WriterConfig{ - Brokers: strings.Split(KafkaBrokers, ","), - Topic: KafkaTopic, - Balancer: kafka.CRC32Balancer{}, - Async: true, - //Logger: log.StandardLogger(), - }) + wp := util.NewWorkerPool(10, 10000) + config := sarama.NewConfig() + config.Version = sarama.V2_1_0_0 + w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) + if err != nil { + log.Fatal("sarama.NewAsyncProducer failed %+v", err) + } defer w.Close() - ctx := context.Background() - var b []byte - var lines, size int64 - var prevLines, prevSize int64 - var speedLine, speedSize int64 - var ts, prevTS int64 for day := 0; ; day++ { tsDay := rounded.Add(time.Duration(24*day) * time.Hour) for step := 0; step < 24*60*60; step++ { - timestamp := tsDay.Add(time.Duration(step) * time.Second) for bus := 0; bus < BusinessNum; bus++ { for ins := 0; ins < InstanceNum; ins++ { - if lines%1000000 == 0 { - ts = time.Now().Unix() - if lines != 0 { - speedLine = (lines - prevLines) / (ts - prevTS) - speedSize = (size - prevSize) / (ts - prevTS) - } - prevLines = lines - prevSize = size - prevTS = ts - log.Infof("generated %+v lines %+v Bytes, speedLine: %v lines/s, speedSize: %v B/s", lines, size, speedLine, speedSize) - } + timestamp := tsDay.Add(time.Duration(step) * time.Second) metric := Metric{ Time: timestamp, ItemGUID: fmt.Sprintf("bus%03d_ins%03d", bus, ins), @@ -162,16 +152,21 @@ func generate() (err error) { SpikeTag: int32(rand.Intn(65535)), IsMissing: int32(rand.Intn(1)), } - if b, err = json.Marshal(&metric); err != nil { - err = errors.Wrapf(err, "got error") - return - } - if err = w.WriteMessages(ctx, kafka.Message{Key: []byte(metric.ItemGUID), Value: b}); err != nil { - err = errors.Wrapf(err, "got error") - return - } - lines++ - size += int64(len(b)) + + wp.Submit(func() { + var b []byte + if b, err = json.Marshal(&metric); err != nil { + err = errors.Wrapf(err, "") + log.Fatalf("got error %+v", err) + } + w.Input() <- &sarama.ProducerMessage{ + Topic: KafkaTopic, + Key: sarama.StringEncoder(metric.ItemGUID), + Value: sarama.ByteEncoder(b), + } + atomic.AddInt64(&gLines, int64(1)) + atomic.AddInt64(&gSize, int64(len(b))) + }) } } } @@ -184,7 +179,7 @@ func main() { %s kakfa_brokers topic This util fill some fields with random content, serialize and send to kafka. kakfa_brokers: for example, 192.168.102.114:9092,192.168.102.115:9092 -topic: for example, apache_access_log`, os.Args[0], os.Args[0]) +topic: for example, sensor_dt_result_online`, os.Args[0], os.Args[0]) log.Infof(usage) } flag.Parse() @@ -196,7 +191,31 @@ topic: for example, apache_access_log`, os.Args[0], os.Args[0]) KafkaBrokers = args[0] KafkaTopic = args[1] log.Infof("KafkaBrokers: %v\nKafkaTopic: %v\nBusinessNum: %v\nInstanceNum: %v\n", KafkaBrokers, KafkaTopic, BusinessNum, InstanceNum) - if err := generate(); err != nil { - log.Fatalf("got error %+v", err) + + if err := agent.Listen(agent.Options{}); err != nil { + log.Fatal(err) + } + + var prevLines, prevSize int64 + ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) + go generate(ctx) + + ticker := time.NewTicker(10 * time.Second) +LOOP: + for { + select { + case <-ctx.Done(): + log.Infof("quit due to context been canceled") + break LOOP + case <-ticker.C: + var speedLine, speedSize int64 + if gLines != 0 { + speedLine = (gLines - prevLines) / int64(10) + speedSize = (gSize - prevSize) / int64(10) + } + prevLines = gLines + prevSize = gSize + log.Infof("generated %+v lines, %+v Bytes, speedLine: %v lines/s, speedSize: %v B/s", gLines, gSize, speedLine, speedSize) + } } } diff --git a/go.mod b/go.mod index 6e6c966b..c63501e1 100644 --- a/go.mod +++ b/go.mod @@ -7,6 +7,7 @@ require ( github.com/Shopify/sarama v1.29.0 github.com/cespare/xxhash/v2 v2.1.1 github.com/fagongzi/goetty v1.7.0 + github.com/google/gops v0.3.18 github.com/nacos-group/nacos-sdk-go v1.0.7 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.10.0 diff --git a/go.sum b/go.sum index faa3e226..cb5ab2c8 100644 --- a/go.sum +++ b/go.sum @@ -12,6 +12,7 @@ github.com/Shopify/sarama v1.29.0 h1:ARid8o8oieau9XrHI55f/L3EoRAhm9px6sonbD7yuUE github.com/Shopify/sarama v1.29.0/go.mod h1:2QpgD79wpdAESqNQMxNc0KYMkycd4slxGdV3TWSVqrU= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= +github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= @@ -93,6 +94,7 @@ github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgO github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= @@ -130,6 +132,8 @@ github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.4 h1:L8R9j+yAqZuZjsqh/z+F1NCffTKKLShY6zXTItVIZ8M= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/gops v0.3.18 h1:my259V+172PVFmduS2RAsq4FKH+HjKqdh7pLr17Ot8c= +github.com/google/gops v0.3.18/go.mod h1:Pfp8hWGIFdV/7rY9/O/U5WgdjYQXf/GiEK4NVuVd2ZE= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= @@ -201,6 +205,7 @@ github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7 github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= @@ -329,6 +334,7 @@ github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0 github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= +github.com/shirou/gopsutil/v3 v3.21.2/go.mod h1:ghfMypLDrFSWN2c9cDYFLHyynQ+QUht0cv/18ZqVczw= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= @@ -365,6 +371,8 @@ github.com/tidwall/match v1.0.3 h1:FQUVvBImDutD8wJLN6c5eMzWtjgONK9MwIBCOrUJKeE= github.com/tidwall/match v1.0.3/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= github.com/tidwall/pretty v1.1.0 h1:K3hMW5epkdAVwibsQEfR/7Zj0Qgt4DxtNumTq/VloO8= github.com/tidwall/pretty v1.1.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= +github.com/tklauser/go-sysconf v0.3.4/go.mod h1:Cl2c8ZRWfHD5IrfHo9VN+FX9kCFjIOyVklgXycLB6ek= +github.com/tklauser/numcpus v0.2.1/go.mod h1:9aU+wOc6WjUIZEwWMP62PL/41d65P+iks1gBkr4QyP8= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 h1:kF/7m/ZU+0D4Jj5eZ41Zm3IH/J8OElK1Qtd7tVKAwLk= github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3/go.mod h1:QDlpd3qS71vYtakd2hmdpqhJ9nwv6mD6A30bQ1BPBFE= @@ -387,6 +395,7 @@ github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0 github.com/xdg/stringprep v1.0.3 h1:cmL5Enob4W83ti/ZHuZLuKD/xqJfus4fVPwE+/BDm+4= github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= +github.com/xlab/treeprint v1.0.0/go.mod h1:IoImgRak9i3zJyuxOKUP1v4UZd1tMoKkq/Cimt1uhCg= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= @@ -478,6 +487,7 @@ golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da h1:b3NXsE2LusjYGGjL5bxEVZZORm/YEFFrWFjR8eFrw/c= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -569,5 +579,6 @@ honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +rsc.io/goversion v1.2.0/go.mod h1:Eih9y/uIBS3ulggl7KNJ09xGSLcuNaLgmvvqa07sgfo= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= From 7ccb355307f29b074242fd3593efb2a89d0ada78 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 11 Jun 2021 08:14:05 +0000 Subject: [PATCH 129/404] added benchmark of flink pipeline --- docs/dev/introduction.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 6a034a93..fc4e20f9 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -50,6 +50,8 @@ Note: ## Benchmark +### clickhouse_sinker + - ClickHouse cluster: 3 shards, 2 physical hosts in each shard. Each host contains 48 cpu, 256 GB RAM, 12TB HDD RAID5. - ZooKeeper cluster: on three hosts of ClickHouse cluster. - Kafka cluster: 2 nodes on three hosts of ClickHouse cluster. Share the same zookeeper cluster wich ClickHouse. @@ -66,6 +68,23 @@ Note: | 2 kafka partition, 2 sinker | 275 K | 22 cpu, 8 GB | 1.3 cpu | | 4 kafka partition, 2 sinker | 301 K | 25 cpu, 18 GB | 1.5 cpu | +### Flink pipeline + +Here's the Flink pipeline which moves date from kafka to ClickHouse. The cpu hotspot of the Flink pipeline is JSON decode, and Row.setField. + +Kafka Source -> JSON decode -> DateTime formart conversion -> Interger type conversion -> JDBCSinkJob + +| config | thoughput(rows/s) | writer total cost | clickhouse cost per node | +|-----------------------------|-------------------|---------------|---------------| +| 1 kafka partition, pipeline Parallelism: 20 | 44.7 K | 13.8 cpu, 20 GB | 1.1 cpu | + +### Conclusion + +- clickhouse_sinker is 3x fast as the Flink pipeline, and cost much less connection and cpu overhead on clickhouse-server. +- clickhouse_sinker retry other replicas on writing failures. +- clickhouse_sinker get table schema from ClickHouse. The pipeline need manual config of all fields. +- clickhouse_sinker detect DateTime format. The pipeline need dedicated steps to do format and type conversion. + ## Configuration Refers to how [integration test](https://github.com/housepower/clickhouse_sinker/blob/master/go.test.sh) use the example config. Also refers to [code](https://github.com/housepower/clickhouse_sinker/blob/master/config/config.go) for all config items. From fb4ccdae7ca09f9e5fa135e3973767b30a551345 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 11 Jun 2021 08:29:20 +0000 Subject: [PATCH 130/404] pass lint --- .github/workflows/lint.yml | 4 ++-- Makefile | 2 +- cmd/kafka_gen_log/main.go | 8 ++++---- cmd/kafka_gen_metric/main.go | 8 ++++---- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index fdb6b28b..9feebb15 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -14,13 +14,13 @@ jobs: uses: golangci/golangci-lint-action@v2 with: # Required: the version of golangci-lint is required and must be specified without patch version: we always use the latest patch version. - version: v1.40 + version: v1.40.1 # Optional: working directory, useful for monorepos # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,revive,tagliatelle + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true diff --git a/Makefile b/Makefile index 9a5612ed..236042d5 100644 --- a/Makefile +++ b/Makefile @@ -25,6 +25,6 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,revive,tagliatelle + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.json diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index bdbc8850..75c25fbd 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -230,7 +230,7 @@ func (g *LogGenerator) getLine() (fp string, lineno int, line string) { } } -func (g *LogGenerator) Run(ctx context.Context) { +func (g *LogGenerator) Run() { toRound := time.Now() // refers to time.Time.Truncate rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) @@ -240,7 +240,7 @@ func (g *LogGenerator) Run(ctx context.Context) { config.Version = sarama.V2_1_0_0 w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) if err != nil { - log.Fatal("sarama.NewAsyncProducer failed %+v", err) + log.Fatalf("sarama.NewAsyncProducer failed %+v", err) } defer w.Close() @@ -278,7 +278,7 @@ func (g *LogGenerator) Run(ctx context.Context) { Verb: randElement(ListVerb), Xforwardfor: "", } - wp.Submit(func() { + _ = wp.Submit(func() { if b, err = json.Marshal(&logObj); err != nil { err = errors.Wrapf(err, "") log.Fatalf("got error %+v", err) @@ -327,7 +327,7 @@ log_file_pattern: file name pattern, for example, '^secure.*$'`, os.Args[0], os. if err := g.Init(); err != nil { log.Fatalf("got error %+v", err) } - go g.Run(ctx) + go g.Run() var prevLines, prevSize int64 ticker := time.NewTicker(10 * time.Second) diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index be01472f..bdb61679 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -108,7 +108,7 @@ func randElement(list []string) string { return list[off] } -func generate(ctx context.Context) { +func generate() { toRound := time.Now().Add(time.Duration(-30*24) * time.Hour) // refers to time.Time.Truncate rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) @@ -118,7 +118,7 @@ func generate(ctx context.Context) { config.Version = sarama.V2_1_0_0 w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) if err != nil { - log.Fatal("sarama.NewAsyncProducer failed %+v", err) + log.Fatalf("sarama.NewAsyncProducer failed %+v", err) } defer w.Close() @@ -153,7 +153,7 @@ func generate(ctx context.Context) { IsMissing: int32(rand.Intn(1)), } - wp.Submit(func() { + _ = wp.Submit(func() { var b []byte if b, err = json.Marshal(&metric); err != nil { err = errors.Wrapf(err, "") @@ -198,7 +198,7 @@ topic: for example, sensor_dt_result_online`, os.Args[0], os.Args[0]) var prevLines, prevSize int64 ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) - go generate(ctx) + go generate() ticker := time.NewTicker(10 * time.Second) LOOP: From 4b6e155653bb418919a4551c3c3e3e821e8b44da Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 18 Jun 2021 18:06:40 +0800 Subject: [PATCH 131/404] minor --- cmd/kafka_gen_log/main.go | 3 ++- cmd/kafka_gen_metric/main.go | 3 ++- model/message.go | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 75c25fbd..6d90def7 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -243,6 +243,7 @@ func (g *LogGenerator) Run() { log.Fatalf("sarama.NewAsyncProducer failed %+v", err) } defer w.Close() + chInput := w.Input() var b []byte for day := 0; ; day++ { @@ -283,7 +284,7 @@ func (g *LogGenerator) Run() { err = errors.Wrapf(err, "") log.Fatalf("got error %+v", err) } - w.Input() <- &sarama.ProducerMessage{ + chInput <- &sarama.ProducerMessage{ Topic: KafkaTopic, Key: sarama.StringEncoder(logObj.Hostname), Value: sarama.ByteEncoder(b), diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index bdb61679..fc307091 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -121,6 +121,7 @@ func generate() { log.Fatalf("sarama.NewAsyncProducer failed %+v", err) } defer w.Close() + chInput := w.Input() for day := 0; ; day++ { tsDay := rounded.Add(time.Duration(24*day) * time.Hour) @@ -159,7 +160,7 @@ func generate() { err = errors.Wrapf(err, "") log.Fatalf("got error %+v", err) } - w.Input() <- &sarama.ProducerMessage{ + chInput <- &sarama.ProducerMessage{ Topic: KafkaTopic, Key: sarama.StringEncoder(metric.ItemGUID), Value: sarama.ByteEncoder(b), diff --git a/model/message.go b/model/message.go index cd10eaf7..eda52ce6 100644 --- a/model/message.go +++ b/model/message.go @@ -44,7 +44,7 @@ type Batch struct { } //BatchGroup consists of multiple batches. -//The `before` relationship could be impossilbe if messages of a partition are distributed to multiple batches. +//The `before` relationship could be impossible if messages of a partition are distributed to multiple batches. //So those batches need to be committed after ALL of them have been written to clickhouse. type BatchGroup struct { Batchs []*Batch From 4196a35f8f509a8a9fd1e483c3b8533fb26621c1 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 25 Jun 2021 10:09:55 +0800 Subject: [PATCH 132/404] introduced sonic, a faster json encoder --- cmd/kafka_gen_log/main.go | 4 ++-- cmd/kafka_gen_metric/main.go | 4 ++-- go.mod | 1 + go.sum | 10 ++++++++++ 4 files changed, 15 insertions(+), 4 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 6d90def7..e62e86ee 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -39,7 +39,6 @@ CREATE TABLE dist_apache_access_log ON CLUSTER abc AS apache_access_log ENGINE = import ( "bufio" "context" - "encoding/json" "flag" "fmt" "math/rand" @@ -54,6 +53,7 @@ import ( "time" "github.com/Shopify/sarama" + "github.com/bytedance/sonic" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" @@ -280,7 +280,7 @@ func (g *LogGenerator) Run() { Xforwardfor: "", } _ = wp.Submit(func() { - if b, err = json.Marshal(&logObj); err != nil { + if b, err = sonic.Marshal(&logObj); err != nil { err = errors.Wrapf(err, "") log.Fatalf("got error %+v", err) } diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index fc307091..f80a7040 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -34,7 +34,6 @@ CREATE TABLE dist_sensor_dt_result_online ON CLUSTER abc AS sensor_dt_result_onl import ( "context" - "encoding/json" "flag" "fmt" "math/rand" @@ -46,6 +45,7 @@ import ( "time" "github.com/Shopify/sarama" + "github.com/bytedance/sonic" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" @@ -156,7 +156,7 @@ func generate() { _ = wp.Submit(func() { var b []byte - if b, err = json.Marshal(&metric); err != nil { + if b, err = sonic.Marshal(&metric); err != nil { err = errors.Wrapf(err, "") log.Fatalf("got error %+v", err) } diff --git a/go.mod b/go.mod index c63501e1..6266c766 100644 --- a/go.mod +++ b/go.mod @@ -5,6 +5,7 @@ go 1.14 require ( github.com/ClickHouse/clickhouse-go v1.4.5 github.com/Shopify/sarama v1.29.0 + github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a github.com/cespare/xxhash/v2 v2.1.1 github.com/fagongzi/goetty v1.7.0 github.com/google/gops v0.3.18 diff --git a/go.sum b/go.sum index cb5ab2c8..aafd6042 100644 --- a/go.sum +++ b/go.sum @@ -40,11 +40,15 @@ github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAK github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= +github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a h1:6ikHdu0dgMWdck/K6ZlMcN+Gx44c8PeZ54mAsQ5LpOg= +github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a/go.mod h1:Jag0ltWFtHzILaHByar6cc/Ru+9qv0JJzyOCfCE7jtk= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chenzhuoyu/base64x v0.0.0-20210528162528-3c6c11c43ee5 h1:7AStn2tanqGY99xzW+Ve1p6YYqnRr1m/yswJ4h0BhcY= +github.com/chenzhuoyu/base64x v0.0.0-20210528162528-3c6c11c43ee5/go.mod h1:NfDzX8KeqVNX62apij1OkqoeDdq1VR3g0TRZo99kkBA= github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= @@ -97,6 +101,8 @@ github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/goccy/go-json v0.7.1 h1:VMhnh5gcc8De8f6m2DLvSqY1x8Jwl3btet+EqMP0QNs= +github.com/goccy/go-json v0.7.1/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -211,6 +217,8 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.12.2 h1:2KCfW3I9M7nSc5wOqXAlW2v2U6v+w6cbjvbfp+OykW8= github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= +github.com/klauspost/cpuid/v2 v2.0.6 h1:dQ5ueTiftKxp0gyjKSx5+8BtPWkyQbd95m8Gys/RarI= +github.com/klauspost/cpuid/v2 v2.0.6/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -378,6 +386,8 @@ github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 h1:kF/7m/ZU+0D github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3/go.mod h1:QDlpd3qS71vYtakd2hmdpqhJ9nwv6mD6A30bQ1BPBFE= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= +github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= +github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= From 11b3200d58c18bfd3a0076d94dd0dfe077f67b46 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 9 Aug 2021 08:09:56 +0000 Subject: [PATCH 133/404] more log --- cmd/kafka_gen_metric/main.go | 2 +- output/clickhouse.go | 6 +++--- pool/conn.go | 7 +++++++ 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index f80a7040..9967bc1a 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -24,7 +24,7 @@ CREATE TABLE sensor_dt_result_online ON CLUSTER abc ( season_tag Int32, spike_tag Int32, is_missing Int32 -) ENGINE=ReplicatedReplacingMergeTree('/clickhouse/tables/{cluster}/{shard}/default/sensor_dt_result_online', '{replica}') +) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/{database}/{table}', '{replica}') PARTITION BY toYYYYMMDD(`@time`) ORDER BY (`@time`, `@ItemGUID`, `@MetricName`); diff --git a/output/clickhouse.go b/output/clickhouse.go index 8ea6518d..67f762d5 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -120,10 +120,10 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( return } -func shouldReconnect(err error) bool { +func shouldReconnect(err error, sc *pool.ShardConn) bool { var exp *clickhouse.Exception if errors.As(err, &exp) { - util.Logger.Error("this is an exception from clickhouse-server", zap.Reflect("exception", exp)) + util.Logger.Error("this is an exception from clickhouse-server", zap.String("dsn", sc.GetDsn()), zap.Reflect("exception", exp)) var replicaSpecific bool for _, ec := range replicaSpecificErrorCodes { if ec == exp.Code { @@ -162,7 +162,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { util.Logger.Error("flush batch failed", zap.String("task", c.cfg.Task.Name), zap.Int("try", times), zap.Error(err)) statistics.FlushMsgsErrorTotal.WithLabelValues(c.cfg.Task.Name).Add(float64(batch.RealSize)) times++ - reconnect = shouldReconnect(err) + reconnect = shouldReconnect(err, sc) if reconnect && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { time.Sleep(10 * time.Second) } else { diff --git a/pool/conn.go b/pool/conn.go index bb1cd6bf..0cbecd4d 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -50,6 +50,13 @@ type ShardConn struct { nextRep int //index of next replica } +// Close closes the current replica connection +func (sc *ShardConn) GetDsn() string { + sc.lock.Lock() + defer sc.lock.Unlock() + return sc.dsn +} + // Close closes the current replica connection func (sc *ShardConn) Close() { sc.lock.Lock() From 60101302721919da9245295769a360c8bea88fce Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 16 Aug 2021 13:54:30 +0800 Subject: [PATCH 134/404] added Decimal support --- cmd/kafka_gen_log/main.go | 2 +- cmd/kafka_gen_metric/main.go | 2 +- docker-compose.yml | 4 +++- docs/dev/introduction.md | 2 ++ go.test.sh | 6 ++++-- model/value.go | 5 +++++ 6 files changed, 16 insertions(+), 5 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index e62e86ee..11a9fd61 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -28,7 +28,7 @@ CREATE TABLE apache_access_log ON CLUSTER abc ( userAgent_minor LowCardinality(String), verb LowCardinality(String), xforwardfor LowCardinality(String) -) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/default/apache_access_log', '{replica}') +) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{database}/{table}/{shard}', '{replica}') PARTITION BY toYYYYMMDD(timestamp) ORDER BY (timestamp, `@hostname`, `@path`, `@lineno`); diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index 9967bc1a..011e6b57 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -24,7 +24,7 @@ CREATE TABLE sensor_dt_result_online ON CLUSTER abc ( season_tag Int32, spike_tag Int32, is_missing Int32 -) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/{database}/{table}', '{replica}') +) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{database}/{table}/{shard}', '{replica}') PARTITION BY toYYYYMMDD(`@time`) ORDER BY (`@time`, `@ItemGUID`, `@MetricName`); diff --git a/docker-compose.yml b/docker-compose.yml index b56fa9b7..c2276091 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -3,11 +3,13 @@ version: '3' services: zookeeper: - image: zookeeper:3.4.12 + image: quay.io/bitnami/zookeeper:3.7.0 restart: always hostname: zookeeper ports: - "2181:2181" + environment: + ALLOW_ANONYMOUS_LOGIN: 1 security_opt: - label:disable kafka: diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index fc4e20f9..a7c3c1f3 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -27,6 +27,7 @@ Refers to [design](./design.md) for how it works. - [x] UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 - [x] Float32, Float64 +- [x] Decimal, Decimal32, Decimal64, Decimal128, Decimal256 - [x] String, FixedString, LowCardinality(String) - [x] Date, DateTime, DateTime64. Assuming that all values of a field of kafka message has the same layout, and layouts of each field are unrelated. Automatically detect the layout from [these date layouts](https://github.com/housepower/clickhouse_sinker/blob/master/parser/parser.go) till the first successful detection and reuse that layout forever. - [x] Array(T), where T is one of above basic types @@ -43,6 +44,7 @@ Note: |:--------------------:|:-------------:|:-----------------------------------:|:-------------------------------------:| | Int8, Int16, ... | 0 | Bool, Number | Int8 [-128,127], ... | | Float32, Float64 | 0.0 | Number | Float32 [-MaxFloat32,MaxFloat32], ... | +| Decimal, ... | 0.0 | Number | [decimal-value-ranges](https://clickhouse.tech/docs/en/sql-reference/data-types/decimal/#decimal-value-ranges) | | String, ... | "" | Bool, Number, String, Object, Array | N/A | | Date, DateTime, ... | EPOCH | Number, String | [EPOCH,MaxUint32_seconds_since_epoch) | | Nullable(T) | NULL | (The same as T) | (The same as T) | diff --git a/go.test.sh b/go.test.sh index 9f352f53..f1d70196 100755 --- a/go.test.sh +++ b/go.test.sh @@ -7,7 +7,8 @@ curl "localhost:8123" -d 'CREATE TABLE test_fixed_schema `day` Date DEFAULT toDate(time), `time` DateTime, `name` String, - `value` Float64 + `value` Float64, + `price` Decimal32(3) ) ENGINE = MergeTree PARTITION BY day @@ -25,7 +26,8 @@ echo "Got initial row counts => $counts" now=`date --rfc-3339=ns` for i in `seq 1 10000`;do - echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i }" + price=`echo "scale = 3; $i / 1000" | bc -q` + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"price\" : $price }" done > a.json for i in `seq 10001 30000`;do echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey01\" : $i }" diff --git a/model/value.go b/model/value.go index 9a9356c2..48900ac4 100644 --- a/model/value.go +++ b/model/value.go @@ -86,6 +86,11 @@ func WhichType(typ string) (dataType int, nullable bool) { } else if strings.HasPrefix(typ, "Array(DateTime64") { dataType = DateTimeArray nullable = false + } else if strings.HasPrefix(typ, "Decimal") { + dataType = Float + } else if strings.HasPrefix(typ, "Array(Decimal") { + dataType = FloatArray + nullable = false } else { util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported ClickHouse data type %v", typ)) } From 0a19adc7796a6cc14f981cbaffc6076162e6c811 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 16 Aug 2021 14:51:37 +0800 Subject: [PATCH 135/404] adjusted maxFlushInterval --- config/config.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/config.go b/config/config.go index ffc42d43..ce73b970 100644 --- a/config/config.go +++ b/config/config.go @@ -146,7 +146,7 @@ type TaskConfig struct { } const ( - maxFlushInterval = 10 + maxFlushInterval = 600 defaultFlushInterval = 5 MaxBufferSize = 1 << 20 //1048576 defaultBufferSize = 1 << 18 //262144 From b5bda82cc9b80e61b2675d3f560bdce16098b339 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 16 Aug 2021 15:27:58 +0800 Subject: [PATCH 136/404] Config.Tasks --- cmd/clickhouse_sinker/main.go | 166 +++++++++++++++++++++++----------- config/config.go | 73 ++++++++------- go.test.sh | 11 +-- input/input.go | 2 +- input/kafka_go.go | 20 ++-- input/kafka_sarama.go | 17 ++-- output/clickhouse.go | 59 +++++------- parser/parser_test.go | 10 +- task/ring.go | 6 +- task/sharding.go | 6 +- task/task.go | 52 ++++++----- util/workerpool.go | 6 +- 12 files changed, 249 insertions(+), 179 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index f9d7a4ff..6fcc7f23 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -30,9 +30,7 @@ import ( "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/health" - "github.com/housepower/clickhouse_sinker/input" - "github.com/housepower/clickhouse_sinker/output" - "github.com/housepower/clickhouse_sinker/parser" + "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/task" "github.com/housepower/clickhouse_sinker/util" @@ -135,16 +133,6 @@ func init() { selfIP = ip.String() } -// GenTask generate a task via config -func GenTask(cfg *config.Config) (taskImpl *task.Service) { - taskCfg := &cfg.Task - ck := output.NewClickHouse(cfg) - pp, _ := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone) - inputer := input.NewInputer(taskCfg.KafkaClient) - taskImpl = task.NewTaskService(inputer, ck, pp, cfg) - return -} - func main() { util.Run("clickhouse_sinker", func() error { // Initialize http server for metrics and debug @@ -227,8 +215,9 @@ func main() { // Sinker object maintains number of task for each partition type Sinker struct { curCfg *config.Config + numCfg int pusher *statistics.Pusher - task *task.Service + tasks map[string]*task.Service rcm config.RemoteConfManager ctx context.Context cancel context.CancelFunc @@ -237,7 +226,7 @@ type Sinker struct { // NewSinker get an instance of sinker with the task list func NewSinker(rcm config.RemoteConfManager) *Sinker { ctx, cancel := context.WithCancel(context.Background()) - s := &Sinker{rcm: rcm, ctx: ctx, cancel: cancel} + s := &Sinker{tasks: make(map[string]*task.Service), rcm: rcm, ctx: ctx, cancel: cancel} return s } @@ -264,7 +253,7 @@ func (s *Sinker) Run() { return } } else { - util.Logger.Fatal("expect --local-cfg-file or --local-cfg-dir") + util.Logger.Fatal("expect --local-cfg-file or --nacos-dataid") return } if err = newCfg.Normallize(); err != nil { @@ -301,22 +290,32 @@ func (s *Sinker) Run() { // Close shutdown task func (s *Sinker) Close() { - // Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). + // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). + s.stopAllTasks() + // 2. Stop Sinker.Run main loop + s.cancel() + // 3. Stop pusher + if s.pusher != nil { + s.pusher.Stop() + s.pusher = nil + } +} + +func (s *Sinker) stopAllTasks() { + for taskName, task := range s.tasks { + task.NotifyStop() + delete(s.tasks, taskName) + } util.Logger.Info("stopping parsing pool") util.GlobalParsingPool.StopWait() util.Logger.Info("stopping writing pool") util.GlobalWritingPool.StopWait() util.Logger.Info("stopping timer wheel") util.GlobalTimerWheel.Stop() - if s.task != nil { - s.task.Stop() - s.task = nil + for taskName, task := range s.tasks { + task.Stop() + delete(s.tasks, taskName) } - if s.pusher != nil { - s.pusher.Stop() - s.pusher = nil - } - s.cancel() } func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { @@ -324,7 +323,7 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { if s.curCfg == nil { // The first time invoking of applyConfig err = s.applyFirstConfig(newCfg) - } else if !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { + } else if !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) { err = s.applyAnotherConfig(newCfg) } return @@ -332,35 +331,47 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { util.Logger.Info("going to apply the first config", zap.Reflect("config", newCfg)) - // 1. Start goroutine pools. + + // 1. Initialize clickhouse connections + chCfg := &newCfg.Clickhouse + if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify); err != nil { + return + } + + // 2. Start goroutine pools. util.InitGlobalTimerWheel() util.InitGlobalParsingPool() util.InitGlobalWritingPool(len(newCfg.Clickhouse.Hosts)) - // 2. Generate, initialize and run task - s.task = GenTask(newCfg) - if err = s.task.Init(); err != nil { - return + // 3. Generate, initialize and run task + for _, taskCfg := range newCfg.Tasks { + task := task.NewTaskService(newCfg, taskCfg) + if err = task.Init(); err != nil { + return + } + s.tasks[taskCfg.Name] = task + } + for _, task := range s.tasks { + go task.Run(s.ctx) } s.curCfg = newCfg - go s.task.Run(s.ctx) return } func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { - util.Logger.Info("going to apply another config", zap.Reflect("config", newCfg)) - - if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Task, s.curCfg.Task) { - // 1. Stop task gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). - util.Logger.Info("stopping parsing pool") - util.GlobalParsingPool.StopWait() - util.Logger.Info("stopping writing pool") - util.GlobalWritingPool.StopWait() - util.Logger.Info("stopping timer wheel") - util.GlobalTimerWheel.Stop() - s.task.Stop() - - // 2. Restart goroutine pools. + util.Logger.Info("going to apply another config", zap.Int("number", s.numCfg), zap.Reflect("config", newCfg)) + s.numCfg++ + + if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) { + // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). + s.stopAllTasks() + // 2. Initialize clickhouse connections. + chCfg := &newCfg.Clickhouse + if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify); err != nil { + return + } + + // 3. Restart goroutine pools. util.Logger.Info("restarting parsing, writing and timer pool") util.InitGlobalTimerWheel() util.GlobalParsingPool.Restart() @@ -368,14 +379,65 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { util.GlobalWritingPool.Restart() util.Logger.Info("resized parsing pool", zap.Int("maxWorkers", len(newCfg.Clickhouse.Hosts))) - // 3. Generate, initialize and run task - s.task = GenTask(newCfg) - if err = s.task.Init(); err != nil { - return + // 4. Generate, initialize and run tasks. + for _, taskCfg := range newCfg.Tasks { + task := task.NewTaskService(newCfg, taskCfg) + if err = task.Init(); err != nil { + return + } + s.tasks[taskCfg.Name] = task + } + for _, task := range s.tasks { + go task.Run(s.ctx) + } + } else if !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) { + //1. Find tasks need to stop. + var tasksToStop []string + curCfgTasks := make(map[string]*config.TaskConfig) + newCfgTasks := make(map[string]*config.TaskConfig) + for _, taskCfg := range s.curCfg.Tasks { + curCfgTasks[taskCfg.Name] = taskCfg + } + for _, taskCfg := range newCfg.Tasks { + newCfgTasks[taskCfg.Name] = taskCfg + } + for taskName := range s.tasks { + curTaskCfg := curCfgTasks[taskName] + newTaskCfg, ok := newCfgTasks[taskName] + if !ok || !reflect.DeepEqual(newTaskCfg, curTaskCfg) { + tasksToStop = append(tasksToStop, taskName) + } + } + // 2. Stop tasks in parallel found at the previous step. + for _, taskName := range tasksToStop { + task := s.tasks[taskName] + task.NotifyStop() + } + for _, taskName := range tasksToStop { + task := s.tasks[taskName] + task.Stop() + delete(s.tasks, taskName) + } + // 3. Initailize tasks which are new or their config differ. + var newTasks []*task.Service + for taskName, taskCfg := range newCfgTasks { + if _, ok := s.tasks[taskName]; ok { + continue + } + task := task.NewTaskService(newCfg, taskCfg) + if err = task.Init(); err != nil { + return + } + s.tasks[taskName] = task + newTasks = append(newTasks, task) + } + + // 4. Start new tasks. We don't do it at step 3 in order to avoid goroutine leak due to errors raised by later steps. + for _, task := range newTasks { + go task.Run(s.ctx) } - // Record the new config - s.curCfg = newCfg - go s.task.Run(s.ctx) } + // Record the new config + s.curCfg = newCfg return } diff --git a/config/config.go b/config/config.go index ce73b970..dcced2bc 100644 --- a/config/config.go +++ b/config/config.go @@ -38,7 +38,8 @@ type RemoteConfManager interface { type Config struct { Kafka KafkaConfig Clickhouse ClickHouseConfig - Task TaskConfig + Task *TaskConfig + Tasks []*TaskConfig LogLevel string LogPaths []string } @@ -172,7 +173,7 @@ func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { // normallize and validate configuration func (cfg *Config) Normallize() (err error) { - if len(cfg.Clickhouse.Hosts) == 0 || cfg.Kafka.Brokers == "" || cfg.Task.Name == "" { + if len(cfg.Clickhouse.Hosts) == 0 || cfg.Kafka.Brokers == "" { err = errors.Errorf("invalid configuration") return } @@ -194,41 +195,47 @@ func (cfg *Config) Normallize() (err error) { cfg.Clickhouse.RetryTimes = 0 } - if cfg.Kafka.Sasl.Enable && cfg.Kafka.Sasl.Username == "" { - //kafka-go doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 - cfg.Task.KafkaClient = "sarama" - } else if cfg.Task.KafkaClient == "" { - cfg.Task.KafkaClient = "kafka-go" - } - if cfg.Task.Parser == "" || cfg.Task.Parser == "json" { - cfg.Task.Parser = "fastjson" + if cfg.Task != nil { + cfg.Tasks = append(cfg.Tasks, cfg.Task) + cfg.Task = nil } + for _, taskCfg := range cfg.Tasks { + if cfg.Kafka.Sasl.Enable && cfg.Kafka.Sasl.Username == "" { + //kafka-go doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 + taskCfg.KafkaClient = "sarama" + } else if taskCfg.KafkaClient == "" { + taskCfg.KafkaClient = "kafka-go" + } + if taskCfg.Parser == "" || taskCfg.Parser == "json" { + taskCfg.Parser = "fastjson" + } - for i := range cfg.Task.Dims { - if cfg.Task.Dims[i].SourceName == "" { - cfg.Task.Dims[i].SourceName = util.GetSourceName(cfg.Task.Dims[i].Name) + for i := range taskCfg.Dims { + if taskCfg.Dims[i].SourceName == "" { + taskCfg.Dims[i].SourceName = util.GetSourceName(taskCfg.Dims[i].Name) + } } - } - if cfg.Task.FlushInterval <= 0 { - cfg.Task.FlushInterval = defaultFlushInterval - } else if cfg.Task.FlushInterval > maxFlushInterval { - cfg.Task.FlushInterval = maxFlushInterval - } - if cfg.Task.BufferSize <= 0 { - cfg.Task.BufferSize = defaultBufferSize - } else if cfg.Task.BufferSize > MaxBufferSize { - cfg.Task.BufferSize = MaxBufferSize - } else { - cfg.Task.BufferSize = 1 << util.GetShift(cfg.Task.BufferSize) - } - if cfg.Task.TimeZone == "" { - cfg.Task.TimeZone = defaultTimeZone - } - if cfg.Task.DynamicSchema.Enable { - if cfg.Task.Parser != "fastjson" { - err = errors.Errorf("Parser %s doesn't support DynamicSchema", cfg.Task.Parser) - return + if taskCfg.FlushInterval <= 0 { + taskCfg.FlushInterval = defaultFlushInterval + } else if taskCfg.FlushInterval > maxFlushInterval { + taskCfg.FlushInterval = maxFlushInterval + } + if taskCfg.BufferSize <= 0 { + taskCfg.BufferSize = defaultBufferSize + } else if taskCfg.BufferSize > MaxBufferSize { + taskCfg.BufferSize = MaxBufferSize + } else { + taskCfg.BufferSize = 1 << util.GetShift(taskCfg.BufferSize) + } + if taskCfg.TimeZone == "" { + taskCfg.TimeZone = defaultTimeZone + } + if taskCfg.DynamicSchema.Enable { + if taskCfg.Parser != "fastjson" { + err = errors.Errorf("Parser %s doesn't support DynamicSchema", taskCfg.Parser) + return + } } } switch strings.ToLower(cfg.LogLevel) { diff --git a/go.test.sh b/go.test.sh index f1d70196..5feb1cd2 100755 --- a/go.test.sh +++ b/go.test.sh @@ -69,12 +69,6 @@ count=`curl "localhost:8123" -d 'SELECT count() FROM test_dynamic_schema'` echo "Got test_dynamic_schema count => $count" [ $count -eq 100000 ] || exit 1 - -echo "reset kafka consumer-group offsets" -sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_fixed_schema --all-topics --to-earliest -sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_auto_schema --all-topics --to-earliest -sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_dynamic_schema --all-topics --to-earliest - echo "truncate tables" curl "localhost:8123" -d 'TRUNCATE TABLE test_fixed_schema' curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' @@ -86,8 +80,13 @@ echo "publish clickhouse_sinker config" ./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.json echo "start clickhouse_sinker to consume" +sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_fixed_schema --all-topics --to-earliest timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema + +sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_auto_schema --all-topics --to-earliest timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema + +sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_dynamic_schema --all-topics --to-earliest timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema echo "check result 2" diff --git a/input/input.go b/input/input.go index ac4cde40..797df185 100644 --- a/input/input.go +++ b/input/input.go @@ -16,7 +16,7 @@ const ( ) type Inputer interface { - Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) error + Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage)) error Run(ctx context.Context) Stop() error CommitMessages(ctx context.Context, message *model.InputMessage) error diff --git a/input/kafka_go.go b/input/kafka_go.go index 2ec1adbe..f2d07be1 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -39,6 +39,7 @@ var _ Inputer = (*KafkaGo)(nil) // KafkaGo implements input.Inputer type KafkaGo struct { cfg *config.Config + taskCfg *config.TaskConfig r *kafka.Reader stopped chan struct{} putFn func(msg model.InputMessage) @@ -50,19 +51,20 @@ func NewKafkaGo() *KafkaGo { } // Init Initialise the kafka instance with configuration -func (k *KafkaGo) Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) (err error) { +func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage)) (err error) { k.cfg = cfg + k.taskCfg = taskCfg kfkCfg := &cfg.Kafka k.stopped = make(chan struct{}) k.putFn = putFn offset := kafka.LastOffset - if k.cfg.Task.Earliest { + if k.taskCfg.Earliest { offset = kafka.FirstOffset } readerCfg := &kafka.ReaderConfig{ Brokers: strings.Split(kfkCfg.Brokers, ","), - GroupID: k.cfg.Task.ConsumerGroup, - Topic: k.cfg.Task.Topic, + GroupID: k.taskCfg.ConsumerGroup, + Topic: k.taskCfg.Topic, StartOffset: offset, MinBytes: 1024 * 1024, // sarama.Consumer.Fetch.Min MaxBytes: 100 * 1024 * 1024, // sarama.MaxResponseSize @@ -129,15 +131,15 @@ LOOP_KAFKA_GO: var msg kafka.Message if msg, err = k.r.FetchMessage(ctx); err != nil { if errors.Is(err, context.Canceled) { - util.Logger.Info("Kafka.Run quit due to context has been canceled", zap.String("task", k.cfg.Task.Name)) + util.Logger.Info("Kafka.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) break LOOP_KAFKA_GO } else if errors.Is(err, io.EOF) { - util.Logger.Info("Kafka.Run quit due to reader has been closed", zap.String("task", k.cfg.Task.Name)) + util.Logger.Info("Kafka.Run quit due to reader has been closed", zap.String("task", k.taskCfg.Name)) break LOOP_KAFKA_GO } else { - statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.cfg.Task.Name).Inc() + statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.taskCfg.Name).Inc() err = errors.Wrap(err, "") - util.Logger.Error("k.r.FetchMessage failed", zap.String("task", k.cfg.Task.Name), zap.Error(err)) + util.Logger.Error("k.r.FetchMessage failed", zap.String("task", k.taskCfg.Name), zap.Error(err)) continue } } @@ -174,5 +176,5 @@ func (k *KafkaGo) Stop() error { // Description of this kafka consumer, which topic it reads from func (k *KafkaGo) Description() string { - return "kafka consumer of topic " + k.cfg.Task.Topic + return "kafka consumer of topic " + k.taskCfg.Topic } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index b448520e..eb920c8f 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -39,6 +39,7 @@ var _ Inputer = (*KafkaSarama)(nil) // KafkaSarama implements input.Inputer type KafkaSarama struct { cfg *config.Config + taskCfg *config.TaskConfig cg sarama.ConsumerGroup sess sarama.ConsumerGroupSession stopped chan struct{} @@ -59,7 +60,7 @@ func (h MyConsumerGroupHandler) Setup(sess sarama.ConsumerGroupSession) error { return nil } func (h MyConsumerGroupHandler) Cleanup(_ sarama.ConsumerGroupSession) error { - util.Logger.Info("consumer group cleanup", zap.String("task", h.k.cfg.Task.Name), zap.String("consumer group", h.k.cfg.Task.ConsumerGroup)) + util.Logger.Info("consumer group cleanup", zap.String("task", h.k.taskCfg.Name), zap.String("consumer group", h.k.taskCfg.ConsumerGroup)) time.Sleep(5 * time.Second) return nil } @@ -79,10 +80,10 @@ func (h MyConsumerGroupHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, c } // Init Initialise the kafka instance with configuration -func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) (err error) { +func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage)) (err error) { k.cfg = cfg + k.taskCfg = taskCfg kfkCfg := &cfg.Kafka - taskCfg := &cfg.Task k.stopped = make(chan struct{}) k.putFn = putFn config := sarama.NewConfig() @@ -138,7 +139,7 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskName string, putFn func(msg m // kafka main loop func (k *KafkaSarama) Run(ctx context.Context) { - taskCfg := &k.cfg.Task + taskCfg := k.taskCfg LOOP_SARAMA: for { handler := MyConsumerGroupHandler{k} @@ -147,15 +148,15 @@ LOOP_SARAMA: // recreated to get the new claims if err := k.cg.Consume(ctx, []string{taskCfg.Topic}, handler); err != nil { if errors.Is(err, context.Canceled) { - util.Logger.Info("Kafka.Run quit due to context has been canceled", zap.String("task", k.cfg.Task.Name)) + util.Logger.Info("Kafka.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) break LOOP_SARAMA } else if errors.Is(err, sarama.ErrClosedConsumerGroup) { - util.Logger.Info("Kafka.Run quit due to consumer group has been closed", zap.String("task", k.cfg.Task.Name)) + util.Logger.Info("Kafka.Run quit due to consumer group has been closed", zap.String("task", k.taskCfg.Name)) break LOOP_SARAMA } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() err = errors.Wrap(err, "") - util.Logger.Error("Kafka.Run got error %+v", zap.String("task", k.cfg.Task.Name), zap.Error(err)) + util.Logger.Error("Kafka.Run got error %+v", zap.String("task", k.taskCfg.Name), zap.Error(err)) continue } } @@ -175,7 +176,7 @@ func (k *KafkaSarama) Stop() error { // Description of this kafka consumer, which topic it reads from func (k *KafkaSarama) Description() string { - return "kafka consumer of topic " + k.cfg.Task.Topic + return "kafka consumer of topic " + k.taskCfg.Topic } // Predefined SCRAMClientGeneratorFunc, copied from https://github.com/Shopify/sarama/blob/master/examples/sasl_scram_client/scram_client.go diff --git a/output/clickhouse.go b/output/clickhouse.go index 67f762d5..1770af8c 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -49,34 +49,29 @@ var ( // ClickHouse is an output service consumers from kafka messages type ClickHouse struct { - Dims []*model.ColumnWithType - Dms []string - // Table Configs - cfg *config.Config - + Dims []*model.ColumnWithType + Dms []string + cfg *config.Config + taskCfg *config.TaskConfig prepareSQL string } // NewClickHouse new a clickhouse instance -func NewClickHouse(cfg *config.Config) *ClickHouse { - return &ClickHouse{cfg: cfg} +func NewClickHouse(cfg *config.Config, taskCfg *config.TaskConfig) *ClickHouse { + return &ClickHouse{cfg: cfg, taskCfg: taskCfg} } // Init the clickhouse intance func (c *ClickHouse) Init() (err error) { - chCfg := &c.cfg.Clickhouse - if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify); err != nil { - return - } return c.initSchema() } // Send a batch to clickhouse func (c *ClickHouse) Send(batch *model.Batch) { - statistics.WritingPoolBacklog.WithLabelValues(c.cfg.Task.Name).Inc() + statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Inc() _ = util.GlobalWritingPool.Submit(func() { c.loopWrite(batch) - statistics.WritingPoolBacklog.WithLabelValues(c.cfg.Task.Name).Dec() + statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Dec() }) } @@ -116,7 +111,7 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( err = errors.Wrapf(err, "tx.Commit") return } - statistics.FlushMsgsTotal.WithLabelValues(c.cfg.Task.Name).Add(float64(batch.RealSize)) + statistics.FlushMsgsTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) return } @@ -150,42 +145,36 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { } // Note: kafka_go and sarama commit give different error when context is cancceled. if errors.Is(err, context.Canceled) || errors.Is(err, io.ErrClosedPipe) { - util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.cfg.Task.Name)) + util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) return } - util.Logger.Fatal("committing offset failed with permanent error %+v", zap.String("task", c.cfg.Task.Name), zap.Error(err)) + util.Logger.Fatal("committing offset failed with permanent error %+v", zap.String("task", c.taskCfg.Name), zap.Error(err)) } if errors.Is(err, context.Canceled) { - util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.cfg.Task.Name)) + util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) return } - util.Logger.Error("flush batch failed", zap.String("task", c.cfg.Task.Name), zap.Int("try", times), zap.Error(err)) - statistics.FlushMsgsErrorTotal.WithLabelValues(c.cfg.Task.Name).Add(float64(batch.RealSize)) + util.Logger.Error("flush batch failed", zap.String("task", c.taskCfg.Name), zap.Int("try", times), zap.Error(err)) + statistics.FlushMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) times++ reconnect = shouldReconnect(err, sc) if reconnect && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { time.Sleep(10 * time.Second) } else { - util.Logger.Fatal("ClickHouse.loopWrite failed", zap.String("task", c.cfg.Task.Name)) + util.Logger.Fatal("ClickHouse.loopWrite failed", zap.String("task", c.taskCfg.Name)) } } } -// Stop free clickhouse connections -func (c *ClickHouse) Stop() error { - pool.FreeClusterConn() - return nil -} - func (c *ClickHouse) initSchema() (err error) { - if c.cfg.Task.AutoSchema { + if c.taskCfg.AutoSchema { sc := pool.GetShardConn(0) var conn *sql.DB if conn, _, err = sc.NextGoodReplica(0); err != nil { return } var rs *sql.Rows - if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, c.cfg.Clickhouse.DB, c.cfg.Task.TableName)); err != nil { + if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, c.cfg.Clickhouse.DB, c.taskCfg.TableName)); err != nil { err = errors.Wrapf(err, "") return err } @@ -199,18 +188,18 @@ func (c *ClickHouse) initSchema() (err error) { return err } typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") - if !util.StringContains(c.cfg.Task.ExcludeColumns, name) && defaultKind != "MATERIALIZED" { + if !util.StringContains(c.taskCfg.ExcludeColumns, name) && defaultKind != "MATERIALIZED" { tp, nullable := model.WhichType(typ) c.Dims = append(c.Dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, SourceName: util.GetSourceName(name)}) } } if len(c.Dims) == 0 { - err = errors.Errorf("Table %s.%s doesn't exist", c.cfg.Clickhouse.DB, c.cfg.Task.TableName) + err = errors.Errorf("Table %s.%s doesn't exist", c.cfg.Clickhouse.DB, c.taskCfg.TableName) return } } else { c.Dims = make([]*model.ColumnWithType, 0) - for _, dim := range c.cfg.Task.Dims { + for _, dim := range c.taskCfg.Dims { tp, nullable := model.WhichType(dim.Type) c.Dims = append(c.Dims, &model.ColumnWithType{ Name: dim.Name, @@ -231,17 +220,17 @@ func (c *ClickHouse) initSchema() (err error) { for i := range params { params[i] = "?" } - c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.cfg.Task.TableName + " (" + strings.Join(quotedDms, ",") + ") " + + c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.taskCfg.TableName + " (" + strings.Join(quotedDms, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" - util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.cfg.Task.Name)) + util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.taskCfg.Name)) return nil } func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { var queries []string var onCluster string - taskCfg := &c.cfg.Task + taskCfg := c.taskCfg chCfg := &c.cfg.Clickhouse if chCfg.Cluster != "" { onCluster = fmt.Sprintf("ON CLUSTER %s", chCfg.Cluster) @@ -321,7 +310,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { - taskCfg := &c.cfg.Task + taskCfg := c.taskCfg chCfg := &c.cfg.Clickhouse sc := pool.GetShardConn(0) var conn *sql.DB diff --git a/parser/parser_test.go b/parser/parser_test.go index 79d172ea..6b1318eb 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -148,7 +148,7 @@ var ( ) var initialize sync.Once -var initErr error +var errInit error var names = []string{"fastjson", "gjson", "csv"} var metrics map[string]model.Metric @@ -188,8 +188,8 @@ func initMetrics() { sample = jsonSample } parser = pp.Get() - if metric, initErr = parser.Parse(sample); initErr != nil { - msg := fmt.Sprintf("parser.Parse failed: %+v\n", initErr) + if metric, errInit = parser.Parse(sample); errInit != nil { + msg := fmt.Sprintf("parser.Parse failed: %+v\n", errInit) panic(msg) } metrics[name] = metric @@ -209,7 +209,7 @@ func sliceContains(list []string, target string) bool { func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { t.Helper() initialize.Do(initMetrics) - require.Nil(t, initErr) + require.Nil(t, errInit) for i := range names { name := names[i] metric := metrics[name] @@ -423,7 +423,7 @@ func TestParserElasticDateTime(t *testing.T) { func TestParserArray(t *testing.T) { initialize.Do(initMetrics) - require.Nil(t, initErr) + require.Nil(t, errInit) testCases := []ArrayCase{ {"not_exist", model.Float, []float64{}}, diff --git a/task/ring.go b/task/ring.go index 0f8af519..01e7e1d7 100644 --- a/task/ring.go +++ b/task/ring.go @@ -33,7 +33,7 @@ type Ring struct { func (ring *Ring) PutElem(msgRow model.MsgRow) { var err error - taskCfg := &ring.service.cfg.Task + taskCfg := ring.service.taskCfg msgOffset := msgRow.Msg.Offset ring.mux.Lock() defer ring.mux.Unlock() @@ -82,7 +82,7 @@ type OffsetRange struct { func (ring *Ring) ForceBatchOrShard(arg interface{}) { var newMsg *model.InputMessage - taskCfg := &ring.service.cfg.Task + taskCfg := ring.service.taskCfg select { case <-ring.service.ctx.Done(): util.Logger.Error("Ring.ForceBatchOrShard quit due to the context has been canceled", zap.String("task", taskCfg.Name)) @@ -144,7 +144,7 @@ func (ring *Ring) genBatchOrShard(expNewGroundOff int64) { if expNewGroundOff <= ring.ringGroundOff { return } - taskCfg := &ring.service.cfg.Task + taskCfg := ring.service.taskCfg var gaps []OffsetRange var msgCnt, parseErrs int endOff := (ring.ringGroundOff | int64(1< 0 { diff --git a/task/task.go b/task/task.go index 23c5cc86..3446ec54 100644 --- a/task/task.go +++ b/task/task.go @@ -49,6 +49,7 @@ type Service struct { clickhouse *output.ClickHouse pp *parser.Pool cfg *config.Config + taskCfg *config.TaskConfig dims []*model.ColumnWithType knownKeys sync.Map @@ -65,20 +66,25 @@ type Service struct { } // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances -func NewTaskService(inputer input.Inputer, clickhouse *output.ClickHouse, pp *parser.Pool, cfg *config.Config) *Service { +func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) *Service { + ck := output.NewClickHouse(cfg, taskCfg) + pp, _ := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone) + inputer := input.NewInputer(taskCfg.KafkaClient) return &Service{ stopped: make(chan struct{}), inputer: inputer, - clickhouse: clickhouse, + clickhouse: ck, started: false, pp: pp, cfg: cfg, + taskCfg: taskCfg, } } // Init initializes the kafak and clickhouse task associated with this service func (service *Service) Init() (err error) { - util.Logger.Info("task initializing", zap.String("task", service.cfg.Task.Name)) + taskCfg := service.taskCfg + util.Logger.Info("task initializing", zap.String("task", taskCfg.Name)) if err = service.clickhouse.Init(); err != nil { return } @@ -90,24 +96,23 @@ func (service *Service) Init() (err error) { service.limiter3 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.rings = make([]*Ring, 0) - if service.cfg.Task.ShardingKey != "" { + if taskCfg.ShardingKey != "" { if service.sharder, err = NewSharder(service); err != nil { return } } - if err = service.inputer.Init(service.cfg, service.cfg.Task.Name, service.put); err != nil { + if err = service.inputer.Init(service.cfg, taskCfg, service.put); err != nil { return } - taskCfg := &service.cfg.Task if taskCfg.DynamicSchema.Enable { maxDims := math.MaxInt16 - if service.cfg.Task.DynamicSchema.MaxDims > 0 { + if taskCfg.DynamicSchema.MaxDims > 0 { maxDims = taskCfg.DynamicSchema.MaxDims } if maxDims <= len(service.dims) { - service.cfg.Task.DynamicSchema.Enable = false + taskCfg.DynamicSchema.Enable = false util.Logger.Warn(fmt.Sprintf("disabled DynamicSchema since the number of columns reaches upper limit %d", maxDims), zap.String("task", taskCfg.Name)) } else { for _, dim := range service.dims { @@ -126,19 +131,20 @@ func (service *Service) Init() (err error) { // Run starts the task func (service *Service) Run(ctx context.Context) { var err error + taskCfg := service.taskCfg service.started = true service.parentCtx = ctx service.ctx, service.cancel = context.WithCancel(ctx) - util.Logger.Info("task started", zap.String("task", service.cfg.Task.Name)) + util.Logger.Info("task started", zap.String("task", taskCfg.Name)) go service.inputer.Run(service.ctx) if service.sharder != nil { // schedule a delayed ForceFlush - if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(service.cfg.Task.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { + if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { if errors.Is(err, goetty.ErrSystemStopped) { util.Logger.Info("Service.Run scheduling timer to a stopped timer wheel") } else { err = errors.Wrap(err, "") - util.Logger.Fatal("scheduling timer filed", zap.String("task", service.cfg.Task.Name), zap.Error(err)) + util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) } } } @@ -150,7 +156,7 @@ LOOP: break LOOP case batch := <-service.batchChan: if err := service.flush(batch); err != nil { - util.Logger.Fatal("service.flush failed", zap.String("task", service.cfg.Task.Name), zap.Error(err)) + util.Logger.Fatal("service.flush failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } } @@ -158,12 +164,12 @@ LOOP: } func (service *Service) fnCommit(partition int, offset int64) error { - msg := model.InputMessage{Topic: service.cfg.Task.Topic, Partition: partition, Offset: offset} + msg := model.InputMessage{Topic: service.taskCfg.Topic, Partition: partition, Offset: offset} return service.inputer.CommitMessages(service.ctx, &msg) } func (service *Service) put(msg model.InputMessage) { - taskCfg := &service.cfg.Task + taskCfg := service.taskCfg statistics.ConsumeMsgsTotal.WithLabelValues(taskCfg.Name).Inc() // ensure ring for this message exist service.Lock() @@ -219,7 +225,7 @@ func (service *Service) put(msg model.InputMessage) { return } if msg.Offset >= ringGroundOff+ring.ringCap && atomic.LoadInt32(&service.cntNewKeys) == 0 { - statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(service.cfg.Task.Name).Inc() + statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter3.Allow() { util.Logger.Warn(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) right to the range [%v, %v)", msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap), zap.String("task", taskCfg.Name)) @@ -243,7 +249,7 @@ func (service *Service) put(msg model.InputMessage) { statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter1.Allow() { util.Logger.Error(fmt.Sprintf("failed to parse message(topic %v, partition %d, offset %v)", - msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", service.cfg.Task.Name), zap.Error(err)) + msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", taskCfg.Name), zap.Error(err)) } } else { row = model.MetricToRow(metric, msg, service.dims) @@ -300,7 +306,7 @@ func (service *Service) flush(batch *model.Batch) (err error) { func (service *Service) changeSchema(arg interface{}) { var err error - taskCfg := &service.cfg.Task + taskCfg := service.taskCfg // change schema if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { util.Logger.Fatal("clickhouse.ChangeSchema failed", zap.String("task", taskCfg.Name), zap.Error(err)) @@ -313,9 +319,16 @@ func (service *Service) changeSchema(arg interface{}) { go service.Run(service.parentCtx) } +// NotifyStop notify task to stop, This is non-blocking. +func (service *Service) NotifyStop() { + taskCfg := service.taskCfg + util.Logger.Info("notified stopping task service...", zap.String("task", taskCfg.Name)) + service.cancel() +} + // Stop stop kafka and clickhouse client. This is blocking. func (service *Service) Stop() { - taskCfg := &service.cfg.Task + taskCfg := service.taskCfg if !service.started { util.Logger.Info("stopped a already stopped task service", zap.String("task", taskCfg.Name)) return @@ -327,9 +340,6 @@ func (service *Service) Stop() { } util.Logger.Info("stopped input", zap.String("task", taskCfg.Name)) - _ = service.clickhouse.Stop() - util.Logger.Info("stopped output", zap.String("task", taskCfg.Name)) - if service.sharder != nil { service.sharder.tid.Stop() } diff --git a/util/workerpool.go b/util/workerpool.go index 13d03135..23798e2e 100644 --- a/util/workerpool.go +++ b/util/workerpool.go @@ -46,8 +46,8 @@ func NewWorkerPool(maxWorkers int, queueSize int) *WorkerPool { } var ( - // ErrorStopped when stopped - ErrorStopped = errors.New("WorkerPool already stopped") + // ErrStopped when stopped + ErrStopped = errors.New("WorkerPool already stopped") ) func (w *WorkerPool) wokerFunc() { @@ -95,7 +95,7 @@ func (w *WorkerPool) Resize(maxWorkers int) { // Submit will block regardless if there is no free workers. func (w *WorkerPool) Submit(fn func()) (err error) { if atomic.LoadUint32(&w.state) == StateStopped { - return ErrorStopped + return ErrStopped } w.Lock() From cef7088af60075c000046d23eb2d7373d1b508f2 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 18 Aug 2021 17:58:16 +0800 Subject: [PATCH 137/404] introduced Config.Clickhouse.MaxOpenConns --- cmd/clickhouse_sinker/main.go | 8 ++++---- config/config.go | 11 ++++++++--- pool/conn.go | 33 ++++++++++++++++----------------- 3 files changed, 28 insertions(+), 24 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 6fcc7f23..23df4f1c 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -334,14 +334,14 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { // 1. Initialize clickhouse connections chCfg := &newCfg.Clickhouse - if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify); err != nil { + if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { return } // 2. Start goroutine pools. util.InitGlobalTimerWheel() util.InitGlobalParsingPool() - util.InitGlobalWritingPool(len(newCfg.Clickhouse.Hosts)) + util.InitGlobalWritingPool(len(chCfg.Hosts) * chCfg.MaxOpenConns) // 3. Generate, initialize and run task for _, taskCfg := range newCfg.Tasks { @@ -367,7 +367,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { s.stopAllTasks() // 2. Initialize clickhouse connections. chCfg := &newCfg.Clickhouse - if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify); err != nil { + if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { return } @@ -375,7 +375,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { util.Logger.Info("restarting parsing, writing and timer pool") util.InitGlobalTimerWheel() util.GlobalParsingPool.Restart() - util.GlobalWritingPool.Resize(len(newCfg.Clickhouse.Hosts)) + util.GlobalWritingPool.Resize(len(newCfg.Clickhouse.Hosts) * newCfg.Clickhouse.MaxOpenConns) util.GlobalWritingPool.Restart() util.Logger.Info("resized parsing pool", zap.Int("maxWorkers", len(newCfg.Clickhouse.Hosts))) diff --git a/config/config.go b/config/config.go index dcced2bc..76273344 100644 --- a/config/config.go +++ b/config/config.go @@ -102,7 +102,8 @@ type ClickHouseConfig struct { // Whether skip verify clickhouse-server cert InsecureSkipVerify bool - RetryTimes int //<=0 means retry infinitely + RetryTimes int //<=0 means retry infinitely + MaxOpenConns int } // Task configuration parameters @@ -147,13 +148,14 @@ type TaskConfig struct { } const ( - maxFlushInterval = 600 - defaultFlushInterval = 5 MaxBufferSize = 1 << 20 //1048576 defaultBufferSize = 1 << 18 //262144 + maxFlushInterval = 600 + defaultFlushInterval = 5 defaultTimeZone = "Local" defaultLogLevel = "info" defaultKerberosConfigPath = "/etc/krb5.conf" + defaultMaxOpenConns = 1 ) func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { @@ -194,6 +196,9 @@ func (cfg *Config) Normallize() (err error) { if cfg.Clickhouse.RetryTimes < 0 { cfg.Clickhouse.RetryTimes = 0 } + if cfg.Clickhouse.MaxOpenConns < 0 { + cfg.Clickhouse.MaxOpenConns = defaultMaxOpenConns + } if cfg.Task != nil { cfg.Tasks = append(cfg.Tasks, cfg.Task) diff --git a/pool/conn.go b/pool/conn.go index 0cbecd4d..a527f45d 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -42,12 +42,13 @@ var ( // ShardConn a datastructure for storing the clickhouse connection type ShardConn struct { - lock sync.Mutex - db *sql.DB - dbVer int - dsn string - replicas []string //ip:port list of replicas - nextRep int //index of next replica + lock sync.Mutex + db *sql.DB + dbVer int + dsn string + replicas []string //ip:port list of replicas + maxOpenConns int + nextRep int //index of next replica } // Close closes the current replica connection @@ -106,7 +107,12 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db *sql.DB, dbVer int, err util.Logger.Warn("sqlDB.Ping failed", zap.String("dsn", sc.dsn), zap.Error(err)) continue } - setDBParams(sqlDB) + + // WARN: clickHouse-server creates a thread for each TCP/HTTP connection. + // If the number of sinkers is close to clickhouse max_concurrent_queries(default 100), user queries could be blocked or refused. + sqlDB.SetMaxOpenConns(sc.maxOpenConns) + sqlDB.SetMaxIdleConns(0) + sqlDB.SetConnMaxIdleTime(10 * time.Second) sc.db = sqlDB sc.dbVer++ util.Logger.Info("sql.Open and sqlDB.Ping succeeded", zap.Int("dbVer", sc.dbVer), zap.String("dsn", sc.dsn)) @@ -119,7 +125,7 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db *sql.DB, dbVer int, err return nil, sc.dbVer, err } -func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool) (err error) { +func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool, maxOpenConns int) (err error) { lock.Lock() defer lock.Unlock() freeClusterConn() @@ -144,7 +150,8 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) } sc := &ShardConn{ - replicas: replicaAddrs, + replicas: replicaAddrs, + maxOpenConns: maxOpenConns, } if _, _, err = sc.NextGoodReplica(0); err != nil { return @@ -154,14 +161,6 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara return } -// TODO: ClickHouse creates a thread for each TCP/HTTP connection. -// If the number of sinkers is close to clickhouse max_concurrent_queries(default 100), user queries could be blocked or refused. -func setDBParams(sqlDB *sql.DB) { - sqlDB.SetMaxOpenConns(1) - sqlDB.SetMaxIdleConns(0) - sqlDB.SetConnMaxIdleTime(10 * time.Second) -} - func freeClusterConn() { for _, sc := range clusterConn { sc.Close() From 18631f149a08fa21c4e8fd698c3e813014d045ae Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 18 Aug 2021 18:50:45 +0800 Subject: [PATCH 138/404] publish replicas --- cmd/nacos_publish_config/main.go | 14 ++++++++++++++ go.mod | 1 + go.sum | 2 ++ 3 files changed, 17 insertions(+) diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index b510c571..5754b6da 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -17,6 +17,7 @@ package main import ( "flag" + "fmt" "os" "reflect" "time" @@ -24,6 +25,7 @@ import ( _ "github.com/ClickHouse/clickhouse-go" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/util" + "github.com/jinzhu/copier" "go.uber.org/zap" ) @@ -42,6 +44,7 @@ var ( `nacos data id`) localCfgFile = flag.String("local-cfg-file", "/etc/clickhouse_sinker.json", "local config file") + replicas = flag.Int("replicas", 1, "replicate each task to multiple ones with the same config except task name, consumer group and table name") ) // Empty is not valid namespaceID @@ -73,6 +76,17 @@ func PublishSinkerConfig() { util.Logger.Fatal("cfg.Normallize failed", zap.Error(err)) return } + tasks := cfg.Tasks + for i := 1; i < *replicas; i++ { + for j := 0; j < len(tasks); j++ { + taskCfg := &config.TaskConfig{} + copier.Copy(taskCfg, tasks[j]) + taskCfg.Name = fmt.Sprintf("%s_r%d", taskCfg.Name, i) + taskCfg.ConsumerGroup = fmt.Sprintf("%s_r%d", taskCfg.ConsumerGroup, i) + taskCfg.TableName = fmt.Sprintf("%s_r%d", taskCfg.TableName, i) + cfg.Tasks = append(cfg.Tasks, taskCfg) + } + } ncm := config.NacosConfManager{} properties := getProperties() diff --git a/go.mod b/go.mod index 6266c766..7ff41290 100644 --- a/go.mod +++ b/go.mod @@ -9,6 +9,7 @@ require ( github.com/cespare/xxhash/v2 v2.1.1 github.com/fagongzi/goetty v1.7.0 github.com/google/gops v0.3.18 + github.com/jinzhu/copier v0.3.2 github.com/nacos-group/nacos-sdk-go v1.0.7 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.10.0 diff --git a/go.sum b/go.sum index aafd6042..12839b3e 100644 --- a/go.sum +++ b/go.sum @@ -195,6 +195,8 @@ github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZ github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uczJe5YQdrYB16oTJlGSC/OyZDqUk9xX4= github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869/go.mod h1:cJ6Cj7dQo+O6GJNiMx+Pa94qKj+TG8ONdKHgMNIyyag= +github.com/jinzhu/copier v0.3.2 h1:QdBOCbaouLDYaIPFfi1bKv5F5tPpeTwXe4sD0jqtz5w= +github.com/jinzhu/copier v0.3.2/go.mod h1:24xnZezI2Yqac9J61UC6/dG/k76ttpq0DdJI3QmUvro= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= From 303dcc6a775474dfdec064fbd312a96f565cbda1 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 19 Aug 2021 10:13:51 +0800 Subject: [PATCH 139/404] publish maxOpenConns --- cmd/nacos_publish_config/main.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 5754b6da..103f7c45 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -45,6 +45,7 @@ var ( localCfgFile = flag.String("local-cfg-file", "/etc/clickhouse_sinker.json", "local config file") replicas = flag.Int("replicas", 1, "replicate each task to multiple ones with the same config except task name, consumer group and table name") + maxOpenConns = flag.Int("max-open-conns", 0, "max open connections per shard") ) // Empty is not valid namespaceID @@ -87,6 +88,9 @@ func PublishSinkerConfig() { cfg.Tasks = append(cfg.Tasks, taskCfg) } } + if *maxOpenConns > 0 { + cfg.Clickhouse.MaxOpenConns = *maxOpenConns + } ncm := config.NacosConfManager{} properties := getProperties() From b369e1055f74a09a94743132441ac29f3b6c6b02 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 19 Aug 2021 11:18:50 +0800 Subject: [PATCH 140/404] fix cfg.Clickhouse.MaxOpenConns --- config/config.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/config.go b/config/config.go index 76273344..fb568aca 100644 --- a/config/config.go +++ b/config/config.go @@ -196,7 +196,7 @@ func (cfg *Config) Normallize() (err error) { if cfg.Clickhouse.RetryTimes < 0 { cfg.Clickhouse.RetryTimes = 0 } - if cfg.Clickhouse.MaxOpenConns < 0 { + if cfg.Clickhouse.MaxOpenConns <= 0 { cfg.Clickhouse.MaxOpenConns = defaultMaxOpenConns } From 9d65d19a9d9844ca2c6100376f083f92731e9f1f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 19 Aug 2021 18:45:56 +0800 Subject: [PATCH 141/404] fix deadlock of quit --- cmd/clickhouse_sinker/main.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 23df4f1c..62091265 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -302,16 +302,16 @@ func (s *Sinker) Close() { } func (s *Sinker) stopAllTasks() { - for taskName, task := range s.tasks { - task.NotifyStop() - delete(s.tasks, taskName) - } util.Logger.Info("stopping parsing pool") util.GlobalParsingPool.StopWait() util.Logger.Info("stopping writing pool") util.GlobalWritingPool.StopWait() util.Logger.Info("stopping timer wheel") util.GlobalTimerWheel.Stop() + for taskName, task := range s.tasks { + task.NotifyStop() + delete(s.tasks, taskName) + } for taskName, task := range s.tasks { task.Stop() delete(s.tasks, taskName) From 2704ba0606fb3d1fcf4ec0325c84b244415c2b79 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 19 Aug 2021 22:25:13 +0800 Subject: [PATCH 142/404] a bit less log --- task/ring.go | 6 ++++-- task/task.go | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/task/ring.go b/task/ring.go index 01e7e1d7..2c1681ac 100644 --- a/task/ring.go +++ b/task/ring.go @@ -69,7 +69,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { if errors.Is(err, goetty.ErrSystemStopped) { util.Logger.Info("Ring.PutElem scheduling timer to a stopped timer wheel", zap.String("task", taskCfg.Name), zap.Error(err)) } else { - util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) + util.Logger.Fatal("scheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } } @@ -94,7 +94,9 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { defer ring.mux.Unlock() if arg != nil { newMsg, _ = arg.(*model.InputMessage) - util.Logger.Warn(fmt.Sprintf("Ring.ForceBatchOrShard partition %d message range [%d, %d)", newMsg.Partition, ring.ringGroundOff, newMsg.Offset), zap.String("task", taskCfg.Name)) + if newMsg.Offset > ring.ringGroundOff { + util.Logger.Warn(fmt.Sprintf("Ring.ForceBatchOrShard partition %d message range [%d, %d)", newMsg.Partition, ring.ringGroundOff, newMsg.Offset), zap.String("task", taskCfg.Name)) + } } if !ring.isIdle { if newMsg == nil { diff --git a/task/task.go b/task/task.go index 3446ec54..7153c939 100644 --- a/task/task.go +++ b/task/task.go @@ -281,7 +281,7 @@ func (service *Service) put(msg model.InputMessage) { util.Logger.Info("Service.put scheduling timer to a stopped timer wheel") } else { err = errors.Wrap(err, "") - util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) + util.Logger.Fatal("scheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } } From d7a4ab2a0b1f65fbc0647e05050acff2ca459ed7 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 19 Aug 2021 23:11:33 +0800 Subject: [PATCH 143/404] fix stopAllTasks --- cmd/clickhouse_sinker/main.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 62091265..d7d8aa8b 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -308,9 +308,8 @@ func (s *Sinker) stopAllTasks() { util.GlobalWritingPool.StopWait() util.Logger.Info("stopping timer wheel") util.GlobalTimerWheel.Stop() - for taskName, task := range s.tasks { + for _, task := range s.tasks { task.NotifyStop() - delete(s.tasks, taskName) } for taskName, task := range s.tasks { task.Stop() @@ -373,6 +372,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 3. Restart goroutine pools. util.Logger.Info("restarting parsing, writing and timer pool") + util.GlobalTimerWheel = nil util.InitGlobalTimerWheel() util.GlobalParsingPool.Restart() util.GlobalWritingPool.Resize(len(newCfg.Clickhouse.Hosts) * newCfg.Clickhouse.MaxOpenConns) From 8889154c5d680f671d1c6e00cfc88a0cd9da5fd6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 19 Aug 2021 23:44:12 +0800 Subject: [PATCH 144/404] KafkaSarama.Stop KafkaGo.Stop wait --- input/kafka_go.go | 2 ++ input/kafka_sarama.go | 2 ++ 2 files changed, 4 insertions(+) diff --git a/input/kafka_go.go b/input/kafka_go.go index f2d07be1..579f730e 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -152,6 +152,7 @@ LOOP_KAFKA_GO: Timestamp: &msg.Time, }) } + k.stopped <- struct{}{} } func (k *KafkaGo) CommitMessages(ctx context.Context, msg *model.InputMessage) (err error) { @@ -170,6 +171,7 @@ func (k *KafkaGo) CommitMessages(ctx context.Context, msg *model.InputMessage) ( func (k *KafkaGo) Stop() error { if k.r != nil { k.r.Close() + <-k.stopped } return nil } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index eb920c8f..8c82003b 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -161,6 +161,7 @@ LOOP_SARAMA: } } } + k.stopped <- struct{}{} } func (k *KafkaSarama) CommitMessages(ctx context.Context, msg *model.InputMessage) error { @@ -171,6 +172,7 @@ func (k *KafkaSarama) CommitMessages(ctx context.Context, msg *model.InputMessag // Stop kafka consumer and close all connections func (k *KafkaSarama) Stop() error { k.cg.Close() + <-k.stopped return nil } From 8443706b54b4f4f8b3e44f37b5312186a652ade2 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 21 Aug 2021 12:39:33 +0800 Subject: [PATCH 145/404] adjusted queueSize --- cmd/clickhouse_sinker/main.go | 5 +++-- util/common.go | 9 ++++++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index d7d8aa8b..9ad13ec2 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -375,9 +375,10 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { util.GlobalTimerWheel = nil util.InitGlobalTimerWheel() util.GlobalParsingPool.Restart() - util.GlobalWritingPool.Resize(len(newCfg.Clickhouse.Hosts) * newCfg.Clickhouse.MaxOpenConns) + maxWorkers := len(newCfg.Clickhouse.Hosts) * newCfg.Clickhouse.MaxOpenConns + util.GlobalWritingPool.Resize(maxWorkers) util.GlobalWritingPool.Restart() - util.Logger.Info("resized parsing pool", zap.Int("maxWorkers", len(newCfg.Clickhouse.Hosts))) + util.Logger.Info("resized writing pool", zap.Int("maxWorkers", maxWorkers)) // 4. Generate, initialize and run tasks. for _, taskCfg := range newCfg.Tasks { diff --git a/util/common.go b/util/common.go index 55e4ed68..d00f22ba 100644 --- a/util/common.go +++ b/util/common.go @@ -69,8 +69,9 @@ func InitGlobalParsingPool() { } else { maxWorkers = 1 } - GlobalParsingPool = NewWorkerPool(maxWorkers, 1<<16) - Logger.Info("initialized parsing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", 100*runtime.NumCPU())) + queueSize := 1 << 16 + GlobalParsingPool = NewWorkerPool(maxWorkers, queueSize) + Logger.Info("initialized parsing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", queueSize)) } // InitGlobalWritingPool initialize GlobalWritingPool @@ -78,9 +79,11 @@ func InitGlobalWritingPool(maxWorkers int) { if GlobalWritingPool != nil { return } - queueSize := runtime.NumCPU() / 4 + queueSize := runtime.NumCPU() if queueSize < 3 { queueSize = 3 + } else if queueSize > 100 { + queueSize = 100 } GlobalWritingPool = NewWorkerPool(maxWorkers, queueSize) Logger.Info("initialized writing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", queueSize)) From 63a66954be0c2d0f1db45e884311920a8bfccc60 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 26 Aug 2021 23:42:48 +0800 Subject: [PATCH 146/404] building image is fragile --- .gitignore | 1 - .goreleaser.yml | 11 ----------- Dockerfile | 2 +- Dockerfile_goreleaser | 8 +++----- 4 files changed, 4 insertions(+), 18 deletions(-) diff --git a/.gitignore b/.gitignore index 2000211d..804e3dd9 100644 --- a/.gitignore +++ b/.gitignore @@ -8,7 +8,6 @@ package-lock.json *.log .idea *.bin -.goreleaser.yml dist vendor diff --git a/.goreleaser.yml b/.goreleaser.yml index 35c5acf7..0be27caf 100644 --- a/.goreleaser.yml +++ b/.goreleaser.yml @@ -27,17 +27,6 @@ builds: - arm64 main: cmd/nacos_publish_config/main.go binary: nacos_publish_config -dockers: - - - binaries: - - clickhouse_sinker - - nacos_publish_config - goos: linux - goarch: amd64 - dockerfile: Dockerfile_goreleaser - image_templates: - - quay.io/housepower/clickhouse_sinker:latest - - quay.io/housepower/clickhouse_sinker:{{ .Tag }} archives: - replacements: darwin: Darwin diff --git a/Dockerfile b/Dockerfile index 153f058b..b1c0fbc2 100644 --- a/Dockerfile +++ b/Dockerfile @@ -13,7 +13,7 @@ COPY --from=builder /app/nacos_publish_config /usr/local/bin/nacos_publish_confi COPY --from=builder /app/kafka_gen_log /usr/local/bin/kafka_gen_log COPY --from=builder /app/kafka_gen_metric /usr/local/bin/kafka_gen_metric -# clickhouse_sinker gets config from local directory "/etc/clickhouse_sinker" by default. +# clickhouse_sinker gets config from local file "/etc/clickhouse_sinker.json" by default. # Customize behavior with following env variables: # - V # - HTTP_PORT diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index 43846abc..cb66589d 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -1,12 +1,10 @@ FROM alpine:latest RUN apk --no-cache add ca-certificates tzdata RUN echo "UTC" > /etc/timezone -ADD clickhouse_sinker /usr/local/bin/clickhouse_sinker -ADD nacos_publish_config /usr/local/bin/nacos_publish_config -ADD kafka_gen_log /usr/local/bin/kafka_gen_log -ADD kafka_gen_metric /usr/local/bin/kafka_gen_metric +ADD dist/clickhouse_sinker_linux_amd64/clickhouse_sinker /usr/local/bin/clickhouse_sinker +ADD dist/nacos_publish_config_linux_amd64/nacos_publish_config /usr/local/bin/nacos_publish_config -# clickhouse_sinker gets config from local directory "/etc/clickhouse_sinker" by default. +# clickhouse_sinker gets config from local file "/etc/clickhouse_sinker.json" by default. # Customize behavior with following env variables: # - V # - HTTP_PORT From ff04a25253ad038239d93629d69183bd18293436 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 2 Sep 2021 11:18:39 +0800 Subject: [PATCH 147/404] introduced ClickHouse.Stop, and fixed batch.Commit failure due to canceled context --- cmd/clickhouse_sinker/main.go | 8 +++---- output/clickhouse.go | 40 +++++++++++++++++++++++++++++++---- task/ring.go | 4 ++-- task/task.go | 5 ++++- 4 files changed, 46 insertions(+), 11 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 9ad13ec2..656ffc90 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -304,10 +304,6 @@ func (s *Sinker) Close() { func (s *Sinker) stopAllTasks() { util.Logger.Info("stopping parsing pool") util.GlobalParsingPool.StopWait() - util.Logger.Info("stopping writing pool") - util.GlobalWritingPool.StopWait() - util.Logger.Info("stopping timer wheel") - util.GlobalTimerWheel.Stop() for _, task := range s.tasks { task.NotifyStop() } @@ -315,6 +311,10 @@ func (s *Sinker) stopAllTasks() { task.Stop() delete(s.tasks, taskName) } + util.Logger.Info("stopping timer wheel") + util.GlobalTimerWheel.Stop() + util.Logger.Info("stopping writing pool") + util.GlobalWritingPool.StopWait() } func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { diff --git a/output/clickhouse.go b/output/clickhouse.go index 1770af8c..65b0bb7e 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -25,6 +25,7 @@ import ( "sort" "strings" "sync" + "sync/atomic" "time" "github.com/ClickHouse/clickhouse-go" @@ -54,11 +55,17 @@ type ClickHouse struct { cfg *config.Config taskCfg *config.TaskConfig prepareSQL string + stopped int32 + numFlying int32 + mux sync.Mutex + taskDone *sync.Cond } // NewClickHouse new a clickhouse instance func NewClickHouse(cfg *config.Config, taskCfg *config.TaskConfig) *ClickHouse { - return &ClickHouse{cfg: cfg, taskCfg: taskCfg} + ck := &ClickHouse{cfg: cfg, taskCfg: taskCfg} + ck.taskDone = sync.NewCond(&ck.mux) + return ck } // Init the clickhouse intance @@ -66,11 +73,32 @@ func (c *ClickHouse) Init() (err error) { return c.initSchema() } +// Stop drains flying batchs +func (c *ClickHouse) Stop() { + atomic.StoreInt32(&c.stopped, 1) + c.mux.Lock() + defer c.mux.Unlock() + for atomic.LoadInt32(&c.numFlying) != 0 { + c.taskDone.Wait() + } + return +} + // Send a batch to clickhouse func (c *ClickHouse) Send(batch *model.Batch) { + if atomic.LoadInt32(&c.stopped) != 0 { + return + } + atomic.AddInt32(&c.numFlying, 1) statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Inc() _ = util.GlobalWritingPool.Submit(func() { c.loopWrite(batch) + numFlying := atomic.AddInt32(&c.numFlying, -1) + if numFlying == 0 { + c.mux.Lock() + c.taskDone.Signal() + c.mux.Unlock() + } statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Dec() }) } @@ -139,19 +167,23 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { var dbVer int sc := pool.GetShardConn(batch.BatchIdx) for { + if atomic.LoadInt32(&c.stopped) != 0 { + util.Logger.Info("ClickHouse.loopWrite quit due to stopped be true", zap.String("task", c.taskCfg.Name)) + return + } if err = c.write(batch, sc, &dbVer); err == nil { if err = batch.Commit(); err == nil { return } // Note: kafka_go and sarama commit give different error when context is cancceled. if errors.Is(err, context.Canceled) || errors.Is(err, io.ErrClosedPipe) { - util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) + util.Logger.Warn("Batch.Commit failed due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) return } - util.Logger.Fatal("committing offset failed with permanent error %+v", zap.String("task", c.taskCfg.Name), zap.Error(err)) + util.Logger.Fatal("Batch.Commit failed with permanent error %+v", zap.String("task", c.taskCfg.Name), zap.Error(err)) } if errors.Is(err, context.Canceled) { - util.Logger.Info("ClickHouse.loopWrite quit due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) + util.Logger.Info("ClickHouse.write failed due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) return } util.Logger.Error("flush batch failed", zap.String("task", c.taskCfg.Name), zap.Int("try", times), zap.Error(err)) diff --git a/task/ring.go b/task/ring.go index 2c1681ac..0140357e 100644 --- a/task/ring.go +++ b/task/ring.go @@ -85,7 +85,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { taskCfg := ring.service.taskCfg select { case <-ring.service.ctx.Done(): - util.Logger.Error("Ring.ForceBatchOrShard quit due to the context has been canceled", zap.String("task", taskCfg.Name)) + util.Logger.Warn("Ring.ForceBatchOrShard quit due to the context has been canceled", zap.String("task", taskCfg.Name)) return default: } @@ -133,7 +133,7 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { var err error if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Info("Ring.ForceBatchOrShard scheduling timer to a stopped timer wheel", zap.String("task", taskCfg.Name), zap.Error(err)) + util.Logger.Warn("Ring.ForceBatchOrShard scheduling timer to a stopped timer wheel", zap.String("task", taskCfg.Name), zap.Error(err)) } else { err = errors.Wrap(err, "") util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) diff --git a/task/task.go b/task/task.go index 7153c939..9d5c51cb 100644 --- a/task/task.go +++ b/task/task.go @@ -153,6 +153,7 @@ LOOP: for { select { case <-service.ctx.Done(): + util.Logger.Info("Service.Run quit due to context has been canceled", zap.String("task", service.taskCfg.Name)) break LOOP case batch := <-service.batchChan: if err := service.flush(batch); err != nil { @@ -165,7 +166,7 @@ LOOP: func (service *Service) fnCommit(partition int, offset int64) error { msg := model.InputMessage{Topic: service.taskCfg.Topic, Partition: partition, Offset: offset} - return service.inputer.CommitMessages(service.ctx, &msg) + return service.inputer.CommitMessages(service.parentCtx, &msg) } func (service *Service) put(msg model.InputMessage) { @@ -335,6 +336,8 @@ func (service *Service) Stop() { } util.Logger.Info("stopping task service...", zap.String("task", taskCfg.Name)) service.cancel() + service.clickhouse.Stop() + util.Logger.Info("stopped output", zap.String("task", taskCfg.Name)) if err := service.inputer.Stop(); err != nil { util.Logger.Fatal("service.inputer.Stop failed", zap.Error(err)) } From 00189f6444835a3bc8f94efd462bcfeadfa7eba6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 2 Sep 2021 20:51:51 +0800 Subject: [PATCH 148/404] introduced ClickHouse.NotifyStop --- cmd/clickhouse_sinker/main.go | 2 ++ input/kafka_go.go | 6 +++--- input/kafka_sarama.go | 6 +++--- output/clickhouse.go | 5 +++++ task/task.go | 3 ++- 5 files changed, 15 insertions(+), 7 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 656ffc90..e5841d74 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -410,6 +410,8 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { } } // 2. Stop tasks in parallel found at the previous step. + // They must drain flying batchs as quickly as possible to allow another clickhouse_sinker + // instance take over partitions safely. for _, taskName := range tasksToStop { task := s.tasks[taskName] task.NotifyStop() diff --git a/input/kafka_go.go b/input/kafka_go.go index 579f730e..f710053e 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -131,15 +131,15 @@ LOOP_KAFKA_GO: var msg kafka.Message if msg, err = k.r.FetchMessage(ctx); err != nil { if errors.Is(err, context.Canceled) { - util.Logger.Info("Kafka.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) + util.Logger.Info("KafkaGo.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) break LOOP_KAFKA_GO } else if errors.Is(err, io.EOF) { - util.Logger.Info("Kafka.Run quit due to reader has been closed", zap.String("task", k.taskCfg.Name)) + util.Logger.Info("KafkaGo.Run quit due to reader has been closed", zap.String("task", k.taskCfg.Name)) break LOOP_KAFKA_GO } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.taskCfg.Name).Inc() err = errors.Wrap(err, "") - util.Logger.Error("k.r.FetchMessage failed", zap.String("task", k.taskCfg.Name), zap.Error(err)) + util.Logger.Error("kafka.Reader.FetchMessage failed", zap.String("task", k.taskCfg.Name), zap.Error(err)) continue } } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 8c82003b..fa6c93e4 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -148,15 +148,15 @@ LOOP_SARAMA: // recreated to get the new claims if err := k.cg.Consume(ctx, []string{taskCfg.Topic}, handler); err != nil { if errors.Is(err, context.Canceled) { - util.Logger.Info("Kafka.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) + util.Logger.Info("KafkaSarama.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) break LOOP_SARAMA } else if errors.Is(err, sarama.ErrClosedConsumerGroup) { - util.Logger.Info("Kafka.Run quit due to consumer group has been closed", zap.String("task", k.taskCfg.Name)) + util.Logger.Info("KafkaSarama.Run quit due to consumer group has been closed", zap.String("task", k.taskCfg.Name)) break LOOP_SARAMA } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() err = errors.Wrap(err, "") - util.Logger.Error("Kafka.Run got error %+v", zap.String("task", k.taskCfg.Name), zap.Error(err)) + util.Logger.Error("sarama.ConsumerGroup.Consume failed", zap.String("task", k.taskCfg.Name), zap.Error(err)) continue } } diff --git a/output/clickhouse.go b/output/clickhouse.go index 65b0bb7e..52672445 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -73,6 +73,11 @@ func (c *ClickHouse) Init() (err error) { return c.initSchema() } +// NotifyStop notify loopWrite to quit +func (c *ClickHouse) NotifyStop() { + atomic.StoreInt32(&c.stopped, 1) +} + // Stop drains flying batchs func (c *ClickHouse) Stop() { atomic.StoreInt32(&c.stopped, 1) diff --git a/task/task.go b/task/task.go index 9d5c51cb..850ae1ec 100644 --- a/task/task.go +++ b/task/task.go @@ -166,7 +166,7 @@ LOOP: func (service *Service) fnCommit(partition int, offset int64) error { msg := model.InputMessage{Topic: service.taskCfg.Topic, Partition: partition, Offset: offset} - return service.inputer.CommitMessages(service.parentCtx, &msg) + return service.inputer.CommitMessages(context.Background(), &msg) } func (service *Service) put(msg model.InputMessage) { @@ -325,6 +325,7 @@ func (service *Service) NotifyStop() { taskCfg := service.taskCfg util.Logger.Info("notified stopping task service...", zap.String("task", taskCfg.Name)) service.cancel() + service.clickhouse.NotifyStop() } // Stop stop kafka and clickhouse client. This is blocking. From cbc1797295130ed1f1ac78ae7ccc9724b621b6aa Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 4 Sep 2021 22:44:39 +0800 Subject: [PATCH 149/404] bump dependencies version --- config/config.go | 2 +- docker/test_auto_schema.json | 2 +- docker/test_dynamic_schema.json | 2 +- docker/test_fixed_schema.json | 2 +- docs/configuration/config.md | 2 +- docs/dev/introduction.md | 18 +- go.mod | 26 ++- go.sum | 312 ++++++-------------------------- 8 files changed, 84 insertions(+), 282 deletions(-) diff --git a/config/config.go b/config/config.go index fb568aca..831bc013 100644 --- a/config/config.go +++ b/config/config.go @@ -180,7 +180,7 @@ func (cfg *Config) Normallize() (err error) { return } if cfg.Kafka.Version == "" { - cfg.Kafka.Version = "2.2.1" + cfg.Kafka.Version = "2.6.0" } cfg.convertKfkSecurity() diff --git a/docker/test_auto_schema.json b/docker/test_auto_schema.json index 72f1cb59..435bc405 100644 --- a/docker/test_auto_schema.json +++ b/docker/test_auto_schema.json @@ -13,7 +13,7 @@ }, "kafka": { "brokers": "127.0.0.1:9093", - "version": "2.2.1" + "version": "2.6.0" }, "task": { "name": "test_auto_schema", diff --git a/docker/test_dynamic_schema.json b/docker/test_dynamic_schema.json index d251cd04..263be263 100644 --- a/docker/test_dynamic_schema.json +++ b/docker/test_dynamic_schema.json @@ -14,7 +14,7 @@ }, "kafka": { "brokers": "127.0.0.1:9093", - "version": "2.2.1" + "version": "2.6.0" }, "task": { "name": "test_dynamic_schema", diff --git a/docker/test_fixed_schema.json b/docker/test_fixed_schema.json index d9f11164..56c4d7a1 100644 --- a/docker/test_fixed_schema.json +++ b/docker/test_fixed_schema.json @@ -13,7 +13,7 @@ }, "kafka": { "brokers": "127.0.0.1:9093", - "version": "2.2.1" + "version": "2.6.0" }, "task": { "name": "test_fixed_schema", diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 8c90d663..dd5c5a78 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -85,7 +85,7 @@ }, // kafka version, if you use sarama, the version must be specified - "version": "2.2.1" + "version": "2.6.0" }, "task": { diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index a7c3c1f3..27492411 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -103,7 +103,7 @@ An example kafka config: "kafka": { "brokers": "192.168.31.64:9092", "@version": "Required if you use sarama. It's the the Kafka server version.", - "version": "2.2.1" + "version": "2.6.0" } ``` @@ -114,7 +114,7 @@ An example kafka config: ```json "kafka": { "brokers": "192.168.31.64:9093", - "version": "2.2.1", + "version": "2.6.0", "tls": { "enable": true, "@trustStoreLocation": "ssl.truststore.location which kafka-console-consumer.sh uses", @@ -132,7 +132,7 @@ Or if you have extracted certificates from JKS, use the following config: ```json "kafka": { "brokers": "192.168.31.64:9093", - "version": "2.2.1", + "version": "2.6.0", "tls": { "enable": true, "@caCertFiles": "Required. It's the CA certificate with which Kafka brokers certs be signed. This cert is added to kafka.client.truststore.jks which kafka-console-consumer.sh uses", @@ -169,7 +169,7 @@ An example kafka config: "kafka": { "brokers": "192.168.31.64:9092", "@version": "Required if you use sarama. It's the the Kafka server version.", - "version": "2.2.1" + "version": "2.6.0" } ``` @@ -180,7 +180,7 @@ An example kafka config: ```json "kafka": { "brokers": "192.168.31.64:9094", - "version": "2.2.1", + "version": "2.6.0", "sasl": { "enable": true, "mechanism": "PLAIN", @@ -211,7 +211,7 @@ An example kafka config: ```json "kafka": { "brokers": "192.168.31.64:9094", - "version": "2.2.1", + "version": "2.6.0", "sasl": { "enable": true, "@mechanism": "SCRAM-SHA-256 or SCRAM-SHA-512", @@ -243,7 +243,7 @@ An example kafka config: ```json "kafka": { "brokers": "192.168.31.64:9094", - "version": "2.2.1", + "version": "2.6.0", "sasl": { "enable": true, "mechanism": "GSSAPI", @@ -359,7 +359,7 @@ type RemoteConfManager interface { ## Kafka Compatibility -Kafka broker [exposes versions of various APIs it supports since 0.10.0.0](https://kafka.apache.org/protocol#api_versions). +Kafka release history is at [here](https://kafka.apache.org/downloads). Kafka broker [exposes versions of various APIs it supports since 0.10.0.0](https://kafka.apache.org/protocol#api_versions). ### Kafka-go @@ -368,7 +368,7 @@ Kafka broker [exposes versions of various APIs it supports since 0.10.0.0](https ### Sarama -- Sarama guarantees compatibility [with Kafka 2.4 through 2.6](https://github.com/Shopify/sarama/blob/master/README.md#compatibility-and-api-stability). +- Sarama guarantees compatibility [with Kafka 2.6 through 2.8](https://github.com/Shopify/sarama/blob/master/README.md#compatibility-and-api-stability). - Sarama [has tied it's protocol usage to the Version field in Config](https://github.com/Shopify/sarama/issues/1732). ### Conclusion diff --git a/go.mod b/go.mod index 7ff41290..3e0a53e6 100644 --- a/go.mod +++ b/go.mod @@ -3,25 +3,35 @@ module github.com/housepower/clickhouse_sinker go 1.14 require ( - github.com/ClickHouse/clickhouse-go v1.4.5 - github.com/Shopify/sarama v1.29.0 + github.com/ClickHouse/clickhouse-go v1.4.7 + github.com/Shopify/sarama v1.29.1 github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a - github.com/cespare/xxhash/v2 v2.1.1 + github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 + github.com/golang/snappy v0.0.4 // indirect github.com/google/gops v0.3.18 github.com/jinzhu/copier v0.3.2 + github.com/klauspost/compress v1.13.5 // indirect github.com/nacos-group/nacos-sdk-go v1.0.7 + github.com/pierrec/lz4 v2.6.1+incompatible // indirect github.com/pkg/errors v0.9.1 - github.com/prometheus/client_golang v1.10.0 - github.com/prometheus/common v0.24.0 - github.com/segmentio/kafka-go v0.4.8 + github.com/prometheus/client_golang v1.11.0 + github.com/prometheus/common v0.26.0 + github.com/segmentio/kafka-go v0.4.17 github.com/sirupsen/logrus v1.6.0 + github.com/smartystreets/goconvey v1.6.4 // indirect github.com/stretchr/testify v1.7.0 - github.com/tidwall/gjson v1.8.0 + github.com/tidwall/gjson v1.9.0 + github.com/tidwall/pretty v1.2.0 // indirect github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.0.2 - go.uber.org/zap v1.16.0 + go.uber.org/atomic v1.9.0 // indirect + go.uber.org/multierr v1.7.0 // indirect + go.uber.org/zap v1.19.0 + golang.org/x/crypto v0.0.0-20210817164053-32db794688a5 // indirect + golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f // indirect golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba + golang.org/x/tools v0.0.0-20200103221440-774c71fcf114 // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) diff --git a/go.sum b/go.sum index 12839b3e..6de8427a 100644 --- a/go.sum +++ b/go.sum @@ -1,20 +1,15 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/clickhouse-go v1.4.5 h1:FfhyEnv6/BaWldyjgT2k4gDDmeNwJ9C4NbY/MXxJlXk= -github.com/ClickHouse/clickhouse-go v1.4.5/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= +github.com/ClickHouse/clickhouse-go v1.4.7 h1:NNZQmlW8dVxGn19pF65BmWr0vq8Pj5Iy8ykyBKhFCPw= +github.com/ClickHouse/clickhouse-go v1.4.7/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0= -github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= -github.com/Shopify/sarama v1.29.0 h1:ARid8o8oieau9XrHI55f/L3EoRAhm9px6sonbD7yuUE= -github.com/Shopify/sarama v1.29.0/go.mod h1:2QpgD79wpdAESqNQMxNc0KYMkycd4slxGdV3TWSVqrU= +github.com/Shopify/sarama v1.29.1 h1:wBAacXbYVLmWieEA/0X/JagDdCZ8NVFOfS6l6+2u5S0= +github.com/Shopify/sarama v1.29.1/go.mod h1:mdtqvCSg8JOxk8PmpTNGyo6wzd4BMm4QXSfDnTXmgkE= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= -github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= -github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= @@ -22,79 +17,48 @@ github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRF github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 h1:zOVTBdCKFd9JbCKz9/nt+FovbjPFmb7mUnp8nH9fQBA= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= -github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= -github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= -github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= -github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= -github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a/go.mod h1:DAHtR1m6lCRdSC2Tm3DSWRPvIPr6xNKyeHdqDQSQT+A= -github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU= -github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= -github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= +github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= +github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAKk= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a h1:6ikHdu0dgMWdck/K6ZlMcN+Gx44c8PeZ54mAsQ5LpOg= github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a/go.mod h1:Jag0ltWFtHzILaHByar6cc/Ru+9qv0JJzyOCfCE7jtk= -github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= -github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= +github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chenzhuoyu/base64x v0.0.0-20210528162528-3c6c11c43ee5 h1:7AStn2tanqGY99xzW+Ve1p6YYqnRr1m/yswJ4h0BhcY= github.com/chenzhuoyu/base64x v0.0.0-20210528162528-3c6c11c43ee5/go.mod h1:NfDzX8KeqVNX62apij1OkqoeDdq1VR3g0TRZo99kkBA= -github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= -github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= -github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= -github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= github.com/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q= github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= -github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M= -github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4safvEdbitLhGGK48rN6g= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/fagongzi/goetty v1.7.0 h1:Z0uoEVqP4uQSQW+HR3bg5GGwmisZpJQ1sK/ab9HK7q0= github.com/fagongzi/goetty v1.7.0/go.mod h1:lLUyHhtjlOqatxVXgyLocwoI2o359JzLE7EWRGZiGw4= github.com/fastly/go-utils v0.0.0-20180712184237-d95a45783239 h1:Ghm4eQYC0nEPnSJdVkTrXpu9KtoVCSo1hg7mtI7G9KU= github.com/fastly/go-utils v0.0.0-20180712184237-d95a45783239/go.mod h1:Gdwt2ce0yfBxPvZrHkprdPPTTS3N5rwmLE8T22KBXlw= -github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= -github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= -github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= -github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= -github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= @@ -103,15 +67,8 @@ github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/goccy/go-json v0.7.1 h1:VMhnh5gcc8De8f6m2DLvSqY1x8Jwl3btet+EqMP0QNs= github.com/goccy/go-json v0.7.1/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= -github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -125,62 +82,28 @@ github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvq github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.3 h1:JjCZWpVbqXDqFVmTfYWEVTMIYrL/NPdPSCHPJ0T/raM= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA= github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= +github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.4 h1:L8R9j+yAqZuZjsqh/z+F1NCffTKKLShY6zXTItVIZ8M= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gops v0.3.18 h1:my259V+172PVFmduS2RAsq4FKH+HjKqdh7pLr17Ot8c= github.com/google/gops v0.3.18/go.mod h1:Pfp8hWGIFdV/7rY9/O/U5WgdjYQXf/GiEK4NVuVd2ZE= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= -github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= -github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/securecookie v1.1.1 h1:miw7JPhV+b/lAHSXz4qd/nN9jRiAFV5FwjeKyCS8BvQ= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1 h1:DHd3rPN5lE3Ts3D8rKkQ8x/0kqfeNmBAaiSi+o7FsgI= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= -github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= -github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= -github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE= -github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= -github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= -github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= -github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= -github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= -github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= -github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= -github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= -github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= -github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= -github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= -github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= -github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= -github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= -github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= -github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= @@ -205,20 +128,20 @@ github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22 github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.10 h1:Kz6Cvnvv2wGdaG/V8yMvfkmNiXq9Ya2KUv4rouJJr68= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.11 h1:uVUAXhF2To8cbw/3xN3pxj6kk7TYKs98NIrTqPlMWAQ= +github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= -github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.12.2 h1:2KCfW3I9M7nSc5wOqXAlW2v2U6v+w6cbjvbfp+OykW8= github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= +github.com/klauspost/compress v1.13.5 h1:9O69jUPDcsT9fEm74W92rZL9FQY7rCdaXVneq+yyzl4= +github.com/klauspost/compress v1.13.5/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/cpuid/v2 v2.0.6 h1:dQ5ueTiftKxp0gyjKSx5+8BtPWkyQbd95m8Gys/RarI= github.com/klauspost/cpuid/v2 v2.0.6/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -239,24 +162,9 @@ github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f/go.mod github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 h1:Bvq8AziQ5jFF4BHGAEDSqwPW1NJS3XshxbRCxtjFAZc= github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042/go.mod h1:TPpsiPUEh0zFL1Snz4crhMlBe60PYxRHr5oFF3rRYg0= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= -github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743/go.mod h1:qklhhLq1aX+mtWk9cPHPzaBjWImj5ULL6C7HFJtXQMM= -github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4= -github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= -github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= -github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= -github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= -github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= -github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= -github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= -github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= -github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -267,85 +175,44 @@ github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRW github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nacos-group/nacos-sdk-go v1.0.7 h1:Am1tJFe7GUTNCREKsZ5ok0H2OspHDRmRcsxn7DiSwhA= github.com/nacos-group/nacos-sdk-go v1.0.7/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= -github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= -github.com/nats-io/jwt v0.3.2/go.mod h1:/euKqTS1ZD+zzjYrY7pseZrTtWQSjujC7xjPc8wL6eU= -github.com/nats-io/nats-server/v2 v2.1.2/go.mod h1:Afk+wRZqkMQs/p45uXdrVLuab3gwv3Z8C4HTBu8GD/k= -github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= -github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= -github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= -github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= -github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= -github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= -github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= -github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= -github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= -github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= -github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= -github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/openzipkin-contrib/zipkin-go-opentracing v0.4.5/go.mod h1:/wsWhb9smxSfWAKL3wpBW7V8scJMt8N8gnaMCS9E/cA= -github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw= -github.com/openzipkin/zipkin-go v0.2.1/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= -github.com/openzipkin/zipkin-go v0.2.2/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= -github.com/pact-foundation/pact-go v1.0.4/go.mod h1:uExwJY4kCzNPcHRj+hCR/HBbOOIwwtUjcrb0b5/5kLM= -github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= -github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= -github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac= -github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= -github.com/pierrec/lz4 v2.6.0+incompatible h1:Ix9yFKn1nSPBLFl/yZknTp8TU5G4Ps0JDmguYK6iH1A= github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= +github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= +github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.10.0 h1:/o0BDeWzLWXNZ+4q5gXltUvaMpJqckTa+jTNoB+z4cg= -github.com/prometheus/client_golang v1.10.0/go.mod h1:WJM3cc3yu7XKBKa/I8WeZm+V3eltZnBwfENSU7mdogU= +github.com/prometheus/client_golang v1.11.0 h1:HNkLOAEQMIDv/K+04rukrLx6ch7msSRwf3/SASFAGtQ= +github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.18.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= -github.com/prometheus/common v0.24.0 h1:aIycr3wRFxPUq8XlLQlGQ9aNXV3dFi5y62pe/SB262k= -github.com/prometheus/common v0.24.0/go.mod h1:H6QK/N6XVT42whUeIdI3dp36w49c+/iMDk7UAI2qm7Q= +github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ= +github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= -github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= -github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= -github.com/segmentio/kafka-go v0.4.8 h1:LO36H2tb7RcCRjsYzT/qf7xE+vRBXgddZDD82e1eiWY= -github.com/segmentio/kafka-go v0.4.8/go.mod h1:Inh7PqOsxmfgasV8InZYKVXWsdjcCq2d9tFV75GLbuM= +github.com/segmentio/kafka-go v0.4.17 h1:IyqRstL9KUTDb3kyGPOOa5VffokKWSEzN6geJ92dSDY= +github.com/segmentio/kafka-go v0.4.17/go.mod h1:19+Eg7KwrNKy/PFhiIthEPkO8k+ac7/ZYXwYM9Df10w= github.com/shirou/gopsutil/v3 v3.21.2/go.mod h1:ghfMypLDrFSWN2c9cDYFLHyynQ+QUht0cv/18ZqVczw= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= @@ -355,15 +222,8 @@ github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1 github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= -github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= -github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= -github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= -github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= -github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5NYJe/zRYDwOu9ku6YHy+Iw7l5DM= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= -github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= @@ -375,23 +235,22 @@ github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5Cc github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/tebeka/strftime v0.1.3 h1:5HQXOqWKYRFfNyBMNVc9z5+QzuBtIXy03psIhtdJYto= github.com/tebeka/strftime v0.1.3/go.mod h1:7wJm3dZlpr4l/oVK0t1HYIc4rMzQ2XJlOMIUJUJH6XQ= -github.com/tidwall/gjson v1.8.0 h1:Qt+orfosKn0rbNTZqHYDqBrmm3UDA4KRkv70fDzG+PQ= github.com/tidwall/gjson v1.8.0/go.mod h1:5/xDoumyyDNerp2U36lyolv46b3uF/9Bu6OfyQ9GImk= +github.com/tidwall/gjson v1.9.0 h1:+Od7AE26jAaMgVC31cQV/Ope5iKXulNMflrlB7k+F9E= +github.com/tidwall/gjson v1.9.0/go.mod h1:5/xDoumyyDNerp2U36lyolv46b3uF/9Bu6OfyQ9GImk= github.com/tidwall/match v1.0.3 h1:FQUVvBImDutD8wJLN6c5eMzWtjgONK9MwIBCOrUJKeE= github.com/tidwall/match v1.0.3/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= -github.com/tidwall/pretty v1.1.0 h1:K3hMW5epkdAVwibsQEfR/7Zj0Qgt4DxtNumTq/VloO8= github.com/tidwall/pretty v1.1.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= +github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= +github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= github.com/tklauser/go-sysconf v0.3.4/go.mod h1:Cl2c8ZRWfHD5IrfHo9VN+FX9kCFjIOyVklgXycLB6ek= github.com/tklauser/numcpus v0.2.1/go.mod h1:9aU+wOc6WjUIZEwWMP62PL/41d65P+iks1gBkr4QyP8= -github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 h1:kF/7m/ZU+0D4Jj5eZ41Zm3IH/J8OElK1Qtd7tVKAwLk= github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3/go.mod h1:QDlpd3qS71vYtakd2hmdpqhJ9nwv6mD6A30bQ1BPBFE= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= -github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= -github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= github.com/valyala/fastjson v1.6.3/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= @@ -406,92 +265,59 @@ github.com/xdg/scram v1.0.3/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49 github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= github.com/xdg/stringprep v1.0.3 h1:cmL5Enob4W83ti/ZHuZLuKD/xqJfus4fVPwE+/BDm+4= github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= -github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/xlab/treeprint v1.0.0/go.mod h1:IoImgRak9i3zJyuxOKUP1v4UZd1tMoKkq/Cimt1uhCg= -go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= -go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= -go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= -go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= -go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= +go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= +go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= +go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= +go.uber.org/goleak v1.1.10 h1:z+mqJhf6ss6BSfSM671tgKyZBFPTTJM+HLxnhPC3wu0= +go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= -go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= +go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= +go.uber.org/multierr v1.7.0 h1:zaiO/rmgFjbmCXdSYJWQcdvOCsthmdaHfr3Gm2Kx4Ec= +go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= -go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.16.0 h1:uFRZXykJGK9lLY4HtgSw44DnIcAM+kRBP7x5m+NpAOM= -go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= +go.uber.org/zap v1.19.0 h1:mZQZefskPPCMIBCSEH0v2/iUqqLrYtaeqwD6FUGUnFE= +go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190506204251-e1dfcc566284/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b h1:7mWr3k41Qtv8XlltBkDkl8LoP3mpSgBW8BUoxtEdbXg= -golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20210817164053-32db794688a5 h1:HWj/xjIHfjYU5nVXpTM0s39J9CbLn7Cc5a7IC5rwsMQ= +golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190125091013-d26f9f9a57f3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a h1:njMmldwFTyDLqonHMagNXKBWptTBeDZOdblgaDsNEGQ= -golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/net v0.0.0-20210614182718-04defd469f4e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f h1:w6wWR0H+nyVpbSAQbzVEIACVyr/h8l/BEkY6Sokc7Eg= +golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181122145206-62eef0e2fa9b/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -500,9 +326,10 @@ golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210423082822-04245dca01da h1:b3NXsE2LusjYGGjL5bxEVZZORm/YEFFrWFjR8eFrw/c= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 h1:SrN+KX8Art/Sf4HNj6Zcz06G7VEz+7w9tdXTPOZ7+l4= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= @@ -510,23 +337,16 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba h1:O8mE0/t419eoIwhTFpKVkHiTs/Igowgfkj25AcZrtiE= golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200103221440-774c71fcf114 h1:DnSr2mCsxyCE6ZgIkmcWUQY2R5cH/6wL7eIxEmQOMSE= golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -534,63 +354,35 @@ golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190530194941-fb225487d101/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.20.0/go.mod h1:chYK+tFQF0nDUGJgXMSgLCQk3phJEuONr2DCgLDdAQM= -google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= -google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= -google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.26.0-rc.1 h1:7QnIQpGRHE5RnLKnESfDoxm2dTapTZua5a0kS0A+VXQ= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o= gopkg.in/ini.v1 v1.42.0 h1:7N3gPTt50s8GuLortA00n8AqRTk75qOP98+mTPpgzRk= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= -gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= -gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= -gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI= -gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= rsc.io/goversion v1.2.0/go.mod h1:Eih9y/uIBS3ulggl7KNJ09xGSLcuNaLgmvvqa07sgfo= -sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= -sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= From 7984fb9024dd5694343d4b8220d8e4f8a810b0df Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 5 Sep 2021 00:07:59 +0800 Subject: [PATCH 150/404] sarama generation cleanup --- cmd/clickhouse_sinker/main.go | 35 ++++++++----- config/config.go | 8 +-- docs/configuration/config.md | 30 ++++++++--- docs/dev/introduction.md | 3 +- input/input.go | 2 +- input/kafka_go.go | 2 +- input/kafka_sarama.go | 22 ++++---- output/clickhouse.go | 19 +------ task/ring.go | 18 +++++-- task/sharding.go | 6 +-- task/task.go | 96 +++++++++++++++++++---------------- 11 files changed, 136 insertions(+), 105 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index e5841d74..b2cf725a 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -26,6 +26,7 @@ import ( "os" "reflect" "strings" + "sync" "time" "github.com/housepower/clickhouse_sinker/config" @@ -290,11 +291,9 @@ func (s *Sinker) Run() { // Close shutdown task func (s *Sinker) Close() { - // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). + // 1. Stop tasks gracefully. s.stopAllTasks() - // 2. Stop Sinker.Run main loop - s.cancel() - // 3. Stop pusher + // 2. Stop pusher if s.pusher != nil { s.pusher.Stop() s.pusher = nil @@ -302,15 +301,20 @@ func (s *Sinker) Close() { } func (s *Sinker) stopAllTasks() { - util.Logger.Info("stopping parsing pool") - util.GlobalParsingPool.StopWait() - for _, task := range s.tasks { - task.NotifyStop() + var wg sync.WaitGroup + for _, tsk := range s.tasks { + wg.Add(1) + go func(tsk *task.Service) { + tsk.Stop() + wg.Done() + }(tsk) } - for taskName, task := range s.tasks { - task.Stop() + wg.Wait() + for taskName := range s.tasks { delete(s.tasks, taskName) } + util.Logger.Info("stopping parsing pool") + util.GlobalParsingPool.StopWait() util.Logger.Info("stopping timer wheel") util.GlobalTimerWheel.Stop() util.Logger.Info("stopping writing pool") @@ -412,13 +416,16 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 2. Stop tasks in parallel found at the previous step. // They must drain flying batchs as quickly as possible to allow another clickhouse_sinker // instance take over partitions safely. + var wg sync.WaitGroup for _, taskName := range tasksToStop { - task := s.tasks[taskName] - task.NotifyStop() + wg.Add(1) + go func(tsk *task.Service) { + tsk.Stop() + wg.Done() + }(s.tasks[taskName]) } + wg.Wait() for _, taskName := range tasksToStop { - task := s.tasks[taskName] - task.Stop() delete(s.tasks, taskName) } // 3. Initailize tasks which are new or their config differ. diff --git a/config/config.go b/config/config.go index 831bc013..2baaf20c 100644 --- a/config/config.go +++ b/config/config.go @@ -205,11 +205,11 @@ func (cfg *Config) Normallize() (err error) { cfg.Task = nil } for _, taskCfg := range cfg.Tasks { - if cfg.Kafka.Sasl.Enable && cfg.Kafka.Sasl.Username == "" { - //kafka-go doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 + if taskCfg.KafkaClient == "" || (cfg.Kafka.Sasl.Enable && cfg.Kafka.Sasl.Username == "") { + // known limitations of kafka-go: + // - The Reader API is too high-level. There's no generation cleanup callback which sarama provides. + // - Doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 taskCfg.KafkaClient = "sarama" - } else if taskCfg.KafkaClient == "" { - taskCfg.KafkaClient = "kafka-go" } if taskCfg.Parser == "" || taskCfg.Parser == "json" { taskCfg.Parser = "fastjson" diff --git a/docs/configuration/config.md b/docs/configuration/config.md index dd5c5a78..c64d0f9d 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -35,6 +35,8 @@ "insecureSkipVerify": false, // retryTimes when error occurs in inserting datas "retryTimes": 0, + // max open connections with each clickhouse node. default to 1. + "maxOpenConns": 1 }, // Kafka config @@ -57,7 +59,7 @@ // Required if Kafka brokers require client authentication. "clientCertFile": "", // Required if and only if ClientCertFile is present. - "clientKeyFile": "", + "clientKeyFile": "" }, // SASL @@ -90,6 +92,8 @@ "task": { "name": "test_dynamic_schema", + // kafka client, possible values: sarama, kafka-go. (defaults to sarama) + "kafkaClient": "sarama", // kafka topic "topic": "topic", // kafka consume from earliest or latest @@ -106,16 +110,26 @@ // columns of the table "dims": [ { - "name": "day", - "type": "Date", - "sourceName": "day" + // column name + "name": "timestamp", + // column type + "type": "DateTime" }, - ... + { + "name": "name", + "type": "String" + }, + { + "name": "value", + "type": "Float32", + // json field name. This must be specified if it doesn't match with the column name. + "sourcename": "val" + } ], - // if it's specified, the schema will be auto mapped from clickhouse, + // if it's specified, clickhouse_sinker will detect table schema instead of using the fixed schema given by "dims". "autoSchema" : true, - // "this columns will be excluded by insert SQL " + // these columns will be excluded from the detected table schema. This takes effect only if "autoSchema" is true. "excludeColumns": [] // (experiment feature) detect new fields and their type, and add columns to the ClickHouse table accordingly. This feature requires parser be "fastjson", and support following ClickHouse data types: Int64, Float64, String. @@ -131,7 +145,7 @@ // shardingPolicy is `stripe,`(requires ShardingKey be numerical) or `hash`(requires ShardingKey be string) "shardingPolicy": "", - // interval of flushing the batch. Default to 5, max to 10. + // interval of flushing the batch. Default to 5, max to 600. "flushInterval": 5, // batch size to insert into clickhouse. sinker will round upward it to the the nearest 2^n. Default to 262114, max to 1048576. "bufferSize": 262114, diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 27492411..7751c27e 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -370,8 +370,9 @@ Kafka release history is at [here](https://kafka.apache.org/downloads). Kafka br - Sarama guarantees compatibility [with Kafka 2.6 through 2.8](https://github.com/Shopify/sarama/blob/master/README.md#compatibility-and-api-stability). - Sarama [has tied it's protocol usage to the Version field in Config](https://github.com/Shopify/sarama/issues/1732). +- Sarama consumer API provides generation cleanup callback. This ensures `exactly once` when consumer-group rebalance occur. ### Conclusion -- Neither Kafka-go nor sarama is mature as Java clients. You need to try both if clickhouse_sinker fails to connect with Kafka. +- Sarama is better than kafka-go, though neither is as mature as the officaial Kafka Java client. You need to try both if clickhouse_sinker fails to connect with Kafka. - Our experience is sarama can't work well with new kafka server if set its `Config.Version` to "0.11.0.0". So we suggest `KafkaConfig.Version` in clickhouse_sinker config matchs the Kafka server. diff --git a/input/input.go b/input/input.go index 797df185..1204598e 100644 --- a/input/input.go +++ b/input/input.go @@ -16,7 +16,7 @@ const ( ) type Inputer interface { - Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage)) error + Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage), cleanupFn func()) error Run(ctx context.Context) Stop() error CommitMessages(ctx context.Context, message *model.InputMessage) error diff --git a/input/kafka_go.go b/input/kafka_go.go index f710053e..5772ee91 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -51,7 +51,7 @@ func NewKafkaGo() *KafkaGo { } // Init Initialise the kafka instance with configuration -func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage)) (err error) { +func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage), cleanupFn func()) (err error) { k.cfg = cfg k.taskCfg = taskCfg kfkCfg := &cfg.Kafka diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index fa6c93e4..039e989f 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -38,12 +38,13 @@ var _ Inputer = (*KafkaSarama)(nil) // KafkaSarama implements input.Inputer type KafkaSarama struct { - cfg *config.Config - taskCfg *config.TaskConfig - cg sarama.ConsumerGroup - sess sarama.ConsumerGroupSession - stopped chan struct{} - putFn func(msg model.InputMessage) + cfg *config.Config + taskCfg *config.TaskConfig + cg sarama.ConsumerGroup + sess sarama.ConsumerGroupSession + stopped chan struct{} + putFn func(msg model.InputMessage) + cleanupFn func() } // NewKafkaSarama get instance of kafka reader @@ -59,9 +60,11 @@ func (h MyConsumerGroupHandler) Setup(sess sarama.ConsumerGroupSession) error { h.k.sess = sess return nil } + func (h MyConsumerGroupHandler) Cleanup(_ sarama.ConsumerGroupSession) error { - util.Logger.Info("consumer group cleanup", zap.String("task", h.k.taskCfg.Name), zap.String("consumer group", h.k.taskCfg.ConsumerGroup)) - time.Sleep(5 * time.Second) + begin := time.Now() + h.k.cleanupFn() + util.Logger.Info("consumer group cleanup", zap.String("task", h.k.taskCfg.Name), zap.String("consumer group", h.k.taskCfg.ConsumerGroup), zap.Int32("generation id", h.k.sess.GenerationID()), zap.Duration("cost", time.Since(begin))) return nil } @@ -80,12 +83,13 @@ func (h MyConsumerGroupHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, c } // Init Initialise the kafka instance with configuration -func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage)) (err error) { +func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage), cleanupFn func()) (err error) { k.cfg = cfg k.taskCfg = taskCfg kfkCfg := &cfg.Kafka k.stopped = make(chan struct{}) k.putFn = putFn + k.cleanupFn = cleanupFn config := sarama.NewConfig() if config.Version, err = sarama.ParseKafkaVersion(kfkCfg.Version); err != nil { err = errors.Wrapf(err, "") diff --git a/output/clickhouse.go b/output/clickhouse.go index 52672445..b9712c35 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -55,7 +55,6 @@ type ClickHouse struct { cfg *config.Config taskCfg *config.TaskConfig prepareSQL string - stopped int32 numFlying int32 mux sync.Mutex taskDone *sync.Cond @@ -73,27 +72,17 @@ func (c *ClickHouse) Init() (err error) { return c.initSchema() } -// NotifyStop notify loopWrite to quit -func (c *ClickHouse) NotifyStop() { - atomic.StoreInt32(&c.stopped, 1) -} - -// Stop drains flying batchs -func (c *ClickHouse) Stop() { - atomic.StoreInt32(&c.stopped, 1) +// Drain drains flying batchs +func (c *ClickHouse) Drain() { c.mux.Lock() defer c.mux.Unlock() for atomic.LoadInt32(&c.numFlying) != 0 { c.taskDone.Wait() } - return } // Send a batch to clickhouse func (c *ClickHouse) Send(batch *model.Batch) { - if atomic.LoadInt32(&c.stopped) != 0 { - return - } atomic.AddInt32(&c.numFlying, 1) statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Inc() _ = util.GlobalWritingPool.Submit(func() { @@ -172,10 +161,6 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { var dbVer int sc := pool.GetShardConn(batch.BatchIdx) for { - if atomic.LoadInt32(&c.stopped) != 0 { - util.Logger.Info("ClickHouse.loopWrite quit due to stopped be true", zap.String("task", c.taskCfg.Name)) - return - } if err = c.write(batch, sc, &dbVer); err == nil { if err = batch.Commit(); err == nil { return diff --git a/task/ring.go b/task/ring.go index 0140357e..09b907c7 100644 --- a/task/ring.go +++ b/task/ring.go @@ -3,6 +3,7 @@ package task import ( "fmt" "sync" + "sync/atomic" "time" "github.com/fagongzi/goetty" @@ -156,7 +157,18 @@ func (ring *Ring) genBatchOrShard(expNewGroundOff int64) { if endOff > ring.ringCeilingOff { endOff = ring.ringCeilingOff } - if ring.service.sharder != nil { + if atomic.LoadUint32(&ring.service.state) != util.StateRunning { + for i := ring.ringGroundOff; i < endOff; i++ { + msgRow := &ring.ringBuf[i&(ring.ringCap-1)] + if msgRow.Msg != nil { + msgCnt++ + } + } + util.Logger.Info(fmt.Sprintf("discarded a batch for topic %v patittion %d, offset [%d,%d), messages %d", taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, msgCnt), + zap.String("task", taskCfg.Name)) + statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) + return + } else if ring.service.sharder != nil { msgCnt = ring.service.sharder.PutElems(ring.partition, ring.ringBuf, ring.ringGroundOff, endOff, ring.ringCap) statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } else { @@ -189,13 +201,13 @@ func (ring *Ring) genBatchOrShard(expNewGroundOff int64) { } if batch.RealSize > 0 { - util.Logger.Debug(fmt.Sprintf("going to flush a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", taskCfg.Topic, ring.partition, endOff-1, + util.Logger.Debug(fmt.Sprintf("going to flush a batch for topic %v patittion %d, offset [%d,%d), messages %d, gaps: %+v, parse errors: %d", taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, batch.RealSize, gaps, parseErrs), zap.String("task", taskCfg.Name)) batch.BatchIdx = (endOff - 1) >> ring.batchSizeShift ring.batchSys.CreateBatchGroupSingle(batch, ring.partition, endOff-1) - ring.service.batchChan <- batch + ring.service.Flush(batch) if gaps == nil { statistics.RingNormalBatchsTotal.WithLabelValues(taskCfg.Name).Inc() } else { diff --git a/task/sharding.go b/task/sharding.go index 78d73e32..90fda23d 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -184,8 +184,8 @@ func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOf maxBatchSize = batchSize } } - util.Logger.Debug(fmt.Sprintf("sharded a batch for topic %v patittion %d, offset %d, messages %d, gaps: %+v, parse errors: %d", - taskCfg.Topic, partition, endOff-1, msgCnt, gaps, parseErrs), + util.Logger.Debug(fmt.Sprintf("sharded a batch for topic %v patittion %d, offset [%d, %d), messages %d, gaps: %+v, parse errors: %d", + taskCfg.Topic, partition, begOff, endOff, msgCnt, gaps, parseErrs), zap.String("task", taskCfg.Name)) if maxBatchSize >= taskCfg.BufferSize { sh.doFlush(nil) @@ -224,7 +224,7 @@ func (sh *Sharder) doFlush(_ interface{}) { sh.offsets = sh.offsets[:0] // ALL batches in a group shall be populated before sending any one to next stage. for _, batch := range batches { - sh.service.batchChan <- batch + sh.service.Flush(batch) } statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } diff --git a/task/task.go b/task/task.go index 850ae1ec..691844bc 100644 --- a/task/task.go +++ b/task/task.go @@ -40,7 +40,6 @@ import ( type Service struct { sync.Mutex - parentCtx context.Context ctx context.Context cancel context.CancelFunc started bool @@ -57,12 +56,15 @@ type Service struct { cntNewKeys int32 // size of newKeys tid goetty.Timeout - rings []*Ring - sharder *Sharder - batchChan chan *model.Batch - limiter1 *rate.Limiter - limiter2 *rate.Limiter - limiter3 *rate.Limiter + rings []*Ring + sharder *Sharder + limiter1 *rate.Limiter + limiter2 *rate.Limiter + limiter3 *rate.Limiter + + state uint32 + numFlying int32 + taskDone *sync.Cond } // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances @@ -85,12 +87,13 @@ func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) *Service { func (service *Service) Init() (err error) { taskCfg := service.taskCfg util.Logger.Info("task initializing", zap.String("task", taskCfg.Name)) + service.numFlying = 0 + service.taskDone = sync.NewCond(service) if err = service.clickhouse.Init(); err != nil { return } service.dims = service.clickhouse.Dims - service.batchChan = make(chan *model.Batch, 32) service.limiter1 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter2 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter3 = rate.NewLimiter(rate.Every(10*time.Second), 1) @@ -102,7 +105,7 @@ func (service *Service) Init() (err error) { } } - if err = service.inputer.Init(service.cfg, taskCfg, service.put); err != nil { + if err = service.inputer.Init(service.cfg, taskCfg, service.put, service.drain); err != nil { return } @@ -133,10 +136,8 @@ func (service *Service) Run(ctx context.Context) { var err error taskCfg := service.taskCfg service.started = true - service.parentCtx = ctx - service.ctx, service.cancel = context.WithCancel(ctx) + service.ctx = ctx util.Logger.Info("task started", zap.String("task", taskCfg.Name)) - go service.inputer.Run(service.ctx) if service.sharder != nil { // schedule a delayed ForceFlush if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { @@ -148,19 +149,7 @@ func (service *Service) Run(ctx context.Context) { } } } - -LOOP: - for { - select { - case <-service.ctx.Done(): - util.Logger.Info("Service.Run quit due to context has been canceled", zap.String("task", service.taskCfg.Name)) - break LOOP - case batch := <-service.batchChan: - if err := service.flush(batch); err != nil { - util.Logger.Fatal("service.flush failed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - } - } + service.inputer.Run(service.ctx) service.stopped <- struct{}{} } @@ -237,12 +226,21 @@ func (service *Service) put(msg model.InputMessage) { } // submit message to a goroutine pool + atomic.AddInt32(&service.numFlying, 1) statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Inc() _ = util.GlobalParsingPool.Submit(func() { var row *model.Row var foundNewKeys bool var metric model.Metric - defer statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Dec() + defer func() { + numFlying := atomic.AddInt32(&service.numFlying, -1) + if numFlying == 0 { + service.Lock() + service.taskDone.Signal() + service.Unlock() + } + statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Dec() + }() p := service.pp.Get() metric, err = p.Parse(msg.Value) // WARNNING: Always PutElem even if there's parsing error, so that this message can be acked to Kafka and skipped writing to ClickHouse. @@ -297,7 +295,30 @@ func (service *Service) put(msg model.InputMessage) { }) } -func (service *Service) flush(batch *model.Batch) (err error) { +// drain ensure we have completeted procession(discard or write&commit) for all received messages, and cleared service state. +// assumes service.put() not be invoked during drain(). +func (service *Service) drain() { + atomic.StoreUint32(&service.state, util.StateStopped) + defer atomic.StoreUint32(&service.state, util.StateRunning) + service.Lock() + for service.numFlying != 0 { + service.taskDone.Wait() + } + for _, ring := range service.rings { + if ring != nil { + ring.ForceBatchOrShard(nil) + ring.tid.Stop() + } + } + service.rings = make([]*Ring, 0) + service.Unlock() + if service.sharder != nil { + service.sharder.ForceFlush(nil) + } + service.clickhouse.Drain() +} + +func (service *Service) Flush(batch *model.Batch) (err error) { if (len(*batch.Rows)) == 0 { return batch.Commit() } @@ -317,15 +338,7 @@ func (service *Service) changeSchema(arg interface{}) { if err = service.Init(); err != nil { util.Logger.Fatal("service.Init failed", zap.String("task", taskCfg.Name), zap.Error(err)) } - go service.Run(service.parentCtx) -} - -// NotifyStop notify task to stop, This is non-blocking. -func (service *Service) NotifyStop() { - taskCfg := service.taskCfg - util.Logger.Info("notified stopping task service...", zap.String("task", taskCfg.Name)) - service.cancel() - service.clickhouse.NotifyStop() + go service.Run(service.ctx) } // Stop stop kafka and clickhouse client. This is blocking. @@ -336,9 +349,6 @@ func (service *Service) Stop() { return } util.Logger.Info("stopping task service...", zap.String("task", taskCfg.Name)) - service.cancel() - service.clickhouse.Stop() - util.Logger.Info("stopped output", zap.String("task", taskCfg.Name)) if err := service.inputer.Stop(); err != nil { util.Logger.Fatal("service.inputer.Stop failed", zap.Error(err)) } @@ -347,14 +357,12 @@ func (service *Service) Stop() { if service.sharder != nil { service.sharder.tid.Stop() } - for _, ring := range service.rings { - if ring != nil { - ring.tid.Stop() - } - } service.tid.Stop() util.Logger.Info("stopped internal timers", zap.String("task", taskCfg.Name)) + service.drain() + util.Logger.Info("drained flying messages", zap.String("task", taskCfg.Name)) + if service.started { <-service.stopped } From bdde6f832b87bae5b873ea815d15c558a8744603 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 5 Sep 2021 22:59:09 +0800 Subject: [PATCH 151/404] let lint happly --- cmd/clickhouse_sinker/main.go | 6 ++++-- cmd/nacos_publish_config/main.go | 4 +++- input/kafka_sarama.go | 6 +++++- statistics/statistics.go | 3 ++- task/ring.go | 8 +++++--- task/task.go | 1 - 6 files changed, 19 insertions(+), 9 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index b2cf725a..3060e60e 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -337,7 +337,8 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { // 1. Initialize clickhouse connections chCfg := &newCfg.Clickhouse - if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { + if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, + chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { return } @@ -370,7 +371,8 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { s.stopAllTasks() // 2. Initialize clickhouse connections. chCfg := &newCfg.Clickhouse - if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { + if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, + chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { return } diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 103f7c45..dac113f6 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -81,7 +81,9 @@ func PublishSinkerConfig() { for i := 1; i < *replicas; i++ { for j := 0; j < len(tasks); j++ { taskCfg := &config.TaskConfig{} - copier.Copy(taskCfg, tasks[j]) + if err = copier.Copy(taskCfg, tasks[j]); err != nil { + util.Logger.Fatal("copier.Copy failed", zap.Error(err)) + } taskCfg.Name = fmt.Sprintf("%s_r%d", taskCfg.Name, i) taskCfg.ConsumerGroup = fmt.Sprintf("%s_r%d", taskCfg.ConsumerGroup, i) taskCfg.TableName = fmt.Sprintf("%s_r%d", taskCfg.TableName, i) diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 039e989f..ae41118b 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -64,7 +64,11 @@ func (h MyConsumerGroupHandler) Setup(sess sarama.ConsumerGroupSession) error { func (h MyConsumerGroupHandler) Cleanup(_ sarama.ConsumerGroupSession) error { begin := time.Now() h.k.cleanupFn() - util.Logger.Info("consumer group cleanup", zap.String("task", h.k.taskCfg.Name), zap.String("consumer group", h.k.taskCfg.ConsumerGroup), zap.Int32("generation id", h.k.sess.GenerationID()), zap.Duration("cost", time.Since(begin))) + util.Logger.Info("consumer group cleanup", + zap.String("task", h.k.taskCfg.Name), + zap.String("consumer group", h.k.taskCfg.ConsumerGroup), + zap.Int32("generation id", h.k.sess.GenerationID()), + zap.Duration("cost", time.Since(begin))) return nil } diff --git a/statistics/statistics.go b/statistics/statistics.go index 40bd370f..284520eb 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -22,6 +22,7 @@ import ( "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/collectors" "github.com/prometheus/client_golang/prometheus/push" "github.com/prometheus/common/expfmt" "go.uber.org/zap" @@ -170,7 +171,7 @@ func init() { prometheus.MustRegister(ShardMsgs) prometheus.MustRegister(ParsingPoolBacklog) prometheus.MustRegister(WritingPoolBacklog) - prometheus.MustRegister(prometheus.NewBuildInfoCollector()) + prometheus.MustRegister(collectors.NewBuildInfoCollector()) } // Pusher is the service to push the metrics to pushgateway diff --git a/task/ring.go b/task/ring.go index 09b907c7..b33263d2 100644 --- a/task/ring.go +++ b/task/ring.go @@ -96,7 +96,9 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { if arg != nil { newMsg, _ = arg.(*model.InputMessage) if newMsg.Offset > ring.ringGroundOff { - util.Logger.Warn(fmt.Sprintf("Ring.ForceBatchOrShard partition %d message range [%d, %d)", newMsg.Partition, ring.ringGroundOff, newMsg.Offset), zap.String("task", taskCfg.Name)) + util.Logger.Warn(fmt.Sprintf("Ring.ForceBatchOrShard partition %d message range [%d, %d)", + newMsg.Partition, ring.ringGroundOff, newMsg.Offset), + zap.String("task", taskCfg.Name)) } } if !ring.isIdle { @@ -201,8 +203,8 @@ func (ring *Ring) genBatchOrShard(expNewGroundOff int64) { } if batch.RealSize > 0 { - util.Logger.Debug(fmt.Sprintf("going to flush a batch for topic %v patittion %d, offset [%d,%d), messages %d, gaps: %+v, parse errors: %d", taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, - batch.RealSize, gaps, parseErrs), + util.Logger.Debug(fmt.Sprintf("going to flush a batch for topic %v patittion %d, offset [%d,%d), messages %d, gaps: %+v, parse errors: %d", + taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, batch.RealSize, gaps, parseErrs), zap.String("task", taskCfg.Name)) batch.BatchIdx = (endOff - 1) >> ring.batchSizeShift diff --git a/task/task.go b/task/task.go index 691844bc..03d19805 100644 --- a/task/task.go +++ b/task/task.go @@ -41,7 +41,6 @@ type Service struct { sync.Mutex ctx context.Context - cancel context.CancelFunc started bool stopped chan struct{} inputer input.Inputer From b3632d81b7ad68ebd9c9a708fd21329c035bdbd0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 6 Sep 2021 10:49:00 +0800 Subject: [PATCH 152/404] better handling of out of range messages --- docker-compose.yml | 4 +- model/message.go | 20 ++-- statistics/statistics.go | 9 -- task/ring.go | 202 +++++++++++++++++++-------------------- task/sharding.go | 55 ++++------- task/task.go | 68 ++++++++----- 6 files changed, 172 insertions(+), 186 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index c2276091..13cd066f 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -3,7 +3,7 @@ version: '3' services: zookeeper: - image: quay.io/bitnami/zookeeper:3.7.0 + image: zookeeper:3.7.0 restart: always hostname: zookeeper ports: @@ -13,7 +13,7 @@ services: security_opt: - label:disable kafka: - image: confluentinc/cp-kafka:5.0.1-1 + image: confluentinc/cp-kafka:6.1.1 container_name: kafka restart: always hostname: kafka diff --git a/model/message.go b/model/message.go index eda52ce6..c829b763 100644 --- a/model/message.go +++ b/model/message.go @@ -14,6 +14,7 @@ import ( var ( rowsPool sync.Pool + FakedRow Row = make([]interface{}, 0) ) // MsgWithMeta abstract messages @@ -48,7 +49,7 @@ type Batch struct { //So those batches need to be committed after ALL of them have been written to clickhouse. type BatchGroup struct { Batchs []*Batch - Offsets []int64 + Offsets map[int]int64 Sys *BatchSys PendWrite int32 //how many batches in this group are pending to wirte to ClickHouse } @@ -76,12 +77,10 @@ LOOP: } // commit the whole group for j, off := range grp.Offsets { - if off >= 0 { - if err := bs.fnCommit(j, off); err != nil { - return err - } - statistics.ConsumeOffsets.WithLabelValues(bs.taskCfg.Name, bs.taskCfg.Topic, strconv.Itoa(j)).Set(float64(off)) + if err := bs.fnCommit(j, off); err != nil { + return err } + statistics.ConsumeOffsets.WithLabelValues(bs.taskCfg.Name, bs.taskCfg.Topic, strconv.Itoa(j)).Set(float64(off)) } eNext := e.Next() bs.groups.Remove(e) @@ -94,23 +93,20 @@ func (bs *BatchSys) CreateBatchGroupSingle(batch *Batch, partition int, offset i bg := &BatchGroup{ Sys: bs, Batchs: []*Batch{batch}, - Offsets: make([]int64, partition+1), + Offsets: make(map[int]int64), PendWrite: 1, } bg.Batchs[0].Group = bg - for i := 0; i < partition; i++ { - bg.Offsets[i] = -1 - } bg.Offsets[partition] = offset bs.mux.Lock() bs.groups.PushBack(bg) bs.mux.Unlock() } -func (bs *BatchSys) CreateBatchGroupMulti(batches []*Batch, offsets []int64) { +func (bs *BatchSys) CreateBatchGroupMulti(batches []*Batch, offsets map[int]int64) { bg := &BatchGroup{Sys: bs, PendWrite: int32(len(batches))} bg.Batchs = append(bg.Batchs, batches...) - bg.Offsets = append(bg.Offsets, offsets...) + bg.Offsets = offsets for _, batch := range bg.Batchs { batch.Group = bg } diff --git a/statistics/statistics.go b/statistics/statistics.go index 284520eb..faa37c2a 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -88,13 +88,6 @@ var ( }, []string{"task"}, ) - RingForceBatchAllGapTotal = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Name: prefix + "ring_force_batch_all_gap_total", - Help: "total num of force batch_all generated with some offset gap", - }, - []string{"task"}, - ) FlushMsgsTotal = prometheus.NewCounterVec( prometheus.CounterOpts{ Name: prefix + "flush_msgs_total", @@ -162,7 +155,6 @@ func init() { prometheus.MustRegister(RingNormalBatchsTotal) prometheus.MustRegister(RingForceBatchsTotal) prometheus.MustRegister(RingForceBatchAllTotal) - prometheus.MustRegister(RingForceBatchAllGapTotal) prometheus.MustRegister(FlushMsgsTotal) prometheus.MustRegister(FlushMsgsErrorTotal) prometheus.MustRegister(ConsumeOffsets) @@ -253,7 +245,6 @@ func (p *Pusher) reconnect() { Collector(RingNormalBatchsTotal). Collector(RingForceBatchsTotal). Collector(RingForceBatchAllTotal). - Collector(RingForceBatchAllGapTotal). Collector(FlushMsgsTotal). Collector(FlushMsgsErrorTotal). Collector(ConsumeOffsets). diff --git a/task/ring.go b/task/ring.go index b33263d2..7b0763ec 100644 --- a/task/ring.go +++ b/task/ring.go @@ -17,8 +17,10 @@ import ( type Ring struct { mux sync.Mutex //protect ring* + available *sync.Cond ringBuf []model.MsgRow ringCap int64 //message is allowed to insert into the ring if its offset in inside [ringGroundOff, ringGroundOff+ringCap) + ringCapMask int64 ringGroundOff int64 //min message offset inside the ring ringCeilingOff int64 //1 + max message offset inside the ring ringFilledOffset int64 //every message which's offset inside range [ringGroundOff, ringFilledOffset) is in the ring @@ -32,38 +34,48 @@ type Ring struct { service *Service } +// assumes ring.mux is locked +func (ring *Ring) QuitIdle() { + if ring.isIdle { + ring.idleCnt = 0 + ring.isIdle = false + ring.ringBuf = make([]model.MsgRow, ring.ringCap) + } +} + +// assumes ring.mux is locked, and msg.Offset is in range [ring.ringGroundOff, ring.ringGroundOff+ring.ringCap) +func (ring *Ring) PutMsgNolock(msg *model.InputMessage) { + ring.QuitIdle() + ring.ringBuf[msg.Offset&ring.ringCapMask].Msg = msg + statistics.RingMsgs.WithLabelValues(ring.service.taskCfg.Name).Inc() +} + func (ring *Ring) PutElem(msgRow model.MsgRow) { var err error taskCfg := ring.service.taskCfg msgOffset := msgRow.Msg.Offset + pMsgRow := &ring.ringBuf[msgOffset&ring.ringCapMask] ring.mux.Lock() defer ring.mux.Unlock() - if msgOffset < ring.ringFilledOffset { + if msgOffset < ring.ringFilledOffset || pMsgRow.Msg != msgRow.Msg { return } - // ring.mux is locked at this point - if ring.isIdle { - ring.idleCnt = 0 - ring.isIdle = false - ring.ringBuf = make([]model.MsgRow, ring.ringCap) - util.Logger.Info(fmt.Sprintf("topic %s partition %d became busy", taskCfg.Topic, ring.partition), zap.String("task", taskCfg.Name)) - } // assert(msgOffset < ring.ringGroundOff + ring.ringCap) if msgOffset >= ring.ringCeilingOff { ring.ringCeilingOff = msgOffset + 1 } - if ring.service.sharder != nil && msgRow.Row != nil { + pMsgRow.Row = msgRow.Row + if ring.service.sharder != nil && msgRow.Row != &model.FakedRow { if msgRow.Shard, err = ring.service.sharder.Calc(msgRow.Row); err != nil { util.Logger.Fatal("shard number calculation failed", zap.String("task", taskCfg.Name), zap.Error(err)) } + pMsgRow.Shard = msgRow.Shard } - statistics.RingMsgs.WithLabelValues(taskCfg.Name).Inc() - ring.ringBuf[msgOffset&(ring.ringCap-1)] = msgRow - for ; ring.ringFilledOffset < ring.ringCeilingOff && ring.ringBuf[ring.ringFilledOffset&(ring.ringCap-1)].Msg != nil; ring.ringFilledOffset++ { + for ; ring.ringFilledOffset < ring.ringCeilingOff && ring.ringBuf[ring.ringFilledOffset&(ring.ringCapMask)].Row != nil; ring.ringFilledOffset++ { } if (ring.ringFilledOffset >> ring.batchSizeShift) != (ring.ringGroundOff >> ring.batchSizeShift) { - ring.genBatchOrShard(ring.ringFilledOffset) + ring.genBatchOrShard() // reschedule the delayed ForceBatchOrShard ring.tid.Stop() if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { @@ -76,58 +88,69 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { } } -type OffsetRange struct { - Begin int64 //inclusive - End int64 //exclusive -} - -func (ring *Ring) ForceBatchOrShard(arg interface{}) { - var newMsg *model.InputMessage +func (ring *Ring) MakeRoom(newMsg *model.InputMessage) { + // assert(!ring.isIdle) taskCfg := ring.service.taskCfg - select { - case <-ring.service.ctx.Done(): - util.Logger.Warn("Ring.ForceBatchOrShard quit due to the context has been canceled", zap.String("task", taskCfg.Name)) - return - default: - } - ring.mux.Lock() defer ring.mux.Unlock() - if arg != nil { - newMsg, _ = arg.(*model.InputMessage) - if newMsg.Offset > ring.ringGroundOff { - util.Logger.Warn(fmt.Sprintf("Ring.ForceBatchOrShard partition %d message range [%d, %d)", - newMsg.Partition, ring.ringGroundOff, newMsg.Offset), - zap.String("task", taskCfg.Name)) + statistics.RingForceBatchAllTotal.WithLabelValues(taskCfg.Name).Inc() + ring.idleCnt = 0 + prevMsgOff := newMsg.Offset - 1 + if newMsg.Offset != ring.ringGroundOff+ring.ringCap || + ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg == nil { + var msgCnt int + for off := ring.ringGroundOff; off < ring.ringGroundOff+ring.ringCap; off++ { + msgRow := &ring.ringBuf[off&ring.ringCapMask] + if msgRow.Msg != nil { + msgCnt++ + } + msgRow.Msg = nil + msgRow.Row = nil } + util.Logger.Info(fmt.Sprintf("Ring.MakeRoom discarded %d messages for topic %v patittion %d, offset [%d,%d)", + msgCnt, taskCfg.Topic, ring.partition, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap), + zap.String("task", taskCfg.Name)) + ring.ringGroundOff = newMsg.Offset + ring.ringFilledOffset = newMsg.Offset + ring.ringCeilingOff = newMsg.Offset + } else { + for ; prevMsgOff > ring.ringGroundOff && ring.ringBuf[(prevMsgOff-1)&ring.ringCapMask].Msg != nil; prevMsgOff-- { + } + // assert(ring.ringFilledOffset < prevMsgOff) + var msgCnt int + for off := ring.ringGroundOff; off < prevMsgOff; off++ { + msgRow := &ring.ringBuf[off&ring.ringCapMask] + if msgRow.Msg != nil { + msgCnt++ + } + msgRow.Msg = nil + msgRow.Row = nil + } + util.Logger.Info(fmt.Sprintf("Ring.MakeRoom discarded %d messages for topic %v patittion %d, offset [%d,%d)", + msgCnt, taskCfg.Topic, ring.partition, ring.ringGroundOff, prevMsgOff), + zap.String("task", taskCfg.Name)) + ring.ringGroundOff = prevMsgOff + ring.ringFilledOffset = newMsg.Offset + ring.ringCeilingOff = newMsg.Offset } +} + +func (ring *Ring) ForceBatchOrShard(_ interface{}) { + taskCfg := ring.service.taskCfg + ring.mux.Lock() + defer ring.mux.Unlock() if !ring.isIdle { - if newMsg == nil { - if ring.ringFilledOffset > ring.ringGroundOff { - ring.genBatchOrShard(ring.ringFilledOffset) + if ring.ringFilledOffset > ring.ringGroundOff { + ring.genBatchOrShard() + ring.idleCnt = 0 + } else if ring.ringGroundOff == ring.ringCeilingOff { + ring.idleCnt++ + if ring.idleCnt >= 2 { ring.idleCnt = 0 - } else if ring.ringGroundOff == ring.ringCeilingOff { - ring.idleCnt++ - if ring.idleCnt >= 2 { - ring.idleCnt = 0 - ring.isIdle = true - ring.ringBuf = nil - util.Logger.Info(fmt.Sprintf("topic %s partition %d became idle", taskCfg.Topic, ring.partition), zap.String("task", taskCfg.Name)) - } + ring.isIdle = true + ring.ringBuf = nil + util.Logger.Info(fmt.Sprintf("topic %s partition %d became idle", taskCfg.Topic, ring.partition), zap.String("task", taskCfg.Name)) } - } else { - statistics.RingForceBatchAllTotal.WithLabelValues(taskCfg.Name).Inc() - LOOP: - for { - ring.genBatchOrShard(ring.ringCeilingOff) - if ring.ringGroundOff == ring.ringCeilingOff { - break LOOP - } - } - ring.ringGroundOff = newMsg.Offset - ring.ringFilledOffset = newMsg.Offset - ring.ringCeilingOff = newMsg.Offset - ring.idleCnt = 0 } } @@ -144,83 +167,54 @@ func (ring *Ring) ForceBatchOrShard(arg interface{}) { } } +// generate a batch for messages [ring.ringGroundOff, ring.ringFilledOffset) // assume ring.mux is locked -func (ring *Ring) genBatchOrShard(expNewGroundOff int64) { - if expNewGroundOff <= ring.ringGroundOff { - return - } +func (ring *Ring) genBatchOrShard() { taskCfg := ring.service.taskCfg - var gaps []OffsetRange - var msgCnt, parseErrs int - endOff := (ring.ringGroundOff | int64(1< expNewGroundOff { - endOff = expNewGroundOff - } - if endOff > ring.ringCeilingOff { - endOff = ring.ringCeilingOff - } + var parseErrs int + endOff := ring.ringFilledOffset + msgCnt := endOff - ring.ringGroundOff if atomic.LoadUint32(&ring.service.state) != util.StateRunning { - for i := ring.ringGroundOff; i < endOff; i++ { - msgRow := &ring.ringBuf[i&(ring.ringCap-1)] - if msgRow.Msg != nil { - msgCnt++ - } - } - util.Logger.Info(fmt.Sprintf("discarded a batch for topic %v patittion %d, offset [%d,%d), messages %d", taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, msgCnt), + util.Logger.Info(fmt.Sprintf("Ring.genBatchOrShard discarded a batch for topic %v patittion %d, offset [%d,%d), messages %d", + taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, msgCnt), zap.String("task", taskCfg.Name)) statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) return } else if ring.service.sharder != nil { - msgCnt = ring.service.sharder.PutElems(ring.partition, ring.ringBuf, ring.ringGroundOff, endOff, ring.ringCap) + ring.service.sharder.PutElems(ring.partition, ring.ringBuf, ring.ringGroundOff, endOff, ring.ringCapMask) statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } else { - gapBegOff := int64(-1) batch := model.NewBatch() for i := ring.ringGroundOff; i < endOff; i++ { - msgRow := &ring.ringBuf[i&(ring.ringCap-1)] - if msgRow.Msg != nil { - msgCnt++ - //assert msg.Offset==i - if gapBegOff >= 0 { - gaps = append(gaps, OffsetRange{Begin: gapBegOff, End: i}) - gapBegOff = -1 - } - if msgRow.Row != nil { - *batch.Rows = append(*batch.Rows, msgRow.Row) - } else { - parseErrs++ - } - } else if gapBegOff < 0 { - gapBegOff = i + msgRow := &ring.ringBuf[i&(ring.ringCapMask)] + if msgRow.Row != &model.FakedRow { + *batch.Rows = append(*batch.Rows, msgRow.Row) + } else { + parseErrs++ } msgRow.Msg = nil msgRow.Row = nil msgRow.Shard = -1 } batch.RealSize = len(*batch.Rows) - if gapBegOff >= 0 { - gaps = append(gaps, OffsetRange{Begin: gapBegOff, End: endOff}) - } if batch.RealSize > 0 { - util.Logger.Debug(fmt.Sprintf("going to flush a batch for topic %v patittion %d, offset [%d,%d), messages %d, gaps: %+v, parse errors: %d", - taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, batch.RealSize, gaps, parseErrs), + util.Logger.Debug(fmt.Sprintf("going to flush a batch for topic %v patittion %d, offset [%d,%d), messages %d, parse errors: %d", + taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, batch.RealSize, parseErrs), zap.String("task", taskCfg.Name)) batch.BatchIdx = (endOff - 1) >> ring.batchSizeShift ring.batchSys.CreateBatchGroupSingle(batch, ring.partition, endOff-1) ring.service.Flush(batch) - if gaps == nil { - statistics.RingNormalBatchsTotal.WithLabelValues(taskCfg.Name).Inc() - } else { - statistics.RingForceBatchAllGapTotal.WithLabelValues(taskCfg.Name).Inc() - } + statistics.RingNormalBatchsTotal.WithLabelValues(taskCfg.Name).Inc() } statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } ring.ringGroundOff = endOff + //util.Logger.Debug(fmt.Sprintf("genBatchOrShard changed ring %p ringGroundOff to %d", ring, ring.ringGroundOff)) if ring.ringFilledOffset < ring.ringGroundOff { ring.ringFilledOffset = ring.ringGroundOff } + ring.available.Signal() } diff --git a/task/sharding.go b/task/sharding.go index 90fda23d..a0d1735b 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -107,7 +107,7 @@ type Sharder struct { ckNum int mux sync.Mutex msgBuf []*model.Rows - offsets []int64 + offsets map[int]int64 tid goetty.Timeout } @@ -124,7 +124,7 @@ func NewSharder(service *Service) (sh *Sharder, err error) { batchSys: model.NewBatchSys(taskCfg, service.fnCommit), ckNum: ckNum, msgBuf: make([]*model.Rows, ckNum), - offsets: make([]int64, 0), + offsets: make(map[int]int64), } for i := 0; i < ckNum; i++ { sh.msgBuf[i] = model.GetRows() @@ -136,47 +136,31 @@ func (sh *Sharder) Calc(row *model.Row) (int, error) { return sh.policy.Calc(row) } -func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOff, ringCap int64) (msgCnt int) { +func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOff, ringCapMask int64) { + if begOff <= endOff { + return + } + msgCnt := endOff - begOff sh.mux.Lock() defer sh.mux.Unlock() - var gaps []OffsetRange var parseErrs int taskCfg := sh.service.taskCfg - gapBegOff := int64(-1) for i := begOff; i < endOff; i++ { - msgRow := &ringBuf[i&(ringCap-1)] - if msgRow.Msg != nil { - msgCnt++ - //assert msg.Offset==i - if msgRow.Row != nil { - rows := sh.msgBuf[msgRow.Shard] - *rows = append(*rows, msgRow.Row) - } else { - parseErrs++ - } - if gapBegOff >= 0 { - gaps = append(gaps, OffsetRange{Begin: gapBegOff, End: i}) - gapBegOff = -1 - } - } else if gapBegOff < 0 { - gapBegOff = i + msgRow := &ringBuf[i&ringCapMask] + //assert msg.Offset==i + if msgRow.Row != &model.FakedRow { + rows := sh.msgBuf[msgRow.Shard] + *rows = append(*rows, msgRow.Row) + } else { + parseErrs++ } msgRow.Msg = nil msgRow.Row = nil msgRow.Shard = -1 } - if gapBegOff >= 0 { - gaps = append(gaps, OffsetRange{Begin: gapBegOff, End: endOff}) - } - gap := partition + 1 - len(sh.offsets) - for i := 0; i < gap; i++ { - sh.offsets = append(sh.offsets, -1) - } - if msgCnt > 0 { - sh.offsets[partition] = endOff - 1 - statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Add(float64(msgCnt)) - } + sh.offsets[partition] = endOff - 1 + statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Add(float64(msgCnt)) var maxBatchSize int for i := 0; i < sh.ckNum; i++ { batchSize := len(*sh.msgBuf[i]) @@ -184,13 +168,12 @@ func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOf maxBatchSize = batchSize } } - util.Logger.Debug(fmt.Sprintf("sharded a batch for topic %v patittion %d, offset [%d, %d), messages %d, gaps: %+v, parse errors: %d", - taskCfg.Topic, partition, begOff, endOff, msgCnt, gaps, parseErrs), + util.Logger.Debug(fmt.Sprintf("sharded a batch for topic %v patittion %d, offset [%d, %d), messages %d, parse errors: %d", + taskCfg.Topic, partition, begOff, endOff, msgCnt, parseErrs), zap.String("task", taskCfg.Name)) if maxBatchSize >= taskCfg.BufferSize { sh.doFlush(nil) } - return } func (sh *Sharder) ForceFlush(arg interface{}) { @@ -221,7 +204,7 @@ func (sh *Sharder) doFlush(_ interface{}) { if msgCnt > 0 { util.Logger.Debug(fmt.Sprintf("going to flush batch group for topic %v, offsets %+v, messages %d", taskCfg.Topic, sh.offsets, msgCnt), zap.String("task", taskCfg.Name)) sh.batchSys.CreateBatchGroupMulti(batches, sh.offsets) - sh.offsets = sh.offsets[:0] + sh.offsets = make(map[int]int64) // ALL batches in a group shall be populated before sending any one to next stage. for _, batch := range batches { sh.service.Flush(batch) diff --git a/task/task.go b/task/task.go index 03d19805..136e1719 100644 --- a/task/task.go +++ b/task/task.go @@ -59,7 +59,6 @@ type Service struct { sharder *Sharder limiter1 *rate.Limiter limiter2 *rate.Limiter - limiter3 *rate.Limiter state uint32 numFlying int32 @@ -67,11 +66,11 @@ type Service struct { } // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances -func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) *Service { +func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) (service *Service) { ck := output.NewClickHouse(cfg, taskCfg) pp, _ := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone) inputer := input.NewInputer(taskCfg.KafkaClient) - return &Service{ + service = &Service{ stopped: make(chan struct{}), inputer: inputer, clickhouse: ck, @@ -80,6 +79,8 @@ func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) *Service { cfg: cfg, taskCfg: taskCfg, } + service.taskDone = sync.NewCond(service) + return } // Init initializes the kafak and clickhouse task associated with this service @@ -87,7 +88,7 @@ func (service *Service) Init() (err error) { taskCfg := service.taskCfg util.Logger.Info("task initializing", zap.String("task", taskCfg.Name)) service.numFlying = 0 - service.taskDone = sync.NewCond(service) + atomic.StoreUint32(&service.state, util.StateRunning) if err = service.clickhouse.Init(); err != nil { return } @@ -95,7 +96,6 @@ func (service *Service) Init() (err error) { service.dims = service.clickhouse.Dims service.limiter1 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter2 = rate.NewLimiter(rate.Every(10*time.Second), 1) - service.limiter3 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.rings = make([]*Ring, 0) if taskCfg.ShardingKey != "" { @@ -158,6 +158,9 @@ func (service *Service) fnCommit(partition int, offset int64) error { } func (service *Service) put(msg model.InputMessage) { + if atomic.LoadUint32(&service.state) != util.StateRunning { + return + } taskCfg := service.taskCfg statistics.ConsumeMsgsTotal.WithLabelValues(taskCfg.Name).Inc() // ensure ring for this message exist @@ -178,6 +181,7 @@ func (service *Service) put(msg model.InputMessage) { ring := &Ring{ ringBuf: make([]model.MsgRow, ringCap), ringCap: ringCap, + ringCapMask: ringCap - 1, ringGroundOff: msg.Offset, ringCeilingOff: msg.Offset, ringFilledOffset: msg.Offset, @@ -188,6 +192,8 @@ func (service *Service) put(msg model.InputMessage) { batchSys: model.NewBatchSys(taskCfg, service.fnCommit), service: service, } + ring.available = sync.NewCond(&ring.mux) + ring.PutMsgNolock(&msg) // schedule a delayed ForceBatchOrShard if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { if errors.Is(err, goetty.ErrSystemStopped) { @@ -201,26 +207,37 @@ func (service *Service) put(msg model.InputMessage) { service.Unlock() } else { service.Unlock() - var ringGroundOff, ringFilledOffset int64 ring.mux.Lock() - ringGroundOff, ringFilledOffset = ring.ringGroundOff, ring.ringFilledOffset - ring.mux.Unlock() - if msg.Offset < ringFilledOffset { + ring.QuitIdle() + if msg.Offset < ring.ringFilledOffset { statistics.RingMsgsOffTooSmallErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter2.Allow() { util.Logger.Warn(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) left to %v", - msg.Topic, msg.Partition, msg.Offset, ringFilledOffset), zap.String("task", taskCfg.Name)) + msg.Topic, msg.Partition, msg.Offset, ring.ringFilledOffset), zap.String("task", taskCfg.Name)) } + ring.mux.Unlock() return - } - if msg.Offset >= ringGroundOff+ring.ringCap && atomic.LoadInt32(&service.cntNewKeys) == 0 { - statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(taskCfg.Name).Inc() - if service.limiter3.Allow() { - util.Logger.Warn(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) right to the range [%v, %v)", + } else if msg.Offset < ring.ringGroundOff+ring.ringCap { + ring.PutMsgNolock(&msg) + ring.mux.Unlock() + } else { + prevMsgOff := msg.Offset - 1 + for msg.Offset == ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil { + // wait ring.PutElem to make room + ring.available.Wait() + } + if msg.Offset == ring.ringGroundOff || (msg.Offset < ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil) { + ring.PutMsgNolock(&msg) + ring.mux.Unlock() + } else { + // discard messages to make room + ring.mux.Unlock() + statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(taskCfg.Name).Inc() + util.Logger.Warn(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) which's previous one is absent in ring offsets [%v, %v)", msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap), zap.String("task", taskCfg.Name)) + ring.MakeRoom(&msg) + ring.PutMsgNolock(&msg) } - time.Sleep(1 * time.Second) - ring.ForceBatchOrShard(&msg) } } @@ -244,6 +261,7 @@ func (service *Service) put(msg model.InputMessage) { metric, err = p.Parse(msg.Value) // WARNNING: Always PutElem even if there's parsing error, so that this message can be acked to Kafka and skipped writing to ClickHouse. if err != nil { + row = &model.FakedRow statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() if service.limiter1.Allow() { util.Logger.Error(fmt.Sprintf("failed to parse message(topic %v, partition %d, offset %v)", @@ -295,10 +313,10 @@ func (service *Service) put(msg model.InputMessage) { } // drain ensure we have completeted procession(discard or write&commit) for all received messages, and cleared service state. -// assumes service.put() not be invoked during drain(). func (service *Service) drain() { + savedState := atomic.LoadUint32(&service.state) atomic.StoreUint32(&service.state, util.StateStopped) - defer atomic.StoreUint32(&service.state, util.StateRunning) + defer atomic.StoreUint32(&service.state, savedState) service.Lock() for service.numFlying != 0 { service.taskDone.Wait() @@ -348,10 +366,7 @@ func (service *Service) Stop() { return } util.Logger.Info("stopping task service...", zap.String("task", taskCfg.Name)) - if err := service.inputer.Stop(); err != nil { - util.Logger.Fatal("service.inputer.Stop failed", zap.Error(err)) - } - util.Logger.Info("stopped input", zap.String("task", taskCfg.Name)) + atomic.StoreUint32(&service.state, util.StateStopped) if service.sharder != nil { service.sharder.tid.Stop() @@ -362,8 +377,15 @@ func (service *Service) Stop() { service.drain() util.Logger.Info("drained flying messages", zap.String("task", taskCfg.Name)) + // Note: inputer needs be stopped *after* drain() since a closed kafka-go client cannot commit offsets. + if err := service.inputer.Stop(); err != nil { + util.Logger.Fatal("service.inputer.Stop failed", zap.Error(err)) + } + util.Logger.Info("stopped input", zap.String("task", taskCfg.Name)) + if service.started { <-service.stopped } + service.started = false util.Logger.Info("stopped", zap.String("task", taskCfg.Name)) } From 0c75840412be47dee59cddcea5682eb269a72b1c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 7 Sep 2021 17:49:50 +0800 Subject: [PATCH 153/404] changed default kafka version --- .github/workflows/tests.yml | 2 +- config/config.go | 2 +- docker-compose.yml | 2 +- docker/test_auto_schema.json | 2 +- docker/test_fixed_schema.json | 2 +- docs/configuration/config.md | 2 +- docs/dev/introduction.md | 14 +++++++------- 7 files changed, 13 insertions(+), 13 deletions(-) diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index fa4c4fe7..fd7bb26f 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -16,7 +16,7 @@ jobs: - name: Set up Go 1.x uses: actions/setup-go@v2 with: - go-version: ^1.13 + go-version: 1.16.7 id: go - name: Check out code into the Go module directory diff --git a/config/config.go b/config/config.go index 2baaf20c..c2995e19 100644 --- a/config/config.go +++ b/config/config.go @@ -180,7 +180,7 @@ func (cfg *Config) Normallize() (err error) { return } if cfg.Kafka.Version == "" { - cfg.Kafka.Version = "2.6.0" + cfg.Kafka.Version = "2.5.0" } cfg.convertKfkSecurity() diff --git a/docker-compose.yml b/docker-compose.yml index 13cd066f..a2337a63 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -13,7 +13,7 @@ services: security_opt: - label:disable kafka: - image: confluentinc/cp-kafka:6.1.1 + image: confluentinc/cp-kafka:5.5.0 container_name: kafka restart: always hostname: kafka diff --git a/docker/test_auto_schema.json b/docker/test_auto_schema.json index 435bc405..4e3c7ed4 100644 --- a/docker/test_auto_schema.json +++ b/docker/test_auto_schema.json @@ -13,7 +13,7 @@ }, "kafka": { "brokers": "127.0.0.1:9093", - "version": "2.6.0" + "version": "2.5.0" }, "task": { "name": "test_auto_schema", diff --git a/docker/test_fixed_schema.json b/docker/test_fixed_schema.json index 56c4d7a1..c49d4b51 100644 --- a/docker/test_fixed_schema.json +++ b/docker/test_fixed_schema.json @@ -13,7 +13,7 @@ }, "kafka": { "brokers": "127.0.0.1:9093", - "version": "2.6.0" + "version": "2.5.0" }, "task": { "name": "test_fixed_schema", diff --git a/docs/configuration/config.md b/docs/configuration/config.md index c64d0f9d..cac420a9 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -87,7 +87,7 @@ }, // kafka version, if you use sarama, the version must be specified - "version": "2.6.0" + "version": "2.5.0" }, "task": { diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 7751c27e..d6fe45e6 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -103,7 +103,7 @@ An example kafka config: "kafka": { "brokers": "192.168.31.64:9092", "@version": "Required if you use sarama. It's the the Kafka server version.", - "version": "2.6.0" + "version": "2.5.0" } ``` @@ -114,7 +114,7 @@ An example kafka config: ```json "kafka": { "brokers": "192.168.31.64:9093", - "version": "2.6.0", + "version": "2.5.0", "tls": { "enable": true, "@trustStoreLocation": "ssl.truststore.location which kafka-console-consumer.sh uses", @@ -132,7 +132,7 @@ Or if you have extracted certificates from JKS, use the following config: ```json "kafka": { "brokers": "192.168.31.64:9093", - "version": "2.6.0", + "version": "2.5.0", "tls": { "enable": true, "@caCertFiles": "Required. It's the CA certificate with which Kafka brokers certs be signed. This cert is added to kafka.client.truststore.jks which kafka-console-consumer.sh uses", @@ -169,7 +169,7 @@ An example kafka config: "kafka": { "brokers": "192.168.31.64:9092", "@version": "Required if you use sarama. It's the the Kafka server version.", - "version": "2.6.0" + "version": "2.5.0" } ``` @@ -180,7 +180,7 @@ An example kafka config: ```json "kafka": { "brokers": "192.168.31.64:9094", - "version": "2.6.0", + "version": "2.5.0", "sasl": { "enable": true, "mechanism": "PLAIN", @@ -211,7 +211,7 @@ An example kafka config: ```json "kafka": { "brokers": "192.168.31.64:9094", - "version": "2.6.0", + "version": "2.5.0", "sasl": { "enable": true, "@mechanism": "SCRAM-SHA-256 or SCRAM-SHA-512", @@ -243,7 +243,7 @@ An example kafka config: ```json "kafka": { "brokers": "192.168.31.64:9094", - "version": "2.6.0", + "version": "2.5.0", "sasl": { "enable": true, "mechanism": "GSSAPI", From 50ebf1d3e60efcb40fcdc3249018129ef7216715 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 8 Sep 2021 15:26:30 +0800 Subject: [PATCH 154/404] kafka-go WatchPartitionChanges --- input/kafka_go.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/input/kafka_go.go b/input/kafka_go.go index 5772ee91..ef3bfddc 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -66,10 +66,13 @@ func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn fun GroupID: k.taskCfg.ConsumerGroup, Topic: k.taskCfg.Topic, StartOffset: offset, - MinBytes: 1024 * 1024, // sarama.Consumer.Fetch.Min - MaxBytes: 100 * 1024 * 1024, // sarama.MaxResponseSize - MaxWait: time.Duration(100) * time.Millisecond, // sarama.Consumer.MaxWaitTime - CommitInterval: time.Second, // flushes commits to Kafka every second + MinBytes: 1024 * 1024, // sarama.Config.Consumer.Fetch.Min + MaxBytes: 100 * 1024 * 1024, // sarama.Config.MaxResponseSize + MaxWait: time.Duration(100) * time.Millisecond, // sarama.Config.Consumer.MaxWaitTime + CommitInterval: time.Second, // sarama.Config.Consumer.Offsets.AutoCommit.Interval + // PartitionWatchInterval is only used when GroupID is set and WatchPartitionChanges is set. + PartitionWatchInterval: 600 * time.Second, // sarama.Config.Metadata.RefreshFrequency + WatchPartitionChanges: true, } if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { if kfkCfg.TLS.CaCertFiles, _, err = util.JksToPem(kfkCfg.TLS.TrustStoreLocation, kfkCfg.TLS.TrustStorePassword, false); err != nil { From ef3aa8a706b12ee6945922f728ed9b62165fd2e6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 15 Sep 2021 11:20:44 +0800 Subject: [PATCH 155/404] decreased writing pool queue size --- util/common.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/util/common.go b/util/common.go index d00f22ba..53d93589 100644 --- a/util/common.go +++ b/util/common.go @@ -79,12 +79,7 @@ func InitGlobalWritingPool(maxWorkers int) { if GlobalWritingPool != nil { return } - queueSize := runtime.NumCPU() - if queueSize < 3 { - queueSize = 3 - } else if queueSize > 100 { - queueSize = 100 - } + queueSize := 3 GlobalWritingPool = NewWorkerPool(maxWorkers, queueSize) Logger.Info("initialized writing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", queueSize)) } From 934ae21ae2a294a7f41bbc5278d55305f8cc19fe Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 15 Sep 2021 17:47:06 +0800 Subject: [PATCH 156/404] enabled sarama log --- input/kafka_sarama.go | 1 + 1 file changed, 1 insertion(+) diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index ae41118b..ed6785b1 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -141,6 +141,7 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn if err != nil { return err } + //sarama.Logger, _ = zap.NewStdLogAt(util.Logger.With(zap.String("name", "sarama")), zapcore.DebugLevel) k.cg = cg return nil } From 895aefdd0cf5b8bcfc2110468eeb9dff98283747 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 1 Sep 2021 10:39:21 +0800 Subject: [PATCH 157/404] assignment loop --- cmd/clickhouse_sinker/main.go | 74 ++++-- cmd/nacos_publish_config/main.go | 5 +- config/config.go | 33 ++- config/nacos.go | 120 --------- config_manager/lags.go | 112 +++++++++ config_manager/nacos.go | 419 +++++++++++++++++++++++++++++++ config_manager/rcm.go | 21 ++ input/kafka_sarama.go | 60 +++-- output/clickhouse.go | 2 +- parser/parser_test.go | 2 +- statistics/statistics_test.go | 2 +- util/common.go | 24 +- 12 files changed, 687 insertions(+), 187 deletions(-) delete mode 100644 config/nacos.go create mode 100644 config_manager/lags.go create mode 100644 config_manager/nacos.go create mode 100644 config_manager/rcm.go diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 3060e60e..c80b8c8d 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -30,6 +30,7 @@ import ( "time" "github.com/housepower/clickhouse_sinker/config" + cm "github.com/housepower/clickhouse_sinker/config_manager" "github.com/housepower/clickhouse_sinker/health" "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" @@ -43,7 +44,9 @@ import ( type CmdOptions struct { ShowVer bool - HTTPPort int + LogLevel string // "debug", "info", "warn", "error", "dpanic", "panic", "fatal" + LogPaths string // comma-separated paths. "stdout" means the console stdout + HTTPPort int // 0 menas a randomly OS chosen port PushGatewayAddrs string PushInterval int LocalCfgFile string @@ -53,6 +56,7 @@ type CmdOptions struct { NacosUsername string NacosPassword string NacosDataID string + NacosServiceName string // participate in assignment management if not empty } var ( @@ -73,7 +77,9 @@ func initCmdOptions() { // 1. Set options to default value. cmdOps = CmdOptions{ ShowVer: false, - HTTPPort: 0, // 0 menas a randomly OS chosen port + LogLevel: "info", + LogPaths: "stdout,clickhouse_sinker.log", + HTTPPort: 0, PushGatewayAddrs: "", PushInterval: 10, LocalCfgFile: "/etc/clickhouse_sinker.json", @@ -83,10 +89,13 @@ func initCmdOptions() { NacosUsername: "nacos", NacosPassword: "nacos", NacosDataID: "", + NacosServiceName: "", } // 2. Replace options with the corresponding env variable if present. util.EnvBoolVar(&cmdOps.ShowVer, "v") + util.EnvStringVar(&cmdOps.LogLevel, "log-level") + util.EnvStringVar(&cmdOps.LogPaths, "log-paths") util.EnvIntVar(&cmdOps.HTTPPort, "http-port") util.EnvStringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs") util.EnvIntVar(&cmdOps.PushInterval, "push-interval") @@ -97,9 +106,12 @@ func initCmdOptions() { util.EnvStringVar(&cmdOps.NacosNamespaceID, "nacos-namespace-id") util.EnvStringVar(&cmdOps.NacosGroup, "nacos-group") util.EnvStringVar(&cmdOps.NacosDataID, "nacos-dataid") + util.EnvStringVar(&cmdOps.NacosServiceName, "nacos-service-name") // 3. Replace options with the corresponding CLI parameter if present. flag.BoolVar(&cmdOps.ShowVer, "v", cmdOps.ShowVer, "show build version and quit") + flag.StringVar(&cmdOps.LogLevel, "log-level", cmdOps.LogLevel, "one of debug, info, warn, error, dpanic, panic, fatal") + flag.StringVar(&cmdOps.LogPaths, "log-paths", cmdOps.LogPaths, "a list of comma-separated log file path. stdout means the console stdout") flag.IntVar(&cmdOps.HTTPPort, "http-port", cmdOps.HTTPPort, "http listen port") flag.StringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs", cmdOps.PushGatewayAddrs, "a list of comma-separated prometheus push gatway address") flag.IntVar(&cmdOps.PushInterval, "push-interval", cmdOps.PushInterval, "push interval in seconds") @@ -112,6 +124,7 @@ func initCmdOptions() { `nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work!`) flag.StringVar(&cmdOps.NacosGroup, "nacos-group", cmdOps.NacosGroup, `nacos group name. Empty string doesn't work!`) flag.StringVar(&cmdOps.NacosDataID, "nacos-dataid", cmdOps.NacosDataID, "nacos dataid") + flag.StringVar(&cmdOps.NacosServiceName, "nacos-service-name", cmdOps.NacosServiceName, "nacos service name") flag.Parse() } @@ -120,8 +133,10 @@ func getVersion() string { } func init() { - util.InitLogger("info", []string{"stdout"}) initCmdOptions() + logPaths := strings.Split(cmdOps.LogPaths, ",") + util.InitLogger(logPaths) + util.SetLogLevel(cmdOps.LogLevel) util.Logger.Info(getVersion()) if cmdOps.ShowVer { os.Exit(0) @@ -181,12 +196,12 @@ func main() { } }() - var rcm config.RemoteConfManager + var rcm cm.RemoteConfManager var properties map[string]interface{} if cmdOps.NacosDataID != "" { util.Logger.Info(fmt.Sprintf("get config from nacos serverAddrs %s, namespaceId %s, group %s, dataId %s", cmdOps.NacosAddr, cmdOps.NacosNamespaceID, cmdOps.NacosGroup, cmdOps.NacosDataID)) - rcm = &config.NacosConfManager{} + rcm = &cm.NacosConfManager{} properties = make(map[string]interface{}) properties["serverAddrs"] = cmdOps.NacosAddr properties["username"] = cmdOps.NacosUsername @@ -194,6 +209,7 @@ func main() { properties["namespaceId"] = cmdOps.NacosNamespaceID properties["group"] = cmdOps.NacosGroup properties["dataId"] = cmdOps.NacosDataID + properties["serviceName"] = cmdOps.NacosServiceName } else { util.Logger.Info(fmt.Sprintf("get config from local file %s", cmdOps.LocalCfgFile)) } @@ -201,6 +217,11 @@ func main() { if err := rcm.Init(properties); err != nil { util.Logger.Fatal("rcm.Init failed", zap.Error(err)) } + if cmdOps.NacosServiceName != "" { + if err := rcm.Register(selfIP, httpPort); err != nil { + util.Logger.Fatal("rcm.Init failed", zap.Error(err)) + } + } } runner = NewSinker(rcm) return runner.Init() @@ -219,13 +240,13 @@ type Sinker struct { numCfg int pusher *statistics.Pusher tasks map[string]*task.Service - rcm config.RemoteConfManager + rcm cm.RemoteConfManager ctx context.Context cancel context.CancelFunc } // NewSinker get an instance of sinker with the task list -func NewSinker(rcm config.RemoteConfManager) *Sinker { +func NewSinker(rcm cm.RemoteConfManager) *Sinker { ctx, cancel := context.WithCancel(context.Background()) s := &Sinker{tasks: make(map[string]*task.Service), rcm: rcm, ctx: ctx, cancel: cancel} return s @@ -267,6 +288,9 @@ func (s *Sinker) Run() { } <-s.ctx.Done() } else { + if cmdOps.NacosServiceName != "" { + go s.rcm.Run(s.ctx) + } for { select { case <-s.ctx.Done(): @@ -293,7 +317,11 @@ func (s *Sinker) Run() { func (s *Sinker) Close() { // 1. Stop tasks gracefully. s.stopAllTasks() - // 2. Stop pusher + // 2. Stop rcm + if s.rcm != nil { + s.rcm.Stop() + } + // 3. Stop pusher if s.pusher != nil { s.pusher.Stop() s.pusher = nil @@ -314,19 +342,28 @@ func (s *Sinker) stopAllTasks() { delete(s.tasks, taskName) } util.Logger.Info("stopping parsing pool") - util.GlobalParsingPool.StopWait() + if util.GlobalParsingPool != nil { + util.GlobalParsingPool.StopWait() + } util.Logger.Info("stopping timer wheel") - util.GlobalTimerWheel.Stop() + if util.GlobalTimerWheel != nil { + util.GlobalTimerWheel.Stop() + } util.Logger.Info("stopping writing pool") - util.GlobalWritingPool.StopWait() + if util.GlobalWritingPool != nil { + util.GlobalWritingPool.StopWait() + } } func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { - util.InitLogger(newCfg.LogLevel, newCfg.LogPaths) + util.SetLogLevel(newCfg.LogLevel) if s.curCfg == nil { // The first time invoking of applyConfig err = s.applyFirstConfig(newCfg) - } else if !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) { + } else if !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || + !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || + !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) || + !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { err = s.applyAnotherConfig(newCfg) } return @@ -349,6 +386,9 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { // 3. Generate, initialize and run task for _, taskCfg := range newCfg.Tasks { + if cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(httpAddr, taskCfg.Name) { + continue + } task := task.NewTaskService(newCfg, taskCfg) if err = task.Init(); err != nil { return @@ -388,6 +428,9 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 4. Generate, initialize and run tasks. for _, taskCfg := range newCfg.Tasks { + if cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(httpAddr, taskCfg.Name) { + continue + } task := task.NewTaskService(newCfg, taskCfg) if err = task.Init(); err != nil { return @@ -397,7 +440,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { for _, task := range s.tasks { go task.Run(s.ctx) } - } else if !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) { + } else if !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) || !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { //1. Find tasks need to stop. var tasksToStop []string curCfgTasks := make(map[string]*config.TaskConfig) @@ -406,6 +449,9 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { curCfgTasks[taskCfg.Name] = taskCfg } for _, taskCfg := range newCfg.Tasks { + if cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(httpAddr, taskCfg.Name) { + continue + } newCfgTasks[taskCfg.Name] = taskCfg } for taskName := range s.tasks { diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index dac113f6..74b3c3ad 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -24,6 +24,7 @@ import ( _ "github.com/ClickHouse/clickhouse-go" "github.com/housepower/clickhouse_sinker/config" + cm "github.com/housepower/clickhouse_sinker/config_manager" "github.com/housepower/clickhouse_sinker/util" "github.com/jinzhu/copier" "go.uber.org/zap" @@ -94,7 +95,7 @@ func PublishSinkerConfig() { cfg.Clickhouse.MaxOpenConns = *maxOpenConns } - ncm := config.NacosConfManager{} + ncm := cm.NacosConfManager{} properties := getProperties() if err = ncm.Init(properties); err != nil { util.Logger.Fatal("ncm.Init failed", zap.Error(err)) @@ -116,7 +117,7 @@ func PublishSinkerConfig() { } func main() { - util.InitLogger("info", []string{"stdout"}) + util.InitLogger([]string{"stdout"}) flag.Parse() PublishSinkerConfig() } diff --git a/config/config.go b/config/config.go index c2995e19..2fcfea2d 100644 --- a/config/config.go +++ b/config/config.go @@ -25,23 +25,14 @@ import ( "github.com/pkg/errors" ) -// RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... -type RemoteConfManager interface { - Init(properties map[string]interface{}) error - // GetConfig fetchs the config. The manager shall not reference the returned Config object after call. - GetConfig() (conf *Config, err error) - // PublishConfig publishs the config. - PublishConfig(conf *Config) (err error) -} - // Config struct used for different configurations use type Config struct { Kafka KafkaConfig Clickhouse ClickHouseConfig Task *TaskConfig Tasks []*TaskConfig + Assignment Assignment LogLevel string - LogPaths []string } // KafkaConfig configuration parameters @@ -147,6 +138,13 @@ type TaskConfig struct { TimeZone string `json:"timezone"` } +type Assignment struct { + Version int + UpdatedAt int64 // timestamp when created + UpdatedBy string // leader instance + Map map[string][]string // map instance to a list of task_name +} + const ( MaxBufferSize = 1 << 20 //1048576 defaultBufferSize = 1 << 18 //262144 @@ -248,9 +246,6 @@ func (cfg *Config) Normallize() (err error) { default: cfg.LogLevel = defaultLogLevel } - if len(cfg.LogPaths) == 0 { - cfg.LogPaths = []string{"stdout"} - } return } @@ -337,6 +332,18 @@ func (cfg *Config) convertKfkSecurity() { } } +func (cfg *Config) IsAssigned(instance, task string) (assigned bool) { + if taskNames, ok := cfg.Assignment.Map[instance]; ok { + for _, taskName := range taskNames { + if taskName == task { + assigned = true + return + } + } + } + return +} + func readConfig(config string) map[string]string { configMap := make(map[string]string) config = strings.TrimSuffix(config, ";") diff --git a/config/nacos.go b/config/nacos.go deleted file mode 100644 index f8c00776..00000000 --- a/config/nacos.go +++ /dev/null @@ -1,120 +0,0 @@ -package config - -import ( - "encoding/json" - "path/filepath" - "strconv" - "strings" - - "github.com/nacos-group/nacos-sdk-go/clients" - "github.com/nacos-group/nacos-sdk-go/clients/config_client" - "github.com/nacos-group/nacos-sdk-go/common/constant" - "github.com/nacos-group/nacos-sdk-go/vo" - "github.com/pkg/errors" -) - -var _ RemoteConfManager = (*NacosConfManager)(nil) - -type NacosConfManager struct { - configClient config_client.IConfigClient - group string - dataID string -} - -func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) { - var sc []constant.ServerConfig - serverAddrs := strings.Split(properties["serverAddrs"].(string), ",") - for _, serverAddr := range serverAddrs { - serverAddrFields := strings.SplitN(serverAddr, ":", 2) - var nacosPort uint64 - if nacosPort, err = strconv.ParseUint(serverAddrFields[1], 10, 64); err != nil { - err = errors.Wrapf(err, "") - return - } - sc = append(sc, constant.ServerConfig{ - IpAddr: serverAddrFields[0], - Port: nacosPort, - }) - } - - var clientDir string - if v, ok := properties["clientDir"]; ok { - clientDir, _ = v.(string) - } else { - clientDir = "/tmp/nacos" - } - var namespaceID string //Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work! - group := constant.DEFAULT_GROUP //Empty string doesn't work! - var ok bool - if _, ok = properties["namespaceId"]; ok { - namespaceID, _ = properties["namespaceId"].(string) - } - if _, ok = properties["group"]; ok { - group, _ = properties["group"].(string) - } - cc := constant.ClientConfig{ - NamespaceId: namespaceID, - TimeoutMs: 5000, - ListenInterval: 10000, - NotLoadCacheAtStart: true, - LogDir: filepath.Join(clientDir, "log"), - CacheDir: filepath.Join(clientDir, "cache"), - RotateTime: "1h", - MaxAge: 3, - LogLevel: "debug", - Username: properties["username"].(string), - Password: properties["password"].(string), - } - - ncm.configClient, err = clients.CreateConfigClient(map[string]interface{}{ - "serverConfigs": sc, - "clientConfig": cc, - }) - if err != nil { - err = errors.Wrapf(err, "") - return - } - - ncm.group = group - if _, ok = properties["dataId"]; ok { - ncm.dataID, _ = properties["dataId"].(string) - } - return -} - -func (ncm *NacosConfManager) GetConfig() (conf *Config, err error) { - var content string - content, err = ncm.configClient.GetConfig(vo.ConfigParam{ - DataId: ncm.dataID, - Group: ncm.group, - }) - if err != nil { - err = errors.Wrapf(err, "") - return - } - conf = &Config{} - if err = json.Unmarshal([]byte(content), conf); err != nil { - err = errors.Wrapf(err, "") - return - } - return -} - -func (ncm *NacosConfManager) PublishConfig(conf *Config) (err error) { - var bs []byte - if bs, err = json.Marshal(*conf); err != nil { - err = errors.Wrapf(err, "") - return - } - content := string(bs) - _, err = ncm.configClient.PublishConfig(vo.ConfigParam{ - DataId: ncm.dataID, - Group: ncm.group, - Content: content, - }) - if err != nil { - err = errors.Wrapf(err, "") - return - } - return -} diff --git a/config_manager/lags.go b/config_manager/lags.go new file mode 100644 index 00000000..43ad0a49 --- /dev/null +++ b/config_manager/lags.go @@ -0,0 +1,112 @@ +package rcm + +import ( + "strings" + + "github.com/Shopify/sarama" + "github.com/housepower/clickhouse_sinker/config" + "github.com/housepower/clickhouse_sinker/input" + "github.com/housepower/clickhouse_sinker/util" + "github.com/pkg/errors" + "go.uber.org/zap" +) + +// GetTaskLags inspired by https://github.com/cloudhut/kminion/blob/1ffd02ba94a5edc26d4f11e57191ed3479d8a111/prometheus/collect_consumer_group_lags.go +func GetTaskLags(cfg *config.Config) (taskLags map[string]int64, err error) { + var adminClient sarama.ClusterAdmin + var client sarama.Client + var sarCfg *sarama.Config + taskLags = make(map[string]int64) // taskName -> totalLags + if sarCfg, err = input.GetSaramaConfig(&cfg.Kafka); err != nil { + return + } + if adminClient, err = sarama.NewClusterAdmin(strings.Split(cfg.Kafka.Brokers, ","), sarCfg); err != nil { + return + } + if client, err = sarama.NewClient(strings.Split(cfg.Kafka.Brokers, ","), sarCfg); err != nil { + return + } + defer func() { + if err2 := client.Close(); err2 != nil { + util.Logger.Error("failed to close Kafka client", zap.Error(err2)) + } + if err2 := adminClient.Close(); err2 != nil { + util.Logger.Error("failed to close Kafka admin client", zap.Error(err2)) + } + }() + + // Get topics' partition id list + var topics []string + topicPartitions := make(map[string]int) //topic -> number of partitions + for _, taskCfg := range cfg.Tasks { + topicPartitions[taskCfg.Topic] = 0 + } + for topic := range topicPartitions { + topics = append(topics, topic) + } + var topicsMeta []*sarama.TopicMetadata + if topicsMeta, err = adminClient.DescribeTopics(topics); err != nil { + return + } + for i, topicMeta := range topicsMeta { + if topicMeta != nil { + topicPartitions[topics[i]] = len(topicMeta.Partitions) + } + } + + // Get partitions' oldest and newest offset + topicOldestOffsets := make(map[string][]int64) //topic -> list of partitions' oldestOffset + topicNewestOffsets := make(map[string][]int64) //topic -> list of partitions' newestOffset + for _, topic := range topics { + if partitions, ok := topicPartitions[topic]; ok { + var oldestOffsets, newestOffsets []int64 + var oldestOffset, newestOffset int64 + for partition := 0; partition < partitions; partition++ { + oldestOffset, err = client.GetOffset(topic, int32(partition), sarama.OffsetOldest) + if err != nil { + err = errors.Wrapf(err, "failed to get topic/partition offsets for %q partition %q", topic, partition) + return + } + newestOffset, err = client.GetOffset(topic, int32(partition), sarama.OffsetNewest) + if err != nil { + err = errors.Wrapf(err, "failed to get topic/partition offsets for %q partition %q", topic, partition) + return + } + oldestOffsets = append(oldestOffsets, oldestOffset) + newestOffsets = append(newestOffsets, newestOffset) + } + topicOldestOffsets[topic] = oldestOffsets + topicNewestOffsets[topic] = newestOffsets + } + } + + // Get consumer groups' offset + for _, taskCfg := range cfg.Tasks { + topic := taskCfg.Topic + var totalLags int64 + oldestOffsets := topicOldestOffsets[topic] + newestOffsets := topicNewestOffsets[topic] + if partitions, ok := topicPartitions[topic]; ok { + pidList := make([]int32, partitions) + for partition := 0; partition < partitions; partition++ { + pidList[partition] = int32(partition) + } + var rep *sarama.OffsetFetchResponse + if rep, err = adminClient.ListConsumerGroupOffsets(taskCfg.ConsumerGroup, map[string][]int32{topic: pidList}); err != nil { + for partition := 0; partition < partitions; partition++ { + totalLags += newestOffsets[partition] - oldestOffsets[partition] + 1 + } + } else { + for partition := 0; partition < partitions; partition++ { + block := rep.GetBlock(topic, int32(partition)) + lag := newestOffsets[partition] - block.Offset - 1 + if lag > 0 { + totalLags += lag + } + } + } + taskLags[taskCfg.Name] = totalLags + } + } + return +} diff --git a/config_manager/nacos.go b/config_manager/nacos.go new file mode 100644 index 00000000..36da5bb2 --- /dev/null +++ b/config_manager/nacos.go @@ -0,0 +1,419 @@ +package rcm + +import ( + "context" + "encoding/json" + "fmt" + "math" + "path/filepath" + "reflect" + "sort" + "strconv" + "strings" + "sync" + "time" + + "github.com/housepower/clickhouse_sinker/config" + "github.com/housepower/clickhouse_sinker/util" + "github.com/nacos-group/nacos-sdk-go/clients" + "github.com/nacos-group/nacos-sdk-go/clients/config_client" + "github.com/nacos-group/nacos-sdk-go/clients/naming_client" + "github.com/nacos-group/nacos-sdk-go/common/constant" + "github.com/nacos-group/nacos-sdk-go/model" + "github.com/nacos-group/nacos-sdk-go/vo" + "github.com/pkg/errors" + "go.uber.org/zap" +) + +var _ RemoteConfManager = (*NacosConfManager)(nil) + +type NacosConfManager struct { + configClient config_client.IConfigClient + namingClient naming_client.INamingClient + group string + dataID string + serviceName string + instance string // ip:port + + // state of assignment loop + ctx context.Context + cancel context.CancelFunc + mux sync.Mutex //protect curInsts, curCfg, curVer + curInsts []string + curCfg *config.Config + curVer int +} + +func toInstanceID(ip string, port int) string { + return fmt.Sprintf("%s:%d", ip, port) +} + +func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) { + var sc []constant.ServerConfig + serverAddrs := strings.Split(properties["serverAddrs"].(string), ",") + for _, serverAddr := range serverAddrs { + serverAddrFields := strings.SplitN(serverAddr, ":", 2) + var nacosPort uint64 + if nacosPort, err = strconv.ParseUint(serverAddrFields[1], 10, 64); err != nil { + err = errors.Wrapf(err, "") + return + } + sc = append(sc, constant.ServerConfig{ + IpAddr: serverAddrFields[0], + Port: nacosPort, + }) + } + + var clientDir string + if v, ok := properties["clientDir"]; ok { + clientDir, _ = v.(string) + } else { + clientDir = "/tmp/nacos" + } + var namespaceID string //Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work! + group := constant.DEFAULT_GROUP //Empty string doesn't work! + if pop, ok := properties["namespaceId"]; ok { + namespaceID, _ = pop.(string) + } + if pop, ok := properties["group"]; ok { + group, _ = pop.(string) + } + cc := constant.ClientConfig{ + NamespaceId: namespaceID, + TimeoutMs: 5000, + ListenInterval: 10000, + NotLoadCacheAtStart: true, + LogDir: filepath.Join(clientDir, "log"), + CacheDir: filepath.Join(clientDir, "cache"), + RotateTime: "1h", + MaxAge: 3, + LogLevel: "debug", + Username: properties["username"].(string), + Password: properties["password"].(string), + } + + ncm.configClient, err = clients.CreateConfigClient(map[string]interface{}{ + "serverConfigs": sc, + "clientConfig": cc, + }) + if err != nil { + err = errors.Wrapf(err, "") + return + } + ncm.namingClient, err = clients.CreateNamingClient(map[string]interface{}{ + "serverConfigs": sc, + "clientConfig": cc, + }) + if err != nil { + err = errors.Wrapf(err, "") + return + } + + ncm.group = group + if pop, ok := properties["dataId"]; ok { + ncm.dataID, _ = pop.(string) + } + if pop, ok := properties["serviceName"]; ok { + ncm.serviceName, _ = pop.(string) + } + return +} + +func (ncm *NacosConfManager) GetConfig() (conf *config.Config, err error) { + var content string + content, err = ncm.configClient.GetConfig(vo.ConfigParam{ + DataId: ncm.dataID, + Group: ncm.group, + }) + if err != nil { + err = errors.Wrapf(err, "") + return + } + conf = &config.Config{} + if err = json.Unmarshal([]byte(content), conf); err != nil { + err = errors.Wrapf(err, "") + return + } + return +} + +func (ncm *NacosConfManager) PublishConfig(conf *config.Config) (err error) { + var bs []byte + if bs, err = json.Marshal(*conf); err != nil { + err = errors.Wrapf(err, "") + return + } + content := string(bs) + _, err = ncm.configClient.PublishConfig(vo.ConfigParam{ + DataId: ncm.dataID, + Group: ncm.group, + Content: content, + }) + if err != nil { + err = errors.Wrapf(err, "") + return + } + return +} + +func (ncm *NacosConfManager) Register(ip string, port int) (err error) { + _, err = ncm.namingClient.RegisterInstance(vo.RegisterInstanceParam{ + Ip: ip, + Port: uint64(port), + ServiceName: ncm.serviceName, + GroupName: ncm.group, + Enable: true, + Healthy: true, + Ephemeral: true, + }) + if err != nil { + err = errors.Wrapf(err, "") + } + ncm.instance = toInstanceID(ip, port) + return +} + +func (ncm *NacosConfManager) Deregister(ip string, port int) (err error) { + _, err = ncm.namingClient.DeregisterInstance( + vo.DeregisterInstanceParam{ + Ip: ip, + Port: uint64(port), + ServiceName: ncm.serviceName, + GroupName: ncm.group, + Ephemeral: true, + }) + if err != nil { + err = errors.Wrapf(err, "") + } + return +} + +func (ncm *NacosConfManager) Run(ctx context.Context) { + var err error + ncm.ctx, ncm.cancel = context.WithCancel(ctx) + + // Assign the first time + util.Logger.Debug("assign first") + if err = ncm.assign(); err != nil { + util.Logger.Error("first assign failed", zap.Error(err)) + } + + // Listen to service and config change, and assign if necessary + configParam := vo.ConfigParam{ + DataId: ncm.dataID, + Group: ncm.group, + OnChange: ncm.configOnChange, + } + if err = ncm.configClient.ListenConfig(configParam); err != nil { + util.Logger.Fatal("ncm.configClient.ListenConfig failed with permanent error", zap.Error(err)) + } + + subParam := vo.SubscribeParam{ + GroupName: ncm.group, + ServiceName: ncm.serviceName, + SubscribeCallback: ncm.serviceOnChange, + } + if err = ncm.namingClient.Subscribe(&subParam); err != nil { + util.Logger.Fatal("ncm.namingClient.Subscribe failed with permanent error", zap.Error(err)) + } + + // Assign regularly to handle lag change + for { + select { + case <-ncm.ctx.Done(): + return + case <-time.After(5 * time.Minute): + util.Logger.Debug("assign triggered by 5 min timer") + if err := ncm.assign(); err != nil { + util.Logger.Error("assign failed", zap.Error(err)) + } + } + } +} + +func (ncm *NacosConfManager) Stop() { + if ncm.cancel != nil { + ncm.cancel() + } + var err error + configParam := vo.ConfigParam{ + DataId: ncm.dataID, + Group: ncm.group, + OnChange: ncm.configOnChange, + } + if err = ncm.configClient.CancelListenConfig(configParam); err != nil { + util.Logger.Error("ncm.configClient.CancelListenConfig failed", zap.Error(err)) + } + + subParam := vo.SubscribeParam{ + GroupName: ncm.group, + ServiceName: ncm.serviceName, + SubscribeCallback: ncm.serviceOnChange, + } + if err = ncm.namingClient.Unsubscribe(&subParam); err != nil { + util.Logger.Error("ncm.namingClient.Unsubscribe failed", zap.Error(err)) + } + util.Logger.Info("stopped nacos config manager") +} + +func (ncm *NacosConfManager) configOnChange(namespace, group, dataID, data string) { + if group != ncm.group || dataID != ncm.dataID { + return + } + util.Logger.Debug("assign triggered by config change") + if err := ncm.assign(); err != nil { + util.Logger.Error("configOnChange failed", zap.Error(err)) + } +} + +func (ncm *NacosConfManager) serviceOnChange(services []model.SubscribeService, err error) { + if err != nil { + return + } + util.Logger.Debug("assign triggered by service change") + if err = ncm.assign(); err != nil { + util.Logger.Error("serviceOnChange failed", zap.Error(err)) + } +} + +type InstanceAssignment struct { + Instance string + TotalLag int64 + TaskLags []TaskLag +} + +type TaskLag struct { + Task string + Lag int64 +} + +func (ncm *NacosConfManager) assign() (err error) { + ncm.mux.Lock() + defer ncm.mux.Unlock() + getServiceParam := vo.GetServiceParam{ + GroupName: ncm.group, + ServiceName: ncm.serviceName, + } + var service model.Service + if service, err = ncm.namingClient.GetService(getServiceParam); err != nil { + err = errors.Wrapf(err, "ncm.namingClient.GetService failed") + return + } + var newInsts []string + for _, inst := range service.Hosts { + newInsts = append(newInsts, toInstanceID(inst.Ip, int(inst.Port))) + } + sort.Strings(newInsts) + if newInsts == nil || newInsts[0] != ncm.instance { + // Only the first instance is capable to assgin + return + } + + var newCfg *config.Config + if newCfg, err = ncm.GetConfig(); err != nil { + err = errors.Wrapf(err, "ncm.GetConfig failed") + return + } + if reflect.DeepEqual(ncm.curInsts, newInsts) && + reflect.DeepEqual(ncm.curCfg, newCfg) && + ncm.curCfg.Assignment.UpdatedBy == ncm.instance && + time.Unix(ncm.curCfg.Assignment.UpdatedAt, 0).Add(10*time.Minute).After(time.Now()) { + util.Logger.Info("Both instances and config are up-to-date, and the config was published by myself in less than 10 minutes.") + return + } + + var taskLags map[string]int64 + if taskLags, err = GetTaskLags(newCfg); err != nil { + return + } + util.Logger.Debug(fmt.Sprintf("task lags %+v", taskLags)) + + var validTasks []string + for _, taskCfg := range newCfg.Tasks { + if _, ok := taskLags[taskCfg.Name]; ok { + validTasks = append(validTasks, taskCfg.Name) + } + } + sort.Slice(validTasks, func(i, j int) bool { + taskNameI := validTasks[i] + lagI := taskLags[taskNameI] + taskNameJ := validTasks[j] + lagJ := taskLags[taskNameJ] + return (lagI > lagJ) || (lagI == lagJ && taskNameI < taskNameJ) + }) + + instAgs := make([]*InstanceAssignment, len(newInsts)) + for i, instance := range newInsts { + instAgs[i] = &InstanceAssignment{ + Instance: instance, + } + } + // distribute tasks in snake way + for idxTask := 0; idxTask < len(validTasks); idxTask++ { + idxInst := idxTask % len(newInsts) + if (idxTask/len(newInsts))%2 == 1 { + idxInst = len(newInsts) - 1 - idxInst + } + taskName := validTasks[idxTask] + taskLag := taskLags[taskName] + instAg := instAgs[idxInst] + instAg.TotalLag += taskLag + instAg.TaskLags = append(instAg.TaskLags, TaskLag{Task: taskName, Lag: taskLag}) + } + // balance + if len(newInsts) >= 2 && len(validTasks) > len(newInsts) { + last := len(newInsts) - 1 + for { + sort.Slice(instAgs, func(i, j int) bool { + return (instAgs[i].TotalLag > instAgs[j].TotalLag) || (instAgs[i].TotalLag == instAgs[j].TotalLag && instAgs[i].Instance < instAgs[j].Instance) + }) + diffLag := float64(instAgs[0].TotalLag - instAgs[last].TotalLag) + diffLagAbs := math.Abs(diffLag) + if diffLag == 0.0 { + break + } + var moved bool + for idx := 0; idx < len(instAgs[0].TaskLags); idx++ { + movingTask := instAgs[0].TaskLags[idx] + if math.Abs(diffLag-float64(2*movingTask.Lag)) < diffLagAbs { + instAgs[0].TotalLag -= movingTask.Lag + instAgs[last].TotalLag += movingTask.Lag + instAgs[0].TaskLags = append(instAgs[0].TaskLags[:idx], instAgs[0].TaskLags[idx+1:]...) + instAgs[last].TaskLags = append(instAgs[last].TaskLags, movingTask) + sort.Slice(instAgs[last].TaskLags, func(i, j int) bool { + return instAgs[last].TaskLags[i].Lag > instAgs[last].TaskLags[j].Lag + }) + moved = true + break + } + } + if !moved { + break + } + } + } + + // publish assignment + newVer := ncm.curVer + 1 + util.Logger.Debug("going to publish assignment", zap.Int("version", newVer), zap.Reflect("assignment", instAgs)) + newCfg.Assignment.Map = make(map[string][]string) + for _, instAg := range instAgs { + var tasks []string + for _, taskLag := range instAg.TaskLags { + tasks = append(tasks, taskLag.Task) + } + sort.Strings(tasks) + newCfg.Assignment.Map[instAg.Instance] = tasks + } + newCfg.Assignment.Version = newVer + newCfg.Assignment.UpdatedBy = ncm.instance + newCfg.Assignment.UpdatedAt = time.Now().Unix() + if err = ncm.PublishConfig(newCfg); err != nil { + return + } + ncm.curCfg = newCfg + ncm.curInsts = newInsts + ncm.curVer = newVer + + return +} diff --git a/config_manager/rcm.go b/config_manager/rcm.go new file mode 100644 index 00000000..98b841b2 --- /dev/null +++ b/config_manager/rcm.go @@ -0,0 +1,21 @@ +package rcm + +import ( + "context" + + "github.com/housepower/clickhouse_sinker/config" +) + +// RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... +type RemoteConfManager interface { + Init(properties map[string]interface{}) error + GetConfig() (conf *config.Config, err error) + // PublishConfig publishs the config. + PublishConfig(conf *config.Config) (err error) + Register(ip string, port int) (err error) + Deregister(ip string, port int) (err error) + + // Assignment loop + Run(ctx context.Context) + Stop() +} diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index ed6785b1..707423c5 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -90,12 +90,29 @@ func (h MyConsumerGroupHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, c func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage), cleanupFn func()) (err error) { k.cfg = cfg k.taskCfg = taskCfg - kfkCfg := &cfg.Kafka k.stopped = make(chan struct{}) k.putFn = putFn k.cleanupFn = cleanupFn - config := sarama.NewConfig() - if config.Version, err = sarama.ParseKafkaVersion(kfkCfg.Version); err != nil { + kfkCfg := &cfg.Kafka + sarCfg, err := GetSaramaConfig(&cfg.Kafka) + if err != nil { + return err + } + if taskCfg.Earliest { + sarCfg.Consumer.Offsets.Initial = sarama.OffsetOldest + } + cg, err := sarama.NewConsumerGroup(strings.Split(kfkCfg.Brokers, ","), taskCfg.ConsumerGroup, sarCfg) + if err != nil { + return err + } + //sarama.Logger, _ = zap.NewStdLogAt(util.Logger.With(zap.String("name", "sarama")), zapcore.DebugLevel) + k.cg = cg + return nil +} + +func GetSaramaConfig(kfkCfg *config.KafkaConfig) (sarCfg *sarama.Config, err error) { + sarCfg = sarama.NewConfig() + if sarCfg.Version, err = sarama.ParseKafkaVersion(kfkCfg.Version); err != nil { err = errors.Wrapf(err, "") return } @@ -110,40 +127,31 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn } } if kfkCfg.TLS.Enable { - config.Net.TLS.Enable = true - if config.Net.TLS.Config, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.EndpIdentAlgo == ""); err != nil { + sarCfg.Net.TLS.Enable = true + if sarCfg.Net.TLS.Config, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.EndpIdentAlgo == ""); err != nil { return } } // check for authentication if kfkCfg.Sasl.Enable { - config.Net.SASL.Enable = true - if config.Version.IsAtLeast(sarama.V1_0_0_0) { - config.Net.SASL.Version = sarama.SASLHandshakeV1 + sarCfg.Net.SASL.Enable = true + if sarCfg.Version.IsAtLeast(sarama.V1_0_0_0) { + sarCfg.Net.SASL.Version = sarama.SASLHandshakeV1 } - config.Net.SASL.Mechanism = (sarama.SASLMechanism)(kfkCfg.Sasl.Mechanism) - switch config.Net.SASL.Mechanism { + sarCfg.Net.SASL.Mechanism = (sarama.SASLMechanism)(kfkCfg.Sasl.Mechanism) + switch sarCfg.Net.SASL.Mechanism { case "SCRAM-SHA-256": - config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &XDGSCRAMClient{HashGeneratorFcn: SHA256} } + sarCfg.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &XDGSCRAMClient{HashGeneratorFcn: SHA256} } case "SCRAM-SHA-512": - config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &XDGSCRAMClient{HashGeneratorFcn: SHA256} } + sarCfg.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &XDGSCRAMClient{HashGeneratorFcn: SHA256} } default: } - config.Net.SASL.User = kfkCfg.Sasl.Username - config.Net.SASL.Password = kfkCfg.Sasl.Password - config.Net.SASL.GSSAPI = kfkCfg.Sasl.GSSAPI - } - if taskCfg.Earliest { - config.Consumer.Offsets.Initial = sarama.OffsetOldest - } - config.ChannelBufferSize = 1024 - cg, err := sarama.NewConsumerGroup(strings.Split(kfkCfg.Brokers, ","), taskCfg.ConsumerGroup, config) - if err != nil { - return err + sarCfg.Net.SASL.User = kfkCfg.Sasl.Username + sarCfg.Net.SASL.Password = kfkCfg.Sasl.Password + sarCfg.Net.SASL.GSSAPI = kfkCfg.Sasl.GSSAPI } - //sarama.Logger, _ = zap.NewStdLogAt(util.Logger.With(zap.String("name", "sarama")), zapcore.DebugLevel) - k.cg = cg - return nil + sarCfg.ChannelBufferSize = 1024 + return } // kafka main loop diff --git a/output/clickhouse.go b/output/clickhouse.go index b9712c35..64460500 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -170,7 +170,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { util.Logger.Warn("Batch.Commit failed due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) return } - util.Logger.Fatal("Batch.Commit failed with permanent error %+v", zap.String("task", c.taskCfg.Name), zap.Error(err)) + util.Logger.Fatal("Batch.Commit failed with permanent error", zap.String("task", c.taskCfg.Name), zap.Error(err)) } if errors.Is(err, context.Canceled) { util.Logger.Info("ClickHouse.write failed due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) diff --git a/parser/parser_test.go b/parser/parser_test.go index 6b1318eb..8cbae607 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -194,7 +194,7 @@ func initMetrics() { } metrics[name] = metric } - util.InitLogger("info", []string{"stdout"}) + util.InitLogger([]string{"stdout"}) } func sliceContains(list []string, target string) bool { diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 63aa5420..02244459 100755 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -25,7 +25,7 @@ import ( ) func TestPusher(t *testing.T) { - util.InitLogger("debug", []string{"stdout"}) + util.InitLogger([]string{"stdout"}) addrs := []string{"172.24.25.1:9091", "172.24.25.2:9091"} interval := 1 selfIP, _ := util.GetOutboundIP() diff --git a/util/common.go b/util/common.go index 53d93589..91b9fe60 100644 --- a/util/common.go +++ b/util/common.go @@ -44,7 +44,7 @@ var ( GlobalParsingPool *WorkerPool //for all tasks' parsing, cpu intensive GlobalWritingPool *WorkerPool //the all tasks' writing ClickHouse, cpu-net balance Logger *zap.Logger - logLevel string + logAtomLevel zap.AtomicLevel logPaths []string ) @@ -249,16 +249,12 @@ func JksToPem(jksPath, jksPassword string, overwrite bool) (certPemPath, keyPemP return } -func InitLogger(newLogLevel string, newLogPaths []string) { - if logLevel == newLogLevel && reflect.DeepEqual(logPaths, newLogPaths) { +func InitLogger(newLogPaths []string) { + if reflect.DeepEqual(logPaths, newLogPaths) { return } - logLevel = newLogLevel + logAtomLevel = zap.NewAtomicLevel() logPaths = newLogPaths - var lvl zapcore.Level - if err := lvl.Set(logLevel); err != nil { - lvl = zap.InfoLevel - } var syncers []zapcore.WriteSyncer for _, p := range logPaths { switch p { @@ -281,7 +277,17 @@ func InitLogger(newLogLevel string, newLogPaths []string) { core := zapcore.NewCore( zapcore.NewJSONEncoder(cfg), zapcore.NewMultiWriteSyncer(syncers...), - lvl, + logAtomLevel, ) Logger = zap.New(core, zap.AddStacktrace(zap.ErrorLevel)) } + +func SetLogLevel(newLogLevel string) { + if Logger != nil { + var lvl zapcore.Level + if err := lvl.Set(newLogLevel); err != nil { + lvl = zap.InfoLevel + } + logAtomLevel.SetLevel(lvl) + } +} From ef9cf7319130b25a4a77a74da1a19a5d9fc91b6d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 19 Sep 2021 10:57:16 +0800 Subject: [PATCH 158/404] changed recommended kafka client to sarama --- docs/dev/introduction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index d6fe45e6..3d415c6e 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -12,7 +12,7 @@ Refers to [design](./design.md) for how it works. - Uses native ClickHouse client-server TCP protocol, with higher performance than HTTP. - Easy to use and deploy, you don't need write any hard code, just care about the configuration file - Support multiple parsers: fastjson(recommended), gjson, csv. -- Support multiple Kafka client: kafka-go(recommended), sarama. +- Support multiple Kafka client: sarama(recommended), kafka-go. - Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/SCRAM, SASL/GSSAPI and combinations of them. - Bulk insert (by config `bufferSize` and `flushInterval`). - Parse messages concurrently. From ea2479c9ab3468d39511c6563f45532d14572919 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 19 Sep 2021 21:01:29 +0800 Subject: [PATCH 159/404] revised Dockerfile --- Dockerfile | 3 +++ Dockerfile_goreleaser | 3 +++ 2 files changed, 6 insertions(+) diff --git a/Dockerfile b/Dockerfile index b1c0fbc2..96ce6662 100644 --- a/Dockerfile +++ b/Dockerfile @@ -16,6 +16,8 @@ COPY --from=builder /app/kafka_gen_metric /usr/local/bin/kafka_gen_metric # clickhouse_sinker gets config from local file "/etc/clickhouse_sinker.json" by default. # Customize behavior with following env variables: # - V +# - LOG_LEVEL +# - LOG_PATHS # - HTTP_PORT # - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL @@ -26,6 +28,7 @@ COPY --from=builder /app/kafka_gen_metric /usr/local/bin/kafka_gen_metric # - NACOS_NAMESPACE_ID # - NACOS_GROUP # - NACOS_DATAID +# - NACOS_SERVICE_NAME # See cmd/clickhouse_sinker/main.go for details. ENTRYPOINT ["/usr/local/bin/clickhouse_sinker"] diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index cb66589d..8c4b6160 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -7,6 +7,8 @@ ADD dist/nacos_publish_config_linux_amd64/nacos_publish_config /usr/local/bin/na # clickhouse_sinker gets config from local file "/etc/clickhouse_sinker.json" by default. # Customize behavior with following env variables: # - V +# - LOG_LEVEL +# - LOG_PATHS # - HTTP_PORT # - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL @@ -17,6 +19,7 @@ ADD dist/nacos_publish_config_linux_amd64/nacos_publish_config /usr/local/bin/na # - NACOS_NAMESPACE_ID # - NACOS_GROUP # - NACOS_DATAID +# - NACOS_SERVICE_NAME # See cmd/clickhouse_sinker/main.go for details. ENTRYPOINT ["/usr/local/bin/clickhouse_sinker"] From bd86687ca12e23cf8ccacd62cb4e28953ea17209 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 19 Sep 2021 22:52:11 +0800 Subject: [PATCH 160/404] fixed Ring.ForceBatchOrShard --- task/ring.go | 43 ++++++++++++++++++++++++++----------------- 1 file changed, 26 insertions(+), 17 deletions(-) diff --git a/task/ring.go b/task/ring.go index 7b0763ec..53342c2d 100644 --- a/task/ring.go +++ b/task/ring.go @@ -40,6 +40,14 @@ func (ring *Ring) QuitIdle() { ring.idleCnt = 0 ring.isIdle = false ring.ringBuf = make([]model.MsgRow, ring.ringCap) + var err error + if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(ring.service.taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { + if errors.Is(err, goetty.ErrSystemStopped) { + util.Logger.Info("Ring.QuitIdle scheduling timer to a stopped timer wheel", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) + } else { + util.Logger.Fatal("scheduling timer failed", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) + } + } } } @@ -139,10 +147,21 @@ func (ring *Ring) ForceBatchOrShard(_ interface{}) { taskCfg := ring.service.taskCfg ring.mux.Lock() defer ring.mux.Unlock() + ring.tid.Stop() if !ring.isIdle { if ring.ringFilledOffset > ring.ringGroundOff { ring.genBatchOrShard() ring.idleCnt = 0 + // reschedule the delayed ForceBatchOrShard + var err error + if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { + if errors.Is(err, goetty.ErrSystemStopped) { + util.Logger.Warn("Ring.ForceBatchOrShard scheduling timer to a stopped timer wheel", zap.String("task", taskCfg.Name), zap.Error(err)) + } else { + err = errors.Wrap(err, "") + util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) + } + } } else if ring.ringGroundOff == ring.ringCeilingOff { ring.idleCnt++ if ring.idleCnt >= 2 { @@ -153,18 +172,6 @@ func (ring *Ring) ForceBatchOrShard(_ interface{}) { } } } - - // reschedule the delayed ForceBatchOrShard - ring.tid.Stop() - var err error - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { - if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Warn("Ring.ForceBatchOrShard scheduling timer to a stopped timer wheel", zap.String("task", taskCfg.Name), zap.Error(err)) - } else { - err = errors.Wrap(err, "") - util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - } } // generate a batch for messages [ring.ringGroundOff, ring.ringFilledOffset) @@ -178,11 +185,14 @@ func (ring *Ring) genBatchOrShard() { util.Logger.Info(fmt.Sprintf("Ring.genBatchOrShard discarded a batch for topic %v patittion %d, offset [%d,%d), messages %d", taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, msgCnt), zap.String("task", taskCfg.Name)) - statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) - return + for i := ring.ringGroundOff; i < endOff; i++ { + msgRow := &ring.ringBuf[i&(ring.ringCapMask)] + msgRow.Msg = nil + msgRow.Row = nil + msgRow.Shard = -1 + } } else if ring.service.sharder != nil { ring.service.sharder.PutElems(ring.partition, ring.ringBuf, ring.ringGroundOff, endOff, ring.ringCapMask) - statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } else { batch := model.NewBatch() for i := ring.ringGroundOff; i < endOff; i++ { @@ -208,9 +218,8 @@ func (ring *Ring) genBatchOrShard() { ring.service.Flush(batch) statistics.RingNormalBatchsTotal.WithLabelValues(taskCfg.Name).Inc() } - statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } - + statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) ring.ringGroundOff = endOff //util.Logger.Debug(fmt.Sprintf("genBatchOrShard changed ring %p ringGroundOff to %d", ring, ring.ringGroundOff)) if ring.ringFilledOffset < ring.ringGroundOff { From 1b154d127a795aaa0ca116c8eb1788b3432bcac2 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 20 Sep 2021 22:26:56 +0800 Subject: [PATCH 161/404] fixed Sinker.Close --- cmd/clickhouse_sinker/main.go | 75 ++++++++++++++++++++++++----------- config_manager/nacos.go | 16 +++++--- config_manager/rcm.go | 4 +- input/input.go | 5 +-- input/kafka_go.go | 18 +++++---- input/kafka_sarama.go | 10 +++-- statistics/statistics.go | 10 +++-- statistics/statistics_test.go | 4 +- task/task.go | 23 +++++------ util/common.go | 1 + 10 files changed, 99 insertions(+), 67 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index c80b8c8d..8f8e7246 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -25,6 +25,7 @@ import ( "net/http/pprof" "os" "reflect" + "sort" "strings" "sync" "time" @@ -236,19 +237,26 @@ func main() { // Sinker object maintains number of task for each partition type Sinker struct { - curCfg *config.Config - numCfg int - pusher *statistics.Pusher - tasks map[string]*task.Service - rcm cm.RemoteConfManager - ctx context.Context - cancel context.CancelFunc + curCfg *config.Config + numCfg int + pusher *statistics.Pusher + tasks map[string]*task.Service + rcm cm.RemoteConfManager + ctx context.Context + cancel context.CancelFunc + stopped chan struct{} } // NewSinker get an instance of sinker with the task list func NewSinker(rcm cm.RemoteConfManager) *Sinker { ctx, cancel := context.WithCancel(context.Background()) - s := &Sinker{tasks: make(map[string]*task.Service), rcm: rcm, ctx: ctx, cancel: cancel} + s := &Sinker{ + tasks: make(map[string]*task.Service), + rcm: rcm, + ctx: ctx, + cancel: cancel, + stopped: make(chan struct{}), + } return s } @@ -256,17 +264,20 @@ func (s *Sinker) Init() (err error) { return } -// Run rull task in different go routines +// Run is the mainloop to get and apply config func (s *Sinker) Run() { var err error var newCfg *config.Config + defer func() { + s.stopped <- struct{}{} + }() if cmdOps.PushGatewayAddrs != "" { addrs := strings.Split(cmdOps.PushGatewayAddrs, ",") s.pusher = statistics.NewPusher(addrs, cmdOps.PushInterval, httpAddr) if err = s.pusher.Init(); err != nil { return } - go s.pusher.Run(s.ctx) + go s.pusher.Run() } if s.rcm == nil { if _, err = os.Stat(cmdOps.LocalCfgFile); err == nil { @@ -289,11 +300,12 @@ func (s *Sinker) Run() { <-s.ctx.Done() } else { if cmdOps.NacosServiceName != "" { - go s.rcm.Run(s.ctx) + go s.rcm.Run() } for { select { case <-s.ctx.Done(): + util.Logger.Info("Sinker.Run quit due to context has been canceled") return case <-time.After(10 * time.Second): if newCfg, err = s.rcm.GetConfig(); err != nil { @@ -315,13 +327,17 @@ func (s *Sinker) Run() { // Close shutdown task func (s *Sinker) Close() { - // 1. Stop tasks gracefully. - s.stopAllTasks() - // 2. Stop rcm + // 1. Stop rcm if s.rcm != nil { s.rcm.Stop() + s.rcm = nil } - // 3. Stop pusher + // 2. Quit Run mainloop + s.cancel() + <-s.stopped + // 3. Stop tasks gracefully. + s.stopAllTasks() + // 4. Stop pusher if s.pusher != nil { s.pusher.Stop() s.pusher = nil @@ -341,18 +357,19 @@ func (s *Sinker) stopAllTasks() { for taskName := range s.tasks { delete(s.tasks, taskName) } - util.Logger.Info("stopping parsing pool") + util.Logger.Info("stopped all tasks") if util.GlobalParsingPool != nil { util.GlobalParsingPool.StopWait() } - util.Logger.Info("stopping timer wheel") + util.Logger.Info("stopped parsing pool") if util.GlobalTimerWheel != nil { util.GlobalTimerWheel.Stop() } - util.Logger.Info("stopping writing pool") + util.Logger.Info("stopped timer wheel") if util.GlobalWritingPool != nil { util.GlobalWritingPool.StopWait() } + util.Logger.Info("stopped writing pool") } func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { @@ -371,7 +388,6 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { util.Logger.Info("going to apply the first config", zap.Reflect("config", newCfg)) - // 1. Initialize clickhouse connections chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, @@ -396,16 +412,15 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { s.tasks[taskCfg.Name] = task } for _, task := range s.tasks { - go task.Run(s.ctx) + go task.Run() } s.curCfg = newCfg + util.Logger.Info("applied the first config") return } func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { util.Logger.Info("going to apply another config", zap.Int("number", s.numCfg), zap.Reflect("config", newCfg)) - s.numCfg++ - if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) { // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). s.stopAllTasks() @@ -427,6 +442,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { util.Logger.Info("resized writing pool", zap.Int("maxWorkers", maxWorkers)) // 4. Generate, initialize and run tasks. + var tasksToStart []string for _, taskCfg := range newCfg.Tasks { if cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(httpAddr, taskCfg.Name) { continue @@ -435,11 +451,14 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { if err = task.Init(); err != nil { return } + tasksToStart = append(tasksToStart, taskCfg.Name) s.tasks[taskCfg.Name] = task } for _, task := range s.tasks { - go task.Run(s.ctx) + go task.Run() } + sort.Strings(tasksToStart) + util.Logger.Info("started tasks", zap.Reflect("tasks", tasksToStart)) } else if !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) || !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { //1. Find tasks need to stop. var tasksToStop []string @@ -461,6 +480,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { tasksToStop = append(tasksToStop, taskName) } } + sort.Strings(tasksToStop) // 2. Stop tasks in parallel found at the previous step. // They must drain flying batchs as quickly as possible to allow another clickhouse_sinker // instance take over partitions safely. @@ -476,7 +496,9 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { for _, taskName := range tasksToStop { delete(s.tasks, taskName) } + util.Logger.Info("stopped tasks", zap.Reflect("tasks", tasksToStop)) // 3. Initailize tasks which are new or their config differ. + var tasksToStart []string var newTasks []*task.Service for taskName, taskCfg := range newCfgTasks { if _, ok := s.tasks[taskName]; ok { @@ -487,15 +509,20 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { return } s.tasks[taskName] = task + tasksToStart = append(tasksToStart, taskName) newTasks = append(newTasks, task) } // 4. Start new tasks. We don't do it at step 3 in order to avoid goroutine leak due to errors raised by later steps. for _, task := range newTasks { - go task.Run(s.ctx) + go task.Run() } + sort.Strings(tasksToStart) + util.Logger.Info("started tasks", zap.Reflect("tasks", tasksToStart)) } // Record the new config s.curCfg = newCfg + util.Logger.Info("applied another config", zap.Int("number", s.numCfg)) + s.numCfg++ return } diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 36da5bb2..cb09dce0 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -38,6 +38,7 @@ type NacosConfManager struct { // state of assignment loop ctx context.Context cancel context.CancelFunc + stopped chan struct{} mux sync.Mutex //protect curInsts, curCfg, curVer curInsts []string curCfg *config.Config @@ -116,6 +117,8 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) if pop, ok := properties["serviceName"]; ok { ncm.serviceName, _ = pop.(string) } + ncm.ctx, ncm.cancel = context.WithCancel(context.Background()) + ncm.stopped = make(chan struct{}) return } @@ -188,9 +191,8 @@ func (ncm *NacosConfManager) Deregister(ip string, port int) (err error) { return } -func (ncm *NacosConfManager) Run(ctx context.Context) { +func (ncm *NacosConfManager) Run() { var err error - ncm.ctx, ncm.cancel = context.WithCancel(ctx) // Assign the first time util.Logger.Debug("assign first") @@ -218,10 +220,12 @@ func (ncm *NacosConfManager) Run(ctx context.Context) { } // Assign regularly to handle lag change +LOOP_FOR: for { select { case <-ncm.ctx.Done(): - return + util.Logger.Info("NacosConfManager.Run quit due to context has been canceled") + break LOOP_FOR case <-time.After(5 * time.Minute): util.Logger.Debug("assign triggered by 5 min timer") if err := ncm.assign(); err != nil { @@ -229,12 +233,12 @@ func (ncm *NacosConfManager) Run(ctx context.Context) { } } } + ncm.stopped <- struct{}{} } func (ncm *NacosConfManager) Stop() { - if ncm.cancel != nil { - ncm.cancel() - } + ncm.cancel() + <-ncm.stopped var err error configParam := vo.ConfigParam{ DataId: ncm.dataID, diff --git a/config_manager/rcm.go b/config_manager/rcm.go index 98b841b2..5f368d60 100644 --- a/config_manager/rcm.go +++ b/config_manager/rcm.go @@ -1,8 +1,6 @@ package rcm import ( - "context" - "github.com/housepower/clickhouse_sinker/config" ) @@ -16,6 +14,6 @@ type RemoteConfManager interface { Deregister(ip string, port int) (err error) // Assignment loop - Run(ctx context.Context) + Run() Stop() } diff --git a/input/input.go b/input/input.go index 1204598e..7a1106ff 100644 --- a/input/input.go +++ b/input/input.go @@ -1,7 +1,6 @@ package input import ( - "context" "fmt" "github.com/housepower/clickhouse_sinker/config" @@ -17,9 +16,9 @@ const ( type Inputer interface { Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage), cleanupFn func()) error - Run(ctx context.Context) + Run() Stop() error - CommitMessages(ctx context.Context, message *model.InputMessage) error + CommitMessages(message *model.InputMessage) error } func NewInputer(typ string) Inputer { diff --git a/input/kafka_go.go b/input/kafka_go.go index ef3bfddc..8142a6df 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -41,6 +41,8 @@ type KafkaGo struct { cfg *config.Config taskCfg *config.TaskConfig r *kafka.Reader + ctx context.Context + cancel context.CancelFunc stopped chan struct{} putFn func(msg model.InputMessage) } @@ -55,6 +57,7 @@ func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn fun k.cfg = cfg k.taskCfg = taskCfg kfkCfg := &cfg.Kafka + k.ctx, k.cancel = context.WithCancel(context.Background()) k.stopped = make(chan struct{}) k.putFn = putFn offset := kafka.LastOffset @@ -127,12 +130,12 @@ func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn fun } // kafka main loop -func (k *KafkaGo) Run(ctx context.Context) { +func (k *KafkaGo) Run() { LOOP_KAFKA_GO: for { var err error var msg kafka.Message - if msg, err = k.r.FetchMessage(ctx); err != nil { + if msg, err = k.r.FetchMessage(k.ctx); err != nil { if errors.Is(err, context.Canceled) { util.Logger.Info("KafkaGo.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) break LOOP_KAFKA_GO @@ -158,8 +161,8 @@ LOOP_KAFKA_GO: k.stopped <- struct{}{} } -func (k *KafkaGo) CommitMessages(ctx context.Context, msg *model.InputMessage) (err error) { - if err = k.r.CommitMessages(ctx, kafka.Message{ +func (k *KafkaGo) CommitMessages(msg *model.InputMessage) (err error) { + if err = k.r.CommitMessages(context.Background(), kafka.Message{ Topic: msg.Topic, Partition: msg.Partition, Offset: msg.Offset, @@ -172,10 +175,9 @@ func (k *KafkaGo) CommitMessages(ctx context.Context, msg *model.InputMessage) ( // Stop kafka consumer and close all connections func (k *KafkaGo) Stop() error { - if k.r != nil { - k.r.Close() - <-k.stopped - } + k.cancel() + k.r.Close() + <-k.stopped return nil } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 707423c5..551757ea 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -42,6 +42,8 @@ type KafkaSarama struct { taskCfg *config.TaskConfig cg sarama.ConsumerGroup sess sarama.ConsumerGroupSession + ctx context.Context + cancel context.CancelFunc stopped chan struct{} putFn func(msg model.InputMessage) cleanupFn func() @@ -90,6 +92,7 @@ func (h MyConsumerGroupHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, c func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage), cleanupFn func()) (err error) { k.cfg = cfg k.taskCfg = taskCfg + k.ctx, k.cancel = context.WithCancel(context.Background()) k.stopped = make(chan struct{}) k.putFn = putFn k.cleanupFn = cleanupFn @@ -155,7 +158,7 @@ func GetSaramaConfig(kfkCfg *config.KafkaConfig) (sarCfg *sarama.Config, err err } // kafka main loop -func (k *KafkaSarama) Run(ctx context.Context) { +func (k *KafkaSarama) Run() { taskCfg := k.taskCfg LOOP_SARAMA: for { @@ -163,7 +166,7 @@ LOOP_SARAMA: // `Consume` should be called inside an infinite loop, when a // server-side rebalance happens, the consumer session will need to be // recreated to get the new claims - if err := k.cg.Consume(ctx, []string{taskCfg.Topic}, handler); err != nil { + if err := k.cg.Consume(k.ctx, []string{taskCfg.Topic}, handler); err != nil { if errors.Is(err, context.Canceled) { util.Logger.Info("KafkaSarama.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) break LOOP_SARAMA @@ -181,13 +184,14 @@ LOOP_SARAMA: k.stopped <- struct{}{} } -func (k *KafkaSarama) CommitMessages(ctx context.Context, msg *model.InputMessage) error { +func (k *KafkaSarama) CommitMessages(msg *model.InputMessage) error { k.sess.MarkOffset(msg.Topic, int32(msg.Partition), msg.Offset+1, "") return nil } // Stop kafka consumer and close all connections func (k *KafkaSarama) Stop() error { + k.cancel() k.cg.Close() <-k.stopped return nil diff --git a/statistics/statistics.go b/statistics/statistics.go index faa37c2a..42891204 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -175,6 +175,7 @@ type Pusher struct { instance string ctx context.Context cancel context.CancelFunc + stopped chan struct{} } func NewPusher(addrs []string, interval int, selfAddr string) *Pusher { @@ -183,6 +184,7 @@ func NewPusher(addrs []string, interval int, selfAddr string) *Pusher { pushInterval: interval, inUseAddr: -1, instance: selfAddr, + stopped: make(chan struct{}), } } @@ -195,11 +197,11 @@ func (p *Pusher) Init() error { return errPgwEmpty } p.reconnect() + p.ctx, p.cancel = context.WithCancel(context.Background()) return nil } -func (p *Pusher) Run(ctx context.Context) { - p.ctx, p.cancel = context.WithCancel(ctx) +func (p *Pusher) Run() { ticker := time.NewTicker(time.Second * time.Duration(p.pushInterval)) FOR: for { @@ -215,9 +217,12 @@ FOR: break FOR } } + p.stopped <- struct{}{} } func (p *Pusher) Stop() { + p.cancel() + <-p.stopped // https://stackoverflow.com/questions/63540280/how-to-set-a-retention-time-for-pushgateway-for-metrics-to-expire // https://github.com/prometheus/pushgateway/issues/19 if err := p.pusher.Delete(); err != nil { @@ -225,7 +230,6 @@ func (p *Pusher) Stop() { util.Logger.Error("failed to delete metric group", zap.String("pushgateway", p.pgwAddrs[p.inUseAddr]), zap.String("job", "clickhouse_sinker"), zap.String("instance", p.instance), zap.Error(err)) } - p.cancel() util.Logger.Info("stopped metric pusher") } diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 02244459..c32484f7 100755 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -15,7 +15,6 @@ limitations under the License. package statistics import ( - "context" "fmt" "testing" "time" @@ -36,8 +35,7 @@ func TestPusher(t *testing.T) { err := pusher.Init() require.Nilf(t, err, "pusher init failed") - ctx := context.Background() - go pusher.Run(ctx) + go pusher.Run() time.Sleep(10 * time.Second) pusher.Stop() } diff --git a/task/task.go b/task/task.go index 136e1719..7769e7bd 100644 --- a/task/task.go +++ b/task/task.go @@ -16,7 +16,6 @@ limitations under the License. package task import ( - "context" "fmt" "math" "sync" @@ -39,8 +38,6 @@ import ( // TaskService holds the configuration for each task type Service struct { sync.Mutex - - ctx context.Context started bool stopped chan struct{} inputer input.Inputer @@ -131,12 +128,10 @@ func (service *Service) Init() (err error) { } // Run starts the task -func (service *Service) Run(ctx context.Context) { +func (service *Service) Run() { var err error taskCfg := service.taskCfg service.started = true - service.ctx = ctx - util.Logger.Info("task started", zap.String("task", taskCfg.Name)) if service.sharder != nil { // schedule a delayed ForceFlush if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { @@ -148,13 +143,13 @@ func (service *Service) Run(ctx context.Context) { } } } - service.inputer.Run(service.ctx) + service.inputer.Run() service.stopped <- struct{}{} } func (service *Service) fnCommit(partition int, offset int64) error { msg := model.InputMessage{Topic: service.taskCfg.Topic, Partition: partition, Offset: offset} - return service.inputer.CommitMessages(context.Background(), &msg) + return service.inputer.CommitMessages(&msg) } func (service *Service) put(msg model.InputMessage) { @@ -355,7 +350,7 @@ func (service *Service) changeSchema(arg interface{}) { if err = service.Init(); err != nil { util.Logger.Fatal("service.Init failed", zap.String("task", taskCfg.Name), zap.Error(err)) } - go service.Run(service.ctx) + go service.Run() } // Stop stop kafka and clickhouse client. This is blocking. @@ -365,27 +360,27 @@ func (service *Service) Stop() { util.Logger.Info("stopped a already stopped task service", zap.String("task", taskCfg.Name)) return } - util.Logger.Info("stopping task service...", zap.String("task", taskCfg.Name)) + util.Logger.Debug("stopping task service...", zap.String("task", taskCfg.Name)) atomic.StoreUint32(&service.state, util.StateStopped) if service.sharder != nil { service.sharder.tid.Stop() } service.tid.Stop() - util.Logger.Info("stopped internal timers", zap.String("task", taskCfg.Name)) + util.Logger.Debug("stopped internal timers", zap.String("task", taskCfg.Name)) service.drain() - util.Logger.Info("drained flying messages", zap.String("task", taskCfg.Name)) + util.Logger.Debug("drained flying messages", zap.String("task", taskCfg.Name)) // Note: inputer needs be stopped *after* drain() since a closed kafka-go client cannot commit offsets. if err := service.inputer.Stop(); err != nil { util.Logger.Fatal("service.inputer.Stop failed", zap.Error(err)) } - util.Logger.Info("stopped input", zap.String("task", taskCfg.Name)) + util.Logger.Debug("stopped input", zap.String("task", taskCfg.Name)) if service.started { <-service.stopped } service.started = false - util.Logger.Info("stopped", zap.String("task", taskCfg.Name)) + util.Logger.Debug("stopped task", zap.String("task", taskCfg.Name)) } diff --git a/util/common.go b/util/common.go index 91b9fe60..90866616 100644 --- a/util/common.go +++ b/util/common.go @@ -267,6 +267,7 @@ func InitLogger(newLogPaths []string) { Filename: p, MaxSize: 100, // megabytes MaxBackups: 10, + LocalTime: true, }) syncers = append(syncers, writeFile) } From e24ece1355baf5e2f4e83769bc64f4aa5da1d7d0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 18 Sep 2021 23:24:56 +0800 Subject: [PATCH 162/404] fix cond wait blocked forever --- output/clickhouse.go | 19 ++++++++++--------- task/ring.go | 2 +- task/task.go | 14 ++++++++------ 3 files changed, 19 insertions(+), 16 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 64460500..34e1cc1a 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -25,7 +25,6 @@ import ( "sort" "strings" "sync" - "sync/atomic" "time" "github.com/ClickHouse/clickhouse-go" @@ -75,24 +74,26 @@ func (c *ClickHouse) Init() (err error) { // Drain drains flying batchs func (c *ClickHouse) Drain() { c.mux.Lock() - defer c.mux.Unlock() - for atomic.LoadInt32(&c.numFlying) != 0 { + for c.numFlying != 0 { c.taskDone.Wait() } + c.mux.Unlock() } // Send a batch to clickhouse func (c *ClickHouse) Send(batch *model.Batch) { - atomic.AddInt32(&c.numFlying, 1) + c.mux.Lock() + c.numFlying++ + c.mux.Unlock() statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Inc() _ = util.GlobalWritingPool.Submit(func() { c.loopWrite(batch) - numFlying := atomic.AddInt32(&c.numFlying, -1) - if numFlying == 0 { - c.mux.Lock() - c.taskDone.Signal() - c.mux.Unlock() + c.mux.Lock() + c.numFlying-- + if c.numFlying == 0 { + c.taskDone.Broadcast() } + c.mux.Unlock() statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Dec() }) } diff --git a/task/ring.go b/task/ring.go index 53342c2d..300bcab9 100644 --- a/task/ring.go +++ b/task/ring.go @@ -225,5 +225,5 @@ func (ring *Ring) genBatchOrShard() { if ring.ringFilledOffset < ring.ringGroundOff { ring.ringFilledOffset = ring.ringGroundOff } - ring.available.Signal() + ring.available.Broadcast() } diff --git a/task/task.go b/task/task.go index 7769e7bd..72d55732 100644 --- a/task/task.go +++ b/task/task.go @@ -237,19 +237,21 @@ func (service *Service) put(msg model.InputMessage) { } // submit message to a goroutine pool - atomic.AddInt32(&service.numFlying, 1) + service.Lock() + service.numFlying++ + service.Unlock() statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Inc() _ = util.GlobalParsingPool.Submit(func() { var row *model.Row var foundNewKeys bool var metric model.Metric defer func() { - numFlying := atomic.AddInt32(&service.numFlying, -1) - if numFlying == 0 { - service.Lock() - service.taskDone.Signal() - service.Unlock() + service.Lock() + service.numFlying-- + if service.numFlying == 0 { + service.taskDone.Broadcast() } + service.Unlock() statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Dec() }() p := service.pp.Get() From c80bcb037c76535bcce6d157b0873d9182d6dcd0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 23 Sep 2021 18:33:29 +0800 Subject: [PATCH 163/404] task scheduling doc --- docs/dev/design.md | 20 ++++++++++++++++++-- pool/conn.go | 4 ++-- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/docs/dev/design.md b/docs/dev/design.md index 806d1e94..5132a739 100644 --- a/docs/dev/design.md +++ b/docs/dev/design.md @@ -1,6 +1,7 @@ # Architecture -## Sharding with kafka message offset stripe (default) +## Sharding +### Sharding with kafka message offset stripe (default) clickhouse_sinker guarantee: @@ -18,7 +19,7 @@ The flow is: - Generate a batch if messages in a ring reach a batchSize bondary, or flush timer fire. This ensures offset/batchSize be same for all messages inside a batch. - Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). Batch is routed according to `(kafka_offset/roundup(buffer_size))%clickhouse_shards`. -## Sharding with custom key and policy +### Sharding with custom key and policy clickhouse_sinker guarantee: @@ -36,3 +37,18 @@ The flow is: - Generate batches for all shard slots if messages in one shard slot reach batchSize, or flush timer fire. Those batches form a `BatchGroup`. The `before` relationship could be impossilbe if messages of a partition are distributed to multiple batches. So those batches need to be committed after ALL of them have been written to clickhouse. - Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). + +## Task scheduling + +The clickhouse-server configuration item `max_concurrent_queries`(default 100) is the maximum number of simultaneously processed queries related to MergeTree table. If the number of concurrent INSERT is close to `max_concurrent_queries`, the user queries(`SELECT`) could fail due to the limit. + +If the clickhouse-server is big, ingesting data to >=100 MergeTree tabls via clickhouse_sinker bring pressure to the clickhouse cluster. On the other side, large number of clickhouse_sinker instances requires lot of CPU/MEM resources. + +The solution is, clickhouse_sinker instances coordinate with each other to assign tasks among themselves. + +The task scheduling procedure: + +- Some platform(Kubernetes, Yarn and etc.) start several clickhouse_sinker instances and may start/stop instances dynamically. Every clickhouse_sinker instance register with Nacos as a single service(CLI option `--nacos-service-name`). +- Someone publish(add/delete/modify) a list of tasks(with empty assignment) to Nacos. +- The first clickhouse_sinker(per instance's ip+port) instance(named scheduler) is responsible to generate and publish task assignment regularly. The task list and assignment consist of the whole config. The task list change, service change and task lag change will trigger another assignment. The scheduler ensure Each clickhouse_innker instance's total lag be balanced. +- Each clickhouse_sinker reload the config regularly. This may start/stop tasks. clickhouse_sinker stop tasks gracefully so that there's no message lost/duplication during task transfering. diff --git a/pool/conn.go b/pool/conn.go index a527f45d..634ea81a 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -108,8 +108,8 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db *sql.DB, dbVer int, err continue } - // WARN: clickHouse-server creates a thread for each TCP/HTTP connection. - // If the number of sinkers is close to clickhouse max_concurrent_queries(default 100), user queries could be blocked or refused. + // WARN: + // If the number of concurrent INSERT is close to clickhouse max_concurrent_queries(default 100), user queries could fail due to the limit. sqlDB.SetMaxOpenConns(sc.maxOpenConns) sqlDB.SetMaxIdleConns(0) sqlDB.SetConnMaxIdleTime(10 * time.Second) From 3ca5a98a16696a59d143207d2626d18f29a1d334 Mon Sep 17 00:00:00 2001 From: Dmitrii Raev Date: Wed, 22 Sep 2021 16:44:38 +0300 Subject: [PATCH 164/404] Update value.go The syntax for ClickHouse FixedString type is: FixedString(N) This leads to "LOGIC ERROR: unsupported ClickHouse data type" error I assume that any FixedString type is primitive String type --- model/value.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/model/value.go b/model/value.go index 48900ac4..b797ce31 100644 --- a/model/value.go +++ b/model/value.go @@ -77,6 +77,17 @@ func WhichType(typ string) (dataType int, nullable bool) { dataType, nullable = ti.Type, ti.Nullable return } + + //handle FixedString(N) clickhouse type + fixedString := strings.HasPrefix(typ, "FixedString(") + if fixedString { + ti, ok := typeInfo["String"] + if ok { + dataType, nullable = ti.Type, ti.Nullable + return + } + } + nullable = strings.HasPrefix(typ, "Nullable(") if nullable { typ = typ[len("Nullable(") : len(typ)-1] From 7c8350e5728e49f17d805ee37c9fc76a115cff36 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 24 Sep 2021 09:27:18 +0800 Subject: [PATCH 165/404] fixed FixedString --- model/value.go | 20 +++++++------------- 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/model/value.go b/model/value.go index b797ce31..7012d97d 100644 --- a/model/value.go +++ b/model/value.go @@ -77,17 +77,6 @@ func WhichType(typ string) (dataType int, nullable bool) { dataType, nullable = ti.Type, ti.Nullable return } - - //handle FixedString(N) clickhouse type - fixedString := strings.HasPrefix(typ, "FixedString(") - if fixedString { - ti, ok := typeInfo["String"] - if ok { - dataType, nullable = ti.Type, ti.Nullable - return - } - } - nullable = strings.HasPrefix(typ, "Nullable(") if nullable { typ = typ[len("Nullable(") : len(typ)-1] @@ -102,6 +91,11 @@ func WhichType(typ string) (dataType int, nullable bool) { } else if strings.HasPrefix(typ, "Array(Decimal") { dataType = FloatArray nullable = false + } else if strings.HasPrefix(typ, "FixedString") { + dataType = String + } else if strings.HasPrefix(typ, "Array(FixedString") { + dataType = StringArray + nullable = false } else { util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported ClickHouse data type %v", typ)) } @@ -119,10 +113,10 @@ func init() { for _, t := range []string{"Float32", "Float64"} { primTypeInfo[t] = TypeInfo{Type: Float, Nullable: false} } - for _, t := range []string{"String", "FixedString"} { + for _, t := range []string{"String"} { primTypeInfo[t] = TypeInfo{Type: String, Nullable: false} } - for _, t := range []string{"Date", "DateTime", "DateTime64"} { + for _, t := range []string{"Date", "DateTime"} { primTypeInfo[t] = TypeInfo{Type: DateTime, Nullable: false} } primTypeInfo["ElasticDateTime"] = TypeInfo{Type: ElasticDateTime, Nullable: false} From e0c3eba90dd16f60fbcc449a4ee5e2c61e7e3bbc Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 24 Sep 2021 09:41:00 +0800 Subject: [PATCH 166/404] minor doc --- docs/dev/introduction.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 3d415c6e..2c158d50 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -22,6 +22,7 @@ Refers to [design](./design.md) for how it works. - Tolerate replica single-point-failure. - At-least-once delivery guarantee. - Config management with local file or Nacos. +- One clickhouse_sinker instance assign tasks to all instances in balance of message lag (by config `nacos-service-name`). ## Supported data types From bc9b7fde8cf9037bbb6d629df7f8b3af8c05a898 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 24 Sep 2021 17:40:41 +0800 Subject: [PATCH 167/404] improved several Stop --- config_manager/nacos.go | 9 ++++----- input/kafka_go.go | 9 +++++---- input/kafka_sarama.go | 9 +++++---- task/task.go | 19 +++++-------------- 4 files changed, 19 insertions(+), 27 deletions(-) diff --git a/config_manager/nacos.go b/config_manager/nacos.go index cb09dce0..05b9a9d9 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -38,7 +38,7 @@ type NacosConfManager struct { // state of assignment loop ctx context.Context cancel context.CancelFunc - stopped chan struct{} + wg sync.WaitGroup mux sync.Mutex //protect curInsts, curCfg, curVer curInsts []string curCfg *config.Config @@ -118,7 +118,6 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) ncm.serviceName, _ = pop.(string) } ncm.ctx, ncm.cancel = context.WithCancel(context.Background()) - ncm.stopped = make(chan struct{}) return } @@ -193,7 +192,8 @@ func (ncm *NacosConfManager) Deregister(ip string, port int) (err error) { func (ncm *NacosConfManager) Run() { var err error - + ncm.wg.Add(1) + defer ncm.wg.Done() // Assign the first time util.Logger.Debug("assign first") if err = ncm.assign(); err != nil { @@ -233,12 +233,11 @@ LOOP_FOR: } } } - ncm.stopped <- struct{}{} } func (ncm *NacosConfManager) Stop() { ncm.cancel() - <-ncm.stopped + ncm.wg.Wait() var err error configParam := vo.ConfigParam{ DataId: ncm.dataID, diff --git a/input/kafka_go.go b/input/kafka_go.go index 8142a6df..8f70955f 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -20,6 +20,7 @@ import ( "crypto/tls" "io" "strings" + "sync" "time" "github.com/pkg/errors" @@ -43,7 +44,7 @@ type KafkaGo struct { r *kafka.Reader ctx context.Context cancel context.CancelFunc - stopped chan struct{} + wgRun sync.WaitGroup putFn func(msg model.InputMessage) } @@ -58,7 +59,6 @@ func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn fun k.taskCfg = taskCfg kfkCfg := &cfg.Kafka k.ctx, k.cancel = context.WithCancel(context.Background()) - k.stopped = make(chan struct{}) k.putFn = putFn offset := kafka.LastOffset if k.taskCfg.Earliest { @@ -131,6 +131,8 @@ func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn fun // kafka main loop func (k *KafkaGo) Run() { + k.wgRun.Add(1) + defer k.wgRun.Done() LOOP_KAFKA_GO: for { var err error @@ -158,7 +160,6 @@ LOOP_KAFKA_GO: Timestamp: &msg.Time, }) } - k.stopped <- struct{}{} } func (k *KafkaGo) CommitMessages(msg *model.InputMessage) (err error) { @@ -177,7 +178,7 @@ func (k *KafkaGo) CommitMessages(msg *model.InputMessage) (err error) { func (k *KafkaGo) Stop() error { k.cancel() k.r.Close() - <-k.stopped + k.wgRun.Wait() return nil } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 551757ea..b6673f83 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -21,6 +21,7 @@ import ( "crypto/sha512" "hash" "strings" + "sync" "time" "github.com/Shopify/sarama" @@ -44,7 +45,7 @@ type KafkaSarama struct { sess sarama.ConsumerGroupSession ctx context.Context cancel context.CancelFunc - stopped chan struct{} + wgRun sync.WaitGroup putFn func(msg model.InputMessage) cleanupFn func() } @@ -93,7 +94,6 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn k.cfg = cfg k.taskCfg = taskCfg k.ctx, k.cancel = context.WithCancel(context.Background()) - k.stopped = make(chan struct{}) k.putFn = putFn k.cleanupFn = cleanupFn kfkCfg := &cfg.Kafka @@ -159,6 +159,8 @@ func GetSaramaConfig(kfkCfg *config.KafkaConfig) (sarCfg *sarama.Config, err err // kafka main loop func (k *KafkaSarama) Run() { + k.wgRun.Add(1) + defer k.wgRun.Done() taskCfg := k.taskCfg LOOP_SARAMA: for { @@ -181,7 +183,6 @@ LOOP_SARAMA: } } } - k.stopped <- struct{}{} } func (k *KafkaSarama) CommitMessages(msg *model.InputMessage) error { @@ -193,7 +194,7 @@ func (k *KafkaSarama) CommitMessages(msg *model.InputMessage) error { func (k *KafkaSarama) Stop() error { k.cancel() k.cg.Close() - <-k.stopped + k.wgRun.Wait() return nil } diff --git a/task/task.go b/task/task.go index 72d55732..d3af3c0c 100644 --- a/task/task.go +++ b/task/task.go @@ -38,8 +38,6 @@ import ( // TaskService holds the configuration for each task type Service struct { sync.Mutex - started bool - stopped chan struct{} inputer input.Inputer clickhouse *output.ClickHouse pp *parser.Pool @@ -57,6 +55,7 @@ type Service struct { limiter1 *rate.Limiter limiter2 *rate.Limiter + wgRun sync.WaitGroup state uint32 numFlying int32 taskDone *sync.Cond @@ -68,10 +67,8 @@ func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) (service *Se pp, _ := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone) inputer := input.NewInputer(taskCfg.KafkaClient) service = &Service{ - stopped: make(chan struct{}), inputer: inputer, clickhouse: ck, - started: false, pp: pp, cfg: cfg, taskCfg: taskCfg, @@ -130,8 +127,9 @@ func (service *Service) Init() (err error) { // Run starts the task func (service *Service) Run() { var err error + service.wgRun.Add(1) + defer service.wgRun.Done() taskCfg := service.taskCfg - service.started = true if service.sharder != nil { // schedule a delayed ForceFlush if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { @@ -144,7 +142,6 @@ func (service *Service) Run() { } } service.inputer.Run() - service.stopped <- struct{}{} } func (service *Service) fnCommit(partition int, offset int64) error { @@ -358,10 +355,7 @@ func (service *Service) changeSchema(arg interface{}) { // Stop stop kafka and clickhouse client. This is blocking. func (service *Service) Stop() { taskCfg := service.taskCfg - if !service.started { - util.Logger.Info("stopped a already stopped task service", zap.String("task", taskCfg.Name)) - return - } + util.Logger.Debug("stopping task service...", zap.String("task", taskCfg.Name)) atomic.StoreUint32(&service.state, util.StateStopped) @@ -380,9 +374,6 @@ func (service *Service) Stop() { } util.Logger.Debug("stopped input", zap.String("task", taskCfg.Name)) - if service.started { - <-service.stopped - } - service.started = false + service.wgRun.Wait() util.Logger.Debug("stopped task", zap.String("task", taskCfg.Name)) } From f41c36a93dd823ee336566c3f4463c5bace8946a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 27 Sep 2021 16:11:47 +0800 Subject: [PATCH 168/404] removed logPaths from config --- docker/test_auto_schema.json | 3 +-- docker/test_dynamic_schema.json | 3 +-- docker/test_fixed_schema.json | 3 +-- docs/configuration/config.md | 5 +---- 4 files changed, 4 insertions(+), 10 deletions(-) diff --git a/docker/test_auto_schema.json b/docker/test_auto_schema.json index 4e3c7ed4..34dffbd6 100644 --- a/docker/test_auto_schema.json +++ b/docker/test_auto_schema.json @@ -28,6 +28,5 @@ ], "bufferSize": 50000 }, - "logLevel": "debug", - "logPaths": ["stdout", "test_auto_schema.log"] + "logLevel": "debug" } \ No newline at end of file diff --git a/docker/test_dynamic_schema.json b/docker/test_dynamic_schema.json index 263be263..d7d564d5 100644 --- a/docker/test_dynamic_schema.json +++ b/docker/test_dynamic_schema.json @@ -32,6 +32,5 @@ }, "bufferSize": 50000 }, - "logLevel": "debug", - "logPaths": ["stdout", "test_dynamic_schema.log"] + "logLevel": "debug" } \ No newline at end of file diff --git a/docker/test_fixed_schema.json b/docker/test_fixed_schema.json index c49d4b51..425ed942 100644 --- a/docker/test_fixed_schema.json +++ b/docker/test_fixed_schema.json @@ -38,6 +38,5 @@ ], "bufferSize": 50000 }, - "logLevel": "debug", - "logPaths": ["stdout", "test_fixed_schema.log"] + "logLevel": "debug" } \ No newline at end of file diff --git a/docs/configuration/config.md b/docs/configuration/config.md index cac420a9..7ddef4ba 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -155,9 +155,6 @@ }, // log level, possible value: "debug", "info", "warn", "error", "dpanic", "panic", "fatal". Default to "info". - "logLevel": "debug", - - // log output paths, possible value: "stdout", "stderr", relative file path, absoute file path. Log files will be rotated every 100MB, keep 10 old ones. Default to ["stdout"]. - "logPaths": ["stdout", "test_dynamic_schema.log"] + "logLevel": "debug" } ``` From abd31ec1350126decdfc13c5b6e1f2dd44ed5e26 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 16 Oct 2021 18:06:02 +0800 Subject: [PATCH 169/404] fixed dead wait --- task/ring.go | 1 + task/task.go | 16 +++++++++++++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/task/ring.go b/task/ring.go index 300bcab9..167e272a 100644 --- a/task/ring.go +++ b/task/ring.go @@ -147,6 +147,7 @@ func (ring *Ring) ForceBatchOrShard(_ interface{}) { taskCfg := ring.service.taskCfg ring.mux.Lock() defer ring.mux.Unlock() + defer ring.available.Broadcast() ring.tid.Stop() if !ring.isIdle { if ring.ringFilledOffset > ring.ringGroundOff { diff --git a/task/task.go b/task/task.go index d3af3c0c..d40f5125 100644 --- a/task/task.go +++ b/task/task.go @@ -88,6 +88,7 @@ func (service *Service) Init() (err error) { } service.dims = service.clickhouse.Dims + service.idxSerID = service.clickhouse.IdxSerID service.limiter1 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter2 = rate.NewLimiter(rate.Every(10*time.Second), 1) @@ -214,9 +215,19 @@ func (service *Service) put(msg model.InputMessage) { ring.mux.Unlock() } else { prevMsgOff := msg.Offset - 1 - for msg.Offset == ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil { - // wait ring.PutElem to make room + for atomic.LoadUint32(&service.state) == util.StateRunning && !ring.isIdle && msg.Offset == ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil { + // wait ring.PutElem/ring.ForceBatchOrShard to make room + util.Logger.Debug(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) while the ring is full, waiting...", + msg.Topic, msg.Partition, msg.Offset), zap.String("task", taskCfg.Name)) ring.available.Wait() + util.Logger.Debug(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) while the ring is full, wake-up", + msg.Topic, msg.Partition, msg.Offset), zap.String("task", taskCfg.Name)) + } + if atomic.LoadUint32(&service.state) != util.StateRunning || ring.isIdle { + util.Logger.Debug(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) while the ring.isIdle %v, service.state %v", + msg.Topic, msg.Partition, msg.Offset, ring.isIdle, atomic.LoadUint32(&service.state)), zap.String("task", taskCfg.Name)) + ring.mux.Unlock() + return } if msg.Offset == ring.ringGroundOff || (msg.Offset < ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil) { ring.PutMsgNolock(&msg) @@ -318,7 +329,6 @@ func (service *Service) drain() { for _, ring := range service.rings { if ring != nil { ring.ForceBatchOrShard(nil) - ring.tid.Stop() } } service.rings = make([]*Ring, 0) From 68751c9284c2095bde88acea04a87ace9b3a0d98 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 14 Oct 2021 13:45:35 +0800 Subject: [PATCH 170/404] added parser test case for inf --- parser/parser_test.go | 45 +++++++++++++++++++++++++++---------------- 1 file changed, 28 insertions(+), 17 deletions(-) diff --git a/parser/parser_test.go b/parser/parser_test.go index 8cbae607..d26b0695 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -19,6 +19,7 @@ import ( "errors" "fmt" "log" + "math" "strconv" "strings" "sync" @@ -27,6 +28,7 @@ import ( "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/tidwall/gjson" "github.com/valyala/fastjson" @@ -54,11 +56,11 @@ var jsonSample = []byte(`{ "array_bool": [true,false], "array_num_int_1": [0, 255, 256, 65535, 65536, 4294967295, 4294967296, 18446744073709551615, 18446744073709551616], "array_num_int_2": [-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807], - "array_num_float": [4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308], + "array_num_float": [4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308, -inf, +inf], "array_str": ["aa","bb","cc"], "array_str_int_1": ["0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"], "array_str_int_2": ["-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"], - "array_str_float": ["4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308"], + "array_str_float": ["4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"], "array_str_date_1": ["2009-07-13","2009-07-14","2009-07-15"], "array_str_date_2": ["13/07/2009","14/07/2009","15/07/2009"], "array_str_time_rfc3339": ["2009-07-13T09:07:13Z", "2009-07-13T09:07:13+08:00", "2009-07-13T09:07:13.123Z", "2009-07-13T09:07:13.123+08:00"], @@ -213,11 +215,12 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { for i := range names { name := names[i] metric := metrics[name] + var skipped []string for j := range testCases { var v interface{} - desc := fmt.Sprintf(`%s %s("%s", %s)`, name, method, testCases[j].Field, strconv.FormatBool(testCases[j].Nullable)) + desc := fmt.Sprintf(`%s.%s("%s", %s)`, name, method, testCases[j].Field, strconv.FormatBool(testCases[j].Nullable)) if name == "csv" && (sliceContains([]string{"GetInt", "GetFloat", "GetDateTime", "GetElasticDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || testCases[j].Nullable) { - log.Printf("%s is known to not compatible with fastjson parser, skipping", desc) + skipped = append(skipped, desc) continue } switch method { @@ -236,6 +239,9 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { } require.Equal(t, testCases[j].ExpVal, v, desc) } + if skipped != nil { + log.Printf("Skipped %d cases incompatible with fastjson parser: %v\n", len(skipped), strings.Join(skipped, ", ")) + } } } @@ -460,10 +466,10 @@ func TestParserArray(t *testing.T) { {"array_num_int_2", model.String, []string{"-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"}}, {"array_num_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, UnixInt(127), UnixInt(128), UnixInt(32767), UnixInt(32768), UnixInt(2147483647), UnixInt(2147483648), UnixInt(9223372036854775807)}}, - {"array_num_float", model.Int, []int64{0, 0, 0, 0, 0}}, - {"array_num_float", model.Float, []float64{4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308}}, - {"array_num_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308"}}, - {"array_num_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, UnixFloat(3.40282346638528859811704183484516925440e+38), UnixFloat(1.797693134862315708145274237317043567981e+308)}}, + {"array_num_float", model.Int, []int64{0, 0, 0, 0, 0, 0, 0}}, + {"array_num_float", model.Float, []float64{4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308, math.Inf(-1), math.Inf(1)}}, + {"array_num_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"}}, + {"array_num_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, UnixFloat(3.40282346638528859811704183484516925440e+38), UnixFloat(1.797693134862315708145274237317043567981e+308), UnixFloat(math.Inf(-1)), UnixFloat(math.Inf(1))}}, {"array_str", model.Int, []int64{0, 0, 0}}, {"array_str", model.Float, []float64{0.0, 0.0, 0.0}}, @@ -480,10 +486,10 @@ func TestParserArray(t *testing.T) { {"array_str_int_2", model.String, []string{"-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"}}, {"array_str_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch}}, - {"array_str_float", model.Int, []int64{0, 0, 0, 0, 0}}, - {"array_str_float", model.Float, []float64{0, 0, 0, 0, 0}}, - {"array_str_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308"}}, - {"array_str_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch}}, + {"array_str_float", model.Int, []int64{0, 0, 0, 0, 0, 0, 0}}, + {"array_str_float", model.Float, []float64{0, 0, 0, 0, 0, 0, 0}}, + {"array_str_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"}}, + {"array_str_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch}}, {"array_str_date_1", model.DateTime, []time.Time{bdLocalDate, bdLocalDate.Add(24 * time.Hour), bdLocalDate.Add(48 * time.Hour)}}, {"array_str_date_2", model.DateTime, []time.Time{bdLocalDate, bdLocalDate.Add(24 * time.Hour), bdLocalDate.Add(48 * time.Hour)}}, @@ -494,15 +500,20 @@ func TestParserArray(t *testing.T) { for i := range names { name := names[i] metric := metrics[name] + var skipped []string for j := range testCases { - if name == "csv" && testCases[j].Field == "array_obj" { - // csv parser doesn't support object array yet. + var v interface{} + desc := fmt.Sprintf(`%s.GetArray("%s", %d)`, name, testCases[j].Field, testCases[j].Type) + if (name == "gjson" && testCases[j].Field == "array_num_float") || + (name == "csv" && sliceContains([]string{"array_num_float", "array_str_float"}, testCases[j].Field)) { + skipped = append(skipped, desc) continue } - var v interface{} - desc := fmt.Sprintf(`%s GetArray("%s", %d)`, name, testCases[j].Field, testCases[j].Type) v = metric.GetArray(testCases[j].Field, testCases[j].Type) - require.Equal(t, testCases[j].ExpVal, v, desc) + assert.Equal(t, testCases[j].ExpVal, v, desc) + } + if skipped != nil { + log.Printf("Skipped %d cases incompatible with fastjson parser: %v\n", len(skipped), strings.Join(skipped, ", ")) } } } From 882987bdddc6eb9fabd4784e9158119a7f09b090 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 14 Oct 2021 15:29:56 +0800 Subject: [PATCH 171/404] added __series_id --- model/message.go | 22 ++++++++++-- output/clickhouse.go | 86 +++++++++++++++++++++++++++++++++++--------- task/task.go | 15 ++++---- 3 files changed, 97 insertions(+), 26 deletions(-) diff --git a/model/message.go b/model/message.go index c829b763..469d2edd 100644 --- a/model/message.go +++ b/model/message.go @@ -8,6 +8,7 @@ import ( "sync/atomic" "time" + "github.com/cespare/xxhash/v2" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/statistics" ) @@ -166,10 +167,16 @@ func PutRow(r *Row) { rowPool.Put(r) } -func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType) (row *Row) { +func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType, idxSeriesID int) (row *Row) { row = GetRow() - for _, dim := range dims { - if strings.HasPrefix(dim.Name, "__kafka") { + var dig *xxhash.Digest + if idxSeriesID >= 0 { + dig = xxhash.New() + } + for i, dim := range dims { + if i == idxSeriesID { + *row = append(*row, uint64(0)) + } else if strings.HasPrefix(dim.Name, "__kafka") { if strings.HasSuffix(dim.Name, "_topic") { *row = append(*row, msg.Topic) } else if strings.HasSuffix(dim.Name, "_partition") { @@ -180,7 +187,16 @@ func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType) (row * } else { val := GetValueByType(metric, dim) *row = append(*row, val) + if idxSeriesID >= 0 && dim.Type == String && val != nil { + dig.WriteString("###") + dig.WriteString(dim.Name) + dig.WriteString("###") + dig.WriteString(val.(string)) + } } } + if idxSeriesID >= 0 { + (*row)[idxSeriesID] = dig.Sum64() + } return } diff --git a/output/clickhouse.go b/output/clickhouse.go index 34e1cc1a..92b0b0ba 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -28,6 +28,7 @@ import ( "time" "github.com/ClickHouse/clickhouse-go" + "github.com/RoaringBitmap/roaring/roaring64" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" @@ -51,9 +52,13 @@ var ( type ClickHouse struct { Dims []*model.ColumnWithType Dms []string + IdxSerID int + idxLabels []int cfg *config.Config taskCfg *config.TaskConfig prepareSQL string + promSerSQL string + bmSeries *roaring64.Bitmap numFlying int32 mux sync.Mutex taskDone *sync.Cond @@ -98,29 +103,19 @@ func (c *ClickHouse) Send(batch *model.Batch) { }) } -// Write kvs to clickhouse -func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) (err error) { +func writeRows(prepareSQL string, rows model.Rows, conn *sql.DB) (err error) { var stmt *sql.Stmt var tx *sql.Tx - if len(*batch.Rows) == 0 { - return - } - - var conn *sql.DB - if conn, *dbVer, err = sc.NextGoodReplica(*dbVer); err != nil { - return - } - if tx, err = conn.Begin(); err != nil { - err = errors.Wrapf(err, "conn.Begin") + err = errors.Wrapf(err, "conn.Begin %s", prepareSQL) return } - if stmt, err = tx.Prepare(c.prepareSQL); err != nil { - err = errors.Wrapf(err, "tx.Prepare %s", c.prepareSQL) + if stmt, err = tx.Prepare(prepareSQL); err != nil { + err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) return } defer stmt.Close() - for _, row := range *batch.Rows { + for _, row := range rows { if _, err = stmt.Exec(*row...); err != nil { err = errors.Wrapf(err, "stmt.Exec") break @@ -134,6 +129,58 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( err = errors.Wrapf(err, "tx.Commit") return } + return +} + +// Write a batch to clickhouse +func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) (err error) { + if len(*batch.Rows) == 0 { + return + } + var conn *sql.DB + if conn, *dbVer, err = sc.NextGoodReplica(*dbVer); err != nil { + return + } + + if err = writeRows(c.prepareSQL, *batch.Rows, conn); err != nil { + return + } + if c.IdxSerID >= 0 { + var seriesRows model.Rows + var labels []string + c.mux.Lock() + for _, row := range *batch.Rows { + seriesID := (*row)[c.IdxSerID].(uint64) + if !c.bmSeries.Contains(seriesID) { + seriesRow := make(model.Row, 2+len(c.idxLabels)) //__series_id, lables, ... + if labels == nil { + labels = make([]string, len(c.idxLabels)) + } + seriesRow[0] = seriesID + for i, idxLabel := range c.idxLabels { + seriesRow[2+i] = (*row)[idxLabel] + labelKey := c.Dims[idxLabel].Name + labelVal := (*row)[idxLabel].(string) + labels[i] = fmt.Sprintf(`"%s": "%s"`, labelKey, labelVal) + } + seriesRow[1] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) + seriesRows = append(seriesRows, &seriesRow) + } + } + c.mux.Unlock() + if len(seriesRows) != 0 { + if err = writeRows(c.promSerSQL, seriesRows, conn); err != nil { + return + } + c.mux.Lock() + for _, seriesRow := range seriesRows { + seriesID := (*seriesRow)[0].(uint64) + c.bmSeries.Add(seriesID) + } + c.mux.Unlock() + } + } + statistics.FlushMsgsTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) return } @@ -245,8 +292,15 @@ func (c *ClickHouse) initSchema() (err error) { } c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.taskCfg.TableName + " (" + strings.Join(quotedDms, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" - util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.taskCfg.Name)) + + c.IdxSerID = -1 + for i, dim := range c.Dims { + if dim.Name == "__series_id" { + c.IdxSerID = i + } + } + return nil } diff --git a/task/task.go b/task/task.go index d40f5125..e2773c62 100644 --- a/task/task.go +++ b/task/task.go @@ -38,12 +38,13 @@ import ( // TaskService holds the configuration for each task type Service struct { sync.Mutex - inputer input.Inputer - clickhouse *output.ClickHouse - pp *parser.Pool - cfg *config.Config - taskCfg *config.TaskConfig - dims []*model.ColumnWithType + inputer input.Inputer + clickhouse *output.ClickHouse + pp *parser.Pool + cfg *config.Config + taskCfg *config.TaskConfig + dims []*model.ColumnWithType + idxSeriesID int knownKeys sync.Map newKeys sync.Map @@ -273,7 +274,7 @@ func (service *Service) put(msg model.InputMessage) { msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - row = model.MetricToRow(metric, msg, service.dims) + row = model.MetricToRow(metric, msg, service.dims, service.idxSeriesID) if taskCfg.DynamicSchema.Enable { foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys) } From 65c3e5c63fdba28490d0946f27ee01328ea8ab19 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 15 Oct 2021 17:59:27 +0800 Subject: [PATCH 172/404] series draft 2 --- output/clickhouse.go | 185 ++++++++++++++++++++------------------ output/clickhouse_util.go | 111 +++++++++++++++++++++++ task/task.go | 16 ++-- 3 files changed, 218 insertions(+), 94 deletions(-) create mode 100644 output/clickhouse_util.go diff --git a/output/clickhouse.go b/output/clickhouse.go index 92b0b0ba..a3fbaf28 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -27,7 +27,6 @@ import ( "sync" "time" - "github.com/ClickHouse/clickhouse-go" "github.com/RoaringBitmap/roaring/roaring64" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" @@ -39,6 +38,7 @@ import ( ) var ( + ErrTblNotExist = errors.Errorf("table doesn't exist") selectSQLTemplate = `select name, type, default_kind from system.columns where database = '%s' and table = '%s'` lowCardinalityRegexp = regexp.MustCompile(`LowCardinality\((.+)\)`) @@ -58,6 +58,7 @@ type ClickHouse struct { taskCfg *config.TaskConfig prepareSQL string promSerSQL string + seriesTbl string bmSeries *roaring64.Bitmap numFlying int32 mux sync.Mutex @@ -103,35 +104,6 @@ func (c *ClickHouse) Send(batch *model.Batch) { }) } -func writeRows(prepareSQL string, rows model.Rows, conn *sql.DB) (err error) { - var stmt *sql.Stmt - var tx *sql.Tx - if tx, err = conn.Begin(); err != nil { - err = errors.Wrapf(err, "conn.Begin %s", prepareSQL) - return - } - if stmt, err = tx.Prepare(prepareSQL); err != nil { - err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) - return - } - defer stmt.Close() - for _, row := range rows { - if _, err = stmt.Exec(*row...); err != nil { - err = errors.Wrapf(err, "stmt.Exec") - break - } - } - if err != nil { - _ = tx.Rollback() - return err - } - if err = tx.Commit(); err != nil { - err = errors.Wrapf(err, "tx.Commit") - return - } - return -} - // Write a batch to clickhouse func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) (err error) { if len(*batch.Rows) == 0 { @@ -185,22 +157,6 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( return } -func shouldReconnect(err error, sc *pool.ShardConn) bool { - var exp *clickhouse.Exception - if errors.As(err, &exp) { - util.Logger.Error("this is an exception from clickhouse-server", zap.String("dsn", sc.GetDsn()), zap.Reflect("exception", exp)) - var replicaSpecific bool - for _, ec := range replicaSpecificErrorCodes { - if ec == exp.Code { - replicaSpecific = true - break - } - } - return replicaSpecific - } - return true -} - // LoopWrite will dead loop to write the records func (c *ClickHouse) loopWrite(batch *model.Batch) { var err error @@ -236,36 +192,94 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { } } +func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { + c.bmSeries = roaring64.New() + allSeriesSQL := fmt.Sprintf("SELECT __series_id FROM %s.dist_%s", c.cfg.Clickhouse.DB, c.seriesTbl) + var rs *sql.Rows + var seriesID uint64 + if rs, err = conn.Query(allSeriesSQL); err != nil { + err = errors.Wrapf(err, "") + return err + } + defer rs.Close() + for rs.Next() { + if err = rs.Scan(&seriesID); err != nil { + err = errors.Wrapf(err, "") + return err + } + c.bmSeries.Add(seriesID) + } + return +} + func (c *ClickHouse) initSchema() (err error) { + c.IdxSerID = -1 if c.taskCfg.AutoSchema { sc := pool.GetShardConn(0) var conn *sql.DB if conn, _, err = sc.NextGoodReplica(0); err != nil { return } - var rs *sql.Rows - if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, c.cfg.Clickhouse.DB, c.taskCfg.TableName)); err != nil { - err = errors.Wrapf(err, "") - return err + if c.Dims, err = getDims(c.cfg.Clickhouse.DB, c.taskCfg.TableName, c.taskCfg.ExcludeColumns, conn); err != nil { + return } - defer rs.Close() - - c.Dims = make([]*model.ColumnWithType, 0, 10) - var name, typ, defaultKind string - for rs.Next() { - if err = rs.Scan(&name, &typ, &defaultKind); err != nil { - err = errors.Wrapf(err, "") - return err - } - typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") - if !util.StringContains(c.taskCfg.ExcludeColumns, name) && defaultKind != "MATERIALIZED" { - tp, nullable := model.WhichType(typ) - c.Dims = append(c.Dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, SourceName: util.GetSourceName(name)}) + for i, dim := range c.Dims { + if dim.Name == "__series_id" { + c.IdxSerID = i } } - if len(c.Dims) == 0 { - err = errors.Errorf("Table %s.%s doesn't exist", c.cfg.Clickhouse.DB, c.taskCfg.TableName) - return + if c.IdxSerID >= 0 { + expSeriesDims := []*model.ColumnWithType{ + {Name: "__series_id", Type: model.Int}, + {Name: "labels", Type: model.String}, + } + expSeriesDms := "__series_id UInt64, labels String" + expSeriesDmsQuoted := []string{"__series_id", "labels"} + for i, dim := range c.Dims { + if dim.Type == model.String { + c.idxLabels = append(c.idxLabels, i) + expSeriesDims = append(expSeriesDims, dim) + if dim.Nullable { + expSeriesDms += fmt.Sprintf(", `%s` Nullable(String)", dim.Name) + } else { + expSeriesDms += fmt.Sprintf(", `%s` String", dim.Name) + } + expSeriesDmsQuoted = append(expSeriesDmsQuoted, fmt.Sprintf("`%s`", dim.Name)) + } + } + // Check the series table schema + c.seriesTbl = c.taskCfg.TableName + "_series" + var seriesDims []*model.ColumnWithType + if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.taskCfg.TableName, c.taskCfg.ExcludeColumns, conn); err != nil { + if errors.Is(err, ErrTblNotExist) { + err = errors.Wrapf(err, "Please create table %s(and distributed table) with the following columns: %s", c.seriesTbl, expSeriesDms) + return + } + return + } + if len(seriesDims) != 2+len(c.idxLabels) { + err = errors.Errorf("Missed %d columns in %s. Please modify table %s(and distributed table) to the following columns: %s", 2+len(c.idxLabels)-len(seriesDims), c.seriesTbl, c.seriesTbl, expSeriesDms) + return + } + for i, idxLabel := range c.idxLabels { + dim := c.Dims[idxLabel] + seriesDim := seriesDims[i] + if seriesDim.Name != dim.Name || seriesDim.Type != model.String { + err = errors.Errorf("Column %s(#%d) in %s doesn't math expection. Please modify table %s(and distributed table) to the following columns: %s", seriesDim.Name, i, c.seriesTbl, c.seriesTbl, expSeriesDms) + return + } + } + // Generate SQL for series INSERT + var params = make([]string, len(c.Dims)) + for i := range params { + params[i] = "?" + } + c.promSerSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.seriesTbl + " (" + strings.Join(expSeriesDmsQuoted, ",") + ") " + + "VALUES (" + strings.Join(params, ",") + ")" + // Initialize bmSeries + if err = c.initBmSeries(conn); err != nil { + return + } } } else { c.Dims = make([]*model.ColumnWithType, 0) @@ -293,14 +307,6 @@ func (c *ClickHouse) initSchema() (err error) { c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.taskCfg.TableName + " (" + strings.Join(quotedDms, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.taskCfg.Name)) - - c.IdxSerID = -1 - for i, dim := range c.Dims { - if dim.Name == "__series_id" { - c.IdxSerID = i - } - } - return nil } @@ -329,8 +335,9 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return false } strKey, _ := key.(string) + intVal := value.(int) var strVal string - switch value.(int) { + switch intVal { case model.Int: strVal = "Nullable(Int64)" case model.Float: @@ -353,6 +360,10 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) queries = append(queries, query) + if c.IdxSerID >= 0 && intVal == model.String { + query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, c.seriesTbl, onCluster, strKey, strVal) + queries = append(queries, query) + } return true }) if err != nil { @@ -360,15 +371,17 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } sort.Strings(queries) if chCfg.Cluster != "" { - var distTbls []string - if distTbls, err = c.getDistTbls(); err != nil { - return - } - for _, distTbl := range distTbls { - queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s %s", chCfg.DB, distTbl, onCluster)) - queries = append(queries, fmt.Sprintf("CREATE TABLE %s.%s %s AS %s ENGINE = Distributed(%s, %s, %s);", - chCfg.DB, distTbl, onCluster, taskCfg.TableName, - chCfg.Cluster, chCfg.DB, taskCfg.TableName)) + for _, table := range []string{taskCfg.TableName, c.seriesTbl} { + var distTbls []string + if distTbls, err = c.getDistTbls(table); err != nil { + return + } + for _, distTbl := range distTbls { + queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s %s", chCfg.DB, distTbl, onCluster)) + queries = append(queries, fmt.Sprintf("CREATE TABLE %s.%s %s AS %s ENGINE = Distributed(%s, %s, %s);", + chCfg.DB, distTbl, onCluster, table, + chCfg.Cluster, chCfg.DB, table)) + } } } sc := pool.GetShardConn(0) @@ -386,7 +399,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return } -func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { +func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { taskCfg := c.taskCfg chCfg := &c.cfg.Clickhouse sc := pool.GetShardConn(0) @@ -395,7 +408,7 @@ func (c *ClickHouse) getDistTbls() (distTbls []string, err error) { return } query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, - chCfg.DB, chCfg.DB, taskCfg.TableName) + chCfg.DB, chCfg.DB, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows *sql.Rows diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go new file mode 100644 index 00000000..f66f9da3 --- /dev/null +++ b/output/clickhouse_util.go @@ -0,0 +1,111 @@ +package output + +import ( + "database/sql" + "fmt" + "regexp" + + "github.com/ClickHouse/clickhouse-go" + "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/pool" + "github.com/housepower/clickhouse_sinker/util" + "github.com/pkg/errors" + "go.uber.org/zap" +) + +func shouldReconnect(err error, sc *pool.ShardConn) bool { + var exp *clickhouse.Exception + if errors.As(err, &exp) { + util.Logger.Error("this is an exception from clickhouse-server", zap.String("dsn", sc.GetDsn()), zap.Reflect("exception", exp)) + var replicaSpecific bool + for _, ec := range replicaSpecificErrorCodes { + if ec == exp.Code { + replicaSpecific = true + break + } + } + return replicaSpecific + } + return true +} + +func writeRows(prepareSQL string, rows model.Rows, conn *sql.DB) (err error) { + var stmt *sql.Stmt + var tx *sql.Tx + if tx, err = conn.Begin(); err != nil { + err = errors.Wrapf(err, "conn.Begin %s", prepareSQL) + return + } + if stmt, err = tx.Prepare(prepareSQL); err != nil { + err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) + return + } + defer stmt.Close() + for _, row := range rows { + if _, err = stmt.Exec(*row...); err != nil { + err = errors.Wrapf(err, "stmt.Exec") + break + } + } + if err != nil { + _ = tx.Rollback() + return err + } + if err = tx.Commit(); err != nil { + err = errors.Wrapf(err, "tx.Commit") + return + } + return +} + +func getDims(database, table string, excludedColumns []string, conn *sql.DB) (dims []*model.ColumnWithType, err error) { + var rs *sql.Rows + if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, database, table)); err != nil { + err = errors.Wrapf(err, "") + return + } + defer rs.Close() + + dims = make([]*model.ColumnWithType, 0, 10) + var name, typ, defaultKind string + for rs.Next() { + if err = rs.Scan(&name, &typ, &defaultKind); err != nil { + err = errors.Wrapf(err, "") + return + } + typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") + if !util.StringContains(excludedColumns, name) && defaultKind != "MATERIALIZED" { + tp, nullable := model.WhichType(typ) + dims = append(dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, SourceName: util.GetSourceName(name)}) + } + } + if len(dims) == 0 { + err = errors.Wrapf(ErrTblNotExist, "") + return + } + return +} + +func isReplicated(database, table string, conn *sql.DB) (yes bool, err error) { + var rs *sql.Rows + if rs, err = conn.Query(fmt.Sprintf("SHOW CREATE TABLE %s.%s", database, table)); err != nil { + err = errors.Wrapf(err, "") + return + } + defer rs.Close() + var statement string + var matched bool + for rs.Next() { + if err = rs.Scan(&statement); err != nil { + err = errors.Wrapf(err, "") + return + } + if matched, err = regexp.Match(`ENGINE\s*=\s*Replicated`, []byte(statement)); err != nil { + err = errors.Wrapf(err, "") + return + } + yes = matched + return + } + return +} diff --git a/task/task.go b/task/task.go index e2773c62..26098df1 100644 --- a/task/task.go +++ b/task/task.go @@ -38,13 +38,13 @@ import ( // TaskService holds the configuration for each task type Service struct { sync.Mutex - inputer input.Inputer - clickhouse *output.ClickHouse - pp *parser.Pool - cfg *config.Config - taskCfg *config.TaskConfig - dims []*model.ColumnWithType - idxSeriesID int + inputer input.Inputer + clickhouse *output.ClickHouse + pp *parser.Pool + cfg *config.Config + taskCfg *config.TaskConfig + dims []*model.ColumnWithType + idxSerID int knownKeys sync.Map newKeys sync.Map @@ -274,7 +274,7 @@ func (service *Service) put(msg model.InputMessage) { msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - row = model.MetricToRow(metric, msg, service.dims, service.idxSeriesID) + row = model.MetricToRow(metric, msg, service.dims, service.idxSerID) if taskCfg.DynamicSchema.Enable { foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys) } From 2e058c70431282e9ed3ecd11ee39fc36092d0e3a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 15 Oct 2021 21:44:48 +0800 Subject: [PATCH 173/404] series draft3 --- go.mod | 1 + go.sum | 6 ++++++ output/clickhouse.go | 45 ++++++++++++++++++++++----------------- output/clickhouse_util.go | 2 +- 4 files changed, 33 insertions(+), 21 deletions(-) diff --git a/go.mod b/go.mod index 3e0a53e6..0926e1b6 100644 --- a/go.mod +++ b/go.mod @@ -4,6 +4,7 @@ go 1.14 require ( github.com/ClickHouse/clickhouse-go v1.4.7 + github.com/RoaringBitmap/roaring v0.9.4 github.com/Shopify/sarama v1.29.1 github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a github.com/cespare/xxhash/v2 v2.1.2 diff --git a/go.sum b/go.sum index 6de8427a..3760d9e2 100644 --- a/go.sum +++ b/go.sum @@ -5,6 +5,8 @@ github.com/ClickHouse/clickhouse-go v1.4.7 h1:NNZQmlW8dVxGn19pF65BmWr0vq8Pj5Iy8y github.com/ClickHouse/clickhouse-go v1.4.7/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= +github.com/RoaringBitmap/roaring v0.9.4 h1:ckvZSX5gwCRaJYBNe7syNawCU5oruY9gQmjXlp4riwo= +github.com/RoaringBitmap/roaring v0.9.4/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= github.com/Shopify/sarama v1.29.1 h1:wBAacXbYVLmWieEA/0X/JagDdCZ8NVFOfS6l6+2u5S0= github.com/Shopify/sarama v1.29.1/go.mod h1:mdtqvCSg8JOxk8PmpTNGyo6wzd4BMm4QXSfDnTXmgkE= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= @@ -23,6 +25,8 @@ github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24 github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bits-and-blooms/bitset v1.2.0 h1:Kn4yilvwNtMACtf1eYDlG8H77R07mZSPbMjLyS07ChA= +github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAKk= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= @@ -171,6 +175,8 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/mschoch/smat v0.2.0 h1:8imxQsjDm8yFEAVBe7azKmKSgzSkZXDuKkSq9374khM= +github.com/mschoch/smat v0.2.0/go.mod h1:kc9mz7DoBKqDyiRL7VZN8KvXQMWeTaVnttLRXOlotKw= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nacos-group/nacos-sdk-go v1.0.7 h1:Am1tJFe7GUTNCREKsZ5ok0H2OspHDRmRcsxn7DiSwhA= diff --git a/output/clickhouse.go b/output/clickhouse.go index a3fbaf28..a5357404 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -119,21 +119,19 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( } if c.IdxSerID >= 0 { var seriesRows model.Rows - var labels []string c.mux.Lock() for _, row := range *batch.Rows { seriesID := (*row)[c.IdxSerID].(uint64) if !c.bmSeries.Contains(seriesID) { seriesRow := make(model.Row, 2+len(c.idxLabels)) //__series_id, lables, ... - if labels == nil { - labels = make([]string, len(c.idxLabels)) - } + labels := make([]string, 0) seriesRow[0] = seriesID for i, idxLabel := range c.idxLabels { seriesRow[2+i] = (*row)[idxLabel] labelKey := c.Dims[idxLabel].Name - labelVal := (*row)[idxLabel].(string) - labels[i] = fmt.Sprintf(`"%s": "%s"`, labelKey, labelVal) + if labelVal, ok := (*row)[idxLabel].(string); ok { + labels = append(labels, fmt.Sprintf(`"%s": "%s"`, labelKey, labelVal)) + } } seriesRow[1] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) seriesRows = append(seriesRows, &seriesRow) @@ -193,8 +191,15 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { } func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { - c.bmSeries = roaring64.New() - allSeriesSQL := fmt.Sprintf("SELECT __series_id FROM %s.dist_%s", c.cfg.Clickhouse.DB, c.seriesTbl) + var seriesDistTbls []string + if seriesDistTbls, err = c.getDistTbls(c.seriesTbl); err != nil { + return + } + if seriesDistTbls == nil { + err = errors.Wrapf(err, "Please create distributed table for %s", c.seriesTbl) + return + } + allSeriesSQL := fmt.Sprintf("SELECT __series_id FROM %s.%s", c.cfg.Clickhouse.DB, seriesDistTbls[0]) var rs *sql.Rows var seriesID uint64 if rs, err = conn.Query(allSeriesSQL); err != nil { @@ -202,6 +207,7 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { return err } defer rs.Close() + c.bmSeries = roaring64.New() for rs.Next() { if err = rs.Scan(&seriesID); err != nil { err = errors.Wrapf(err, "") @@ -209,6 +215,7 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { } c.bmSeries.Add(seriesID) } + util.Logger.Info(fmt.Sprintf("loaded %d series from %v", c.bmSeries.GetCardinality(), c.seriesTbl), zap.String("task", c.taskCfg.Name)) return } @@ -250,25 +257,23 @@ func (c *ClickHouse) initSchema() (err error) { // Check the series table schema c.seriesTbl = c.taskCfg.TableName + "_series" var seriesDims []*model.ColumnWithType - if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.taskCfg.TableName, c.taskCfg.ExcludeColumns, conn); err != nil { + if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.seriesTbl, nil, conn); err != nil { if errors.Is(err, ErrTblNotExist) { err = errors.Wrapf(err, "Please create table %s(and distributed table) with the following columns: %s", c.seriesTbl, expSeriesDms) return } return } - if len(seriesDims) != 2+len(c.idxLabels) { - err = errors.Errorf("Missed %d columns in %s. Please modify table %s(and distributed table) to the following columns: %s", 2+len(c.idxLabels)-len(seriesDims), c.seriesTbl, c.seriesTbl, expSeriesDms) - return - } - for i, idxLabel := range c.idxLabels { - dim := c.Dims[idxLabel] - seriesDim := seriesDims[i] - if seriesDim.Name != dim.Name || seriesDim.Type != model.String { - err = errors.Errorf("Column %s(#%d) in %s doesn't math expection. Please modify table %s(and distributed table) to the following columns: %s", seriesDim.Name, i, c.seriesTbl, c.seriesTbl, expSeriesDms) + for i, expDim := range expSeriesDims { + if i < len(seriesDims) && (seriesDims[i].Name != expDim.Name || seriesDims[i].Type != expDim.Type) { + err = errors.Errorf("Column #%d of %s expect to be {%s, %v}, actual {%s, %v}. Please modify table %s(and distributed table) to the following columns: %s", i, expDim.Name, expDim.Type, seriesDims[i].Name, seriesDims[i].Type, c.seriesTbl, c.seriesTbl, expSeriesDms) return } } + if len(seriesDims) != len(expSeriesDims) { + err = errors.Errorf("Columns number of %s expect to be %d, actual %d. Please modify table %s(and distributed table) to the following columns: %s", c.seriesTbl, len(expSeriesDims), len(seriesDims), c.seriesTbl, c.seriesTbl, expSeriesDms) + return + } // Generate SQL for series INSERT var params = make([]string, len(c.Dims)) for i := range params { @@ -407,8 +412,8 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { if conn, _, err = sc.NextGoodReplica(0); err != nil { return } - query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed.*\'%s\',\s*\'%s\'')`, - chCfg.DB, chCfg.DB, table) + query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed\(\'%s\', \'%s\', \'%s\'\)')`, + chCfg.DB, chCfg.Cluster, chCfg.DB, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows *sql.Rows diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index f66f9da3..a1714042 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -80,7 +80,7 @@ func getDims(database, table string, excludedColumns []string, conn *sql.DB) (di } } if len(dims) == 0 { - err = errors.Wrapf(ErrTblNotExist, "") + err = errors.Wrapf(ErrTblNotExist, "%s.%s", database, table) return } return From dffc6bd8901984a4f4b79465942a1a854bba03e2 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 16 Oct 2021 16:15:53 +0800 Subject: [PATCH 174/404] fixed c.idxLabels, handle missed labels in series table --- output/clickhouse.go | 157 +++++++++++++++++++++++++------------- output/clickhouse_util.go | 18 +++++ 2 files changed, 121 insertions(+), 54 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index a5357404..6e66e9d7 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -59,10 +59,14 @@ type ClickHouse struct { prepareSQL string promSerSQL string seriesTbl string - bmSeries *roaring64.Bitmap - numFlying int32 - mux sync.Mutex - taskDone *sync.Cond + + distMetricTbls []string + distSeriesTbls []string + + bmSeries *roaring64.Bitmap + numFlying int32 + mux sync.Mutex + taskDone *sync.Cond } // NewClickHouse new a clickhouse instance @@ -191,15 +195,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { } func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { - var seriesDistTbls []string - if seriesDistTbls, err = c.getDistTbls(c.seriesTbl); err != nil { - return - } - if seriesDistTbls == nil { - err = errors.Wrapf(err, "Please create distributed table for %s", c.seriesTbl) - return - } - allSeriesSQL := fmt.Sprintf("SELECT __series_id FROM %s.%s", c.cfg.Clickhouse.DB, seriesDistTbls[0]) + allSeriesSQL := fmt.Sprintf("SELECT __series_id FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) var rs *sql.Rows var seriesID uint64 if rs, err = conn.Query(allSeriesSQL); err != nil { @@ -220,6 +216,7 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { } func (c *ClickHouse) initSchema() (err error) { + chCfg := &c.cfg.Clickhouse c.IdxSerID = -1 if c.taskCfg.AutoSchema { sc := pool.GetShardConn(0) @@ -236,50 +233,105 @@ func (c *ClickHouse) initSchema() (err error) { } } if c.IdxSerID >= 0 { + // Check distributed series table + c.idxLabels = nil + c.seriesTbl = c.taskCfg.TableName + "_series" + var onCluster string + if chCfg.Cluster != "" { + onCluster = fmt.Sprintf("ON CLUSTER %s", chCfg.Cluster) + if c.distSeriesTbls, err = c.getDistTbls(c.seriesTbl); err != nil { + return + } + if c.distSeriesTbls == nil { + err = errors.Errorf("Please create distributed table for %s.", c.seriesTbl) + return + } + } + // Check the series table schema expSeriesDims := []*model.ColumnWithType{ {Name: "__series_id", Type: model.Int}, {Name: "labels", Type: model.String}, } - expSeriesDms := "__series_id UInt64, labels String" - expSeriesDmsQuoted := []string{"__series_id", "labels"} - for i, dim := range c.Dims { - if dim.Type == model.String { - c.idxLabels = append(c.idxLabels, i) - expSeriesDims = append(expSeriesDims, dim) - if dim.Nullable { - expSeriesDms += fmt.Sprintf(", `%s` Nullable(String)", dim.Name) - } else { - expSeriesDms += fmt.Sprintf(", `%s` String", dim.Name) - } - expSeriesDmsQuoted = append(expSeriesDmsQuoted, fmt.Sprintf("`%s`", dim.Name)) - } - } - // Check the series table schema - c.seriesTbl = c.taskCfg.TableName + "_series" var seriesDims []*model.ColumnWithType if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.seriesTbl, nil, conn); err != nil { if errors.Is(err, ErrTblNotExist) { - err = errors.Wrapf(err, "Please create table %s(and distributed table) with the following columns: %s", c.seriesTbl, expSeriesDms) + err = errors.Wrapf(err, "Please create series table for %s", c.cfg.Clickhouse.DB, c.taskCfg.TableName) return } return } - for i, expDim := range expSeriesDims { - if i < len(seriesDims) && (seriesDims[i].Name != expDim.Name || seriesDims[i].Type != expDim.Type) { - err = errors.Errorf("Column #%d of %s expect to be {%s, %v}, actual {%s, %v}. Please modify table %s(and distributed table) to the following columns: %s", i, expDim.Name, expDim.Type, seriesDims[i].Name, seriesDims[i].Type, c.seriesTbl, c.seriesTbl, expSeriesDms) - return + var badFirst bool + if len(seriesDims) < len(expSeriesDims) { + badFirst = true + } else { + for i := range expSeriesDims { + if seriesDims[i].Name != expSeriesDims[i].Name || + seriesDims[i].Type != expSeriesDims[i].Type { + badFirst = true + break + } } } - if len(seriesDims) != len(expSeriesDims) { - err = errors.Errorf("Columns number of %s expect to be %d, actual %d. Please modify table %s(and distributed table) to the following columns: %s", c.seriesTbl, len(expSeriesDims), len(seriesDims), c.seriesTbl, c.seriesTbl, expSeriesDms) + if badFirst { + err = errors.Errorf(`First two columns of %s are expect to be "__series_id UInt64, labels String".`, c.seriesTbl) return } + for i, serDim := range seriesDims { + if i < 2 { + continue + } + idxLabel := -1 + for j, dim := range c.Dims { + if serDim.Name == dim.Name && serDim.Type == dim.Type { + idxLabel = j + break + } + } + if idxLabel < 0 { + err = errors.Errorf("Column %s exists in %s but not in %s", serDim.Name, c.seriesTbl, c.taskCfg.TableName) + return + } + c.idxLabels = append(c.idxLabels, idxLabel) + } + // Add missed columns to the series table + var missedLabels []string + for i, dim := range c.Dims { + if dim.Type != model.String { + continue + } + var found bool + for _, serDim := range seriesDims { + if serDim.Name == dim.Name { + found = true + break + } + } + if !found { + c.idxLabels = append(c.idxLabels, i) + seriesDims = append(seriesDims, dim) + missedLabels = append(missedLabels, dim.Name) + } + } + if missedLabels != nil { + for _, key := range missedLabels { + query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` Nullable(String)", chCfg.DB, c.seriesTbl, onCluster, key) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) + if _, err = conn.Exec(query); err != nil { + err = errors.Wrapf(err, query) + return + } + } + recreateDistTbls(chCfg.Cluster, chCfg.DB, c.seriesTbl, c.distSeriesTbls, conn) + } + // Generate SQL for series INSERT - var params = make([]string, len(c.Dims)) - for i := range params { + serDimsQuoted := make([]string, len(seriesDims)) + params := make([]string, len(seriesDims)) + for i, serDim := range seriesDims { + serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) params[i] = "?" } - c.promSerSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.seriesTbl + " (" + strings.Join(expSeriesDmsQuoted, ",") + ") " + + c.promSerSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.seriesTbl + " (" + strings.Join(serDimsQuoted, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" // Initialize bmSeries if err = c.initBmSeries(conn); err != nil { @@ -299,13 +351,16 @@ func (c *ClickHouse) initSchema() (err error) { } } // Generate SQL for INSERT + if c.distMetricTbls, err = c.getDistTbls(c.taskCfg.TableName); err != nil { + return + } c.Dms = make([]string, 0, len(c.Dims)) quotedDms := make([]string, 0, len(c.Dims)) for _, d := range c.Dims { c.Dms = append(c.Dms, d.Name) quotedDms = append(quotedDms, fmt.Sprintf("`%s`", d.Name)) } - var params = make([]string, len(c.Dims)) + params := make([]string, len(c.Dims)) for i := range params { params[i] = "?" } @@ -375,20 +430,6 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return } sort.Strings(queries) - if chCfg.Cluster != "" { - for _, table := range []string{taskCfg.TableName, c.seriesTbl} { - var distTbls []string - if distTbls, err = c.getDistTbls(table); err != nil { - return - } - for _, distTbl := range distTbls { - queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s %s", chCfg.DB, distTbl, onCluster)) - queries = append(queries, fmt.Sprintf("CREATE TABLE %s.%s %s AS %s ENGINE = Distributed(%s, %s, %s);", - chCfg.DB, distTbl, onCluster, table, - chCfg.Cluster, chCfg.DB, table)) - } - } - } sc := pool.GetShardConn(0) var conn *sql.DB if conn, _, err = sc.NextGoodReplica(0); err != nil { @@ -401,6 +442,14 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return } } + if chCfg.Cluster != "" { + if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, taskCfg.TableName, c.distMetricTbls, conn); err != nil { + return + } + if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, c.seriesTbl, c.distSeriesTbls, conn); err != nil { + return + } + } return } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index a1714042..dedbcb7c 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -86,6 +86,24 @@ func getDims(database, table string, excludedColumns []string, conn *sql.DB) (di return } +func recreateDistTbls(cluster, database, table string, distTbls []string, conn *sql.DB) (err error) { + var queries []string + for _, distTbl := range distTbls { + queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s ON CLUSTER %s", database, distTbl, cluster)) + queries = append(queries, fmt.Sprintf("CREATE TABLE %s.%s ON CLUSTER %s AS %s ENGINE = Distributed(%s, %s, %s);", + database, distTbl, cluster, table, + cluster, database, table)) + } + for _, query := range queries { + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) + if _, err = conn.Exec(query); err != nil { + err = errors.Wrapf(err, "") + return + } + } + return +} + func isReplicated(database, table string, conn *sql.DB) (yes bool, err error) { var rs *sql.Rows if rs, err = conn.Query(fmt.Sprintf("SHOW CREATE TABLE %s.%s", database, table)); err != nil { From 47fe17239057b3e9118bda32ed619884852a912e Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 17 Oct 2021 21:21:13 +0800 Subject: [PATCH 175/404] removed statistics_test.go --- statistics/statistics_test.go | 41 ----------------------------------- 1 file changed, 41 deletions(-) delete mode 100755 statistics/statistics_test.go diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go deleted file mode 100755 index c32484f7..00000000 --- a/statistics/statistics_test.go +++ /dev/null @@ -1,41 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ -package statistics - -import ( - "fmt" - "testing" - "time" - - "github.com/housepower/clickhouse_sinker/util" - "github.com/stretchr/testify/require" -) - -func TestPusher(t *testing.T) { - util.InitLogger([]string{"stdout"}) - addrs := []string{"172.24.25.1:9091", "172.24.25.2:9091"} - interval := 1 - selfIP, _ := util.GetOutboundIP() - selfPort := util.GetSpareTCPPort(1024) - selfAddr := fmt.Sprintf("%s:%d", selfIP, selfPort) - pusher := NewPusher(addrs, interval, selfAddr) - - err := pusher.Init() - require.Nilf(t, err, "pusher init failed") - - go pusher.Run() - time.Sleep(10 * time.Second) - pusher.Stop() -} From 26f8ca799e1376489106e9e0ca2d997d8dec4f6c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 17 Oct 2021 22:41:43 +0800 Subject: [PATCH 176/404] make lint happy --- .github/workflows/lint.yml | 2 +- Makefile | 2 +- model/message.go | 8 +- output/clickhouse.go | 214 +++++++++++++++++++------------------ output/clickhouse_util.go | 25 ----- 5 files changed, 117 insertions(+), 134 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 9feebb15..d8aeba91 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -20,7 +20,7 @@ jobs: # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true diff --git a/Makefile b/Makefile index 236042d5..e3c574d8 100644 --- a/Makefile +++ b/Makefile @@ -25,6 +25,6 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.json diff --git a/model/message.go b/model/message.go index 469d2edd..bfffbfcf 100644 --- a/model/message.go +++ b/model/message.go @@ -188,10 +188,10 @@ func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType, idxSer val := GetValueByType(metric, dim) *row = append(*row, val) if idxSeriesID >= 0 && dim.Type == String && val != nil { - dig.WriteString("###") - dig.WriteString(dim.Name) - dig.WriteString("###") - dig.WriteString(val.(string)) + _, _ = dig.WriteString("###") + _, _ = dig.WriteString(dim.Name) + _, _ = dig.WriteString("###") + _, _ = dig.WriteString(val.(string)) } } } diff --git a/output/clickhouse.go b/output/clickhouse.go index 6e66e9d7..04d2a2bb 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -215,8 +215,117 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { return } -func (c *ClickHouse) initSchema() (err error) { +func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { chCfg := &c.cfg.Clickhouse + // Check distributed series table + c.idxLabels = nil + c.seriesTbl = c.taskCfg.TableName + "_series" + var onCluster string + if chCfg.Cluster != "" { + onCluster = fmt.Sprintf("ON CLUSTER %s", chCfg.Cluster) + if c.distSeriesTbls, err = c.getDistTbls(c.seriesTbl); err != nil { + return + } + if c.distSeriesTbls == nil { + err = errors.Errorf("Please create distributed table for %s.", c.seriesTbl) + return + } + } + // Check the series table schema + expSeriesDims := []*model.ColumnWithType{ + {Name: "__series_id", Type: model.Int}, + {Name: "labels", Type: model.String}, + } + var seriesDims []*model.ColumnWithType + if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.seriesTbl, nil, conn); err != nil { + if errors.Is(err, ErrTblNotExist) { + err = errors.Wrapf(err, "Please create series table for %s.%s", c.cfg.Clickhouse.DB, c.taskCfg.TableName) + return + } + return + } + var badFirst bool + if len(seriesDims) < len(expSeriesDims) { + badFirst = true + } else { + for i := range expSeriesDims { + if seriesDims[i].Name != expSeriesDims[i].Name || + seriesDims[i].Type != expSeriesDims[i].Type { + badFirst = true + break + } + } + } + if badFirst { + err = errors.Errorf(`First two columns of %s are expect to be "__series_id UInt64, labels String".`, c.seriesTbl) + return + } + for i, serDim := range seriesDims { + if i < 2 { + continue + } + idxLabel := -1 + for j, dim := range c.Dims { + if serDim.Name == dim.Name && serDim.Type == dim.Type { + idxLabel = j + break + } + } + if idxLabel < 0 { + err = errors.Errorf("Column %s exists in %s but not in %s", serDim.Name, c.seriesTbl, c.taskCfg.TableName) + return + } + c.idxLabels = append(c.idxLabels, idxLabel) + } + // Add missed columns to the series table + var missedLabels []string + for i, dim := range c.Dims { + if dim.Type != model.String { + continue + } + var found bool + for _, serDim := range seriesDims { + if serDim.Name == dim.Name { + found = true + break + } + } + if !found { + c.idxLabels = append(c.idxLabels, i) + seriesDims = append(seriesDims, dim) + missedLabels = append(missedLabels, dim.Name) + } + } + if missedLabels != nil { + for _, key := range missedLabels { + query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` Nullable(String)", chCfg.DB, c.seriesTbl, onCluster, key) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) + if _, err = conn.Exec(query); err != nil { + err = errors.Wrapf(err, query) + return + } + } + if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, c.seriesTbl, c.distSeriesTbls, conn); err != nil { + return + } + } + // Generate SQL for series INSERT + serDimsQuoted := make([]string, len(seriesDims)) + params := make([]string, len(seriesDims)) + for i, serDim := range seriesDims { + serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) + params[i] = "?" + } + c.promSerSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.seriesTbl + " (" + strings.Join(serDimsQuoted, ",") + ") " + + "VALUES (" + strings.Join(params, ",") + ")" + // Initialize bmSeries + if err = c.initBmSeries(conn); err != nil { + return + } + return +} + +func (c *ClickHouse) initSchema() (err error) { c.IdxSerID = -1 if c.taskCfg.AutoSchema { sc := pool.GetShardConn(0) @@ -233,108 +342,7 @@ func (c *ClickHouse) initSchema() (err error) { } } if c.IdxSerID >= 0 { - // Check distributed series table - c.idxLabels = nil - c.seriesTbl = c.taskCfg.TableName + "_series" - var onCluster string - if chCfg.Cluster != "" { - onCluster = fmt.Sprintf("ON CLUSTER %s", chCfg.Cluster) - if c.distSeriesTbls, err = c.getDistTbls(c.seriesTbl); err != nil { - return - } - if c.distSeriesTbls == nil { - err = errors.Errorf("Please create distributed table for %s.", c.seriesTbl) - return - } - } - // Check the series table schema - expSeriesDims := []*model.ColumnWithType{ - {Name: "__series_id", Type: model.Int}, - {Name: "labels", Type: model.String}, - } - var seriesDims []*model.ColumnWithType - if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.seriesTbl, nil, conn); err != nil { - if errors.Is(err, ErrTblNotExist) { - err = errors.Wrapf(err, "Please create series table for %s", c.cfg.Clickhouse.DB, c.taskCfg.TableName) - return - } - return - } - var badFirst bool - if len(seriesDims) < len(expSeriesDims) { - badFirst = true - } else { - for i := range expSeriesDims { - if seriesDims[i].Name != expSeriesDims[i].Name || - seriesDims[i].Type != expSeriesDims[i].Type { - badFirst = true - break - } - } - } - if badFirst { - err = errors.Errorf(`First two columns of %s are expect to be "__series_id UInt64, labels String".`, c.seriesTbl) - return - } - for i, serDim := range seriesDims { - if i < 2 { - continue - } - idxLabel := -1 - for j, dim := range c.Dims { - if serDim.Name == dim.Name && serDim.Type == dim.Type { - idxLabel = j - break - } - } - if idxLabel < 0 { - err = errors.Errorf("Column %s exists in %s but not in %s", serDim.Name, c.seriesTbl, c.taskCfg.TableName) - return - } - c.idxLabels = append(c.idxLabels, idxLabel) - } - // Add missed columns to the series table - var missedLabels []string - for i, dim := range c.Dims { - if dim.Type != model.String { - continue - } - var found bool - for _, serDim := range seriesDims { - if serDim.Name == dim.Name { - found = true - break - } - } - if !found { - c.idxLabels = append(c.idxLabels, i) - seriesDims = append(seriesDims, dim) - missedLabels = append(missedLabels, dim.Name) - } - } - if missedLabels != nil { - for _, key := range missedLabels { - query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` Nullable(String)", chCfg.DB, c.seriesTbl, onCluster, key) - util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) - if _, err = conn.Exec(query); err != nil { - err = errors.Wrapf(err, query) - return - } - } - recreateDistTbls(chCfg.Cluster, chCfg.DB, c.seriesTbl, c.distSeriesTbls, conn) - } - - // Generate SQL for series INSERT - serDimsQuoted := make([]string, len(seriesDims)) - params := make([]string, len(seriesDims)) - for i, serDim := range seriesDims { - serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) - params[i] = "?" - } - c.promSerSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.seriesTbl + " (" + strings.Join(serDimsQuoted, ",") + ") " + - "VALUES (" + strings.Join(params, ",") + ")" - // Initialize bmSeries - if err = c.initBmSeries(conn); err != nil { + if err = c.initSeriesSchema(conn); err != nil { return } } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index dedbcb7c..96fa898a 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -3,7 +3,6 @@ package output import ( "database/sql" "fmt" - "regexp" "github.com/ClickHouse/clickhouse-go" "github.com/housepower/clickhouse_sinker/model" @@ -103,27 +102,3 @@ func recreateDistTbls(cluster, database, table string, distTbls []string, conn * } return } - -func isReplicated(database, table string, conn *sql.DB) (yes bool, err error) { - var rs *sql.Rows - if rs, err = conn.Query(fmt.Sprintf("SHOW CREATE TABLE %s.%s", database, table)); err != nil { - err = errors.Wrapf(err, "") - return - } - defer rs.Close() - var statement string - var matched bool - for rs.Next() { - if err = rs.Scan(&statement); err != nil { - err = errors.Wrapf(err, "") - return - } - if matched, err = regexp.Match(`ENGINE\s*=\s*Replicated`, []byte(statement)); err != nil { - err = errors.Wrapf(err, "") - return - } - yes = matched - return - } - return -} From daf0a155731d54602c05f975eb3574c84cc2f1a5 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 18 Oct 2021 09:26:39 +0800 Subject: [PATCH 177/404] fixed duplicated series_id in series table --- output/clickhouse.go | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 04d2a2bb..d3513a17 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -126,7 +126,7 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( c.mux.Lock() for _, row := range *batch.Rows { seriesID := (*row)[c.IdxSerID].(uint64) - if !c.bmSeries.Contains(seriesID) { + if c.bmSeries.CheckedAdd(seriesID) { seriesRow := make(model.Row, 2+len(c.idxLabels)) //__series_id, lables, ... labels := make([]string, 0) seriesRow[0] = seriesID @@ -146,12 +146,6 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( if err = writeRows(c.promSerSQL, seriesRows, conn); err != nil { return } - c.mux.Lock() - for _, seriesRow := range seriesRows { - seriesID := (*seriesRow)[0].(uint64) - c.bmSeries.Add(seriesID) - } - c.mux.Unlock() } } From 8749237e8b7150c831bd47b6a03d4dd87563eaab Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 18 Oct 2021 11:36:53 +0800 Subject: [PATCH 178/404] recoverSeriesSchema, recoverSeriesData --- output/clickhouse.go | 198 +++++++++++++++++++++++++++++++------------ 1 file changed, 146 insertions(+), 52 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index d3513a17..ea4e9be4 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -108,6 +108,38 @@ func (c *ClickHouse) Send(batch *model.Batch) { }) } +func (c *ClickHouse) writeSeries(rows model.Rows, conn *sql.DB) (err error) { + var seriesRows model.Rows + if c.IdxSerID < 0 { + return + } + c.mux.Lock() + for _, row := range rows { + seriesID := (*row)[c.IdxSerID].(uint64) + if c.bmSeries.CheckedAdd(seriesID) { + seriesRow := make(model.Row, 2+len(c.idxLabels)) //__series_id, lables, ... + labels := make([]string, 0) + seriesRow[0] = seriesID + for i, idxLabel := range c.idxLabels { + seriesRow[2+i] = (*row)[idxLabel] + labelKey := c.Dims[idxLabel].Name + if labelVal, ok := (*row)[idxLabel].(string); ok { + labels = append(labels, fmt.Sprintf(`"%s": "%s"`, labelKey, labelVal)) + } + } + seriesRow[1] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) + seriesRows = append(seriesRows, &seriesRow) + } + } + c.mux.Unlock() + if len(seriesRows) != 0 { + if err = writeRows(c.promSerSQL, seriesRows, conn); err != nil { + return + } + } + return +} + // Write a batch to clickhouse func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) (err error) { if len(*batch.Rows) == 0 { @@ -121,32 +153,8 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( if err = writeRows(c.prepareSQL, *batch.Rows, conn); err != nil { return } - if c.IdxSerID >= 0 { - var seriesRows model.Rows - c.mux.Lock() - for _, row := range *batch.Rows { - seriesID := (*row)[c.IdxSerID].(uint64) - if c.bmSeries.CheckedAdd(seriesID) { - seriesRow := make(model.Row, 2+len(c.idxLabels)) //__series_id, lables, ... - labels := make([]string, 0) - seriesRow[0] = seriesID - for i, idxLabel := range c.idxLabels { - seriesRow[2+i] = (*row)[idxLabel] - labelKey := c.Dims[idxLabel].Name - if labelVal, ok := (*row)[idxLabel].(string); ok { - labels = append(labels, fmt.Sprintf(`"%s": "%s"`, labelKey, labelVal)) - } - } - seriesRow[1] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) - seriesRows = append(seriesRows, &seriesRow) - } - } - c.mux.Unlock() - if len(seriesRows) != 0 { - if err = writeRows(c.promSerSQL, seriesRows, conn); err != nil { - return - } - } + if err = c.writeSeries(*batch.Rows, conn); err != nil { + return } statistics.FlushMsgsTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) @@ -189,10 +197,15 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { } func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { - allSeriesSQL := fmt.Sprintf("SELECT __series_id FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) + var query string + if c.cfg.Clickhouse.Cluster != "" { + query = fmt.Sprintf("SELECT __series_id FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) + } else { + query = fmt.Sprintf("SELECT __series_id FROM %s.%s", c.cfg.Clickhouse.DB, c.seriesTbl) + } var rs *sql.Rows var seriesID uint64 - if rs, err = conn.Query(allSeriesSQL); err != nil { + if rs, err = conn.Query(query); err != nil { err = errors.Wrapf(err, "") return err } @@ -209,21 +222,11 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { return } -func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { +func (c *ClickHouse) recoverSeriesSchema(conn *sql.DB) (err error) { chCfg := &c.cfg.Clickhouse - // Check distributed series table - c.idxLabels = nil - c.seriesTbl = c.taskCfg.TableName + "_series" var onCluster string if chCfg.Cluster != "" { onCluster = fmt.Sprintf("ON CLUSTER %s", chCfg.Cluster) - if c.distSeriesTbls, err = c.getDistTbls(c.seriesTbl); err != nil { - return - } - if c.distSeriesTbls == nil { - err = errors.Errorf("Please create distributed table for %s.", c.seriesTbl) - return - } } // Check the series table schema expSeriesDims := []*model.ColumnWithType{ @@ -303,6 +306,7 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { return } } + // Generate SQL for series INSERT serDimsQuoted := make([]string, len(seriesDims)) params := make([]string, len(seriesDims)) @@ -312,15 +316,115 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { } c.promSerSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.seriesTbl + " (" + strings.Join(serDimsQuoted, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" + return +} + +func (c *ClickHouse) recoverSeriesData(conn *sql.DB) (err error) { + chCfg := &c.cfg.Clickhouse + sel := make([]string, len(c.Dims)) + for i, dim := range c.Dims { + if i == c.IdxSerID { + sel[i] = dim.Name + } else { + sel[i] = fmt.Sprintf("any(`%s`)", dim.Name) + } + } + var query string + if chCfg.Cluster != "" { + query = fmt.Sprintf(`SELECT %s FROM %s.%s WHERE __series_id GLOBAL NOT IN (SELECT __series_id FROM %s.%s) GROUP BY __series_id`, strings.Join(sel, ", "), chCfg.DB, c.distMetricTbls[0], chCfg.DB, c.distSeriesTbls[0]) + } else { + query = fmt.Sprintf(`SELECT %s FROM %s.%s WHERE __series_id NOT IN (SELECT __series_id FROM %s.%s) GROUP BY __series_id`, strings.Join(sel, ", "), chCfg.DB, c.taskCfg.TableName, chCfg.DB, c.seriesTbl) + } + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) + rowsMissed := make(model.Rows, 0) + var rs *sql.Rows + if rs, err = conn.Query(query); err != nil { + err = errors.Wrapf(err, "") + return err + } + defer rs.Close() + var cnt int + rowPtr := make(model.Row, len(c.Dims)) + for rs.Next() { + row := make(model.Row, len(c.Dims)) + for i, dim := range c.Dims { + switch dim.Type { + case model.Int: + row[i] = uint64(0) + case model.Float: + row[i] = float64(0.0) + case model.DateTime, model.ElasticDateTime: + row[i] = time.Time{} + case model.String: + row[i] = sql.NullString{} + default: + } + rowPtr[i] = &row[i] + } + if err = rs.Scan(rowPtr...); err != nil { + err = errors.Wrapf(err, "") + return err + } + rowsMissed = append(rowsMissed, &row) + if len(rowsMissed) >= c.taskCfg.BufferSize { + if err = c.writeSeries(rowsMissed, conn); err != nil { + return + } + cnt += len(rowsMissed) + rowsMissed = rowsMissed[:0] + } + } + if err = c.writeSeries(rowsMissed, conn); err != nil { + return + } + cnt += len(rowsMissed) + util.Logger.Info(fmt.Sprintf("recovered %d series to %s", cnt, c.seriesTbl), zap.String("task", c.taskCfg.Name)) + return +} + +func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { + chCfg := &c.cfg.Clickhouse + c.IdxSerID = -1 + for i, dim := range c.Dims { + if dim.Name == "__series_id" { + c.IdxSerID = i + } + } + if c.IdxSerID < 0 { + return + } + // Check distributed series table + c.idxLabels = nil + c.seriesTbl = c.taskCfg.TableName + "_series" + if chCfg.Cluster != "" { + if c.distSeriesTbls, err = c.getDistTbls(c.seriesTbl); err != nil { + return + } + if c.distSeriesTbls == nil { + err = errors.Errorf("Please create distributed table for %s.", c.seriesTbl) + return + } + } // Initialize bmSeries if err = c.initBmSeries(conn); err != nil { return } + // Recover series table columns + if err = c.recoverSeriesSchema(conn); err != nil { + return + } + // Recover series to the series table + if err = c.recoverSeriesData(conn); err != nil { + return + } + return } func (c *ClickHouse) initSchema() (err error) { - c.IdxSerID = -1 + if c.distMetricTbls, err = c.getDistTbls(c.taskCfg.TableName); err != nil { + return + } if c.taskCfg.AutoSchema { sc := pool.GetShardConn(0) var conn *sql.DB @@ -330,15 +434,8 @@ func (c *ClickHouse) initSchema() (err error) { if c.Dims, err = getDims(c.cfg.Clickhouse.DB, c.taskCfg.TableName, c.taskCfg.ExcludeColumns, conn); err != nil { return } - for i, dim := range c.Dims { - if dim.Name == "__series_id" { - c.IdxSerID = i - } - } - if c.IdxSerID >= 0 { - if err = c.initSeriesSchema(conn); err != nil { - return - } + if err = c.initSeriesSchema(conn); err != nil { + return } } else { c.Dims = make([]*model.ColumnWithType, 0) @@ -353,9 +450,6 @@ func (c *ClickHouse) initSchema() (err error) { } } // Generate SQL for INSERT - if c.distMetricTbls, err = c.getDistTbls(c.taskCfg.TableName); err != nil { - return - } c.Dms = make([]string, 0, len(c.Dims)) quotedDms := make([]string, 0, len(c.Dims)) for _, d := range c.Dims { From 3a1a5814480b80821eff8f288f223a99018ce1da Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 18 Oct 2021 14:09:16 +0800 Subject: [PATCH 179/404] make lint happy --- input/input.go | 2 +- input/kafka_go.go | 6 +++--- input/kafka_sarama.go | 6 +++--- model/message.go | 2 +- output/clickhouse.go | 13 ++++++------- task/task.go | 42 ++++++++++++++++++++++++++---------------- 6 files changed, 40 insertions(+), 31 deletions(-) diff --git a/input/input.go b/input/input.go index 7a1106ff..57cde336 100644 --- a/input/input.go +++ b/input/input.go @@ -15,7 +15,7 @@ const ( ) type Inputer interface { - Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage), cleanupFn func()) error + Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg *model.InputMessage), cleanupFn func()) error Run() Stop() error CommitMessages(message *model.InputMessage) error diff --git a/input/kafka_go.go b/input/kafka_go.go index 8f70955f..7dfaee48 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -45,7 +45,7 @@ type KafkaGo struct { ctx context.Context cancel context.CancelFunc wgRun sync.WaitGroup - putFn func(msg model.InputMessage) + putFn func(msg *model.InputMessage) } // NewKafkaGo get instance of kafka reader @@ -54,7 +54,7 @@ func NewKafkaGo() *KafkaGo { } // Init Initialise the kafka instance with configuration -func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage), cleanupFn func()) (err error) { +func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg *model.InputMessage), cleanupFn func()) (err error) { k.cfg = cfg k.taskCfg = taskCfg kfkCfg := &cfg.Kafka @@ -151,7 +151,7 @@ LOOP_KAFKA_GO: continue } } - k.putFn(model.InputMessage{ + k.putFn(&model.InputMessage{ Topic: msg.Topic, Partition: msg.Partition, Key: msg.Key, diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index b6673f83..82749a8f 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -46,7 +46,7 @@ type KafkaSarama struct { ctx context.Context cancel context.CancelFunc wgRun sync.WaitGroup - putFn func(msg model.InputMessage) + putFn func(msg *model.InputMessage) cleanupFn func() } @@ -77,7 +77,7 @@ func (h MyConsumerGroupHandler) Cleanup(_ sarama.ConsumerGroupSession) error { func (h MyConsumerGroupHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error { for msg := range claim.Messages() { - h.k.putFn(model.InputMessage{ + h.k.putFn(&model.InputMessage{ Topic: msg.Topic, Partition: int(msg.Partition), Key: msg.Key, @@ -90,7 +90,7 @@ func (h MyConsumerGroupHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, c } // Init Initialise the kafka instance with configuration -func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg model.InputMessage), cleanupFn func()) (err error) { +func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg *model.InputMessage), cleanupFn func()) (err error) { k.cfg = cfg k.taskCfg = taskCfg k.ctx, k.cancel = context.WithCancel(context.Background()) diff --git a/model/message.go b/model/message.go index bfffbfcf..310cc1a0 100644 --- a/model/message.go +++ b/model/message.go @@ -167,7 +167,7 @@ func PutRow(r *Row) { rowPool.Put(r) } -func MetricToRow(metric Metric, msg InputMessage, dims []*ColumnWithType, idxSeriesID int) (row *Row) { +func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSeriesID int) (row *Row) { row = GetRow() var dig *xxhash.Digest if idxSeriesID >= 0 { diff --git a/output/clickhouse.go b/output/clickhouse.go index ea4e9be4..c492892e 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -320,7 +320,6 @@ func (c *ClickHouse) recoverSeriesSchema(conn *sql.DB) (err error) { } func (c *ClickHouse) recoverSeriesData(conn *sql.DB) (err error) { - chCfg := &c.cfg.Clickhouse sel := make([]string, len(c.Dims)) for i, dim := range c.Dims { if i == c.IdxSerID { @@ -330,10 +329,12 @@ func (c *ClickHouse) recoverSeriesData(conn *sql.DB) (err error) { } } var query string - if chCfg.Cluster != "" { - query = fmt.Sprintf(`SELECT %s FROM %s.%s WHERE __series_id GLOBAL NOT IN (SELECT __series_id FROM %s.%s) GROUP BY __series_id`, strings.Join(sel, ", "), chCfg.DB, c.distMetricTbls[0], chCfg.DB, c.distSeriesTbls[0]) + if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { + query = fmt.Sprintf(`SELECT %s FROM %s.%s WHERE __series_id GLOBAL NOT IN (SELECT __series_id FROM %s.%s) GROUP BY __series_id`, + strings.Join(sel, ", "), chCfg.DB, c.distMetricTbls[0], chCfg.DB, c.distSeriesTbls[0]) } else { - query = fmt.Sprintf(`SELECT %s FROM %s.%s WHERE __series_id NOT IN (SELECT __series_id FROM %s.%s) GROUP BY __series_id`, strings.Join(sel, ", "), chCfg.DB, c.taskCfg.TableName, chCfg.DB, c.seriesTbl) + query = fmt.Sprintf(`SELECT %s FROM %s.%s WHERE __series_id NOT IN (SELECT __series_id FROM %s.%s) GROUP BY __series_id`, + strings.Join(sel, ", "), chCfg.DB, c.taskCfg.TableName, chCfg.DB, c.seriesTbl) } util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) rowsMissed := make(model.Rows, 0) @@ -383,7 +384,6 @@ func (c *ClickHouse) recoverSeriesData(conn *sql.DB) (err error) { } func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { - chCfg := &c.cfg.Clickhouse c.IdxSerID = -1 for i, dim := range c.Dims { if dim.Name == "__series_id" { @@ -396,7 +396,7 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { // Check distributed series table c.idxLabels = nil c.seriesTbl = c.taskCfg.TableName + "_series" - if chCfg.Cluster != "" { + if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { if c.distSeriesTbls, err = c.getDistTbls(c.seriesTbl); err != nil { return } @@ -417,7 +417,6 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { if err = c.recoverSeriesData(conn); err != nil { return } - return } diff --git a/task/task.go b/task/task.go index 26098df1..cd90b108 100644 --- a/task/task.go +++ b/task/task.go @@ -151,10 +151,7 @@ func (service *Service) fnCommit(partition int, offset int64) error { return service.inputer.CommitMessages(&msg) } -func (service *Service) put(msg model.InputMessage) { - if atomic.LoadUint32(&service.state) != util.StateRunning { - return - } +func (service *Service) putToRing(msg *model.InputMessage) (ok bool) { taskCfg := service.taskCfg statistics.ConsumeMsgsTotal.WithLabelValues(taskCfg.Name).Inc() // ensure ring for this message exist @@ -187,7 +184,7 @@ func (service *Service) put(msg model.InputMessage) { service: service, } ring.available = sync.NewCond(&ring.mux) - ring.PutMsgNolock(&msg) + ring.PutMsgNolock(msg) // schedule a delayed ForceBatchOrShard if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { if errors.Is(err, goetty.ErrSystemStopped) { @@ -199,6 +196,7 @@ func (service *Service) put(msg model.InputMessage) { } service.rings[msg.Partition] = ring service.Unlock() + ok = true } else { service.Unlock() ring.mux.Lock() @@ -210,13 +208,14 @@ func (service *Service) put(msg model.InputMessage) { msg.Topic, msg.Partition, msg.Offset, ring.ringFilledOffset), zap.String("task", taskCfg.Name)) } ring.mux.Unlock() - return } else if msg.Offset < ring.ringGroundOff+ring.ringCap { - ring.PutMsgNolock(&msg) + ring.PutMsgNolock(msg) ring.mux.Unlock() + ok = true } else { prevMsgOff := msg.Offset - 1 - for atomic.LoadUint32(&service.state) == util.StateRunning && !ring.isIdle && msg.Offset == ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil { + for atomic.LoadUint32(&service.state) == util.StateRunning && !ring.isIdle && + msg.Offset == ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil { // wait ring.PutElem/ring.ForceBatchOrShard to make room util.Logger.Debug(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) while the ring is full, waiting...", msg.Topic, msg.Partition, msg.Offset), zap.String("task", taskCfg.Name)) @@ -228,29 +227,40 @@ func (service *Service) put(msg model.InputMessage) { util.Logger.Debug(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) while the ring.isIdle %v, service.state %v", msg.Topic, msg.Partition, msg.Offset, ring.isIdle, atomic.LoadUint32(&service.state)), zap.String("task", taskCfg.Name)) ring.mux.Unlock() - return - } - if msg.Offset == ring.ringGroundOff || (msg.Offset < ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil) { - ring.PutMsgNolock(&msg) + } else if msg.Offset == ring.ringGroundOff || (msg.Offset < ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil) { + ring.PutMsgNolock(msg) ring.mux.Unlock() + ok = true } else { // discard messages to make room ring.mux.Unlock() statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(taskCfg.Name).Inc() util.Logger.Warn(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) which's previous one is absent in ring offsets [%v, %v)", msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap), zap.String("task", taskCfg.Name)) - ring.MakeRoom(&msg) - ring.PutMsgNolock(&msg) + ring.MakeRoom(msg) + ring.PutMsgNolock(msg) + ok = true } } } + return +} - // submit message to a goroutine pool +func (service *Service) put(msg *model.InputMessage) { + if atomic.LoadUint32(&service.state) != util.StateRunning { + return + } + if !service.putToRing(msg) { + return + } + // submit message to the parsing pool + taskCfg := service.taskCfg service.Lock() service.numFlying++ service.Unlock() statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Inc() _ = util.GlobalParsingPool.Submit(func() { + var err error var row *model.Row var foundNewKeys bool var metric model.Metric @@ -313,7 +323,7 @@ func (service *Service) put(msg model.InputMessage) { service.Lock() ring = service.rings[msg.Partition] service.Unlock() - ring.PutElem(model.MsgRow{Msg: &msg, Row: row}) + ring.PutElem(model.MsgRow{Msg: msg, Row: row}) } }) } From 23546bcc2a8d8c08ac60f83c2ddedad1bce03606 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 18 Oct 2021 17:04:02 +0800 Subject: [PATCH 180/404] fixed systest --- go.test.sh | 11 +++++------ output/clickhouse.go | 16 ++++++++-------- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/go.test.sh b/go.test.sh index 5feb1cd2..05599e91 100755 --- a/go.test.sh +++ b/go.test.sh @@ -4,14 +4,13 @@ echo "create tables" curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_fixed_schema' curl "localhost:8123" -d 'CREATE TABLE test_fixed_schema ( - `day` Date DEFAULT toDate(time), - `time` DateTime, - `name` String, - `value` Float64, - `price` Decimal32(3) + time DateTime, + name String, + value Float64, + price Decimal32(3) DEFAULT(9.9) ) ENGINE = MergeTree -PARTITION BY day +PARTITION BY toYYYYMMDD(time) ORDER BY (time, name)' curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_auto_schema' diff --git a/output/clickhouse.go b/output/clickhouse.go index c492892e..2938c443 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -424,18 +424,15 @@ func (c *ClickHouse) initSchema() (err error) { if c.distMetricTbls, err = c.getDistTbls(c.taskCfg.TableName); err != nil { return } + sc := pool.GetShardConn(0) + var conn *sql.DB + if conn, _, err = sc.NextGoodReplica(0); err != nil { + return + } if c.taskCfg.AutoSchema { - sc := pool.GetShardConn(0) - var conn *sql.DB - if conn, _, err = sc.NextGoodReplica(0); err != nil { - return - } if c.Dims, err = getDims(c.cfg.Clickhouse.DB, c.taskCfg.TableName, c.taskCfg.ExcludeColumns, conn); err != nil { return } - if err = c.initSeriesSchema(conn); err != nil { - return - } } else { c.Dims = make([]*model.ColumnWithType, 0) for _, dim := range c.taskCfg.Dims { @@ -448,6 +445,9 @@ func (c *ClickHouse) initSchema() (err error) { }) } } + if err = c.initSeriesSchema(conn); err != nil { + return + } // Generate SQL for INSERT c.Dms = make([]string, 0, len(c.Dims)) quotedDms := make([]string, 0, len(c.Dims)) From 825ff2562b48b554de3116de3ea1f63f19712768 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 18 Oct 2021 21:14:51 +0800 Subject: [PATCH 181/404] fixed recoverSeriesData --- output/clickhouse.go | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 2938c443..4d51a985 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -344,7 +344,6 @@ func (c *ClickHouse) recoverSeriesData(conn *sql.DB) (err error) { return err } defer rs.Close() - var cnt int rowPtr := make(model.Row, len(c.Dims)) for rs.Next() { row := make(model.Row, len(c.Dims)) @@ -367,19 +366,12 @@ func (c *ClickHouse) recoverSeriesData(conn *sql.DB) (err error) { return err } rowsMissed = append(rowsMissed, &row) - if len(rowsMissed) >= c.taskCfg.BufferSize { - if err = c.writeSeries(rowsMissed, conn); err != nil { - return - } - cnt += len(rowsMissed) - rowsMissed = rowsMissed[:0] - } } + rs.Close() //rows need be closed before executing another sql if err = c.writeSeries(rowsMissed, conn); err != nil { return } - cnt += len(rowsMissed) - util.Logger.Info(fmt.Sprintf("recovered %d series to %s", cnt, c.seriesTbl), zap.String("task", c.taskCfg.Name)) + util.Logger.Info(fmt.Sprintf("recovered %d series to %s", len(rowsMissed), c.seriesTbl), zap.String("task", c.taskCfg.Name)) return } From c4072d2eb154f1c4db01f3047a5c0249dcbd730d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 19 Oct 2021 13:48:18 +0800 Subject: [PATCH 182/404] don't put __name__ into labels --- output/clickhouse.go | 23 +++++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 4d51a985..4190fb4d 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -53,6 +53,7 @@ type ClickHouse struct { Dims []*model.ColumnWithType Dms []string IdxSerID int + IdxName int idxLabels []int cfg *config.Config taskCfg *config.TaskConfig @@ -116,18 +117,20 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn *sql.DB) (err error) { c.mux.Lock() for _, row := range rows { seriesID := (*row)[c.IdxSerID].(uint64) + name := (*row)[c.IdxName].(string) if c.bmSeries.CheckedAdd(seriesID) { - seriesRow := make(model.Row, 2+len(c.idxLabels)) //__series_id, lables, ... + seriesRow := make(model.Row, 3+len(c.idxLabels)) //__series_id, __name__, lables, ... labels := make([]string, 0) seriesRow[0] = seriesID + seriesRow[1] = name for i, idxLabel := range c.idxLabels { - seriesRow[2+i] = (*row)[idxLabel] + seriesRow[3+i] = (*row)[idxLabel] labelKey := c.Dims[idxLabel].Name if labelVal, ok := (*row)[idxLabel].(string); ok { labels = append(labels, fmt.Sprintf(`"%s": "%s"`, labelKey, labelVal)) } } - seriesRow[1] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) + seriesRow[2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) seriesRows = append(seriesRows, &seriesRow) } } @@ -231,6 +234,7 @@ func (c *ClickHouse) recoverSeriesSchema(conn *sql.DB) (err error) { // Check the series table schema expSeriesDims := []*model.ColumnWithType{ {Name: "__series_id", Type: model.Int}, + {Name: "__name__", Type: model.String}, {Name: "labels", Type: model.String}, } var seriesDims []*model.ColumnWithType @@ -254,11 +258,11 @@ func (c *ClickHouse) recoverSeriesSchema(conn *sql.DB) (err error) { } } if badFirst { - err = errors.Errorf(`First two columns of %s are expect to be "__series_id UInt64, labels String".`, c.seriesTbl) + err = errors.Errorf(`First three columns of %s are expect to be "__series_id UInt64, __name__ String, labels String".`, c.seriesTbl) return } for i, serDim := range seriesDims { - if i < 2 { + if i < 3 { continue } idxLabel := -1 @@ -377,14 +381,21 @@ func (c *ClickHouse) recoverSeriesData(conn *sql.DB) (err error) { func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { c.IdxSerID = -1 + c.IdxName = -1 for i, dim := range c.Dims { - if dim.Name == "__series_id" { + if dim.Name == "__series_id" && dim.Type == model.Int { c.IdxSerID = i + } else if dim.Name == "__name__" && dim.Type == model.String { + c.IdxName = i } } if c.IdxSerID < 0 { return } + if c.IdxName < 0 { + err = errors.Errorf("Table %s shall have columns `__series_id UInt64` and `__name__ String`.", c.taskCfg.TableName) + return + } // Check distributed series table c.idxLabels = nil c.seriesTbl = c.taskCfg.TableName + "_series" From 9f7199a5fe0309956d40d9947d0a0b21fe2513fb Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 19 Oct 2021 14:45:33 +0800 Subject: [PATCH 183/404] respect batchSize boundary --- task/ring.go | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/task/ring.go b/task/ring.go index 167e272a..3764d412 100644 --- a/task/ring.go +++ b/task/ring.go @@ -175,12 +175,16 @@ func (ring *Ring) ForceBatchOrShard(_ interface{}) { } } -// generate a batch for messages [ring.ringGroundOff, ring.ringFilledOffset) +// generate a batch for messages [ring.ringGroundOff, ring.ringFilledOffset), respect batchSize boundary // assume ring.mux is locked func (ring *Ring) genBatchOrShard() { taskCfg := ring.service.taskCfg var parseErrs int - endOff := ring.ringFilledOffset + // Respect batchSize boundary + endOff := ((ring.ringGroundOff >> ring.batchSizeShift) + 1) << ring.batchSizeShift + if endOff > ring.ringFilledOffset { + endOff = ring.ringFilledOffset + } msgCnt := endOff - ring.ringGroundOff if atomic.LoadUint32(&ring.service.state) != util.StateRunning { util.Logger.Info(fmt.Sprintf("Ring.genBatchOrShard discarded a batch for topic %v patittion %d, offset [%d,%d), messages %d", @@ -214,7 +218,7 @@ func (ring *Ring) genBatchOrShard() { taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, batch.RealSize, parseErrs), zap.String("task", taskCfg.Name)) - batch.BatchIdx = (endOff - 1) >> ring.batchSizeShift + batch.BatchIdx = ring.ringGroundOff >> ring.batchSizeShift ring.batchSys.CreateBatchGroupSingle(batch, ring.partition, endOff-1) ring.service.Flush(batch) statistics.RingNormalBatchsTotal.WithLabelValues(taskCfg.Name).Inc() From a52f768676f0dfcbb88f7a15e2231ee934da9387 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 19 Oct 2021 15:14:27 +0800 Subject: [PATCH 184/404] fixed Service.Stop dead lock --- input/kafka_go.go | 18 +++++++++++------- output/clickhouse.go | 1 - task/ring.go | 1 - task/task.go | 12 ++++++++---- 4 files changed, 19 insertions(+), 13 deletions(-) diff --git a/input/kafka_go.go b/input/kafka_go.go index 7dfaee48..7fbc0cb9 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -39,13 +39,14 @@ var _ Inputer = (*KafkaGo)(nil) // KafkaGo implements input.Inputer type KafkaGo struct { - cfg *config.Config - taskCfg *config.TaskConfig - r *kafka.Reader - ctx context.Context - cancel context.CancelFunc - wgRun sync.WaitGroup - putFn func(msg *model.InputMessage) + cfg *config.Config + taskCfg *config.TaskConfig + r *kafka.Reader + ctx context.Context + cancel context.CancelFunc + wgRun sync.WaitGroup + putFn func(msg *model.InputMessage) + cleanupFn func() } // NewKafkaGo get instance of kafka reader @@ -60,6 +61,7 @@ func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn fun kfkCfg := &cfg.Kafka k.ctx, k.cancel = context.WithCancel(context.Background()) k.putFn = putFn + k.cleanupFn = cleanupFn offset := kafka.LastOffset if k.taskCfg.Earliest { offset = kafka.FirstOffset @@ -176,6 +178,8 @@ func (k *KafkaGo) CommitMessages(msg *model.InputMessage) (err error) { // Stop kafka consumer and close all connections func (k *KafkaGo) Stop() error { + k.cleanupFn() + // Note: a closed kafka-go client cannot commit offsets. k.cancel() k.r.Close() k.wgRun.Wait() diff --git a/output/clickhouse.go b/output/clickhouse.go index 4190fb4d..2c450045 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -159,7 +159,6 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( if err = c.writeSeries(*batch.Rows, conn); err != nil { return } - statistics.FlushMsgsTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) return } diff --git a/task/ring.go b/task/ring.go index 3764d412..2ac79a72 100644 --- a/task/ring.go +++ b/task/ring.go @@ -147,7 +147,6 @@ func (ring *Ring) ForceBatchOrShard(_ interface{}) { taskCfg := ring.service.taskCfg ring.mux.Lock() defer ring.mux.Unlock() - defer ring.available.Broadcast() ring.tid.Stop() if !ring.isIdle { if ring.ringFilledOffset > ring.ringGroundOff { diff --git a/task/task.go b/task/task.go index cd90b108..105a765a 100644 --- a/task/task.go +++ b/task/task.go @@ -348,6 +348,7 @@ func (service *Service) drain() { service.sharder.ForceFlush(nil) } service.clickhouse.Drain() + util.Logger.Debug("drained flying messages", zap.String("task", service.taskCfg.Name)) } func (service *Service) Flush(batch *model.Batch) (err error) { @@ -379,6 +380,13 @@ func (service *Service) Stop() { util.Logger.Debug("stopping task service...", zap.String("task", taskCfg.Name)) atomic.StoreUint32(&service.state, util.StateStopped) + for _, ring := range service.rings { + if ring != nil { + ring.mux.Lock() + ring.available.Broadcast() + ring.mux.Unlock() + } + } if service.sharder != nil { service.sharder.tid.Stop() @@ -386,10 +394,6 @@ func (service *Service) Stop() { service.tid.Stop() util.Logger.Debug("stopped internal timers", zap.String("task", taskCfg.Name)) - service.drain() - util.Logger.Debug("drained flying messages", zap.String("task", taskCfg.Name)) - - // Note: inputer needs be stopped *after* drain() since a closed kafka-go client cannot commit offsets. if err := service.inputer.Stop(); err != nil { util.Logger.Fatal("service.inputer.Stop failed", zap.Error(err)) } From bb4c621dea4f2c778795e3e99e6ca6c203c6f704 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 20 Oct 2021 14:15:36 +0800 Subject: [PATCH 185/404] removed le from labels --- output/clickhouse.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 2c450045..28fc85b1 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -125,9 +125,10 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn *sql.DB) (err error) { seriesRow[1] = name for i, idxLabel := range c.idxLabels { seriesRow[3+i] = (*row)[idxLabel] - labelKey := c.Dims[idxLabel].Name - if labelVal, ok := (*row)[idxLabel].(string); ok { - labels = append(labels, fmt.Sprintf(`"%s": "%s"`, labelKey, labelVal)) + if labelKey := c.Dims[idxLabel].Name; labelKey != "le" { + if labelVal, ok := (*row)[idxLabel].(string); ok { + labels = append(labels, fmt.Sprintf(`"%s": "%s"`, labelKey, labelVal)) + } } } seriesRow[2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) From 310358e335f339d4e5b7165dfb1ecc1c4ae0091a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 20 Oct 2021 18:13:25 +0800 Subject: [PATCH 186/404] don't store labels into the metric table --- config/config.go | 6 + model/message.go | 13 +- output/clickhouse.go | 265 +++++++++++--------------------------- output/clickhouse_util.go | 4 +- task/sharding.go | 6 +- 5 files changed, 100 insertions(+), 194 deletions(-) diff --git a/config/config.go b/config/config.go index 2fcfea2d..da8fd928 100644 --- a/config/config.go +++ b/config/config.go @@ -127,6 +127,8 @@ type TaskConfig struct { Enable bool MaxDims int // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack } + // PrometheusSchema expects each message is a Prometheus metric(timestamp, value, metric name and a list of labels). + PrometheusSchema bool // ShardingKey is the column name to which sharding against ShardingKey string `json:"shardingKey,omitempty"` @@ -234,6 +236,10 @@ func (cfg *Config) Normallize() (err error) { if taskCfg.TimeZone == "" { taskCfg.TimeZone = defaultTimeZone } + if taskCfg.PrometheusSchema { + taskCfg.DynamicSchema.Enable = true + taskCfg.AutoSchema = true + } if taskCfg.DynamicSchema.Enable { if taskCfg.Parser != "fastjson" { err = errors.Errorf("Parser %s doesn't support DynamicSchema", taskCfg.Parser) diff --git a/model/message.go b/model/message.go index 310cc1a0..66781a40 100644 --- a/model/message.go +++ b/model/message.go @@ -2,6 +2,7 @@ package model import ( "container/list" + "fmt" "strconv" "strings" "sync" @@ -170,12 +171,15 @@ func PutRow(r *Row) { func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSeriesID int) (row *Row) { row = GetRow() var dig *xxhash.Digest + var labels []string if idxSeriesID >= 0 { dig = xxhash.New() } for i, dim := range dims { - if i == idxSeriesID { + if idxSeriesID >= 0 && i == idxSeriesID { *row = append(*row, uint64(0)) + } else if idxSeriesID >= 0 && i == idxSeriesID+1 { + *row = append(*row, "") } else if strings.HasPrefix(dim.Name, "__kafka") { if strings.HasSuffix(dim.Name, "_topic") { *row = append(*row, msg.Topic) @@ -188,15 +192,20 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe val := GetValueByType(metric, dim) *row = append(*row, val) if idxSeriesID >= 0 && dim.Type == String && val != nil { + labelVal := val.(string) _, _ = dig.WriteString("###") _, _ = dig.WriteString(dim.Name) _, _ = dig.WriteString("###") - _, _ = dig.WriteString(val.(string)) + _, _ = dig.WriteString(labelVal) + if dim.Name != "__name__" && dim.Name != "le" { + labels = append(labels, fmt.Sprintf(`"%s": "%s"`, dim.Name, labelVal)) + } } } } if idxSeriesID >= 0 { (*row)[idxSeriesID] = dig.Sum64() + (*row)[idxSeriesID+1] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) } return } diff --git a/output/clickhouse.go b/output/clickhouse.go index 28fc85b1..26aaf9f0 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -51,10 +51,7 @@ var ( // ClickHouse is an output service consumers from kafka messages type ClickHouse struct { Dims []*model.ColumnWithType - Dms []string IdxSerID int - IdxName int - idxLabels []int cfg *config.Config taskCfg *config.TaskConfig prepareSQL string @@ -111,33 +108,16 @@ func (c *ClickHouse) Send(batch *model.Batch) { func (c *ClickHouse) writeSeries(rows model.Rows, conn *sql.DB) (err error) { var seriesRows model.Rows - if c.IdxSerID < 0 { - return - } c.mux.Lock() for _, row := range rows { seriesID := (*row)[c.IdxSerID].(uint64) - name := (*row)[c.IdxName].(string) if c.bmSeries.CheckedAdd(seriesID) { - seriesRow := make(model.Row, 3+len(c.idxLabels)) //__series_id, __name__, lables, ... - labels := make([]string, 0) - seriesRow[0] = seriesID - seriesRow[1] = name - for i, idxLabel := range c.idxLabels { - seriesRow[3+i] = (*row)[idxLabel] - if labelKey := c.Dims[idxLabel].Name; labelKey != "le" { - if labelVal, ok := (*row)[idxLabel].(string); ok { - labels = append(labels, fmt.Sprintf(`"%s": "%s"`, labelKey, labelVal)) - } - } - } - seriesRow[2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) - seriesRows = append(seriesRows, &seriesRow) + seriesRows = append(seriesRows, row) } } c.mux.Unlock() if len(seriesRows) != 0 { - if err = writeRows(c.promSerSQL, seriesRows, conn); err != nil { + if err = writeRows(c.promSerSQL, seriesRows, c.IdxSerID, len(c.Dims), conn); err != nil { return } } @@ -153,11 +133,16 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( if conn, *dbVer, err = sc.NextGoodReplica(*dbVer); err != nil { return } - - if err = writeRows(c.prepareSQL, *batch.Rows, conn); err != nil { - return + //row[:c.IdxSerID] is for metric table + //row[c.IdxSerID:] is for series table + numDims := len(c.Dims) + if c.taskCfg.PrometheusSchema { + numDims = c.IdxSerID + 1 + if err = c.writeSeries(*batch.Rows, conn); err != nil { + return + } } - if err = c.writeSeries(*batch.Rows, conn); err != nil { + if err = writeRows(c.prepareSQL, *batch.Rows, 0, numDims, conn); err != nil { return } statistics.FlushMsgsTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) @@ -206,6 +191,7 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { } else { query = fmt.Sprintf("SELECT __series_id FROM %s.%s", c.cfg.Clickhouse.DB, c.seriesTbl) } + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs *sql.Rows var seriesID uint64 if rs, err = conn.Query(query); err != nil { @@ -225,16 +211,35 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { return } -func (c *ClickHouse) recoverSeriesSchema(conn *sql.DB) (err error) { - chCfg := &c.cfg.Clickhouse - var onCluster string - if chCfg.Cluster != "" { - onCluster = fmt.Sprintf("ON CLUSTER %s", chCfg.Cluster) +func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { + if !c.taskCfg.PrometheusSchema { + c.IdxSerID = -1 + return + } + // Strip away string columns from metric table, and move column "__series_id" to the last. + var dimSerID *model.ColumnWithType + for i := 0; i < len(c.Dims); { + dim := c.Dims[i] + if dim.Name == "__series_id" && dim.Type == model.Int { + dimSerID = dim + c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) + } else if dim.Type == model.String { + c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) + } else { + i++ + } + } + if dimSerID == nil { + err = errors.Errorf("Metric table %s shall have column `__series_id UInt64`.", c.taskCfg.TableName) + return } - // Check the series table schema + c.IdxSerID = len(c.Dims) + c.Dims = append(c.Dims, dimSerID) + + // Add string columns from series table + c.seriesTbl = c.taskCfg.TableName + "_series" expSeriesDims := []*model.ColumnWithType{ {Name: "__series_id", Type: model.Int}, - {Name: "__name__", Type: model.String}, {Name: "labels", Type: model.String}, } var seriesDims []*model.ColumnWithType @@ -258,58 +263,10 @@ func (c *ClickHouse) recoverSeriesSchema(conn *sql.DB) (err error) { } } if badFirst { - err = errors.Errorf(`First three columns of %s are expect to be "__series_id UInt64, __name__ String, labels String".`, c.seriesTbl) + err = errors.Errorf(`First columns of %s are expect to be "__series_id UInt64, labels String".`, c.seriesTbl) return } - for i, serDim := range seriesDims { - if i < 3 { - continue - } - idxLabel := -1 - for j, dim := range c.Dims { - if serDim.Name == dim.Name && serDim.Type == dim.Type { - idxLabel = j - break - } - } - if idxLabel < 0 { - err = errors.Errorf("Column %s exists in %s but not in %s", serDim.Name, c.seriesTbl, c.taskCfg.TableName) - return - } - c.idxLabels = append(c.idxLabels, idxLabel) - } - // Add missed columns to the series table - var missedLabels []string - for i, dim := range c.Dims { - if dim.Type != model.String { - continue - } - var found bool - for _, serDim := range seriesDims { - if serDim.Name == dim.Name { - found = true - break - } - } - if !found { - c.idxLabels = append(c.idxLabels, i) - seriesDims = append(seriesDims, dim) - missedLabels = append(missedLabels, dim.Name) - } - } - if missedLabels != nil { - for _, key := range missedLabels { - query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` Nullable(String)", chCfg.DB, c.seriesTbl, onCluster, key) - util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) - if _, err = conn.Exec(query); err != nil { - err = errors.Wrapf(err, query) - return - } - } - if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, c.seriesTbl, c.distSeriesTbls, conn); err != nil { - return - } - } + c.Dims = append(c.Dims, seriesDims[1:]...) // Generate SQL for series INSERT serDimsQuoted := make([]string, len(seriesDims)) @@ -320,85 +277,8 @@ func (c *ClickHouse) recoverSeriesSchema(conn *sql.DB) (err error) { } c.promSerSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.seriesTbl + " (" + strings.Join(serDimsQuoted, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" - return -} -func (c *ClickHouse) recoverSeriesData(conn *sql.DB) (err error) { - sel := make([]string, len(c.Dims)) - for i, dim := range c.Dims { - if i == c.IdxSerID { - sel[i] = dim.Name - } else { - sel[i] = fmt.Sprintf("any(`%s`)", dim.Name) - } - } - var query string - if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { - query = fmt.Sprintf(`SELECT %s FROM %s.%s WHERE __series_id GLOBAL NOT IN (SELECT __series_id FROM %s.%s) GROUP BY __series_id`, - strings.Join(sel, ", "), chCfg.DB, c.distMetricTbls[0], chCfg.DB, c.distSeriesTbls[0]) - } else { - query = fmt.Sprintf(`SELECT %s FROM %s.%s WHERE __series_id NOT IN (SELECT __series_id FROM %s.%s) GROUP BY __series_id`, - strings.Join(sel, ", "), chCfg.DB, c.taskCfg.TableName, chCfg.DB, c.seriesTbl) - } - util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) - rowsMissed := make(model.Rows, 0) - var rs *sql.Rows - if rs, err = conn.Query(query); err != nil { - err = errors.Wrapf(err, "") - return err - } - defer rs.Close() - rowPtr := make(model.Row, len(c.Dims)) - for rs.Next() { - row := make(model.Row, len(c.Dims)) - for i, dim := range c.Dims { - switch dim.Type { - case model.Int: - row[i] = uint64(0) - case model.Float: - row[i] = float64(0.0) - case model.DateTime, model.ElasticDateTime: - row[i] = time.Time{} - case model.String: - row[i] = sql.NullString{} - default: - } - rowPtr[i] = &row[i] - } - if err = rs.Scan(rowPtr...); err != nil { - err = errors.Wrapf(err, "") - return err - } - rowsMissed = append(rowsMissed, &row) - } - rs.Close() //rows need be closed before executing another sql - if err = c.writeSeries(rowsMissed, conn); err != nil { - return - } - util.Logger.Info(fmt.Sprintf("recovered %d series to %s", len(rowsMissed), c.seriesTbl), zap.String("task", c.taskCfg.Name)) - return -} - -func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { - c.IdxSerID = -1 - c.IdxName = -1 - for i, dim := range c.Dims { - if dim.Name == "__series_id" && dim.Type == model.Int { - c.IdxSerID = i - } else if dim.Name == "__name__" && dim.Type == model.String { - c.IdxName = i - } - } - if c.IdxSerID < 0 { - return - } - if c.IdxName < 0 { - err = errors.Errorf("Table %s shall have columns `__series_id UInt64` and `__name__ String`.", c.taskCfg.TableName) - return - } // Check distributed series table - c.idxLabels = nil - c.seriesTbl = c.taskCfg.TableName + "_series" if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { if c.distSeriesTbls, err = c.getDistTbls(c.seriesTbl); err != nil { return @@ -408,25 +288,15 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { return } } + // Initialize bmSeries if err = c.initBmSeries(conn); err != nil { return } - // Recover series table columns - if err = c.recoverSeriesSchema(conn); err != nil { - return - } - // Recover series to the series table - if err = c.recoverSeriesData(conn); err != nil { - return - } return } func (c *ClickHouse) initSchema() (err error) { - if c.distMetricTbls, err = c.getDistTbls(c.taskCfg.TableName); err != nil { - return - } sc := pool.GetShardConn(0) var conn *sql.DB if conn, _, err = sc.NextGoodReplica(0); err != nil { @@ -452,19 +322,30 @@ func (c *ClickHouse) initSchema() (err error) { return } // Generate SQL for INSERT - c.Dms = make([]string, 0, len(c.Dims)) - quotedDms := make([]string, 0, len(c.Dims)) - for _, d := range c.Dims { - c.Dms = append(c.Dms, d.Name) - quotedDms = append(quotedDms, fmt.Sprintf("`%s`", d.Name)) - } - params := make([]string, len(c.Dims)) - for i := range params { + numDims := len(c.Dims) + if c.taskCfg.PrometheusSchema { + numDims = c.IdxSerID + 1 + } + quotedDms := make([]string, numDims) + params := make([]string, numDims) + for i := 0; i < numDims; i++ { + quotedDms[i] = fmt.Sprintf("`%s`", c.Dims[i].Name) params[i] = "?" } c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.taskCfg.TableName + " (" + strings.Join(quotedDms, ",") + ") " + "VALUES (" + strings.Join(params, ",") + ")" util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.taskCfg.Name)) + + // Check distributed metric table + if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { + if c.distMetricTbls, err = c.getDistTbls(c.taskCfg.TableName); err != nil { + return + } + if c.distMetricTbls == nil { + err = errors.Errorf("Please create distributed table for %s.", c.taskCfg.TableName) + return + } + } return nil } @@ -486,6 +367,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return } var i int + var affectDistMetric, affectDistSeries bool newKeys.Range(func(key, value interface{}) bool { i++ if i > newKeysQuota { @@ -516,11 +398,16 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { err = errors.Errorf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) return false } - query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) - queries = append(queries, query) - if c.IdxSerID >= 0 && intVal == model.String { - query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, c.seriesTbl, onCluster, strKey, strVal) + if c.taskCfg.PrometheusSchema { + if intVal == model.String { + query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, c.seriesTbl, onCluster, strKey, strVal) + queries = append(queries, query) + affectDistSeries = true + } + } else { + query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) queries = append(queries, query) + affectDistMetric = true } return true }) @@ -541,11 +428,15 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } } if chCfg.Cluster != "" { - if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, taskCfg.TableName, c.distMetricTbls, conn); err != nil { - return + if affectDistMetric { + if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, c.taskCfg.TableName, c.distMetricTbls, conn); err != nil { + return + } } - if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, c.seriesTbl, c.distSeriesTbls, conn); err != nil { - return + if affectDistSeries { + if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, c.seriesTbl, c.distSeriesTbls, conn); err != nil { + return + } } } return diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 96fa898a..15656f7c 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -28,7 +28,7 @@ func shouldReconnect(err error, sc *pool.ShardConn) bool { return true } -func writeRows(prepareSQL string, rows model.Rows, conn *sql.DB) (err error) { +func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn *sql.DB) (err error) { var stmt *sql.Stmt var tx *sql.Tx if tx, err = conn.Begin(); err != nil { @@ -41,7 +41,7 @@ func writeRows(prepareSQL string, rows model.Rows, conn *sql.DB) (err error) { } defer stmt.Close() for _, row := range rows { - if _, err = stmt.Exec(*row...); err != nil { + if _, err = stmt.Exec((*row)[idxBegin:idxEnd]...); err != nil { err = errors.Wrapf(err, "stmt.Exec") break } diff --git a/task/sharding.go b/task/sharding.go index a0d1735b..fc6f0c7a 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -23,11 +23,11 @@ type ShardingPolicy struct { stripe uint64 //=0 means hash, >0 means stripe size } -func NewShardingPolicy(shardingKey, shardingPolicy string, dims []string, ckNum int) (policy *ShardingPolicy, err error) { +func NewShardingPolicy(shardingKey, shardingPolicy string, dims []*model.ColumnWithType, ckNum int) (policy *ShardingPolicy, err error) { policy = &ShardingPolicy{ckNum: ckNum} colSeq := -1 for i, dim := range dims { - if dim == shardingKey { + if dim.Name == shardingKey { colSeq = i } } @@ -115,7 +115,7 @@ func NewSharder(service *Service) (sh *Sharder, err error) { var policy *ShardingPolicy ckNum := pool.NumShard() taskCfg := service.taskCfg - if policy, err = NewShardingPolicy(taskCfg.ShardingKey, taskCfg.ShardingPolicy, service.clickhouse.Dms, ckNum); err != nil { + if policy, err = NewShardingPolicy(taskCfg.ShardingKey, taskCfg.ShardingPolicy, service.clickhouse.Dims, ckNum); err != nil { return } sh = &Sharder{ From 1797a28d9418e3b5ddcd05bc363235c0b615a80a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 22 Oct 2021 22:45:29 +0800 Subject: [PATCH 187/404] gjson detect new keys --- go.mod | 5 +-- go.sum | 23 +++++----- model/value.go | 26 +++++++++++ parser/fastjson_test.go | 27 ------------ parser/gjson.go | 58 ++++++++++++++++++++++++- parser/parser_test.go | 96 ++++++++++++++++++++++++++++------------- 6 files changed, 161 insertions(+), 74 deletions(-) delete mode 100644 parser/fastjson_test.go diff --git a/go.mod b/go.mod index 0926e1b6..ce16a06f 100644 --- a/go.mod +++ b/go.mod @@ -6,7 +6,7 @@ require ( github.com/ClickHouse/clickhouse-go v1.4.7 github.com/RoaringBitmap/roaring v0.9.4 github.com/Shopify/sarama v1.29.1 - github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a + github.com/bytedance/sonic v1.0.0-rc github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 github.com/golang/snappy v0.0.4 // indirect @@ -22,8 +22,7 @@ require ( github.com/sirupsen/logrus v1.6.0 github.com/smartystreets/goconvey v1.6.4 // indirect github.com/stretchr/testify v1.7.0 - github.com/tidwall/gjson v1.9.0 - github.com/tidwall/pretty v1.2.0 // indirect + github.com/tidwall/gjson v1.10.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.0.2 diff --git a/go.sum b/go.sum index 3760d9e2..ad121798 100644 --- a/go.sum +++ b/go.sum @@ -31,13 +31,13 @@ github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAK github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= -github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a h1:6ikHdu0dgMWdck/K6ZlMcN+Gx44c8PeZ54mAsQ5LpOg= -github.com/bytedance/sonic v0.0.0-20210624083732-e9b6634cac5a/go.mod h1:Jag0ltWFtHzILaHByar6cc/Ru+9qv0JJzyOCfCE7jtk= +github.com/bytedance/sonic v1.0.0-rc h1:RMj5thTrWOelnkZQacWpG/Gm06eQY1yNZY8Ze+BxeDY= +github.com/bytedance/sonic v1.0.0-rc/go.mod h1:lqEFq2niUZaKwt5URNYvcyY6r72F21rXzHgnzXS1Fw4= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/chenzhuoyu/base64x v0.0.0-20210528162528-3c6c11c43ee5 h1:7AStn2tanqGY99xzW+Ve1p6YYqnRr1m/yswJ4h0BhcY= -github.com/chenzhuoyu/base64x v0.0.0-20210528162528-3c6c11c43ee5/go.mod h1:NfDzX8KeqVNX62apij1OkqoeDdq1VR3g0TRZo99kkBA= +github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06 h1:1sDoSuDPWzhkdzNVxCxtIaKiAe96ESVPv8coGwc1gZ4= +github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= @@ -69,8 +69,8 @@ github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/goccy/go-json v0.7.1 h1:VMhnh5gcc8De8f6m2DLvSqY1x8Jwl3btet+EqMP0QNs= -github.com/goccy/go-json v0.7.1/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.7.2 h1:MY1gMmtCxRpaI8YGpeHCvXUb+FVIo09pnjqF9Rhh274= +github.com/goccy/go-json v0.7.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= @@ -146,8 +146,8 @@ github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0 github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.13.5 h1:9O69jUPDcsT9fEm74W92rZL9FQY7rCdaXVneq+yyzl4= github.com/klauspost/compress v1.13.5/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/cpuid/v2 v2.0.6 h1:dQ5ueTiftKxp0gyjKSx5+8BtPWkyQbd95m8Gys/RarI= -github.com/klauspost/cpuid/v2 v2.0.6/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= +github.com/klauspost/cpuid/v2 v2.0.9 h1:lgaqFMSdTdQYdZ04uHyN2d/eKdOMyi2YLSvlQIBFYa4= +github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -242,10 +242,11 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/tebeka/strftime v0.1.3 h1:5HQXOqWKYRFfNyBMNVc9z5+QzuBtIXy03psIhtdJYto= github.com/tebeka/strftime v0.1.3/go.mod h1:7wJm3dZlpr4l/oVK0t1HYIc4rMzQ2XJlOMIUJUJH6XQ= github.com/tidwall/gjson v1.8.0/go.mod h1:5/xDoumyyDNerp2U36lyolv46b3uF/9Bu6OfyQ9GImk= -github.com/tidwall/gjson v1.9.0 h1:+Od7AE26jAaMgVC31cQV/Ope5iKXulNMflrlB7k+F9E= -github.com/tidwall/gjson v1.9.0/go.mod h1:5/xDoumyyDNerp2U36lyolv46b3uF/9Bu6OfyQ9GImk= -github.com/tidwall/match v1.0.3 h1:FQUVvBImDutD8wJLN6c5eMzWtjgONK9MwIBCOrUJKeE= +github.com/tidwall/gjson v1.10.1 h1:Midn39zaqkz9SPTBDuEe0RdW9T4T+MpBOAC10e9/qFE= +github.com/tidwall/gjson v1.10.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/match v1.0.3/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= +github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= +github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= github.com/tidwall/pretty v1.1.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= diff --git a/model/value.go b/model/value.go index 7012d97d..965956d7 100644 --- a/model/value.go +++ b/model/value.go @@ -43,6 +43,32 @@ var ( typeInfo map[string]TypeInfo ) +func GetTypeName(typ int) (name string) { + switch typ { + case Int: + name = "Int" + case Float: + name = "Float" + case String: + name = "String" + case DateTime: + name = "DateTime" + case ElasticDateTime: + name = "ElasticDateTime" + case IntArray: + name = "IntArray" + case FloatArray: + name = "FloatArray" + case StringArray: + name = "StringArray" + case DateTimeArray: + name = "DateTimeArray" + default: + name = "TypeUnknown" + } + return +} + // There are only three cases for the value type of metric, (float64, string, map [string] interface {}) func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { name := cwt.SourceName diff --git a/parser/fastjson_test.go b/parser/fastjson_test.go deleted file mode 100644 index 1b0c8a89..00000000 --- a/parser/fastjson_test.go +++ /dev/null @@ -1,27 +0,0 @@ -package parser - -import ( - "testing" - - "github.com/stretchr/testify/require" - "github.com/valyala/fastjson" -) - -func TestFastjsonDetectSchema(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") - parser := pp.Get() - defer pp.Put(parser) - metric, _ := parser.Parse(jsonSample) - - act := make(map[string]string) - c, _ := metric.(*FastjsonMetric) - var obj *fastjson.Object - var err error - if obj, err = c.value.Object(); err != nil { - return - } - obj.Visit(func(key []byte, v *fastjson.Value) { - act[string(key)] = v.Type().String() - }) - require.Equal(t, jsonSchema, act) -} diff --git a/parser/gjson.go b/parser/gjson.go index 38d0cac2..032b1c0c 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -21,6 +21,7 @@ import ( "time" "github.com/tidwall/gjson" + "go.uber.org/zap" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" @@ -205,8 +206,20 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { return } -func (c *GjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool { - return false +func (c *GjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (foundNew bool) { + gjson.Parse(c.raw).ForEach(func(k, v gjson.Result) bool { + strKey := k.Str + if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { + if typ := gjDetectType(v); typ != model.TypeUnknown { + newKeys.Store(strKey, typ) + foundNew = true + } else { + util.Logger.Warn("GjsonMetric.GetNewKeys failed to detect field type", zap.String("key", strKey), zap.String("value", v.String())) + } + } + return true + }) + return } func gjCompatibleInt(r gjson.Result) (ok bool) { @@ -250,3 +263,44 @@ func gjCompatibleDateTime(r gjson.Result) (ok bool) { } return } + +func gjDetectType(v gjson.Result) (typ int) { + switch v.Type { + case gjson.True, gjson.False: + typ = model.Int + case gjson.Number: + typ = model.Float + if float64(v.Int()) == v.Num { + typ = model.Int + } + case gjson.String: + typ = model.String + if _, layout := parseInLocation(string(v.Str), time.Local); layout != "" { + typ = model.DateTime + } + case gjson.JSON: + typ = model.String + array := v.Array() + if array != nil { + switch array[0].Type { + case gjson.True, gjson.False: + typ = model.IntArray + case gjson.Number: + typ = model.FloatArray + if float64(v.Int()) == v.Num { + typ = model.IntArray + } + case gjson.String: + typ = model.StringArray + if _, layout := parseInLocation(string(v.Str), time.Local); layout != "" { + typ = model.DateTimeArray + } + default: + typ = model.StringArray + } + } + default: + typ = model.String + } + return +} diff --git a/parser/parser_test.go b/parser/parser_test.go index d26b0695..7f207a1f 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -70,35 +70,35 @@ var jsonSample = []byte(`{ var jsonSchema = map[string]string{ "null": "null", - "bool_true": "true", - "bool_false": "false", - "num_int": "number", - "num_float": "number", - "str": "string", - "str_int": "string", - "str_float": "string", - "str_date_1": "string", - "str_date_2": "string", - "str_time_rfc3339_1": "string", - "str_time_rfc3339_2": "string", - "str_time_clickhouse_1": "string", - "str_time_clickhouse_2": "string", - "obj": "object", - "array_empty": "array", - "array_null": "array", - "array_bool": "array", - "array_num_int_1": "array", - "array_num_int_2": "array", - "array_num_float": "array", - "array_str": "array", - "array_str_int_1": "array", - "array_str_int_2": "array", - "array_str_float": "array", - "array_str_date_1": "array", - "array_str_date_2": "array", - "array_str_time_rfc3339": "array", - "array_str_time_clickhouse": "array", - "array_obj": "array", + "bool_true": "Int", + "bool_false": "Int", + "num_int": "Int", + "num_float": "Float", + "str": "String", + "str_int": "String", + "str_float": "String", + "str_date_1": "DateTime", + "str_date_2": "DateTime", + "str_time_rfc3339_1": "DateTime", + "str_time_rfc3339_2": "DateTime", + "str_time_clickhouse_1": "DateTime", + "str_time_clickhouse_2": "DateTime", + "obj": "String", + "array_empty": "String", + "array_null": "String", + "array_bool": "IntArray", + "array_num_int_1": "IntArray", + "array_num_int_2": "IntArray", + "array_num_float": "FloatArray", + "array_str": "StringArray", + "array_str_int_1": "StringArray", + "array_str_int_2": "StringArray", + "array_str_float": "StringArray", + "array_str_date_1": "DateTimeArray", + "array_str_date_2": "DateTimeArray", + "array_str_time_rfc3339": "DateTimeArray", + "array_str_time_clickhouse": "DateTimeArray", + "array_obj": "StringArray", } var csvSample = []byte(`null,true,false,123,123.321,"escaped_""ws",123,123.321,2009-07-13,13/07/2009,2009-07-13T09:07:13Z,2009-07-13T09:07:13.123+08:00,2009-07-13 09:07:13,2009-07-13 09:07:13.123,"{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",[],[null],"[true,false]","[0,255,256,65535,65536,4294967295,4294967296,18446744073709551615,18446744073709551616]","[-9223372036854775808,-2147483649,-2147483648,-32769,-32768,-129,-128,0,127,128,32767,32768,2147483647,2147483648,9223372036854775807]","[4.940656458412465441765687928682213723651e-324,1.401298464324817070923729583289916131280e-45,0.0,3.40282346638528859811704183484516925440e+38,1.797693134862315708145274237317043567981e+308]","[""aa"",""bb"",""cc""]","[""0"",""255"",""256"",""65535"",""65536"",""4294967295"",""4294967296"",""18446744073709551615"",""18446744073709551616""]","[""-9223372036854775808"",""-2147483649"",""-2147483648"",""-32769"",""-32768"",""-129"",""-128"",""0"",""127"",""128"",""32767"",""32768"",""2147483647"",""2147483648"",""9223372036854775807""]","[""4.940656458412465441765687928682213723651e-324"",""1.401298464324817070923729583289916131280e-45"",""0.0"",""3.40282346638528859811704183484516925440e+38"",""1.797693134862315708145274237317043567981e+308""]","[""2009-07-13"",""2009-07-14"",""2009-07-15""]","[""13/07/2009"",""14/07/2009"",""15/07/2009""]","[""2009-07-13T09:07:13Z"",""2009-07-13T09:07:13+08:00"",""2009-07-13T09:07:13.123Z"",""2009-07-13T09:07:13.123+08:00""]","[""2009-07-13 09:07:13"",""2009-07-13 09:07:13.123""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]"`) @@ -504,8 +504,7 @@ func TestParserArray(t *testing.T) { for j := range testCases { var v interface{} desc := fmt.Sprintf(`%s.GetArray("%s", %d)`, name, testCases[j].Field, testCases[j].Type) - if (name == "gjson" && testCases[j].Field == "array_num_float") || - (name == "csv" && sliceContains([]string{"array_num_float", "array_str_float"}, testCases[j].Field)) { + if name == "csv" && sliceContains([]string{"array_num_float", "array_str_float"}, testCases[j].Field) { skipped = append(skipped, desc) continue } @@ -661,6 +660,41 @@ func TestParseInt(t *testing.T) { } } +func TestFastjsonDetectSchema(t *testing.T) { + pp, _ := NewParserPool("fastjson", nil, "", "") + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + act := make(map[string]string) + c, _ := metric.(*FastjsonMetric) + var obj *fastjson.Object + var err error + if obj, err = c.value.Object(); err != nil { + return + } + obj.Visit(func(key []byte, v *fastjson.Value) { + act[string(key)] = model.GetTypeName(fjDetectType(v)) + }) + require.Equal(t, jsonSchema, act) +} + +func TestGjsonDetectSchema(t *testing.T) { + pp, _ := NewParserPool("gjson", nil, "", "") + parser := pp.Get() + defer pp.Put(parser) + metric, _ := parser.Parse(jsonSample) + + act := make(map[string]string) + c, _ := metric.(*GjsonMetric) + obj := gjson.Parse(c.raw) + obj.ForEach(func(k, v gjson.Result) bool { + act[k.Str] = model.GetTypeName(gjDetectType(v)) + return true + }) + require.Equal(t, jsonSchema, act) +} + func BenchmarkUnmarshalljson(b *testing.B) { object := map[string]interface{}{} for i := 0; i < b.N; i++ { From 0391e931e1768648740c33807465a0ab0ea765e6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 23 Oct 2021 11:27:00 +0800 Subject: [PATCH 188/404] added gjson dynamicSchema support --- config/config.go | 2 +- docs/configuration/config.md | 2 +- model/value.go | 4 ++-- parser/fastjson.go | 3 ++- parser/gjson.go | 43 +++++++++++++++++++----------------- parser/parser_test.go | 11 ++++----- 6 files changed, 35 insertions(+), 30 deletions(-) diff --git a/config/config.go b/config/config.go index da8fd928..b747c9b4 100644 --- a/config/config.go +++ b/config/config.go @@ -241,7 +241,7 @@ func (cfg *Config) Normallize() (err error) { taskCfg.AutoSchema = true } if taskCfg.DynamicSchema.Enable { - if taskCfg.Parser != "fastjson" { + if taskCfg.Parser != "fastjson" && taskCfg.Parser != "gjson" { err = errors.Errorf("Parser %s doesn't support DynamicSchema", taskCfg.Parser) return } diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 7ddef4ba..6c68d85a 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -132,7 +132,7 @@ // these columns will be excluded from the detected table schema. This takes effect only if "autoSchema" is true. "excludeColumns": [] - // (experiment feature) detect new fields and their type, and add columns to the ClickHouse table accordingly. This feature requires parser be "fastjson", and support following ClickHouse data types: Int64, Float64, String. + // (experiment feature) detect new fields and their type, and add columns to the ClickHouse table accordingly. This feature requires parser be "fastjson" or "gjson". New fields' type will be one of: Int64, Float64, String. "dynamicSchema": { // whether enable this feature, default to false "enable": true, diff --git a/model/value.go b/model/value.go index 965956d7..360af0dc 100644 --- a/model/value.go +++ b/model/value.go @@ -22,7 +22,7 @@ import ( ) const ( - TypeUnknown = iota + Unknown = iota Int Float String @@ -64,7 +64,7 @@ func GetTypeName(typ int) (name string) { case DateTimeArray: name = "DateTimeArray" default: - name = "TypeUnknown" + name = "Unknown" } return } diff --git a/parser/fastjson.go b/parser/fastjson.go index c94d22e1..ce7fad04 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -218,7 +218,7 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (fou obj.Visit(func(key []byte, v *fastjson.Value) { strKey := string(key) if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { - if typ := fjDetectType(v); typ != model.TypeUnknown { + if typ := fjDetectType(v); typ != model.Unknown { newKeys.Store(strKey, typ) foundNew = true } else { @@ -295,6 +295,7 @@ func getDefaultDateTime(nullable bool) (val interface{}) { func fjDetectType(v *fastjson.Value) (typ int) { switch v.Type() { case fastjson.TypeNull: + typ = model.Unknown case fastjson.TypeTrue: typ = model.Int case fastjson.TypeFalse: diff --git a/parser/gjson.go b/parser/gjson.go index 032b1c0c..55e96da6 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -131,7 +131,7 @@ func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) (val interfa func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { r := gjson.Get(c.raw, key) - if !r.Exists() || r.Type != gjson.JSON { + if !r.IsArray() { val = makeArray(typ) return } @@ -210,7 +210,7 @@ func (c *GjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (foundN gjson.Parse(c.raw).ForEach(func(k, v gjson.Result) bool { strKey := k.Str if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { - if typ := gjDetectType(v); typ != model.TypeUnknown { + if typ := gjDetectType(v); typ != model.Unknown { newKeys.Store(strKey, typ) foundNew = true } else { @@ -265,6 +265,7 @@ func gjCompatibleDateTime(r gjson.Result) (ok bool) { } func gjDetectType(v gjson.Result) (typ int) { + typ = model.Unknown switch v.Type { case gjson.True, gjson.False: typ = model.Int @@ -275,32 +276,34 @@ func gjDetectType(v gjson.Result) (typ int) { } case gjson.String: typ = model.String - if _, layout := parseInLocation(string(v.Str), time.Local); layout != "" { + if _, layout := parseInLocation(v.Str, time.Local); layout != "" { typ = model.DateTime } case gjson.JSON: - typ = model.String - array := v.Array() - if array != nil { - switch array[0].Type { - case gjson.True, gjson.False: - typ = model.IntArray - case gjson.Number: - typ = model.FloatArray - if float64(v.Int()) == v.Num { + if v.IsObject() { + typ = model.String + } else if v.IsArray() { + if array := v.Array(); len(array) != 0 { + switch array[0].Type { + case gjson.True, gjson.False: typ = model.IntArray + case gjson.Number: + typ = model.FloatArray + if float64(array[0].Int()) == array[0].Num { + typ = model.IntArray + } + case gjson.String: + typ = model.StringArray + if _, layout := parseInLocation(array[0].Str, time.Local); layout != "" { + typ = model.DateTimeArray + } + case gjson.JSON: + typ = model.StringArray + default: } - case gjson.String: - typ = model.StringArray - if _, layout := parseInLocation(string(v.Str), time.Local); layout != "" { - typ = model.DateTimeArray - } - default: - typ = model.StringArray } } default: - typ = model.String } return } diff --git a/parser/parser_test.go b/parser/parser_test.go index 7f207a1f..5ec14846 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -69,7 +69,7 @@ var jsonSample = []byte(`{ }`) var jsonSchema = map[string]string{ - "null": "null", + "null": "Unknown", "bool_true": "Int", "bool_false": "Int", "num_int": "Int", @@ -84,8 +84,8 @@ var jsonSchema = map[string]string{ "str_time_clickhouse_1": "DateTime", "str_time_clickhouse_2": "DateTime", "obj": "String", - "array_empty": "String", - "array_null": "String", + "array_empty": "Unknown", + "array_null": "Unknown", "array_bool": "IntArray", "array_num_int_1": "IntArray", "array_num_int_2": "IntArray", @@ -503,8 +503,9 @@ func TestParserArray(t *testing.T) { var skipped []string for j := range testCases { var v interface{} - desc := fmt.Sprintf(`%s.GetArray("%s", %d)`, name, testCases[j].Field, testCases[j].Type) - if name == "csv" && sliceContains([]string{"array_num_float", "array_str_float"}, testCases[j].Field) { + desc := fmt.Sprintf(`%s.GetArray("%s", %s)`, name, testCases[j].Field, model.GetTypeName(testCases[j].Type)) + if (name == "gjson" && testCases[j].Field == "array_num_float") || + (name == "csv" && sliceContains([]string{"array_num_float", "array_str_float"}, testCases[j].Field)) { skipped = append(skipped, desc) continue } From 710a75bff16fc9b52fb417781ef10308d9918d5a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 23 Oct 2021 15:14:46 +0800 Subject: [PATCH 189/404] updated dependencies --- go.mod | 20 ++++++++--------- go.sum | 71 ++++++++++++++++++++++++++++++++++------------------------ 2 files changed, 51 insertions(+), 40 deletions(-) diff --git a/go.mod b/go.mod index ce16a06f..7edfc2d8 100644 --- a/go.mod +++ b/go.mod @@ -3,35 +3,33 @@ module github.com/housepower/clickhouse_sinker go 1.14 require ( - github.com/ClickHouse/clickhouse-go v1.4.7 + github.com/ClickHouse/clickhouse-go v1.5.1 github.com/RoaringBitmap/roaring v0.9.4 - github.com/Shopify/sarama v1.29.1 + github.com/Shopify/sarama v1.30.0 github.com/bytedance/sonic v1.0.0-rc github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 - github.com/golang/snappy v0.0.4 // indirect github.com/google/gops v0.3.18 github.com/jinzhu/copier v0.3.2 - github.com/klauspost/compress v1.13.5 // indirect github.com/nacos-group/nacos-sdk-go v1.0.7 - github.com/pierrec/lz4 v2.6.1+incompatible // indirect github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.11.0 github.com/prometheus/common v0.26.0 - github.com/segmentio/kafka-go v0.4.17 - github.com/sirupsen/logrus v1.6.0 + github.com/segmentio/kafka-go v0.4.22 + github.com/sirupsen/logrus v1.8.1 github.com/smartystreets/goconvey v1.6.4 // indirect github.com/stretchr/testify v1.7.0 github.com/tidwall/gjson v1.10.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.0.2 + github.com/xdg/scram v1.0.3 // indirect + github.com/xdg/stringprep v1.0.3 // indirect go.uber.org/atomic v1.9.0 // indirect go.uber.org/multierr v1.7.0 // indirect - go.uber.org/zap v1.19.0 - golang.org/x/crypto v0.0.0-20210817164053-32db794688a5 // indirect - golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f // indirect + go.uber.org/zap v1.19.1 + golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 // indirect + golang.org/x/net v0.0.0-20211020060615-d418f374d309 // indirect golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba - golang.org/x/tools v0.0.0-20200103221440-774c71fcf114 // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) diff --git a/go.sum b/go.sum index ad121798..9b0b34f8 100644 --- a/go.sum +++ b/go.sum @@ -1,16 +1,16 @@ cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/clickhouse-go v1.4.7 h1:NNZQmlW8dVxGn19pF65BmWr0vq8Pj5Iy8ykyBKhFCPw= -github.com/ClickHouse/clickhouse-go v1.4.7/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= +github.com/ClickHouse/clickhouse-go v1.5.1 h1:I8zVFZTz80crCs0FFEBJooIxsPcV0xfthzK1YrkpJTc= +github.com/ClickHouse/clickhouse-go v1.5.1/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v0.9.4 h1:ckvZSX5gwCRaJYBNe7syNawCU5oruY9gQmjXlp4riwo= github.com/RoaringBitmap/roaring v0.9.4/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= -github.com/Shopify/sarama v1.29.1 h1:wBAacXbYVLmWieEA/0X/JagDdCZ8NVFOfS6l6+2u5S0= -github.com/Shopify/sarama v1.29.1/go.mod h1:mdtqvCSg8JOxk8PmpTNGyo6wzd4BMm4QXSfDnTXmgkE= -github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= -github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= +github.com/Shopify/sarama v1.30.0 h1:TOZL6r37xJBDEMLx4yjB77jxbZYXPaDow08TSK6vIL0= +github.com/Shopify/sarama v1.30.0/go.mod h1:zujlQQx1kzHsh4jfV1USnptCQrHAEZ2Hk8fTKCulPVs= +github.com/Shopify/toxiproxy/v2 v2.1.6-0.20210914104332-15ea381dcdae h1:ePgznFqEG1v3AjMklnK8H7BSc++FDSo7xfK9K7Af+0Y= +github.com/Shopify/toxiproxy/v2 v2.1.6-0.20210914104332-15ea381dcdae/go.mod h1:/cvHQkZ1fst0EmZnA5dFtiQdWCNCFYzb+uE2vqVgvx0= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= @@ -40,6 +40,7 @@ github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06 h1:1sDoSuDPWzhk github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= +github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -87,7 +88,6 @@ github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw github.com/golang/protobuf v1.4.3 h1:JjCZWpVbqXDqFVmTfYWEVTMIYrL/NPdPSCHPJ0T/raM= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -102,6 +102,7 @@ github.com/google/gops v0.3.18/go.mod h1:Pfp8hWGIFdV/7rY9/O/U5WgdjYQXf/GiEK4NVuV github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/securecookie v1.1.1 h1:miw7JPhV+b/lAHSXz4qd/nN9jRiAFV5FwjeKyCS8BvQ= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1 h1:DHd3rPN5lE3Ts3D8rKkQ8x/0kqfeNmBAaiSi+o7FsgI= @@ -143,13 +144,11 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8 github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= -github.com/klauspost/compress v1.13.5 h1:9O69jUPDcsT9fEm74W92rZL9FQY7rCdaXVneq+yyzl4= -github.com/klauspost/compress v1.13.5/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= +github.com/klauspost/compress v1.13.6 h1:P76CopJELS0TiO2mebmnzgWaajssP/EszplttgQxcgc= +github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/cpuid/v2 v2.0.9 h1:lgaqFMSdTdQYdZ04uHyN2d/eKdOMyi2YLSvlQIBFYa4= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -216,13 +215,16 @@ github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1 github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/segmentio/kafka-go v0.4.17 h1:IyqRstL9KUTDb3kyGPOOa5VffokKWSEzN6geJ92dSDY= -github.com/segmentio/kafka-go v0.4.17/go.mod h1:19+Eg7KwrNKy/PFhiIthEPkO8k+ac7/ZYXwYM9Df10w= +github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/segmentio/kafka-go v0.4.22 h1:F4k2OTm9Y4+zliuoXgNKJZTktE0miQioZZzofsjhRdk= +github.com/segmentio/kafka-go v0.4.22/go.mod h1:XzMcoMjSzDGHcIwpWUI7GB43iKZ2fTVmryPSGLf/MPg= github.com/shirou/gopsutil/v3 v3.21.2/go.mod h1:ghfMypLDrFSWN2c9cDYFLHyynQ+QUht0cv/18ZqVczw= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= +github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= @@ -258,6 +260,7 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNij github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= +github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= github.com/valyala/fastjson v1.6.3/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= @@ -273,20 +276,21 @@ github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0 github.com/xdg/stringprep v1.0.3 h1:cmL5Enob4W83ti/ZHuZLuKD/xqJfus4fVPwE+/BDm+4= github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= github.com/xlab/treeprint v1.0.0/go.mod h1:IoImgRak9i3zJyuxOKUP1v4UZd1tMoKkq/Cimt1uhCg= +github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10 h1:z+mqJhf6ss6BSfSM671tgKyZBFPTTJM+HLxnhPC3wu0= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= +go.uber.org/goleak v1.1.11-0.20210813005559-691160354723 h1:sHOAIxRGBp443oHZIPB+HsUGaksVCXVQENPxwTfQdH4= +go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/multierr v1.7.0 h1:zaiO/rmgFjbmCXdSYJWQcdvOCsthmdaHfr3Gm2Kx4Ec= go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.19.0 h1:mZQZefskPPCMIBCSEH0v2/iUqqLrYtaeqwD6FUGUnFE= -go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= +go.uber.org/zap v1.19.1 h1:ue41HOKd1vGURxrmeKIgELGb3jPW9DMUDGtsinblHwI= +go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190506204251-e1dfcc566284/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -294,13 +298,14 @@ golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20210817164053-32db794688a5 h1:HWj/xjIHfjYU5nVXpTM0s39J9CbLn7Cc5a7IC5rwsMQ= -golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20210920023735-84f357641f63/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 h1:7I4JAnoQBe7ZtJcBaYHi5UtiO8tQHbUSXxL+pnGRANg= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= -golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.4.2 h1:Gz96sIWK3OalVv/I/qNygP42zyoKp3xptRVCWRFEBvo= +golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -311,20 +316,23 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210614182718-04defd469f4e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f h1:w6wWR0H+nyVpbSAQbzVEIACVyr/h8l/BEkY6Sokc7Eg= -golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210917221730-978cfadd31cf/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211020060615-d418f374d309 h1:A0lJIi+hcTR6aajJH4YqKWwohY4aW9RO7oRMcdv+HKI= +golang.org/x/net v0.0.0-20211020060615-d418f374d309/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -333,7 +341,9 @@ golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 h1:SrN+KX8Art/Sf4HNj6Zcz06G7VEz+7w9tdXTPOZ7+l4= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -342,8 +352,9 @@ golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba h1:O8mE0/t419eoIwhTFpKVkHiTs/Igowgfkj25AcZrtiE= golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -353,9 +364,9 @@ golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBn golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20200103221440-774c71fcf114 h1:DnSr2mCsxyCE6ZgIkmcWUQY2R5cH/6wL7eIxEmQOMSE= -golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.1.5 h1:ouewzE6p+/VEB31YYnTbEJdi8pFqKp4P4n85vwo3DHA= +golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -381,8 +392,10 @@ gopkg.in/ini.v1 v1.42.0 h1:7N3gPTt50s8GuLortA00n8AqRTk75qOP98+mTPpgzRk= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= From f32aee6026f8e5b8e2b2f121b3016369f80cbc1b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 23 Oct 2021 15:59:19 +0800 Subject: [PATCH 190/404] removed logrus --- cmd/kafka_gen_log/main.go | 34 +++++++++++++++++----------------- cmd/kafka_gen_metric/main.go | 19 ++++++++++--------- go.mod | 1 - go.sum | 1 - 4 files changed, 27 insertions(+), 28 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 11a9fd61..490e0eec 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -57,7 +57,7 @@ import ( "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) var ( @@ -179,7 +179,7 @@ func (g *LogGenerator) Init() error { return err } sort.Strings(g.logfiles) - log.Infof("Following files under %v match pattern %v: %+v", LogfileDir, LogfilePattern, g.logfiles) + util.Logger.Info(fmt.Sprintf("Following files under %v match pattern %v: %+v", LogfileDir, LogfilePattern, g.logfiles)) if err := g.next(); err != nil { return err @@ -191,7 +191,6 @@ func (g *LogGenerator) Init() error { func (g *LogGenerator) next() (err error) { g.scanner = nil if g.reader != nil { - log.Debugf("closed %+v", g.fp) g.reader.Close() g.reader = nil } @@ -204,11 +203,11 @@ func (g *LogGenerator) next() (err error) { if reader, err = os.Open(g.fp); err == nil { g.reader = reader g.scanner = bufio.NewScanner(g.reader) - log.Debugf("scanning %+v", g.fp) + util.Logger.Debug(fmt.Sprintf("scanning %+v", g.fp)) return nil } err = errors.Wrapf(err, "") - log.Infof("failed to open %+v, %+v", g.fp, err) + util.Logger.Fatal("os.Open failed", zap.String("path", g.fp), zap.Error(err)) time.Sleep(6000 * time.Second) } err = errors.Errorf("no readable file") @@ -221,11 +220,11 @@ func (g *LogGenerator) getLine() (fp string, lineno int, line string) { g.lineno++ return g.fp, g.lineno, g.scanner.Text() } - if g.scanner.Err() != nil { - log.Warnf("Scan %+v", g.scanner.Err()) + if err := g.scanner.Err(); err != nil { + util.Logger.Fatal("got error", zap.Error(err)) } if err := g.next(); err != nil { - log.Fatalf("got error %+v", err) + util.Logger.Fatal("got error", zap.Error(err)) } } } @@ -240,7 +239,7 @@ func (g *LogGenerator) Run() { config.Version = sarama.V2_1_0_0 w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) if err != nil { - log.Fatalf("sarama.NewAsyncProducer failed %+v", err) + util.Logger.Fatal("sarama.NewAsyncProducer failed", zap.Error(err)) } defer w.Close() chInput := w.Input() @@ -282,7 +281,7 @@ func (g *LogGenerator) Run() { _ = wp.Submit(func() { if b, err = sonic.Marshal(&logObj); err != nil { err = errors.Wrapf(err, "") - log.Fatalf("got error %+v", err) + util.Logger.Fatal("got error", zap.Error(err)) } chInput <- &sarama.ProducerMessage{ Topic: KafkaTopic, @@ -297,6 +296,7 @@ func (g *LogGenerator) Run() { } func main() { + util.InitLogger([]string{"stdout"}) flag.Usage = func() { usage := fmt.Sprintf(`Usage of %s %s kakfa_brokers topic log_file_dir log_file_pattern @@ -305,28 +305,28 @@ kakfa_brokers: for example, 192.168.102.114:9092,192.168.102.115:9092 topic: for example, apache_access_log log_file_dir: log file directory, for example, /var/log log_file_pattern: file name pattern, for example, '^secure.*$'`, os.Args[0], os.Args[0]) - log.Infof(usage) + fmt.Println(usage) + os.Exit(0) } flag.Parse() args := flag.Args() if len(args) != 4 { flag.Usage() - log.Fatal("Invalid CLI arguments!") } KafkaBrokers = args[0] KafkaTopic = args[1] LogfileDir = args[2] LogfilePattern = args[3] - log.Infof("KafkaBrokers: %v\nKafkaTopic: %v\nLogfileDir: %v\nLogFilePattern: %v\n", KafkaBrokers, KafkaTopic, LogfileDir, LogfilePattern) + util.Logger.Info("CLI options", zap.String("KafkaBrokers", KafkaBrokers), zap.String("KafkaTopic", KafkaTopic), zap.String("LogfileDir", LogfileDir), zap.String("LogFilePattern", LogfilePattern)) if err := agent.Listen(agent.Options{}); err != nil { - log.Fatal(err) + util.Logger.Fatal("got error", zap.Error(err)) } ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) g := &LogGenerator{} if err := g.Init(); err != nil { - log.Fatalf("got error %+v", err) + util.Logger.Fatal("got error", zap.Error(err)) } go g.Run() @@ -336,7 +336,7 @@ LOOP: for { select { case <-ctx.Done(): - log.Infof("quit due to context been canceled") + util.Logger.Info("quit due to context been canceled") break LOOP case <-ticker.C: var speedLine, speedSize int64 @@ -347,7 +347,7 @@ LOOP: } prevLines = lines prevSize = size - log.Infof("generated %+v lines, %+v Bytes, speedLine: %v lines/s, speedSize: %v B/s", lines, size, speedLine, speedSize) + util.Logger.Info("status", zap.Int64("lines", lines), zap.Int64("bytes", size), zap.Int64("speed(lines/s)", speedLine), zap.Int64("speed(bytes/s)", speedSize)) } } } diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index 011e6b57..876535b4 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -49,7 +49,7 @@ import ( "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) const ( @@ -118,7 +118,7 @@ func generate() { config.Version = sarama.V2_1_0_0 w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) if err != nil { - log.Fatalf("sarama.NewAsyncProducer failed %+v", err) + util.Logger.Error("sarama.NewAsyncProducer failed", zap.Error(err)) } defer w.Close() chInput := w.Input() @@ -158,7 +158,7 @@ func generate() { var b []byte if b, err = sonic.Marshal(&metric); err != nil { err = errors.Wrapf(err, "") - log.Fatalf("got error %+v", err) + util.Logger.Fatal("got error", zap.Error(err)) } chInput <- &sarama.ProducerMessage{ Topic: KafkaTopic, @@ -175,26 +175,27 @@ func generate() { } func main() { + util.InitLogger([]string{"stdout"}) flag.Usage = func() { usage := fmt.Sprintf(`Usage of %s %s kakfa_brokers topic This util fill some fields with random content, serialize and send to kafka. kakfa_brokers: for example, 192.168.102.114:9092,192.168.102.115:9092 topic: for example, sensor_dt_result_online`, os.Args[0], os.Args[0]) - log.Infof(usage) + fmt.Println(usage) + os.Exit(0) } flag.Parse() args := flag.Args() if len(args) != 2 { flag.Usage() - log.Fatal("Invalid CLI arguments!") } KafkaBrokers = args[0] KafkaTopic = args[1] - log.Infof("KafkaBrokers: %v\nKafkaTopic: %v\nBusinessNum: %v\nInstanceNum: %v\n", KafkaBrokers, KafkaTopic, BusinessNum, InstanceNum) + util.Logger.Info("CLI options", zap.String("KafkaBrokers", KafkaBrokers), zap.String("KafkaTopic", KafkaTopic), zap.Int("BusinessNum", BusinessNum), zap.Int("InstanceNum", InstanceNum)) if err := agent.Listen(agent.Options{}); err != nil { - log.Fatal(err) + util.Logger.Fatal("got error", zap.Error(err)) } var prevLines, prevSize int64 @@ -206,7 +207,7 @@ LOOP: for { select { case <-ctx.Done(): - log.Infof("quit due to context been canceled") + util.Logger.Info("quit due to context been canceled") break LOOP case <-ticker.C: var speedLine, speedSize int64 @@ -216,7 +217,7 @@ LOOP: } prevLines = gLines prevSize = gSize - log.Infof("generated %+v lines, %+v Bytes, speedLine: %v lines/s, speedSize: %v B/s", gLines, gSize, speedLine, speedSize) + util.Logger.Info("status", zap.Int64("lines", gLines), zap.Int64("bytes", gSize), zap.Int64("speed(lines/s)", speedLine), zap.Int64("speed(bytes/s)", speedSize)) } } } diff --git a/go.mod b/go.mod index 7edfc2d8..af42677c 100644 --- a/go.mod +++ b/go.mod @@ -16,7 +16,6 @@ require ( github.com/prometheus/client_golang v1.11.0 github.com/prometheus/common v0.26.0 github.com/segmentio/kafka-go v0.4.22 - github.com/sirupsen/logrus v1.8.1 github.com/smartystreets/goconvey v1.6.4 // indirect github.com/stretchr/testify v1.7.0 github.com/tidwall/gjson v1.10.1 diff --git a/go.sum b/go.sum index 9b0b34f8..31f7a002 100644 --- a/go.sum +++ b/go.sum @@ -223,7 +223,6 @@ github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeV github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= From 241cfb49de1424a1c4e328bf578c3507dc1acf84 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 25 Oct 2021 11:21:06 +0800 Subject: [PATCH 191/404] fix idle --- task/ring.go | 56 ++++++++++++++++++++++------------------------------ task/task.go | 14 ++----------- 2 files changed, 26 insertions(+), 44 deletions(-) diff --git a/task/ring.go b/task/ring.go index 2ac79a72..56bda293 100644 --- a/task/ring.go +++ b/task/ring.go @@ -40,14 +40,8 @@ func (ring *Ring) QuitIdle() { ring.idleCnt = 0 ring.isIdle = false ring.ringBuf = make([]model.MsgRow, ring.ringCap) - var err error - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(ring.service.taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { - if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Info("Ring.QuitIdle scheduling timer to a stopped timer wheel", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) - } else { - util.Logger.Fatal("scheduling timer failed", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) - } - } + util.Logger.Info(fmt.Sprintf("topic %s partition %d quit idle", ring.service.taskCfg.Topic, ring.partition), zap.String("task", ring.service.taskCfg.Name)) + ring.scheduleForchBatchOrShard() } } @@ -84,15 +78,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { } if (ring.ringFilledOffset >> ring.batchSizeShift) != (ring.ringGroundOff >> ring.batchSizeShift) { ring.genBatchOrShard() - // reschedule the delayed ForceBatchOrShard - ring.tid.Stop() - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { - if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Info("Ring.PutElem scheduling timer to a stopped timer wheel", zap.String("task", taskCfg.Name), zap.Error(err)) - } else { - util.Logger.Fatal("scheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - } + ring.scheduleForchBatchOrShard() } } @@ -147,29 +133,35 @@ func (ring *Ring) ForceBatchOrShard(_ interface{}) { taskCfg := ring.service.taskCfg ring.mux.Lock() defer ring.mux.Unlock() - ring.tid.Stop() if !ring.isIdle { - if ring.ringFilledOffset > ring.ringGroundOff { - ring.genBatchOrShard() - ring.idleCnt = 0 - // reschedule the delayed ForceBatchOrShard - var err error - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { - if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Warn("Ring.ForceBatchOrShard scheduling timer to a stopped timer wheel", zap.String("task", taskCfg.Name), zap.Error(err)) - } else { - err = errors.Wrap(err, "") - util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - } - } else if ring.ringGroundOff == ring.ringCeilingOff { + if ring.ringGroundOff == ring.ringCeilingOff { ring.idleCnt++ if ring.idleCnt >= 2 { ring.idleCnt = 0 ring.isIdle = true ring.ringBuf = nil util.Logger.Info(fmt.Sprintf("topic %s partition %d became idle", taskCfg.Topic, ring.partition), zap.String("task", taskCfg.Name)) + return } + } else if ring.ringFilledOffset > ring.ringGroundOff { + ring.genBatchOrShard() + ring.idleCnt = 0 + } + ring.scheduleForchBatchOrShard() + } +} + +// schedule ForchBatchOrShard +// assume ring.mux is locked +func (ring *Ring) scheduleForchBatchOrShard() { + var err error + ring.tid.Stop() + if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(ring.service.taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { + if errors.Is(err, goetty.ErrSystemStopped) { + util.Logger.Warn("Ring.ForceBatchOrShard scheduling timer to a stopped timer wheel", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) + } else { + err = errors.Wrap(err, "") + util.Logger.Fatal("scheduling timer filed", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) } } } diff --git a/task/task.go b/task/task.go index 105a765a..c0f3191c 100644 --- a/task/task.go +++ b/task/task.go @@ -165,12 +165,11 @@ func (service *Service) putToRing(msg *model.InputMessage) (ok bool) { } } - var err error if ring == nil { batchSizeShift := util.GetShift(taskCfg.BufferSize) ringCap := int64(1 << (batchSizeShift + 1)) ring := &Ring{ - ringBuf: make([]model.MsgRow, ringCap), + ringBuf: nil, ringCap: ringCap, ringCapMask: ringCap - 1, ringGroundOff: msg.Offset, @@ -178,22 +177,13 @@ func (service *Service) putToRing(msg *model.InputMessage) (ok bool) { ringFilledOffset: msg.Offset, batchSizeShift: batchSizeShift, idleCnt: 0, - isIdle: false, + isIdle: true, partition: msg.Partition, batchSys: model.NewBatchSys(taskCfg, service.fnCommit), service: service, } ring.available = sync.NewCond(&ring.mux) ring.PutMsgNolock(msg) - // schedule a delayed ForceBatchOrShard - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { - if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Info("Service.put scheduling timer to a stopped timer wheel") - } else { - err = errors.Wrap(err, "") - util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - } service.rings[msg.Partition] = ring service.Unlock() ok = true From cb99978d18d459952ac3022c97a6779c5a836566 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 29 Oct 2021 17:18:47 +0800 Subject: [PATCH 192/404] detect metric name key --- cmd/kafka_gen_log/main.go | 8 ++++++-- cmd/kafka_gen_metric/main.go | 8 ++++++-- model/message.go | 4 ++-- output/clickhouse.go | 11 +++++++++++ task/task.go | 7 +++++-- 5 files changed, 30 insertions(+), 8 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 490e0eec..01afe336 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -305,7 +305,7 @@ kakfa_brokers: for example, 192.168.102.114:9092,192.168.102.115:9092 topic: for example, apache_access_log log_file_dir: log file directory, for example, /var/log log_file_pattern: file name pattern, for example, '^secure.*$'`, os.Args[0], os.Args[0]) - fmt.Println(usage) + util.Logger.Info(usage) os.Exit(0) } flag.Parse() @@ -317,7 +317,11 @@ log_file_pattern: file name pattern, for example, '^secure.*$'`, os.Args[0], os. KafkaTopic = args[1] LogfileDir = args[2] LogfilePattern = args[3] - util.Logger.Info("CLI options", zap.String("KafkaBrokers", KafkaBrokers), zap.String("KafkaTopic", KafkaTopic), zap.String("LogfileDir", LogfileDir), zap.String("LogFilePattern", LogfilePattern)) + util.Logger.Info("CLI options", + zap.String("KafkaBrokers", KafkaBrokers), + zap.String("KafkaTopic", KafkaTopic), + zap.String("LogfileDir", LogfileDir), + zap.String("LogFilePattern", LogfilePattern)) if err := agent.Listen(agent.Options{}); err != nil { util.Logger.Fatal("got error", zap.Error(err)) diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index 876535b4..09c2dc97 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -182,7 +182,7 @@ func main() { This util fill some fields with random content, serialize and send to kafka. kakfa_brokers: for example, 192.168.102.114:9092,192.168.102.115:9092 topic: for example, sensor_dt_result_online`, os.Args[0], os.Args[0]) - fmt.Println(usage) + util.Logger.Info(usage) os.Exit(0) } flag.Parse() @@ -192,7 +192,11 @@ topic: for example, sensor_dt_result_online`, os.Args[0], os.Args[0]) } KafkaBrokers = args[0] KafkaTopic = args[1] - util.Logger.Info("CLI options", zap.String("KafkaBrokers", KafkaBrokers), zap.String("KafkaTopic", KafkaTopic), zap.Int("BusinessNum", BusinessNum), zap.Int("InstanceNum", InstanceNum)) + util.Logger.Info("CLI options", + zap.String("KafkaBrokers", KafkaBrokers), + zap.String("KafkaTopic", KafkaTopic), + zap.Int("BusinessNum", BusinessNum), + zap.Int("InstanceNum", InstanceNum)) if err := agent.Listen(agent.Options{}); err != nil { util.Logger.Fatal("got error", zap.Error(err)) diff --git a/model/message.go b/model/message.go index 66781a40..9e62e54b 100644 --- a/model/message.go +++ b/model/message.go @@ -168,7 +168,7 @@ func PutRow(r *Row) { rowPool.Put(r) } -func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSeriesID int) (row *Row) { +func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSeriesID int, nameKey string) (row *Row) { row = GetRow() var dig *xxhash.Digest var labels []string @@ -197,7 +197,7 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe _, _ = dig.WriteString(dim.Name) _, _ = dig.WriteString("###") _, _ = dig.WriteString(labelVal) - if dim.Name != "__name__" && dim.Name != "le" { + if dim.Name != nameKey && dim.Name != "le" { labels = append(labels, fmt.Sprintf(`"%s": "%s"`, dim.Name, labelVal)) } } diff --git a/output/clickhouse.go b/output/clickhouse.go index 26aaf9f0..a179a393 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -52,6 +52,7 @@ var ( type ClickHouse struct { Dims []*model.ColumnWithType IdxSerID int + NameKey string cfg *config.Config taskCfg *config.TaskConfig prepareSQL string @@ -266,6 +267,16 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { err = errors.Errorf(`First columns of %s are expect to be "__series_id UInt64, labels String".`, c.seriesTbl) return } + c.NameKey = "__name__" // prometheus uses internal "__name__" label for metric name + for i := 2; i < len(seriesDims); i++ { + serDim := seriesDims[i] + if serDim.Type == model.String { + if serDim.Name == c.NameKey { + break + } + c.NameKey = serDim.Name // opentsdb uses "metric" tag for metric name + } + } c.Dims = append(c.Dims, seriesDims[1:]...) // Generate SQL for series INSERT diff --git a/task/task.go b/task/task.go index c0f3191c..c7761078 100644 --- a/task/task.go +++ b/task/task.go @@ -44,7 +44,9 @@ type Service struct { cfg *config.Config taskCfg *config.TaskConfig dims []*model.ColumnWithType - idxSerID int + + idxSerID int + nameKey string knownKeys sync.Map newKeys sync.Map @@ -90,6 +92,7 @@ func (service *Service) Init() (err error) { service.dims = service.clickhouse.Dims service.idxSerID = service.clickhouse.IdxSerID + service.nameKey = service.clickhouse.NameKey service.limiter1 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter2 = rate.NewLimiter(rate.Every(10*time.Second), 1) @@ -274,7 +277,7 @@ func (service *Service) put(msg *model.InputMessage) { msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - row = model.MetricToRow(metric, msg, service.dims, service.idxSerID) + row = model.MetricToRow(metric, msg, service.dims, service.idxSerID, service.nameKey) if taskCfg.DynamicSchema.Enable { foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys) } From 3cea60fe531cb8d100a42676383e13377d28757d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 2 Nov 2021 10:16:41 +0800 Subject: [PATCH 193/404] added white and black list for dynamic key detection --- config/config.go | 105 ++++++++++++++++++++++++++++----------------- model/metric.go | 3 +- parser/csv.go | 3 +- parser/fastjson.go | 17 +++++--- parser/gjson.go | 17 +++++--- task/task.go | 11 ++++- 6 files changed, 104 insertions(+), 52 deletions(-) diff --git a/config/config.go b/config/config.go index b747c9b4..e8774fe7 100644 --- a/config/config.go +++ b/config/config.go @@ -18,6 +18,7 @@ package config import ( "encoding/json" "io/ioutil" + "regexp" "strings" "github.com/housepower/clickhouse_sinker/util" @@ -126,6 +127,13 @@ type TaskConfig struct { DynamicSchema struct { Enable bool MaxDims int // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack + // A column is added for new key K if all following conditions are true: + // - K isn't in ExcludeColumns + // - number of existing columns doesn't reach MaxDims-1 + // - WhiteList is empty, or K matchs WhiteList + // - BlackList is empty, or K doesn't match BlackList + WhiteList string // the regexp of white list + BlackList string // the regexp of black list } // PrometheusSchema expects each message is a Prometheus metric(timestamp, value, metric name and a list of labels). PrometheusSchema bool @@ -205,52 +213,71 @@ func (cfg *Config) Normallize() (err error) { cfg.Task = nil } for _, taskCfg := range cfg.Tasks { - if taskCfg.KafkaClient == "" || (cfg.Kafka.Sasl.Enable && cfg.Kafka.Sasl.Username == "") { - // known limitations of kafka-go: - // - The Reader API is too high-level. There's no generation cleanup callback which sarama provides. - // - Doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 - taskCfg.KafkaClient = "sarama" - } - if taskCfg.Parser == "" || taskCfg.Parser == "json" { - taskCfg.Parser = "fastjson" + if err = cfg.normallizeTask(taskCfg); err != nil { + return } + } + switch strings.ToLower(cfg.LogLevel) { + case "debug", "info", "warn", "error", "dpanic", "panic", "fatal": + default: + cfg.LogLevel = defaultLogLevel + } + return +} - for i := range taskCfg.Dims { - if taskCfg.Dims[i].SourceName == "" { - taskCfg.Dims[i].SourceName = util.GetSourceName(taskCfg.Dims[i].Name) - } - } +func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { + if taskCfg.KafkaClient == "" || (cfg.Kafka.Sasl.Enable && cfg.Kafka.Sasl.Username == "") { + // known limitations of kafka-go: + // - The Reader API is too high-level. There's no generation cleanup callback which sarama provides. + // - Doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 + taskCfg.KafkaClient = "sarama" + } + if taskCfg.Parser == "" || taskCfg.Parser == "json" { + taskCfg.Parser = "fastjson" + } - if taskCfg.FlushInterval <= 0 { - taskCfg.FlushInterval = defaultFlushInterval - } else if taskCfg.FlushInterval > maxFlushInterval { - taskCfg.FlushInterval = maxFlushInterval - } - if taskCfg.BufferSize <= 0 { - taskCfg.BufferSize = defaultBufferSize - } else if taskCfg.BufferSize > MaxBufferSize { - taskCfg.BufferSize = MaxBufferSize - } else { - taskCfg.BufferSize = 1 << util.GetShift(taskCfg.BufferSize) + for i := range taskCfg.Dims { + if taskCfg.Dims[i].SourceName == "" { + taskCfg.Dims[i].SourceName = util.GetSourceName(taskCfg.Dims[i].Name) } - if taskCfg.TimeZone == "" { - taskCfg.TimeZone = defaultTimeZone - } - if taskCfg.PrometheusSchema { - taskCfg.DynamicSchema.Enable = true - taskCfg.AutoSchema = true + } + + if taskCfg.FlushInterval <= 0 { + taskCfg.FlushInterval = defaultFlushInterval + } else if taskCfg.FlushInterval > maxFlushInterval { + taskCfg.FlushInterval = maxFlushInterval + } + if taskCfg.BufferSize <= 0 { + taskCfg.BufferSize = defaultBufferSize + } else if taskCfg.BufferSize > MaxBufferSize { + taskCfg.BufferSize = MaxBufferSize + } else { + taskCfg.BufferSize = 1 << util.GetShift(taskCfg.BufferSize) + } + if taskCfg.TimeZone == "" { + taskCfg.TimeZone = defaultTimeZone + } + if taskCfg.PrometheusSchema { + taskCfg.DynamicSchema.Enable = true + taskCfg.AutoSchema = true + } + if taskCfg.DynamicSchema.Enable { + if taskCfg.Parser != "fastjson" && taskCfg.Parser != "gjson" { + err = errors.Errorf("Parser %s doesn't support DynamicSchema", taskCfg.Parser) + return } - if taskCfg.DynamicSchema.Enable { - if taskCfg.Parser != "fastjson" && taskCfg.Parser != "gjson" { - err = errors.Errorf("Parser %s doesn't support DynamicSchema", taskCfg.Parser) - return - } + } + if taskCfg.DynamicSchema.WhiteList != "" { + if _, err = regexp.Compile(taskCfg.DynamicSchema.WhiteList); err != nil { + err = errors.Wrapf(err, "WhiteList %s is invalid regexp", taskCfg.DynamicSchema.WhiteList) + return } } - switch strings.ToLower(cfg.LogLevel) { - case "debug", "info", "warn", "error", "dpanic", "panic", "fatal": - default: - cfg.LogLevel = defaultLogLevel + if taskCfg.DynamicSchema.BlackList != "" { + if _, err = regexp.Compile(taskCfg.DynamicSchema.BlackList); err != nil { + err = errors.Wrapf(err, "BlackList %s is invalid regexp", taskCfg.DynamicSchema.BlackList) + return + } } return } diff --git a/model/metric.go b/model/metric.go index 0db781d4..9a17d9d8 100644 --- a/model/metric.go +++ b/model/metric.go @@ -16,6 +16,7 @@ limitations under the License. package model import ( + "regexp" "sync" ) @@ -27,7 +28,7 @@ type Metric interface { GetDateTime(key string, nullable bool) (val interface{}) GetElasticDateTime(key string, nullable bool) (val interface{}) GetArray(key string, t int) (val interface{}) - GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool + GetNewKeys(knownKeys, newKeys *sync.Map, white, black *regexp.Regexp) bool } // DimMetrics diff --git a/parser/csv.go b/parser/csv.go index d37372bf..bb7d180c 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -18,6 +18,7 @@ import ( "bytes" "encoding/csv" "fmt" + "regexp" "strconv" "sync" "time" @@ -224,6 +225,6 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { return } -func (c *CsvMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) bool { +func (c *CsvMetric) GetNewKeys(knownKeys, newKeys *sync.Map, white, black *regexp.Regexp) bool { return false } diff --git a/parser/fastjson.go b/parser/fastjson.go index ce7fad04..75ad5663 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -17,6 +17,7 @@ package parser import ( "fmt" + "regexp" "sync" "time" @@ -209,7 +210,7 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { return } -func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (foundNew bool) { +func (c *FastjsonMetric) GetNewKeys(knownKeys, newKeys *sync.Map, white, black *regexp.Regexp) (foundNew bool) { var obj *fastjson.Object var err error if obj, err = c.value.Object(); err != nil { @@ -218,11 +219,17 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (fou obj.Visit(func(key []byte, v *fastjson.Value) { strKey := string(key) if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { - if typ := fjDetectType(v); typ != model.Unknown { - newKeys.Store(strKey, typ) - foundNew = true + if (white == nil || white.MatchString(strKey)) && + (black == nil || !black.MatchString(strKey)) { + if typ := fjDetectType(v); typ != model.Unknown { + newKeys.Store(strKey, typ) + foundNew = true + } else { + util.Logger.Warn("FastjsonMetric.GetNewKeys failed to detect field type", zap.String("key", strKey), zap.String("value", v.String())) + } } else { - util.Logger.Warn("FastjsonMetric.GetNewKeys failed to detect field type", zap.String("key", strKey), zap.String("value", v.String())) + util.Logger.Warn("FastjsonMetric.GetNewKeys ignored new key due to white/black list setting", zap.String("key", strKey), zap.String("value", v.String())) + knownKeys.Store(strKey, nil) } } }) diff --git a/parser/gjson.go b/parser/gjson.go index 55e96da6..546e4066 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -17,6 +17,7 @@ package parser import ( "fmt" + "regexp" "sync" "time" @@ -206,15 +207,21 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { return } -func (c *GjsonMetric) GetNewKeys(knownKeys *sync.Map, newKeys *sync.Map) (foundNew bool) { +func (c *GjsonMetric) GetNewKeys(knownKeys, newKeys *sync.Map, white, black *regexp.Regexp) (foundNew bool) { gjson.Parse(c.raw).ForEach(func(k, v gjson.Result) bool { strKey := k.Str if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { - if typ := gjDetectType(v); typ != model.Unknown { - newKeys.Store(strKey, typ) - foundNew = true + if (white == nil || white.MatchString(strKey)) && + (black == nil || !black.MatchString(strKey)) { + if typ := gjDetectType(v); typ != model.Unknown { + newKeys.Store(strKey, typ) + foundNew = true + } else { + util.Logger.Warn("GjsonMetric.GetNewKeys failed to detect field type", zap.String("key", strKey), zap.String("value", v.String())) + } } else { - util.Logger.Warn("GjsonMetric.GetNewKeys failed to detect field type", zap.String("key", strKey), zap.String("value", v.String())) + util.Logger.Warn("GjsonMetric.GetNewKeys ignored new key due to white/black list setting", zap.String("key", strKey), zap.String("value", v.String())) + knownKeys.Store(strKey, nil) } } return true diff --git a/task/task.go b/task/task.go index c7761078..e71d424f 100644 --- a/task/task.go +++ b/task/task.go @@ -18,6 +18,7 @@ package task import ( "fmt" "math" + "regexp" "sync" "sync/atomic" "time" @@ -43,6 +44,8 @@ type Service struct { pp *parser.Pool cfg *config.Config taskCfg *config.TaskConfig + whiteList *regexp.Regexp + blackList *regexp.Regexp dims []*model.ColumnWithType idxSerID int @@ -77,6 +80,12 @@ func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) (service *Se taskCfg: taskCfg, } service.taskDone = sync.NewCond(service) + if taskCfg.DynamicSchema.WhiteList != "" { + service.whiteList = regexp.MustCompile(taskCfg.DynamicSchema.WhiteList) + } + if taskCfg.DynamicSchema.BlackList != "" { + service.blackList = regexp.MustCompile(taskCfg.DynamicSchema.BlackList) + } return } @@ -279,7 +288,7 @@ func (service *Service) put(msg *model.InputMessage) { } else { row = model.MetricToRow(metric, msg, service.dims, service.idxSerID, service.nameKey) if taskCfg.DynamicSchema.Enable { - foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys) + foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys, service.whiteList, service.blackList) } // Dumping message and result //util.Logger.Debug("parsed kafka message", zap.Int("partition", msg.Partition), zap.Int64("offset", msg.Offset), From eeb903bb9c60d359f1da1ec9280ea2530ff1f1bf Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 2 Nov 2021 11:42:19 +0800 Subject: [PATCH 194/404] fixed incorrect labels --- model/message.go | 15 ++++++++------- output/clickhouse.go | 4 +--- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/model/message.go b/model/message.go index 9e62e54b..873a813a 100644 --- a/model/message.go +++ b/model/message.go @@ -192,13 +192,14 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe val := GetValueByType(metric, dim) *row = append(*row, val) if idxSeriesID >= 0 && dim.Type == String && val != nil { - labelVal := val.(string) - _, _ = dig.WriteString("###") - _, _ = dig.WriteString(dim.Name) - _, _ = dig.WriteString("###") - _, _ = dig.WriteString(labelVal) - if dim.Name != nameKey && dim.Name != "le" { - labels = append(labels, fmt.Sprintf(`"%s": "%s"`, dim.Name, labelVal)) + if labelVal := val.(string); labelVal != "" { + _, _ = dig.WriteString("###") + _, _ = dig.WriteString(dim.Name) + _, _ = dig.WriteString("###") + _, _ = dig.WriteString(labelVal) + if dim.Name != nameKey && dim.Name != "le" { + labels = append(labels, fmt.Sprintf(`"%s": "%s"`, dim.Name, labelVal)) + } } } } diff --git a/output/clickhouse.go b/output/clickhouse.go index a179a393..939e41e7 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -271,10 +271,8 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { for i := 2; i < len(seriesDims); i++ { serDim := seriesDims[i] if serDim.Type == model.String { - if serDim.Name == c.NameKey { - break - } c.NameKey = serDim.Name // opentsdb uses "metric" tag for metric name + break } } c.Dims = append(c.Dims, seriesDims[1:]...) From 2a95229f4c9e274b39aa604f1548bd869432ed94 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 2 Nov 2021 14:50:23 +0800 Subject: [PATCH 195/404] added TaskConfig.TimeUnit --- config/config.go | 10 +++++++--- docs/configuration/config.md | 18 +++++++++++++++--- parser/csv.go | 4 ++-- parser/fastjson.go | 4 ++-- parser/gjson.go | 4 ++-- parser/parser.go | 15 +++++---------- parser/parser_test.go | 33 +++++++++++++++++---------------- task/task.go | 2 +- 8 files changed, 51 insertions(+), 39 deletions(-) diff --git a/config/config.go b/config/config.go index e8774fe7..4d8f9683 100644 --- a/config/config.go +++ b/config/config.go @@ -143,9 +143,10 @@ type TaskConfig struct { // ShardingPolicy is `stripe,`(requires ShardingKey be numerical) or `hash`(requires ShardingKey be string) ShardingPolicy string `json:"shardingPolicy,omitempty"` - FlushInterval int `json:"flushInterval,omitempty"` - BufferSize int `json:"bufferSize,omitempty"` - TimeZone string `json:"timezone"` + FlushInterval int `json:"flushInterval,omitempty"` + BufferSize int `json:"bufferSize,omitempty"` + TimeZone string `json:"timeZone"` + TimeUnit float64 `json:"timeUnit"` } type Assignment struct { @@ -257,6 +258,9 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { if taskCfg.TimeZone == "" { taskCfg.TimeZone = defaultTimeZone } + if taskCfg.TimeUnit == 0.0 { + taskCfg.TimeUnit = float64(1.0) + } if taskCfg.PrometheusSchema { taskCfg.DynamicSchema.Enable = true taskCfg.AutoSchema = true diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 6c68d85a..5cebb7a8 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -130,14 +130,23 @@ // if it's specified, clickhouse_sinker will detect table schema instead of using the fixed schema given by "dims". "autoSchema" : true, // these columns will be excluded from the detected table schema. This takes effect only if "autoSchema" is true. - "excludeColumns": [] + "excludeColumns": [], // (experiment feature) detect new fields and their type, and add columns to the ClickHouse table accordingly. This feature requires parser be "fastjson" or "gjson". New fields' type will be one of: Int64, Float64, String. + // A column is added for new key K if all following conditions are true: + // - K isn't in ExcludeColumns + // - number of existing columns doesn't reach MaxDims-1 + // - WhiteList is empty, or K matchs WhiteList + // - BlackList is empty, or K doesn't match BlackList "dynamicSchema": { // whether enable this feature, default to false "enable": true, // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack - "maxDims": 1024 + "maxDims": 1024, + // the regexp of white list. syntax reference: https://github.com/google/re2/wiki/Syntax + "whiteList": "^[0-9A-Za-z_]+$", + // the regexp of black list + "blackList": "@" }, // shardingKey is the column name to which sharding against @@ -151,7 +160,10 @@ "bufferSize": 262114, // In the absence of time zone information, interprets the time as in the given location. Default to "Local" (aka /etc/localtime of the machine on which sinker runs) - "timezone": "" + "timeZone": "", + // Time unit when interprete a number as time. Default to 1.0. + // Java's timestamp is milliseconds since epoch. Change timeUnit to 0.001 at this case. + "timeUnit": 1.0 }, // log level, possible value: "debug", "info", "warn", "error", "dpanic", "panic", "fatal". Default to "info". diff --git a/parser/csv.go b/parser/csv.go index bb7d180c..1442becd 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -129,7 +129,7 @@ func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}) { val = Epoch } } else { - val = UnixFloat(dd) + val = UnixFloat(dd, c.pp.timeUnit) } return } @@ -207,7 +207,7 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { var t time.Time switch e.Type { case gjson.Number: - t = UnixFloat(e.Num) + t = UnixFloat(e.Num, c.pp.timeUnit) case gjson.String: var err error if t, err = c.pp.ParseDateTime(key, e.Str); err != nil { diff --git a/parser/fastjson.go b/parser/fastjson.go index 75ad5663..eddda10a 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -119,7 +119,7 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} val = getDefaultDateTime(nullable) return } - val = UnixFloat(f) + val = UnixFloat(f, c.pp.timeUnit) case fastjson.TypeString: var b []byte if b, err = v.StringBytes(); err != nil || len(b) == 0 { @@ -188,7 +188,7 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { if f, err := e.Float64(); err != nil { t = Epoch } else { - t = UnixFloat(f) + t = UnixFloat(f, c.pp.timeUnit) } case fastjson.TypeString: if b, err := e.StringBytes(); err != nil || len(b) == 0 { diff --git a/parser/gjson.go b/parser/gjson.go index 546e4066..7f87420e 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -110,7 +110,7 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { } switch r.Type { case gjson.Number: - val = UnixFloat(r.Num) + val = UnixFloat(r.Num, c.pp.timeUnit) case gjson.String: var err error if val, err = c.pp.ParseDateTime(key, r.Str); err != nil { @@ -189,7 +189,7 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { var t time.Time switch e.Type { case gjson.Number: - t = UnixFloat(e.Num) + t = UnixFloat(e.Num, c.pp.timeUnit) case gjson.String: var err error if t, err = c.pp.ParseDateTime(key, e.Str); err != nil { diff --git a/parser/parser.go b/parser/parser.go index 56aee971..30d18364 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -76,12 +76,13 @@ type Pool struct { csvFormat map[string]int delimiter string timeZone *time.Location + timeUnit float64 knownLayouts sync.Map pool sync.Pool } // NewParserPool creates a parser pool -func NewParserPool(name string, csvFormat []string, delimiter string, timezone string) (pp *Pool, err error) { +func NewParserPool(name string, csvFormat []string, delimiter string, timezone string, timeunit float64) (pp *Pool, err error) { var tz *time.Location if timezone == "" { tz = time.Local @@ -93,6 +94,7 @@ func NewParserPool(name string, csvFormat []string, delimiter string, timezone s name: name, delimiter: delimiter, timeZone: tz, + timeUnit: timeunit, } if csvFormat != nil { pp.csvFormat = make(map[string]int) @@ -194,15 +196,8 @@ func parseInLocation(val string, loc *time.Location) (t time.Time, layout string return } -func UnixInt(sec int64) (t time.Time) { - //2^32 seconds since epoch: 2106-02-07T06:28:16Z - if sec < 0 || sec >= 4294967296 { - return Epoch - } - return time.Unix(sec, 0).UTC() -} - -func UnixFloat(sec float64) (t time.Time) { +func UnixFloat(sec, unit float64) (t time.Time) { + sec *= unit //2^32 seconds since epoch: 2106-02-07T06:28:16Z if sec < 0 || sec >= 4294967296.0 { return Epoch diff --git a/parser/parser_test.go b/parser/parser_test.go index 5ec14846..41059364 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -147,6 +147,7 @@ var ( bdLocalNs = bdLocalNsOrig.UTC() bdLocalSec = bdLocalNsOrig.Truncate(1 * time.Second).UTC() bdLocalDate = time.Date(2009, 7, 13, 0, 0, 0, 0, time.Local).UTC() + timeUnit = float64(0.000001) ) var initialize sync.Once @@ -180,13 +181,13 @@ func initMetrics() { for _, name := range names { switch name { case "csv": - pp, _ = NewParserPool("csv", csvSchema, ",", "") + pp, _ = NewParserPool("csv", csvSchema, ",", "", timeUnit) sample = csvSample case "fastjson": - pp, _ = NewParserPool("fastjson", nil, "", "") + pp, _ = NewParserPool("fastjson", nil, "", "", timeUnit) sample = jsonSample case "gjson": - pp, _ = NewParserPool("gjson", nil, "", "") + pp, _ = NewParserPool("gjson", nil, "", "", timeUnit) sample = jsonSample } parser = pp.Get() @@ -354,8 +355,8 @@ func TestParserDateTime(t *testing.T) { {"null", false, Epoch}, {"bool_true", false, Epoch}, {"bool_false", false, Epoch}, - {"num_int", false, UnixInt(123)}, - {"num_float", false, UnixFloat(123.321)}, + {"num_int", false, UnixFloat(123, timeUnit)}, + {"num_float", false, UnixFloat(123.321, timeUnit)}, {"str", false, Epoch}, {"str_int", false, Epoch}, {"str_float", false, Epoch}, @@ -371,8 +372,8 @@ func TestParserDateTime(t *testing.T) { {"null", true, nil}, {"bool_true", true, nil}, {"bool_false", true, nil}, - {"num_int", true, UnixInt(123)}, - {"num_float", true, UnixFloat(123.321)}, + {"num_int", true, UnixFloat(123, timeUnit)}, + {"num_float", true, UnixFloat(123.321, timeUnit)}, {"str", true, nil}, {"str_int", true, nil}, {"str_float", true, nil}, @@ -394,8 +395,8 @@ func TestParserElasticDateTime(t *testing.T) { {"null", false, Epoch.Unix()}, {"bool_true", false, Epoch.Unix()}, {"bool_false", false, Epoch.Unix()}, - {"num_int", false, UnixInt(123).Unix()}, - {"num_float", false, UnixFloat(123.321).Unix()}, + {"num_int", false, UnixFloat(123, timeUnit).Unix()}, + {"num_float", false, UnixFloat(123.321, timeUnit).Unix()}, {"str", false, Epoch.Unix()}, {"str_int", false, Epoch.Unix()}, {"str_float", false, Epoch.Unix()}, @@ -411,8 +412,8 @@ func TestParserElasticDateTime(t *testing.T) { {"null", true, nil}, {"bool_true", true, nil}, {"bool_false", true, nil}, - {"num_int", true, UnixInt(123).Unix()}, - {"num_float", true, UnixFloat(123.321).Unix()}, + {"num_int", true, UnixFloat(123, timeUnit).Unix()}, + {"num_float", true, UnixFloat(123.321, timeUnit).Unix()}, {"str", true, nil}, {"str_int", true, nil}, {"str_float", true, nil}, @@ -459,17 +460,17 @@ func TestParserArray(t *testing.T) { {"array_num_int_1", model.Int, []int64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 0, 0}}, {"array_num_int_1", model.Float, []float64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 18446744073709551615, 18446744073709551616}}, {"array_num_int_1", model.String, []string{"0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"}}, - {"array_num_int_1", model.DateTime, []time.Time{Epoch, UnixInt(255), UnixInt(256), UnixInt(65535), UnixInt(65536), UnixInt(4294967295), UnixInt(4294967296), Epoch, Epoch}}, + {"array_num_int_1", model.DateTime, []time.Time{Epoch, UnixFloat(255, timeUnit), UnixFloat(256, timeUnit), UnixFloat(65535, timeUnit), UnixFloat(65536, timeUnit), UnixFloat(4294967295, timeUnit), UnixFloat(4294967296, timeUnit), Epoch, Epoch}}, {"array_num_int_2", model.Int, []int64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, {"array_num_int_2", model.Float, []float64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, {"array_num_int_2", model.String, []string{"-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"}}, - {"array_num_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, UnixInt(127), UnixInt(128), UnixInt(32767), UnixInt(32768), UnixInt(2147483647), UnixInt(2147483648), UnixInt(9223372036854775807)}}, + {"array_num_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, UnixFloat(127, timeUnit), UnixFloat(128, timeUnit), UnixFloat(32767, timeUnit), UnixFloat(32768, timeUnit), UnixFloat(2147483647, timeUnit), UnixFloat(2147483648, timeUnit), UnixFloat(9223372036854775807, timeUnit)}}, {"array_num_float", model.Int, []int64{0, 0, 0, 0, 0, 0, 0}}, {"array_num_float", model.Float, []float64{4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308, math.Inf(-1), math.Inf(1)}}, {"array_num_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"}}, - {"array_num_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, UnixFloat(3.40282346638528859811704183484516925440e+38), UnixFloat(1.797693134862315708145274237317043567981e+308), UnixFloat(math.Inf(-1)), UnixFloat(math.Inf(1))}}, + {"array_num_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, UnixFloat(3.40282346638528859811704183484516925440e+38, timeUnit), UnixFloat(1.797693134862315708145274237317043567981e+308, timeUnit), UnixFloat(math.Inf(-1), timeUnit), UnixFloat(math.Inf(1), timeUnit)}}, {"array_str", model.Int, []int64{0, 0, 0}}, {"array_str", model.Float, []float64{0.0, 0.0, 0.0}}, @@ -662,7 +663,7 @@ func TestParseInt(t *testing.T) { } func TestFastjsonDetectSchema(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "") + pp, _ := NewParserPool("fastjson", nil, "", "", timeUnit) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -681,7 +682,7 @@ func TestFastjsonDetectSchema(t *testing.T) { } func TestGjsonDetectSchema(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "") + pp, _ := NewParserPool("gjson", nil, "", "", timeUnit) parser := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) diff --git a/task/task.go b/task/task.go index e71d424f..86065270 100644 --- a/task/task.go +++ b/task/task.go @@ -70,7 +70,7 @@ type Service struct { // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) (service *Service) { ck := output.NewClickHouse(cfg, taskCfg) - pp, _ := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone) + pp, _ := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone, taskCfg.TimeUnit) inputer := input.NewInputer(taskCfg.KafkaClient) service = &Service{ inputer: inputer, From 0a0199a02bfbaa1625ebf49e0c5bd0e7c5e42d0f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 3 Nov 2021 22:18:45 +0800 Subject: [PATCH 196/404] added support for UUID and Enum --- docs/dev/introduction.md | 2 ++ model/value.go | 6 +++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 2c158d50..13901703 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -31,6 +31,8 @@ Refers to [design](./design.md) for how it works. - [x] Decimal, Decimal32, Decimal64, Decimal128, Decimal256 - [x] String, FixedString, LowCardinality(String) - [x] Date, DateTime, DateTime64. Assuming that all values of a field of kafka message has the same layout, and layouts of each field are unrelated. Automatically detect the layout from [these date layouts](https://github.com/housepower/clickhouse_sinker/blob/master/parser/parser.go) till the first successful detection and reuse that layout forever. +- [x] UUID +- [x] Enum - [x] Array(T), where T is one of above basic types - [x] Nullable(T), where T is one of above basic types - [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) diff --git a/model/value.go b/model/value.go index 360af0dc..9afab399 100644 --- a/model/value.go +++ b/model/value.go @@ -122,6 +122,10 @@ func WhichType(typ string) (dataType int, nullable bool) { } else if strings.HasPrefix(typ, "Array(FixedString") { dataType = StringArray nullable = false + } else if strings.HasPrefix(typ, "Enum8(") { + dataType = String + } else if strings.HasPrefix(typ, "Enum16(") { + dataType = String } else { util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported ClickHouse data type %v", typ)) } @@ -139,7 +143,7 @@ func init() { for _, t := range []string{"Float32", "Float64"} { primTypeInfo[t] = TypeInfo{Type: Float, Nullable: false} } - for _, t := range []string{"String"} { + for _, t := range []string{"String", "UUID"} { primTypeInfo[t] = TypeInfo{Type: String, Nullable: false} } for _, t := range []string{"Date", "DateTime"} { From f7e85b975bc15f108bdce25650fb875e43261e6c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 4 Nov 2021 14:44:26 +0800 Subject: [PATCH 197/404] added some doc for UUID and Enum --- docs/dev/introduction.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 13901703..6df5b2d9 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -50,6 +50,8 @@ Note: | Decimal, ... | 0.0 | Number | [decimal-value-ranges](https://clickhouse.tech/docs/en/sql-reference/data-types/decimal/#decimal-value-ranges) | | String, ... | "" | Bool, Number, String, Object, Array | N/A | | Date, DateTime, ... | EPOCH | Number, String | [EPOCH,MaxUint32_seconds_since_epoch) | +| UUID | "00000000-0000-0000-0000-000000000000" | String | N/A | +| Enum | N/A | String | N/A | | Nullable(T) | NULL | (The same as T) | (The same as T) | | Array(T) | [] | (The same as T) | (The same as T) | From 8690bd933fbdc466e27679843ce76021fa9b106d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 4 Nov 2021 21:58:20 +0800 Subject: [PATCH 198/404] skip bad rows --- output/clickhouse.go | 12 ++++++++-- output/clickhouse_util.go | 47 ++++++++++++++++++++++++++++++++++----- 2 files changed, 51 insertions(+), 8 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 939e41e7..ba012408 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -118,9 +118,13 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn *sql.DB) (err error) { } c.mux.Unlock() if len(seriesRows) != 0 { - if err = writeRows(c.promSerSQL, seriesRows, c.IdxSerID, len(c.Dims), conn); err != nil { + var numBad int + if numBad, err = writeRows(c.promSerSQL, seriesRows, c.IdxSerID, len(c.Dims), conn); err != nil { return } + if numBad != 0 { + statistics.ParseMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(numBad)) + } } return } @@ -143,9 +147,13 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( return } } - if err = writeRows(c.prepareSQL, *batch.Rows, 0, numDims, conn); err != nil { + var numBad int + if numBad, err = writeRows(c.prepareSQL, *batch.Rows, 0, numDims, conn); err != nil { return } + if numBad != 0 { + statistics.ParseMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(numBad)) + } statistics.FlushMsgsTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) return } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 15656f7c..0662d33a 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -5,6 +5,7 @@ import ( "fmt" "github.com/ClickHouse/clickhouse-go" + "github.com/RoaringBitmap/roaring" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/util" @@ -28,9 +29,10 @@ func shouldReconnect(err error, sc *pool.ShardConn) bool { return true } -func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn *sql.DB) (err error) { +func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn *sql.DB) (numBad int, err error) { var stmt *sql.Stmt var tx *sql.Tx + var errExec error if tx, err = conn.Begin(); err != nil { err = errors.Wrapf(err, "conn.Begin %s", prepareSQL) return @@ -40,15 +42,48 @@ func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn *s return } defer stmt.Close() - for _, row := range rows { + var bmBad *roaring.Bitmap + for i, row := range rows { if _, err = stmt.Exec((*row)[idxBegin:idxEnd]...); err != nil { - err = errors.Wrapf(err, "stmt.Exec") - break + if bmBad == nil { + errExec = errors.Wrapf(err, "stmt.Exec") + bmBad = roaring.NewBitmap() + } + bmBad.AddInt(i) } } - if err != nil { + if errExec != nil { + stmt.Close() _ = tx.Rollback() - return err + numBad = int(bmBad.GetCardinality()) + util.Logger.Warn(fmt.Sprintf("writeRows skipped %d rows of %d due to invalid content", numBad, len(rows)), zap.Error(errExec)) + // write rows again, skip bad ones + if tx, err = conn.Begin(); err != nil { + err = errors.Wrapf(err, "conn.Begin %s", prepareSQL) + return + } + if stmt, err = tx.Prepare(prepareSQL); err != nil { + err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) + return + } + defer stmt.Close() + for i, row := range rows { + if !bmBad.ContainsInt(i) { + if _, err = stmt.Exec((*row)[idxBegin:idxEnd]...); err != nil { + err = errors.Wrapf(err, "stmt.Exec") + break + } + } + } + if err != nil { + _ = tx.Rollback() + return + } + if err = tx.Commit(); err != nil { + err = errors.Wrapf(err, "tx.Commit") + return + } + return } if err = tx.Commit(); err != nil { err = errors.Wrapf(err, "tx.Commit") From 7f89e318a52a85ebf31be9a2da72adcac1157213 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 5 Nov 2021 15:51:35 +0800 Subject: [PATCH 199/404] adjusted replicaSpecificErrorCodes --- output/clickhouse.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index ba012408..1a319979 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -45,7 +45,7 @@ var ( // https://github.com/ClickHouse/ClickHouse/issues/24036 // src/Common/ErrorCodes.cpp // src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp - replicaSpecificErrorCodes = []int32{164, 225, 319, 1000} //READONLY, NO_ZOOKEEPER, UNKNOWN_STATUS_OF_INSERT, POCO_EXCEPTION + replicaSpecificErrorCodes = []int32{242, 319, 1000} //TABLE_IS_READ_ONLY, UNKNOWN_STATUS_OF_INSERT, POCO_EXCEPTION ) // ClickHouse is an output service consumers from kafka messages From d4578fa37ccd635cff7706ea018d90c414cb4f29 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 10 Nov 2021 18:14:09 +0800 Subject: [PATCH 200/404] more columns on kafka meta data --- model/message.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/model/message.go b/model/message.go index 873a813a..05f0f34c 100644 --- a/model/message.go +++ b/model/message.go @@ -185,8 +185,14 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe *row = append(*row, msg.Topic) } else if strings.HasSuffix(dim.Name, "_partition") { *row = append(*row, msg.Partition) - } else { + } else if strings.HasSuffix(dim.Name, "_offset") { *row = append(*row, msg.Offset) + } else if strings.HasSuffix(dim.Name, "_key") { + *row = append(*row, string(msg.Key)) + } else if strings.HasSuffix(dim.Name, "_timestamp") { + *row = append(*row, *msg.Timestamp) + } else { + *row = append(*row, nil) } } else { val := GetValueByType(metric, dim) From f82c1e7a3b74432f866c391e282d6e93778b2b8b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 12 Nov 2021 10:43:37 +0800 Subject: [PATCH 201/404] create_table_query differ among 21.8.9.13, 21.10.1.8013 and 21.10.2.15 --- output/clickhouse.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 1a319979..a6ffc3a7 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -467,7 +467,7 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { if conn, _, err = sc.NextGoodReplica(0); err != nil { return } - query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed\(\'%s\', \'%s\', \'%s\'\)')`, + query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed\(\'%s\', \'%s\', \'%s\'.*\)')`, chCfg.DB, chCfg.Cluster, chCfg.DB, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) From 20f3a850ed1369e2992dd7ef88a69fdd396d16f4 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 12 Nov 2021 23:19:15 +0800 Subject: [PATCH 202/404] fixed ForceBatchOrShard became idle incorrectly --- task/ring.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/task/ring.go b/task/ring.go index 56bda293..b4ace8e9 100644 --- a/task/ring.go +++ b/task/ring.go @@ -134,7 +134,10 @@ func (ring *Ring) ForceBatchOrShard(_ interface{}) { ring.mux.Lock() defer ring.mux.Unlock() if !ring.isIdle { - if ring.ringGroundOff == ring.ringCeilingOff { + if ring.ringFilledOffset > ring.ringGroundOff { + ring.genBatchOrShard() + ring.idleCnt = 0 + } else if ring.ringBuf[ring.ringGroundOff&ring.ringCapMask].Msg == nil { ring.idleCnt++ if ring.idleCnt >= 2 { ring.idleCnt = 0 @@ -143,9 +146,6 @@ func (ring *Ring) ForceBatchOrShard(_ interface{}) { util.Logger.Info(fmt.Sprintf("topic %s partition %d became idle", taskCfg.Topic, ring.partition), zap.String("task", taskCfg.Name)) return } - } else if ring.ringFilledOffset > ring.ringGroundOff { - ring.genBatchOrShard() - ring.idleCnt = 0 } ring.scheduleForchBatchOrShard() } From 544aac3d6f8fe8b234896a4ca7864201a1cf0437 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 15 Nov 2021 10:16:41 +0800 Subject: [PATCH 203/404] fixed time.After leak --- cmd/clickhouse_sinker/main.go | 5 ++++- cmd/kafka_gen_log/main.go | 1 + cmd/kafka_gen_metric/main.go | 1 + config_manager/nacos.go | 4 +++- statistics/statistics.go | 1 + 5 files changed, 10 insertions(+), 2 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 8f8e7246..309fa552 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -302,12 +302,15 @@ func (s *Sinker) Run() { if cmdOps.NacosServiceName != "" { go s.rcm.Run() } + // Golang <-time.After() is not garbage collected before expiry. + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() for { select { case <-s.ctx.Done(): util.Logger.Info("Sinker.Run quit due to context has been canceled") return - case <-time.After(10 * time.Second): + case <-ticker.C: if newCfg, err = s.rcm.GetConfig(); err != nil { util.Logger.Error("s.rcm.GetConfig failed", zap.Error(err)) continue diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 01afe336..ad2dcd35 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -336,6 +336,7 @@ log_file_pattern: file name pattern, for example, '^secure.*$'`, os.Args[0], os. var prevLines, prevSize int64 ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() LOOP: for { select { diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index 09c2dc97..d7f19cf9 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -207,6 +207,7 @@ topic: for example, sensor_dt_result_online`, os.Args[0], os.Args[0]) go generate() ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() LOOP: for { select { diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 05b9a9d9..59ccd3e6 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -220,13 +220,15 @@ func (ncm *NacosConfManager) Run() { } // Assign regularly to handle lag change + ticker := time.NewTicker(5 * time.Minute) + defer ticker.Stop() LOOP_FOR: for { select { case <-ncm.ctx.Done(): util.Logger.Info("NacosConfManager.Run quit due to context has been canceled") break LOOP_FOR - case <-time.After(5 * time.Minute): + case <-ticker.C: util.Logger.Debug("assign triggered by 5 min timer") if err := ncm.assign(); err != nil { util.Logger.Error("assign failed", zap.Error(err)) diff --git a/statistics/statistics.go b/statistics/statistics.go index 42891204..136b8a17 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -203,6 +203,7 @@ func (p *Pusher) Init() error { func (p *Pusher) Run() { ticker := time.NewTicker(time.Second * time.Duration(p.pushInterval)) + defer ticker.Stop() FOR: for { select { From 5ca55ad18ae36fa1c7bdddf6ffeb4d191257633e Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 15 Nov 2021 11:08:03 +0800 Subject: [PATCH 204/404] GetTaskStateAndLags --- config_manager/lags.go | 32 +++++++++++++++++++++++++++----- config_manager/nacos.go | 14 +++++++------- 2 files changed, 34 insertions(+), 12 deletions(-) diff --git a/config_manager/lags.go b/config_manager/lags.go index 43ad0a49..07d85e80 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -11,12 +11,17 @@ import ( "go.uber.org/zap" ) -// GetTaskLags inspired by https://github.com/cloudhut/kminion/blob/1ffd02ba94a5edc26d4f11e57191ed3479d8a111/prometheus/collect_consumer_group_lags.go -func GetTaskLags(cfg *config.Config) (taskLags map[string]int64, err error) { +type StateLag struct { + State string + Lag int64 +} + +// GetTaskStateAndLags is inspired by https://github.com/cloudhut/kminion/blob/1ffd02ba94a5edc26d4f11e57191ed3479d8a111/prometheus/collect_consumer_group_lags.go +func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err error) { var adminClient sarama.ClusterAdmin var client sarama.Client var sarCfg *sarama.Config - taskLags = make(map[string]int64) // taskName -> totalLags + stateLags = make(map[string]StateLag) // taskName -> state and totalLags if sarCfg, err = input.GetSaramaConfig(&cfg.Kafka); err != nil { return } @@ -80,7 +85,7 @@ func GetTaskLags(cfg *config.Config) (taskLags map[string]int64, err error) { } } - // Get consumer groups' offset + // Get consumer groups' offset, calculate lag for _, taskCfg := range cfg.Tasks { topic := taskCfg.Topic var totalLags int64 @@ -105,7 +110,24 @@ func GetTaskLags(cfg *config.Config) (taskLags map[string]int64, err error) { } } } - taskLags[taskCfg.Name] = totalLags + stateLags[taskCfg.Name] = StateLag{Lag: totalLags} + } + } + + // Get consumer groups' state + groups := make([]string, len(cfg.Tasks)) + for i, taskCfg := range cfg.Tasks { + groups[i] = taskCfg.ConsumerGroup + } + var gd []*sarama.GroupDescription + if gd, err = adminClient.DescribeConsumerGroups(groups); err != nil { + err = errors.Wrapf(err, "failed to describe consumer groups") + return + } + for i, taskCfg := range cfg.Tasks { + if stateLag, ok := stateLags[taskCfg.Name]; ok { + stateLag.State = gd[i].State + stateLags[taskCfg.Name] = stateLag } } return diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 59ccd3e6..362f3282 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -327,23 +327,23 @@ func (ncm *NacosConfManager) assign() (err error) { return } - var taskLags map[string]int64 - if taskLags, err = GetTaskLags(newCfg); err != nil { + var stateLags map[string]StateLag + if stateLags, err = GetTaskStateAndLags(newCfg); err != nil { return } - util.Logger.Debug(fmt.Sprintf("task lags %+v", taskLags)) + util.Logger.Debug(fmt.Sprintf("task state and lags %+v", stateLags)) var validTasks []string for _, taskCfg := range newCfg.Tasks { - if _, ok := taskLags[taskCfg.Name]; ok { + if _, ok := stateLags[taskCfg.Name]; ok { validTasks = append(validTasks, taskCfg.Name) } } sort.Slice(validTasks, func(i, j int) bool { taskNameI := validTasks[i] - lagI := taskLags[taskNameI] + lagI := stateLags[taskNameI].Lag taskNameJ := validTasks[j] - lagJ := taskLags[taskNameJ] + lagJ := stateLags[taskNameJ].Lag return (lagI > lagJ) || (lagI == lagJ && taskNameI < taskNameJ) }) @@ -360,7 +360,7 @@ func (ncm *NacosConfManager) assign() (err error) { idxInst = len(newInsts) - 1 - idxInst } taskName := validTasks[idxTask] - taskLag := taskLags[taskName] + taskLag := stateLags[taskName].Lag instAg := instAgs[idxInst] instAg.TotalLag += taskLag instAg.TaskLags = append(instAg.TaskLags, TaskLag{Task: taskName, Lag: taskLag}) From b5e55515cea47574cfc46d7a1399e38231b5e09c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 15 Nov 2021 11:29:20 +0800 Subject: [PATCH 205/404] added state endpoint --- cmd/clickhouse_sinker/main.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 309fa552..5d5031c7 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -17,6 +17,7 @@ package main import ( "context" + "encoding/json" "flag" "fmt" "log" @@ -159,6 +160,7 @@ func main() { ClickHouse Sinker

ClickHouse Sinker

+

State

Metrics

Ready

Ready Full

@@ -168,6 +170,19 @@ func main() { `)) }) + mux.HandleFunc("/state", func(w http.ResponseWriter, r *http.Request) { + w.Header().Set("Content-Type", "application/json") + if runner != nil && runner.curCfg != nil { + var stateLags map[string]cm.StateLag + var bs []byte + var err error + if stateLags, err = cm.GetTaskStateAndLags(runner.curCfg); err == nil { + if bs, err = json.Marshal(stateLags); err == nil { + _, _ = w.Write(bs) + } + } + } + }) mux.Handle("/metrics", httpMetrics) mux.HandleFunc("/ready", health.Health.ReadyEndpoint) // GET /ready?full=1 mux.HandleFunc("/live", health.Health.LiveEndpoint) // GET /live?full=1 From cc7406dd8459391fb16ab48d2a32a08990b5318a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 15 Nov 2021 14:21:02 +0800 Subject: [PATCH 206/404] added GetTaskStateAndLag --- config_manager/lags.go | 195 ++++++++++++++++++++++++----------------- 1 file changed, 113 insertions(+), 82 deletions(-) diff --git a/config_manager/lags.go b/config_manager/lags.go index 07d85e80..09959946 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -16,118 +16,149 @@ type StateLag struct { Lag int64 } -// GetTaskStateAndLags is inspired by https://github.com/cloudhut/kminion/blob/1ffd02ba94a5edc26d4f11e57191ed3479d8a111/prometheus/collect_consumer_group_lags.go +// GetTaskStateAndLags get state and lag of all tasks. func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err error) { + var client sarama.Client var adminClient sarama.ClusterAdmin + var closer func() + if client, adminClient, closer, err = newClient(cfg); err != nil { + return + } + defer closer() + + stateLags = make(map[string]StateLag) + for _, taskCfg := range cfg.Tasks { + var totalLags int64 + if totalLags, err = getLag(client, adminClient, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { + return + } + stateLags[taskCfg.Name] = StateLag{Lag: totalLags} + } + + // Get consumer groups' state + groups := make([]string, len(cfg.Tasks)) + for i, taskCfg := range cfg.Tasks { + groups[i] = taskCfg.ConsumerGroup + } + var gd []*sarama.GroupDescription + if gd, err = adminClient.DescribeConsumerGroups(groups); err != nil { + err = errors.Wrapf(err, "failed to describe consumer groups") + return + } + for i, taskCfg := range cfg.Tasks { + if stateLag, ok := stateLags[taskCfg.Name]; ok { + stateLag.State = gd[i].State + stateLags[taskCfg.Name] = stateLag + } + } + return +} + +// GetTaskStateAndLag get state and lag of a task. +func GetTaskStateAndLag(cfg *config.Config, taskName string) (stateLag StateLag, err error) { var client sarama.Client + var adminClient sarama.ClusterAdmin + var closer func() + if client, adminClient, closer, err = newClient(cfg); err != nil { + return + } + defer closer() + + var taskCfg *config.TaskConfig + for _, tskCfg := range cfg.Tasks { + if tskCfg.Name == taskName { + taskCfg = tskCfg + break + } + } + if taskCfg == nil { + err = errors.Errorf("task %q doesn't exist", taskName) + return + } + if stateLag.Lag, err = getLag(client, adminClient, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { + return + } + + // Get consumer group' state + var gd []*sarama.GroupDescription + if gd, err = adminClient.DescribeConsumerGroups([]string{taskCfg.ConsumerGroup}); err != nil { + err = errors.Wrapf(err, "failed to describe consumer groups") + return + } + stateLag.State = gd[0].State + return +} + +func newClient(cfg *config.Config) (client sarama.Client, adminClient sarama.ClusterAdmin, closer func(), err error) { var sarCfg *sarama.Config - stateLags = make(map[string]StateLag) // taskName -> state and totalLags if sarCfg, err = input.GetSaramaConfig(&cfg.Kafka); err != nil { return } if adminClient, err = sarama.NewClusterAdmin(strings.Split(cfg.Kafka.Brokers, ","), sarCfg); err != nil { + err = errors.Wrapf(err, "sarama.NewClusterAdmin failed") return } if client, err = sarama.NewClient(strings.Split(cfg.Kafka.Brokers, ","), sarCfg); err != nil { + err = errors.Wrapf(err, "sarama.NewClient failed") return } - defer func() { + closer = func() { if err2 := client.Close(); err2 != nil { util.Logger.Error("failed to close Kafka client", zap.Error(err2)) } if err2 := adminClient.Close(); err2 != nil { util.Logger.Error("failed to close Kafka admin client", zap.Error(err2)) } - }() - - // Get topics' partition id list - var topics []string - topicPartitions := make(map[string]int) //topic -> number of partitions - for _, taskCfg := range cfg.Tasks { - topicPartitions[taskCfg.Topic] = 0 - } - for topic := range topicPartitions { - topics = append(topics, topic) } + return +} + +// getLag is inspired by https://github.com/cloudhut/kminion/blob/1ffd02ba94a5edc26d4f11e57191ed3479d8a111/prometheus/collect_consumer_group_lags.go +func getLag(client sarama.Client, adminClient sarama.ClusterAdmin, topic, group string) (totalLags int64, err error) { + // Get number of partitions + var partitions int var topicsMeta []*sarama.TopicMetadata - if topicsMeta, err = adminClient.DescribeTopics(topics); err != nil { + if topicsMeta, err = adminClient.DescribeTopics([]string{topic}); err != nil { + err = errors.Wrapf(err, "failed to describe topic %q", topic) return } - for i, topicMeta := range topicsMeta { - if topicMeta != nil { - topicPartitions[topics[i]] = len(topicMeta.Partitions) - } - } + partitions = len(topicsMeta[0].Partitions) // Get partitions' oldest and newest offset - topicOldestOffsets := make(map[string][]int64) //topic -> list of partitions' oldestOffset - topicNewestOffsets := make(map[string][]int64) //topic -> list of partitions' newestOffset - for _, topic := range topics { - if partitions, ok := topicPartitions[topic]; ok { - var oldestOffsets, newestOffsets []int64 - var oldestOffset, newestOffset int64 - for partition := 0; partition < partitions; partition++ { - oldestOffset, err = client.GetOffset(topic, int32(partition), sarama.OffsetOldest) - if err != nil { - err = errors.Wrapf(err, "failed to get topic/partition offsets for %q partition %q", topic, partition) - return - } - newestOffset, err = client.GetOffset(topic, int32(partition), sarama.OffsetNewest) - if err != nil { - err = errors.Wrapf(err, "failed to get topic/partition offsets for %q partition %q", topic, partition) - return - } - oldestOffsets = append(oldestOffsets, oldestOffset) - newestOffsets = append(newestOffsets, newestOffset) - } - topicOldestOffsets[topic] = oldestOffsets - topicNewestOffsets[topic] = newestOffsets + var oldestOffsets, newestOffsets []int64 + var oldestOffset, newestOffset int64 + for partition := 0; partition < partitions; partition++ { + oldestOffset, err = client.GetOffset(topic, int32(partition), sarama.OffsetOldest) + if err != nil { + err = errors.Wrapf(err, "failed to get oldest offset for topic %q partition %q", topic, partition) + return } - } - - // Get consumer groups' offset, calculate lag - for _, taskCfg := range cfg.Tasks { - topic := taskCfg.Topic - var totalLags int64 - oldestOffsets := topicOldestOffsets[topic] - newestOffsets := topicNewestOffsets[topic] - if partitions, ok := topicPartitions[topic]; ok { - pidList := make([]int32, partitions) - for partition := 0; partition < partitions; partition++ { - pidList[partition] = int32(partition) - } - var rep *sarama.OffsetFetchResponse - if rep, err = adminClient.ListConsumerGroupOffsets(taskCfg.ConsumerGroup, map[string][]int32{topic: pidList}); err != nil { - for partition := 0; partition < partitions; partition++ { - totalLags += newestOffsets[partition] - oldestOffsets[partition] + 1 - } - } else { - for partition := 0; partition < partitions; partition++ { - block := rep.GetBlock(topic, int32(partition)) - lag := newestOffsets[partition] - block.Offset - 1 - if lag > 0 { - totalLags += lag - } - } - } - stateLags[taskCfg.Name] = StateLag{Lag: totalLags} + newestOffset, err = client.GetOffset(topic, int32(partition), sarama.OffsetNewest) + if err != nil { + err = errors.Wrapf(err, "failed to get newest offset for topic %q partition %q", topic, partition) + return } + oldestOffsets = append(oldestOffsets, oldestOffset) + newestOffsets = append(newestOffsets, newestOffset) } - // Get consumer groups' state - groups := make([]string, len(cfg.Tasks)) - for i, taskCfg := range cfg.Tasks { - groups[i] = taskCfg.ConsumerGroup + // Get consumer group' offset, calculate lag + pidList := make([]int32, partitions) + for partition := 0; partition < partitions; partition++ { + pidList[partition] = int32(partition) } - var gd []*sarama.GroupDescription - if gd, err = adminClient.DescribeConsumerGroups(groups); err != nil { - err = errors.Wrapf(err, "failed to describe consumer groups") - return - } - for i, taskCfg := range cfg.Tasks { - if stateLag, ok := stateLags[taskCfg.Name]; ok { - stateLag.State = gd[i].State - stateLags[taskCfg.Name] = stateLag + var rep *sarama.OffsetFetchResponse + if rep, err = adminClient.ListConsumerGroupOffsets(group, map[string][]int32{topic: pidList}); err != nil { + for partition := 0; partition < partitions; partition++ { + totalLags += newestOffsets[partition] - oldestOffsets[partition] + 1 + } + } else { + for partition := 0; partition < partitions; partition++ { + block := rep.GetBlock(topic, int32(partition)) + lag := newestOffsets[partition] - block.Offset - 1 + if lag > 0 { + totalLags += lag + } } } return From 3d07fdb1362156fd418ed326f011693d463ce4f5 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 15 Nov 2021 16:19:37 +0800 Subject: [PATCH 207/404] enabled sarama and kafka-go log --- input/kafka_go.go | 9 +++++++++ input/kafka_sarama.go | 3 ++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/input/kafka_go.go b/input/kafka_go.go index 7fbc0cb9..60bfe6f8 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -49,6 +49,14 @@ type KafkaGo struct { cleanupFn func() } +type KafkaGoLogger struct { + logger *zap.Logger +} + +func (kgl *KafkaGoLogger) Printf(template string, args ...interface{}) { + kgl.logger.Sugar().Debugf(template, args) +} + // NewKafkaGo get instance of kafka reader func NewKafkaGo() *KafkaGo { return &KafkaGo{} @@ -78,6 +86,7 @@ func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn fun // PartitionWatchInterval is only used when GroupID is set and WatchPartitionChanges is set. PartitionWatchInterval: 600 * time.Second, // sarama.Config.Metadata.RefreshFrequency WatchPartitionChanges: true, + ErrorLogger: &KafkaGoLogger{util.Logger}, } if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { if kfkCfg.TLS.CaCertFiles, _, err = util.JksToPem(kfkCfg.TLS.TrustStoreLocation, kfkCfg.TLS.TrustStorePassword, false); err != nil { diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 82749a8f..ea6d1177 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -28,6 +28,7 @@ import ( "github.com/pkg/errors" "github.com/xdg-go/scram" "go.uber.org/zap" + "go.uber.org/zap/zapcore" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" @@ -108,7 +109,7 @@ func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn if err != nil { return err } - //sarama.Logger, _ = zap.NewStdLogAt(util.Logger.With(zap.String("name", "sarama")), zapcore.DebugLevel) + sarama.Logger, _ = zap.NewStdLogAt(util.Logger.With(zap.String("name", "sarama")), zapcore.DebugLevel) k.cg = cg return nil } From c39cc304e3f04aacea2b2d9c8652794fecf1dd0e Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 15 Nov 2021 18:23:00 +0800 Subject: [PATCH 208/404] changed default kafka version to 2.0.0 --- config/config.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/config/config.go b/config/config.go index 4d8f9683..4ec8b4b6 100644 --- a/config/config.go +++ b/config/config.go @@ -189,7 +189,13 @@ func (cfg *Config) Normallize() (err error) { return } if cfg.Kafka.Version == "" { - cfg.Kafka.Version = "2.5.0" + // https://cwiki.apache.org/confluence/display/KAFKA/Compatibility+Matrix + // KIP-35 - Retrieving protocol version introduced a mechanism for dynamically determining the functionality of a Kafka broker. + // https://github.com/Shopify/sarama/issues/1732, + // Historically Sarama has tied it's protocol usage to the Version field in Config. + // https://kafka.apache.org/downloads + // 2.0.0 is released at July 30, 2018. + cfg.Kafka.Version = "2.0.0" } cfg.convertKfkSecurity() From 1b81b649e6e5c2b5c2c229c28b2feccb216b7f4f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 16 Nov 2021 19:32:11 +0800 Subject: [PATCH 209/404] added franz --- go.mod | 1 + go.sum | 11 ++++ input/input.go | 4 +- input/kafka_franz.go | 124 +++++++++++++++++++++++++++++++++++++++++++ 4 files changed, 139 insertions(+), 1 deletion(-) create mode 100644 input/kafka_franz.go diff --git a/go.mod b/go.mod index af42677c..0b0784b8 100644 --- a/go.mod +++ b/go.mod @@ -20,6 +20,7 @@ require ( github.com/stretchr/testify v1.7.0 github.com/tidwall/gjson v1.10.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 + github.com/twmb/franz-go v1.2.4 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.0.2 github.com/xdg/scram v1.0.3 // indirect diff --git a/go.sum b/go.sum index 31f7a002..ed8b83dc 100644 --- a/go.sum +++ b/go.sum @@ -144,6 +144,7 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8 github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.13.5/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.13.6 h1:P76CopJELS0TiO2mebmnzgWaajssP/EszplttgQxcgc= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/cpuid/v2 v2.0.9 h1:lgaqFMSdTdQYdZ04uHyN2d/eKdOMyi2YLSvlQIBFYa4= @@ -184,6 +185,8 @@ github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= +github.com/pierrec/lz4/v4 v4.1.8 h1:ieHkV+i2BRzngO4Wd/3HGowuZStgq6QkPsD1eolNAO4= +github.com/pierrec/lz4/v4 v4.1.8/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -259,6 +262,12 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNij github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= +github.com/twmb/franz-go v1.2.4 h1:APqAImlBNTpv5ukAwOl/9x2kkAVKk3ocPxwlPXkYNC4= +github.com/twmb/franz-go v1.2.4/go.mod h1:e5ZOdNswX/wv+jebWNX49yc9U7zgR18Xovj9ckk6mx8= +github.com/twmb/franz-go/pkg/kmsg v0.0.0-20211104051938-70808186d5f7 h1:YW4mW39H53O1qouKQnlrdNwyqAi5c4P10Oig8yndDKQ= +github.com/twmb/franz-go/pkg/kmsg v0.0.0-20211104051938-70808186d5f7/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/go-rbtree v1.0.0 h1:KxN7dXJ8XaZ4cvmHV1qqXTshxX3EBvX/toG5+UR49Mg= +github.com/twmb/go-rbtree v1.0.0/go.mod h1:UlIAI8gu3KRPkXSobZnmJfVwCJgEhD/liWzT5ppzIyc= github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= github.com/valyala/fastjson v1.6.3/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= @@ -297,6 +306,7 @@ golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210920023735-84f357641f63/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 h1:7I4JAnoQBe7ZtJcBaYHi5UtiO8tQHbUSXxL+pnGRANg= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= @@ -316,6 +326,7 @@ golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210913180222-943fd674d43e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210917221730-978cfadd31cf/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211020060615-d418f374d309 h1:A0lJIi+hcTR6aajJH4YqKWwohY4aW9RO7oRMcdv+HKI= golang.org/x/net v0.0.0-20211020060615-d418f374d309/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= diff --git a/input/input.go b/input/input.go index 57cde336..26b5e6bb 100644 --- a/input/input.go +++ b/input/input.go @@ -11,7 +11,7 @@ import ( const ( TypeKafkaGo = "kafka-go" TypeKafkaSarama = "sarama" - TypePulsar = "pulsar" + TypeKafkaFranz = "franz" ) type Inputer interface { @@ -27,6 +27,8 @@ func NewInputer(typ string) Inputer { return NewKafkaGo() case TypeKafkaSarama: return NewKafkaSarama() + case TypeKafkaFranz: + return NewKafkaFranz() default: util.Logger.Fatal(fmt.Sprintf("BUG: %s is not a supported input type", typ)) return nil diff --git a/input/kafka_franz.go b/input/kafka_franz.go new file mode 100644 index 00000000..ed4172e1 --- /dev/null +++ b/input/kafka_franz.go @@ -0,0 +1,124 @@ +/*Copyright [2019] housepower + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package input + +import ( + "context" + "strings" + "sync" + + "github.com/pkg/errors" + "github.com/twmb/franz-go/pkg/kgo" + "go.uber.org/zap" + + "github.com/housepower/clickhouse_sinker/config" + "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" +) + +var _ Inputer = (*KafkaFranz)(nil) + +// KafkaFranz implements input.Inputer +// refers to examples/group_consuming/main.go +type KafkaFranz struct { + cfg *config.Config + taskCfg *config.TaskConfig + cl *kgo.Client + ctx context.Context + cancel context.CancelFunc + wgRun sync.WaitGroup + putFn func(msg *model.InputMessage) + cleanupFn func() +} + +// NewKafkaFranz get instance of kafka reader +func NewKafkaFranz() *KafkaFranz { + return &KafkaFranz{} +} + +// Init Initialise the kafka instance with configuration +func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg *model.InputMessage), cleanupFn func()) (err error) { + k.cfg = cfg + k.taskCfg = taskCfg + k.ctx, k.cancel = context.WithCancel(context.Background()) + k.putFn = putFn + k.cleanupFn = cleanupFn + kfkCfg := &cfg.Kafka + + opts := []kgo.Opt{ + kgo.SeedBrokers(strings.Split(kfkCfg.Brokers, ",")...), + kgo.ConsumeTopics(taskCfg.Topic), + kgo.ConsumerGroup(taskCfg.ConsumerGroup), + kgo.DisableAutoCommit(), + kgo.FetchMaxBytes(5 << 20), + } + if k.cl, err = kgo.NewClient(opts...); err != nil { + err = errors.Wrap(err, "") + return + } + return nil +} + +// kafka main loop +func (k *KafkaFranz) Run() { + k.wgRun.Add(1) + defer k.wgRun.Done() + for { + fetches := k.cl.PollFetches(k.ctx) + if fetches.IsClientClosed() { + util.Logger.Error("KafkaFranz.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) + return + } + var hasError bool + fetches.EachError(func(_ string, _ int32, err error) { + err = errors.Wrap(err, "") + util.Logger.Error("kgo.Client.PollFetchs() failed", zap.Error(err)) + hasError = true + }) + if hasError { + continue + } + fetches.EachRecord(func(rec *kgo.Record) { + msg := &model.InputMessage{ + Topic: rec.Topic, + Partition: int(rec.Partition), + Key: rec.Key, + Value: rec.Value, + Offset: rec.Offset, + Timestamp: &rec.Timestamp, + } + k.putFn(msg) + }) + } +} + +func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { + k.cl.CommitRecords(context.Background(), &kgo.Record{Topic: msg.Topic, Partition: int32(msg.Partition), Offset: msg.Offset}) + return nil +} + +// Stop kafka consumer and close all connections +func (k *KafkaFranz) Stop() error { + k.cancel() + k.cl.Close() + k.wgRun.Wait() + return nil +} + +// Description of this kafka consumer, which topic it reads from +func (k *KafkaFranz) Description() string { + return "kafka consumer of topic " + k.taskCfg.Topic +} From 682347ccd4cd38a0e8e6784ede67dc3c74ecd49c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 16 Nov 2021 20:50:37 +0800 Subject: [PATCH 210/404] onPartitionRevoked --- input/kafka_franz.go | 19 +++++++++++++++---- task/task.go | 8 ++++++-- 2 files changed, 21 insertions(+), 6 deletions(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index ed4172e1..50890832 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -19,6 +19,7 @@ import ( "context" "strings" "sync" + "time" "github.com/pkg/errors" "github.com/twmb/franz-go/pkg/kgo" @@ -63,6 +64,7 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn kgo.ConsumeTopics(taskCfg.Topic), kgo.ConsumerGroup(taskCfg.ConsumerGroup), kgo.DisableAutoCommit(), + kgo.OnPartitionsRevoked(k.onPartitionRevoked), kgo.FetchMaxBytes(5 << 20), } if k.cl, err = kgo.NewClient(opts...); err != nil { @@ -78,9 +80,8 @@ func (k *KafkaFranz) Run() { defer k.wgRun.Done() for { fetches := k.cl.PollFetches(k.ctx) - if fetches.IsClientClosed() { - util.Logger.Error("KafkaFranz.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) - return + if fetches == nil || fetches.IsClientClosed() { + break } var hasError bool fetches.EachError(func(_ string, _ int32, err error) { @@ -103,6 +104,8 @@ func (k *KafkaFranz) Run() { k.putFn(msg) }) } + k.cl.Close() // will trigger k.onPartitionRevoked + util.Logger.Info("KafkaFranz.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) } func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { @@ -113,7 +116,6 @@ func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { // Stop kafka consumer and close all connections func (k *KafkaFranz) Stop() error { k.cancel() - k.cl.Close() k.wgRun.Wait() return nil } @@ -122,3 +124,12 @@ func (k *KafkaFranz) Stop() error { func (k *KafkaFranz) Description() string { return "kafka consumer of topic " + k.taskCfg.Topic } + +func (k *KafkaFranz) onPartitionRevoked(_ context.Context, _ *kgo.Client, _ map[string][]int32) { + begin := time.Now() + k.cleanupFn() + util.Logger.Info("consumer group cleanup", + zap.String("task", k.taskCfg.Name), + zap.String("consumer group", k.taskCfg.ConsumerGroup), + zap.Duration("cost", time.Since(begin))) +} diff --git a/task/task.go b/task/task.go index 86065270..ed297cbf 100644 --- a/task/task.go +++ b/task/task.go @@ -158,9 +158,13 @@ func (service *Service) Run() { service.inputer.Run() } -func (service *Service) fnCommit(partition int, offset int64) error { +func (service *Service) fnCommit(partition int, offset int64) (err error) { msg := model.InputMessage{Topic: service.taskCfg.Topic, Partition: partition, Offset: offset} - return service.inputer.CommitMessages(&msg) + if err = service.inputer.CommitMessages(&msg); err != nil { + return + } + util.Logger.Debug(fmt.Sprintf("committed topic %s, partition %d, offset %d", msg.Topic, msg.Partition, msg.Offset+1), zap.String("task", service.taskCfg.Name)) + return } func (service *Service) putToRing(msg *model.InputMessage) (ok bool) { From 064ab777b537d8f7afc5c06255333258c4e3cf54 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 17 Nov 2021 10:35:48 +0800 Subject: [PATCH 211/404] fix ErrDataLoss at startup --- input/kafka_franz.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 50890832..ffd2c284 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -109,7 +109,8 @@ func (k *KafkaFranz) Run() { } func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { - k.cl.CommitRecords(context.Background(), &kgo.Record{Topic: msg.Topic, Partition: int32(msg.Partition), Offset: msg.Offset}) + // "LeaderEpoch: -1" will disable leader epoch validation + k.cl.CommitRecords(context.Background(), &kgo.Record{Topic: msg.Topic, Partition: int32(msg.Partition), Offset: msg.Offset, LeaderEpoch: -1}) return nil } From c3c15f2f4ab48c8983eb2c5d1526e749250596b6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 17 Nov 2021 14:21:32 +0800 Subject: [PATCH 212/404] optimized franz config --- go.mod | 3 +-- go.sum | 6 ++++++ input/kafka_franz.go | 6 +++++- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 0b0784b8..b5eff3fb 100644 --- a/go.mod +++ b/go.mod @@ -21,12 +21,11 @@ require ( github.com/tidwall/gjson v1.10.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/twmb/franz-go v1.2.4 + github.com/twmb/franz-go/plugin/kzap v1.0.0 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.0.2 github.com/xdg/scram v1.0.3 // indirect github.com/xdg/stringprep v1.0.3 // indirect - go.uber.org/atomic v1.9.0 // indirect - go.uber.org/multierr v1.7.0 // indirect go.uber.org/zap v1.19.1 golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 // indirect golang.org/x/net v0.0.0-20211020060615-d418f374d309 // indirect diff --git a/go.sum b/go.sum index ed8b83dc..979573b4 100644 --- a/go.sum +++ b/go.sum @@ -262,10 +262,15 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNij github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= +github.com/twmb/franz-go v1.0.0/go.mod h1:cdFLk8d/5/ox88y38xgiDKP3Yo338OO0t5QbTEM2K6I= github.com/twmb/franz-go v1.2.4 h1:APqAImlBNTpv5ukAwOl/9x2kkAVKk3ocPxwlPXkYNC4= github.com/twmb/franz-go v1.2.4/go.mod h1:e5ZOdNswX/wv+jebWNX49yc9U7zgR18Xovj9ckk6mx8= +github.com/twmb/franz-go/pkg/kmsg v0.0.0-20210901051457-3c197a133ddd/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go/pkg/kmsg v0.0.0-20210914042331-106aef61b693/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v0.0.0-20211104051938-70808186d5f7 h1:YW4mW39H53O1qouKQnlrdNwyqAi5c4P10Oig8yndDKQ= github.com/twmb/franz-go/pkg/kmsg v0.0.0-20211104051938-70808186d5f7/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go/plugin/kzap v1.0.0 h1:vHI3IBXBTNspBhw6OXv+t28fqnij03FxOiqp0OUzmGc= +github.com/twmb/franz-go/plugin/kzap v1.0.0/go.mod h1:hBnW68adqgx2m+Us874W03VmKjqgov/WI8wfYaSoZgI= github.com/twmb/go-rbtree v1.0.0 h1:KxN7dXJ8XaZ4cvmHV1qqXTshxX3EBvX/toG5+UR49Mg= github.com/twmb/go-rbtree v1.0.0/go.mod h1:UlIAI8gu3KRPkXSobZnmJfVwCJgEhD/liWzT5ppzIyc= github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= @@ -326,6 +331,7 @@ golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210825183410-e898025ed96a/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210913180222-943fd674d43e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210917221730-978cfadd31cf/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211020060615-d418f374d309 h1:A0lJIi+hcTR6aajJH4YqKWwohY4aW9RO7oRMcdv+HKI= diff --git a/input/kafka_franz.go b/input/kafka_franz.go index ffd2c284..d2376257 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -23,6 +23,7 @@ import ( "github.com/pkg/errors" "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/plugin/kzap" "go.uber.org/zap" "github.com/housepower/clickhouse_sinker/config" @@ -65,7 +66,10 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn kgo.ConsumerGroup(taskCfg.ConsumerGroup), kgo.DisableAutoCommit(), kgo.OnPartitionsRevoked(k.onPartitionRevoked), - kgo.FetchMaxBytes(5 << 20), + kgo.MaxConcurrentFetches(3), + kgo.FetchMaxBytes(1 << 27), //134 MB + kgo.BrokerMaxReadBytes(1 << 27), //134 MB + kgo.WithLogger(kzap.New(util.Logger)), } if k.cl, err = kgo.NewClient(opts...); err != nil { err = errors.Wrap(err, "") From b9db7a23576078577e9c84d4a94e9641ca421fd0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 17 Nov 2021 15:51:39 +0800 Subject: [PATCH 213/404] added franz secrity --- config/config.go | 10 ++++++ go.mod | 1 + input/kafka_franz.go | 76 +++++++++++++++++++++++++++++++++++++++++++ input/kafka_go.go | 10 ------ input/kafka_sarama.go | 10 ------ 5 files changed, 87 insertions(+), 20 deletions(-) diff --git a/config/config.go b/config/config.go index 4ec8b4b6..014f0c7b 100644 --- a/config/config.go +++ b/config/config.go @@ -199,6 +199,16 @@ func (cfg *Config) Normallize() (err error) { } cfg.convertKfkSecurity() + if cfg.Kafka.TLS.CaCertFiles == "" && cfg.Kafka.TLS.TrustStoreLocation != "" { + if cfg.Kafka.TLS.CaCertFiles, _, err = util.JksToPem(cfg.Kafka.TLS.TrustStoreLocation, cfg.Kafka.TLS.TrustStorePassword, false); err != nil { + return + } + } + if cfg.Kafka.TLS.ClientKeyFile == "" && cfg.Kafka.TLS.KeystoreLocation != "" { + if cfg.Kafka.TLS.ClientCertFile, cfg.Kafka.TLS.ClientKeyFile, err = util.JksToPem(cfg.Kafka.TLS.KeystoreLocation, cfg.Kafka.TLS.KeystorePassword, false); err != nil { + return + } + } if cfg.Kafka.Sasl.Enable { cfg.Kafka.Sasl.Mechanism = strings.ToUpper(cfg.Kafka.Sasl.Mechanism) switch cfg.Kafka.Sasl.Mechanism { diff --git a/go.mod b/go.mod index b5eff3fb..22004b73 100644 --- a/go.mod +++ b/go.mod @@ -10,6 +10,7 @@ require ( github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 github.com/google/gops v0.3.18 + github.com/jcmturner/gokrb5/v8 v8.4.2 github.com/jinzhu/copier v0.3.2 github.com/nacos-group/nacos-sdk-go v1.0.7 github.com/pkg/errors v0.9.1 diff --git a/input/kafka_franz.go b/input/kafka_franz.go index d2376257..80959171 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -17,12 +17,20 @@ package input import ( "context" + "crypto/tls" "strings" "sync" "time" + krb5client "github.com/jcmturner/gokrb5/v8/client" + krb5config "github.com/jcmturner/gokrb5/v8/config" + "github.com/jcmturner/gokrb5/v8/keytab" "github.com/pkg/errors" "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/sasl" + "github.com/twmb/franz-go/pkg/sasl/kerberos" + "github.com/twmb/franz-go/pkg/sasl/plain" + "github.com/twmb/franz-go/pkg/sasl/scram" "github.com/twmb/franz-go/plugin/kzap" "go.uber.org/zap" @@ -31,6 +39,16 @@ import ( "github.com/housepower/clickhouse_sinker/util" ) +const ( + TOK_ID_KRB_AP_REQ = 256 + GSS_API_GENERIC_TAG = 0x60 + KRB5_USER_AUTH = 1 + KRB5_KEYTAB_AUTH = 2 + GSS_API_INITIAL = 1 + GSS_API_VERIFY = 2 + GSS_API_FINISH = 3 +) + var _ Inputer = (*KafkaFranz)(nil) // KafkaFranz implements input.Inputer @@ -71,6 +89,64 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn kgo.BrokerMaxReadBytes(1 << 27), //134 MB kgo.WithLogger(kzap.New(util.Logger)), } + if !taskCfg.Earliest { + opts = append(opts, kgo.ConsumeResetOffset(kgo.NewOffset().AtEnd())) + } + if kfkCfg.TLS.Enable { + var tlsCfg *tls.Config + if tlsCfg, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.EndpIdentAlgo == ""); err != nil { + return + } + opts = append(opts, kgo.DialTLSConfig(tlsCfg)) + } + if kfkCfg.Sasl.Enable { + var mch sasl.Mechanism + switch kfkCfg.Sasl.Mechanism { + case "PLAIN": + auth := plain.Auth{ + User: kfkCfg.Sasl.Username, + Pass: kfkCfg.Sasl.Password, + } + mch = auth.AsMechanism() + case "SCRAM-SHA-256", "SCRAM-SHA-512": + auth := scram.Auth{ + User: kfkCfg.Sasl.Username, + Pass: kfkCfg.Sasl.Password, + } + switch kfkCfg.Sasl.Mechanism { + case "SCRAM-SHA-256": + mch = auth.AsSha256Mechanism() + case "SCRAM-SHA-512": + mch = auth.AsSha512Mechanism() + default: + } + case "GSSAPI": + gssapiCfg := kfkCfg.Sasl.GSSAPI + auth := kerberos.Auth{Service: gssapiCfg.ServiceName} + // refers to https://github.com/Shopify/sarama/blob/main/kerberos_client.go + var krbCfg *krb5config.Config + var kt *keytab.Keytab + if krbCfg, err = krb5config.Load(gssapiCfg.KerberosConfigPath); err != nil { + err = errors.Wrap(err, "") + return + } + if gssapiCfg.AuthType == KRB5_KEYTAB_AUTH { + if kt, err = keytab.Load(gssapiCfg.KeyTabPath); err != nil { + err = errors.Wrap(err, "") + return + } + auth.Client = krb5client.NewWithKeytab(gssapiCfg.Username, gssapiCfg.Realm, kt, krbCfg, krb5client.DisablePAFXFAST(gssapiCfg.DisablePAFXFAST)) + } else { + auth.Client = krb5client.NewWithPassword(gssapiCfg.Username, + gssapiCfg.Realm, gssapiCfg.Password, krbCfg, krb5client.DisablePAFXFAST(gssapiCfg.DisablePAFXFAST)) + } + mch = auth.AsMechanism() + } + if mch != nil { + opts = append(opts, kgo.SASL(mch)) + } + } + if k.cl, err = kgo.NewClient(opts...); err != nil { err = errors.Wrap(err, "") return diff --git a/input/kafka_go.go b/input/kafka_go.go index 60bfe6f8..461c1d45 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -88,16 +88,6 @@ func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn fun WatchPartitionChanges: true, ErrorLogger: &KafkaGoLogger{util.Logger}, } - if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { - if kfkCfg.TLS.CaCertFiles, _, err = util.JksToPem(kfkCfg.TLS.TrustStoreLocation, kfkCfg.TLS.TrustStorePassword, false); err != nil { - return - } - } - if kfkCfg.TLS.ClientKeyFile == "" && kfkCfg.TLS.KeystoreLocation != "" { - if kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, err = util.JksToPem(kfkCfg.TLS.KeystoreLocation, kfkCfg.TLS.KeystorePassword, false); err != nil { - return - } - } var dialer *kafka.Dialer if kfkCfg.TLS.Enable { var tlsConfig *tls.Config diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index ea6d1177..a39ebd5c 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -120,16 +120,6 @@ func GetSaramaConfig(kfkCfg *config.KafkaConfig) (sarCfg *sarama.Config, err err err = errors.Wrapf(err, "") return } - if kfkCfg.TLS.CaCertFiles == "" && kfkCfg.TLS.TrustStoreLocation != "" { - if kfkCfg.TLS.CaCertFiles, _, err = util.JksToPem(kfkCfg.TLS.TrustStoreLocation, kfkCfg.TLS.TrustStorePassword, false); err != nil { - return - } - } - if kfkCfg.TLS.ClientKeyFile == "" && kfkCfg.TLS.KeystoreLocation != "" { - if kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, err = util.JksToPem(kfkCfg.TLS.KeystoreLocation, kfkCfg.TLS.KeystorePassword, false); err != nil { - return - } - } if kfkCfg.TLS.Enable { sarCfg.Net.TLS.Enable = true if sarCfg.Net.TLS.Config, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.EndpIdentAlgo == ""); err != nil { From e58aa71ba6ed4f6830e272c7339aa5b1b83cf280 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 17 Nov 2021 16:05:21 +0800 Subject: [PATCH 214/404] revised KafkaFranz, KafkaSarama and KafkaGo Stop --- input/kafka_go.go | 13 +++++-------- input/kafka_sarama.go | 14 ++++++++------ 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/input/kafka_go.go b/input/kafka_go.go index 461c1d45..36e986b7 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -139,11 +139,7 @@ LOOP_KAFKA_GO: var err error var msg kafka.Message if msg, err = k.r.FetchMessage(k.ctx); err != nil { - if errors.Is(err, context.Canceled) { - util.Logger.Info("KafkaGo.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) - break LOOP_KAFKA_GO - } else if errors.Is(err, io.EOF) { - util.Logger.Info("KafkaGo.Run quit due to reader has been closed", zap.String("task", k.taskCfg.Name)) + if errors.Is(err, context.Canceled) || errors.Is(err, io.EOF) { break LOOP_KAFKA_GO } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.taskCfg.Name).Inc() @@ -161,6 +157,10 @@ LOOP_KAFKA_GO: Timestamp: &msg.Time, }) } + // Note: a closed kafka-go client cannot commit offsets. + k.cleanupFn() + k.r.Close() + util.Logger.Info("KafkaGo.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) } func (k *KafkaGo) CommitMessages(msg *model.InputMessage) (err error) { @@ -177,10 +177,7 @@ func (k *KafkaGo) CommitMessages(msg *model.InputMessage) (err error) { // Stop kafka consumer and close all connections func (k *KafkaGo) Stop() error { - k.cleanupFn() - // Note: a closed kafka-go client cannot commit offsets. k.cancel() - k.r.Close() k.wgRun.Wait() return nil } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index a39ebd5c..706418ad 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -155,16 +155,17 @@ func (k *KafkaSarama) Run() { taskCfg := k.taskCfg LOOP_SARAMA: for { + select { + case <-k.ctx.Done(): + break LOOP_SARAMA + default: + } handler := MyConsumerGroupHandler{k} // `Consume` should be called inside an infinite loop, when a // server-side rebalance happens, the consumer session will need to be // recreated to get the new claims if err := k.cg.Consume(k.ctx, []string{taskCfg.Topic}, handler); err != nil { - if errors.Is(err, context.Canceled) { - util.Logger.Info("KafkaSarama.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) - break LOOP_SARAMA - } else if errors.Is(err, sarama.ErrClosedConsumerGroup) { - util.Logger.Info("KafkaSarama.Run quit due to consumer group has been closed", zap.String("task", k.taskCfg.Name)) + if errors.Is(err, context.Canceled) || errors.Is(err, sarama.ErrClosedConsumerGroup) { break LOOP_SARAMA } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() @@ -174,6 +175,8 @@ LOOP_SARAMA: } } } + k.cg.Close() + util.Logger.Info("KafkaSarama.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) } func (k *KafkaSarama) CommitMessages(msg *model.InputMessage) error { @@ -184,7 +187,6 @@ func (k *KafkaSarama) CommitMessages(msg *model.InputMessage) error { // Stop kafka consumer and close all connections func (k *KafkaSarama) Stop() error { k.cancel() - k.cg.Close() k.wgRun.Wait() return nil } From 7a23ba3a4b8ffd9239e7da0ddeaef5b031769006 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 17 Nov 2021 17:12:52 +0800 Subject: [PATCH 215/404] changed default kafka client from sarama to franz --- config/config.go | 4 ++-- docs/dev/introduction.md | 15 +++++++++++---- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/config/config.go b/config/config.go index 014f0c7b..90a7d459 100644 --- a/config/config.go +++ b/config/config.go @@ -243,11 +243,11 @@ func (cfg *Config) Normallize() (err error) { } func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { - if taskCfg.KafkaClient == "" || (cfg.Kafka.Sasl.Enable && cfg.Kafka.Sasl.Username == "") { + if taskCfg.KafkaClient == "" { // known limitations of kafka-go: // - The Reader API is too high-level. There's no generation cleanup callback which sarama provides. // - Doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 - taskCfg.KafkaClient = "sarama" + taskCfg.KafkaClient = "franz" } if taskCfg.Parser == "" || taskCfg.Parser == "json" { taskCfg.Parser = "fastjson" diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 6df5b2d9..4954f0d6 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -368,16 +368,23 @@ Kafka release history is at [here](https://kafka.apache.org/downloads). Kafka br ### Kafka-go -- Kafka-go [negotiate it's protocol Version](https://github.com/segmentio/kafka-go/blob/c66d8ca149e7f1a7905b47a60962745ceb08a6a9/conn.go#L209). +- Kafka-go [negotiate it's protocol version](https://github.com/segmentio/kafka-go/blob/c66d8ca149e7f1a7905b47a60962745ceb08a6a9/conn.go#L209). - Kafka-go [doesn't support Kerberos authentication](https://github.com/segmentio/kafka-go/issues/237). ### Sarama - Sarama guarantees compatibility [with Kafka 2.6 through 2.8](https://github.com/Shopify/sarama/blob/master/README.md#compatibility-and-api-stability). -- Sarama [has tied it's protocol usage to the Version field in Config](https://github.com/Shopify/sarama/issues/1732). +- Sarama [has tied it's protocol usage to the Version field in Config](https://github.com/Shopify/sarama/issues/1732). My experience is setting `Config.Version` to "0.11.0.0" or "2.5.0" cannot cowork with broker 2.2.0. - Sarama consumer API provides generation cleanup callback. This ensures `exactly once` when consumer-group rebalance occur. +### Franz + +- Franz negotiates it's protocol version. +- Franz supports Kerberos authentication. +- Franz supports generation cleanup callback. +- Franz wins Sarama and Kafka-go at benchmark competition. +- Franz project is young but very active. + ### Conclusion -- Sarama is better than kafka-go, though neither is as mature as the officaial Kafka Java client. You need to try both if clickhouse_sinker fails to connect with Kafka. -- Our experience is sarama can't work well with new kafka server if set its `Config.Version` to "0.11.0.0". So we suggest `KafkaConfig.Version` in clickhouse_sinker config matchs the Kafka server. +Franz is the best Golang client library, though none is as mature as the officaial Kafka Java client. You need to try another if clickhouse_sinker fails to connect with Kafka. From 28cb7d64020a3514915530c068d4a23494c2f840 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 17 Nov 2021 17:25:23 +0800 Subject: [PATCH 216/404] fix tx goroutine leak --- output/clickhouse_util.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 0662d33a..144bc431 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -39,6 +39,7 @@ func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn *s } if stmt, err = tx.Prepare(prepareSQL); err != nil { err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) + _ = tx.Rollback() return } defer stmt.Close() @@ -64,6 +65,7 @@ func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn *s } if stmt, err = tx.Prepare(prepareSQL); err != nil { err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) + _ = tx.Rollback() return } defer stmt.Close() From 31e334bec10862a10332fe961980b9e89d437b3b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 18 Nov 2021 22:00:38 +0800 Subject: [PATCH 217/404] quoted cluster, database and table in sql --- output/clickhouse.go | 20 +++++++++++++------- output/clickhouse_util.go | 6 +++--- 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index a6ffc3a7..e9e38308 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -292,8 +292,11 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) params[i] = "?" } - c.promSerSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.seriesTbl + " (" + strings.Join(serDimsQuoted, ",") + ") " + - "VALUES (" + strings.Join(params, ",") + ")" + c.promSerSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s) VALUES (%s)", + c.cfg.Clickhouse.DB, + c.seriesTbl, + strings.Join(serDimsQuoted, ","), + strings.Join(params, ",")) // Check distributed series table if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { @@ -349,8 +352,11 @@ func (c *ClickHouse) initSchema() (err error) { quotedDms[i] = fmt.Sprintf("`%s`", c.Dims[i].Name) params[i] = "?" } - c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.taskCfg.TableName + " (" + strings.Join(quotedDms, ",") + ") " + - "VALUES (" + strings.Join(params, ",") + ")" + c.prepareSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s) VALUES (%s)", + c.cfg.Clickhouse.DB, + c.taskCfg.TableName, + strings.Join(quotedDms, ","), + strings.Join(params, ",")) util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.taskCfg.Name)) // Check distributed metric table @@ -372,7 +378,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { taskCfg := c.taskCfg chCfg := &c.cfg.Clickhouse if chCfg.Cluster != "" { - onCluster = fmt.Sprintf("ON CLUSTER %s", chCfg.Cluster) + onCluster = fmt.Sprintf("ON CLUSTER `%s`", chCfg.Cluster) } maxDims := math.MaxInt16 if taskCfg.DynamicSchema.MaxDims > 0 { @@ -417,12 +423,12 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } if c.taskCfg.PrometheusSchema { if intVal == model.String { - query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, c.seriesTbl, onCluster, strKey, strVal) + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, c.seriesTbl, onCluster, strKey, strVal) queries = append(queries, query) affectDistSeries = true } } else { - query := fmt.Sprintf("ALTER TABLE %s.%s %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) queries = append(queries, query) affectDistMetric = true } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 144bc431..20a5b107 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -125,9 +125,9 @@ func getDims(database, table string, excludedColumns []string, conn *sql.DB) (di func recreateDistTbls(cluster, database, table string, distTbls []string, conn *sql.DB) (err error) { var queries []string for _, distTbl := range distTbls { - queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s ON CLUSTER %s", database, distTbl, cluster)) - queries = append(queries, fmt.Sprintf("CREATE TABLE %s.%s ON CLUSTER %s AS %s ENGINE = Distributed(%s, %s, %s);", - database, distTbl, cluster, table, + queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS `%s`.`%s` ON CLUSTER `%s`", database, distTbl, cluster)) + queries = append(queries, fmt.Sprintf("CREATE TABLE `%s`.`%s` ON CLUSTER `%s` AS `%s`.`%s` ENGINE = Distributed(`%s`, `%s`, `%s`);", + database, distTbl, cluster, database, table, cluster, database, table)) } for _, query := range queries { From 0b9b1dfef1babc7e29389462ab61b79e2f75c81d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 18 Nov 2021 22:27:53 +0800 Subject: [PATCH 218/404] normallizeTask check cluster name --- config/config.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/config/config.go b/config/config.go index 90a7d459..2031ea07 100644 --- a/config/config.go +++ b/config/config.go @@ -286,6 +286,16 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { err = errors.Errorf("Parser %s doesn't support DynamicSchema", taskCfg.Parser) return } + if cfg.Clickhouse.Cluster == "" { + var numHosts int + for _, shard := range cfg.Clickhouse.Hosts { + numHosts += len(shard) + } + if numHosts > 1 { + err = errors.Errorf("Need to set cluster name when DynamicSchema is enabled and number of hosts is more than one") + return + } + } } if taskCfg.DynamicSchema.WhiteList != "" { if _, err = regexp.Compile(taskCfg.DynamicSchema.WhiteList); err != nil { From 512e1407baadfc5bd17557a221e0d736889df045 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 26 Nov 2021 10:51:57 +0800 Subject: [PATCH 219/404] added __mgmt_id --- model/message.go | 30 ++++++++++++++++++++++-------- output/clickhouse.go | 12 +++++++----- 2 files changed, 29 insertions(+), 13 deletions(-) diff --git a/model/message.go b/model/message.go index 05f0f34c..743f0df4 100644 --- a/model/message.go +++ b/model/message.go @@ -172,13 +172,19 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe row = GetRow() var dig *xxhash.Digest var labels []string + var seriesID uint64 if idxSeriesID >= 0 { - dig = xxhash.New() + // ETL could calculate "__series_id" so that clickhouse_sinker needn't filter out those Prometheus native labels. + val := metric.GetInt("__series_id", false) + seriesID = uint64(val.(int64)) + if seriesID == 0 { + dig = xxhash.New() + } } for i, dim := range dims { if idxSeriesID >= 0 && i == idxSeriesID { *row = append(*row, uint64(0)) - } else if idxSeriesID >= 0 && i == idxSeriesID+1 { + } else if idxSeriesID >= 0 && i == idxSeriesID+2 { *row = append(*row, "") } else if strings.HasPrefix(dim.Name, "__kafka") { if strings.HasSuffix(dim.Name, "_topic") { @@ -199,10 +205,14 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe *row = append(*row, val) if idxSeriesID >= 0 && dim.Type == String && val != nil { if labelVal := val.(string); labelVal != "" { - _, _ = dig.WriteString("###") - _, _ = dig.WriteString(dim.Name) - _, _ = dig.WriteString("###") - _, _ = dig.WriteString(labelVal) + // "__series_id" calculation envolves all Prometheus native labels (including metric name) in some order. + if dig != nil { + _, _ = dig.WriteString("###") + _, _ = dig.WriteString(dim.Name) + _, _ = dig.WriteString("###") + _, _ = dig.WriteString(labelVal) + } + // "labels" JSON excludes "le", so that "labels" can be used as group key for histogram queries. if dim.Name != nameKey && dim.Name != "le" { labels = append(labels, fmt.Sprintf(`"%s": "%s"`, dim.Name, labelVal)) } @@ -211,8 +221,12 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe } } if idxSeriesID >= 0 { - (*row)[idxSeriesID] = dig.Sum64() - (*row)[idxSeriesID+1] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) + if dig == nil { + (*row)[idxSeriesID] = seriesID + } else { + (*row)[idxSeriesID] = dig.Sum64() + } + (*row)[idxSeriesID+2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) } return } diff --git a/output/clickhouse.go b/output/clickhouse.go index e9e38308..55629d81 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -112,7 +112,8 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn *sql.DB) (err error) { c.mux.Lock() for _, row := range rows { seriesID := (*row)[c.IdxSerID].(uint64) - if c.bmSeries.CheckedAdd(seriesID) { + mgmtID := uint64((*row)[c.IdxSerID+1].(int64)) + if c.bmSeries.CheckedAdd(seriesID + mgmtID) { seriesRows = append(seriesRows, row) } } @@ -196,9 +197,9 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { var query string if c.cfg.Clickhouse.Cluster != "" { - query = fmt.Sprintf("SELECT __series_id FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) + query = fmt.Sprintf("SELECT toUInt64(toUInt64(__series_id) + toUInt64(__mgmt_id)) FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) } else { - query = fmt.Sprintf("SELECT __series_id FROM %s.%s", c.cfg.Clickhouse.DB, c.seriesTbl) + query = fmt.Sprintf("SELECT toUInt64(toUInt64(__series_id) + toUInt64(__mgmt_id)) FROM %s.%s", c.cfg.Clickhouse.DB, c.seriesTbl) } util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs *sql.Rows @@ -249,6 +250,7 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { c.seriesTbl = c.taskCfg.TableName + "_series" expSeriesDims := []*model.ColumnWithType{ {Name: "__series_id", Type: model.Int}, + {Name: "__mgmt_id", Type: model.Int}, {Name: "labels", Type: model.String}, } var seriesDims []*model.ColumnWithType @@ -272,11 +274,11 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { } } if badFirst { - err = errors.Errorf(`First columns of %s are expect to be "__series_id UInt64, labels String".`, c.seriesTbl) + err = errors.Errorf(`First columns of %s are expect to be "__series_id UInt64, __mgmt_id UInt64, labels String".`, c.seriesTbl) return } c.NameKey = "__name__" // prometheus uses internal "__name__" label for metric name - for i := 2; i < len(seriesDims); i++ { + for i := len(expSeriesDims); i < len(seriesDims); i++ { serDim := seriesDims[i] if serDim.Type == model.String { c.NameKey = serDim.Name // opentsdb uses "metric" tag for metric name From e7eb2d16424a6c5a3e990d9ab01dc5cf47c28554 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 30 Nov 2021 15:12:45 +0800 Subject: [PATCH 220/404] minor comment --- input/kafka_franz.go | 1 + 1 file changed, 1 insertion(+) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 80959171..93e2a773 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -87,6 +87,7 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn kgo.MaxConcurrentFetches(3), kgo.FetchMaxBytes(1 << 27), //134 MB kgo.BrokerMaxReadBytes(1 << 27), //134 MB + //kgo.MetadataMaxAge(...) corresponds to sarama.Config.Metadata.RefreshFrequency kgo.WithLogger(kzap.New(util.Logger)), } if !taskCfg.Earliest { From 1d44572fc5ccd7dc4c3f35a96be321775379cfbc Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 30 Nov 2021 16:31:35 +0800 Subject: [PATCH 221/404] deduplicated per __mgmt_id --- model/message.go | 12 ++++++++---- output/clickhouse.go | 15 +++++++-------- 2 files changed, 15 insertions(+), 12 deletions(-) diff --git a/model/message.go b/model/message.go index 743f0df4..e6b53b02 100644 --- a/model/message.go +++ b/model/message.go @@ -221,11 +221,15 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe } } if idxSeriesID >= 0 { - if dig == nil { - (*row)[idxSeriesID] = seriesID - } else { - (*row)[idxSeriesID] = dig.Sum64() + if dig != nil { + seriesID = dig.Sum64() + } + (*row)[idxSeriesID] = seriesID + mgmtID := uint64((*row)[idxSeriesID+1].(int64)) + if mgmtID == 0 { + mgmtID = seriesID } + (*row)[idxSeriesID+1] = mgmtID (*row)[idxSeriesID+2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) } return diff --git a/output/clickhouse.go b/output/clickhouse.go index 55629d81..a7535bf8 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -111,9 +111,8 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn *sql.DB) (err error) { var seriesRows model.Rows c.mux.Lock() for _, row := range rows { - seriesID := (*row)[c.IdxSerID].(uint64) - mgmtID := uint64((*row)[c.IdxSerID+1].(int64)) - if c.bmSeries.CheckedAdd(seriesID + mgmtID) { + mgmtID := (*row)[c.IdxSerID+1].(uint64) + if c.bmSeries.CheckedAdd(mgmtID) { seriesRows = append(seriesRows, row) } } @@ -197,13 +196,13 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { var query string if c.cfg.Clickhouse.Cluster != "" { - query = fmt.Sprintf("SELECT toUInt64(toUInt64(__series_id) + toUInt64(__mgmt_id)) FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) + query = fmt.Sprintf("SELECT toUInt64(__mgmt_id) FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) } else { - query = fmt.Sprintf("SELECT toUInt64(toUInt64(__series_id) + toUInt64(__mgmt_id)) FROM %s.%s", c.cfg.Clickhouse.DB, c.seriesTbl) + query = fmt.Sprintf("SELECT toUInt64(__mgmt_id) FROM %s.%s", c.cfg.Clickhouse.DB, c.seriesTbl) } util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs *sql.Rows - var seriesID uint64 + var mgmtID uint64 if rs, err = conn.Query(query); err != nil { err = errors.Wrapf(err, "") return err @@ -211,11 +210,11 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { defer rs.Close() c.bmSeries = roaring64.New() for rs.Next() { - if err = rs.Scan(&seriesID); err != nil { + if err = rs.Scan(&mgmtID); err != nil { err = errors.Wrapf(err, "") return err } - c.bmSeries.Add(seriesID) + c.bmSeries.Add(mgmtID) } util.Logger.Info(fmt.Sprintf("loaded %d series from %v", c.bmSeries.GetCardinality(), c.seriesTbl), zap.String("task", c.taskCfg.Name)) return From 1acefdaccbd192c86c0ce06e96e1c8d3714fbeb0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 1 Dec 2021 15:09:05 +0800 Subject: [PATCH 222/404] adjusted __series_id and __mgmt_id from uint64 to int64 --- model/message.go | 17 ++++++++++------- output/clickhouse.go | 14 +++++++------- 2 files changed, 17 insertions(+), 14 deletions(-) diff --git a/model/message.go b/model/message.go index e6b53b02..9eace4d0 100644 --- a/model/message.go +++ b/model/message.go @@ -172,18 +172,22 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe row = GetRow() var dig *xxhash.Digest var labels []string - var seriesID uint64 + var seriesID, mgmtID int64 if idxSeriesID >= 0 { // ETL could calculate "__series_id" so that clickhouse_sinker needn't filter out those Prometheus native labels. val := metric.GetInt("__series_id", false) - seriesID = uint64(val.(int64)) + seriesID = val.(int64) if seriesID == 0 { dig = xxhash.New() } + val = metric.GetInt("__mgmt_id", false) + mgmtID = val.(int64) } for i, dim := range dims { if idxSeriesID >= 0 && i == idxSeriesID { - *row = append(*row, uint64(0)) + *row = append(*row, int64(0)) + } else if idxSeriesID >= 0 && i == idxSeriesID+1 { + *row = append(*row, int64(0)) } else if idxSeriesID >= 0 && i == idxSeriesID+2 { *row = append(*row, "") } else if strings.HasPrefix(dim.Name, "__kafka") { @@ -221,14 +225,13 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe } } if idxSeriesID >= 0 { - if dig != nil { - seriesID = dig.Sum64() + if seriesID == 0 { + seriesID = int64(dig.Sum64()) } - (*row)[idxSeriesID] = seriesID - mgmtID := uint64((*row)[idxSeriesID+1].(int64)) if mgmtID == 0 { mgmtID = seriesID } + (*row)[idxSeriesID] = seriesID (*row)[idxSeriesID+1] = mgmtID (*row)[idxSeriesID+2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) } diff --git a/output/clickhouse.go b/output/clickhouse.go index a7535bf8..629d4c12 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -111,8 +111,8 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn *sql.DB) (err error) { var seriesRows model.Rows c.mux.Lock() for _, row := range rows { - mgmtID := (*row)[c.IdxSerID+1].(uint64) - if c.bmSeries.CheckedAdd(mgmtID) { + mgmtID := (*row)[c.IdxSerID+1].(int64) + if c.bmSeries.CheckedAdd(uint64(mgmtID)) { seriesRows = append(seriesRows, row) } } @@ -196,13 +196,13 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { var query string if c.cfg.Clickhouse.Cluster != "" { - query = fmt.Sprintf("SELECT toUInt64(__mgmt_id) FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) + query = fmt.Sprintf("SELECT toInt64(__mgmt_id) FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) } else { - query = fmt.Sprintf("SELECT toUInt64(__mgmt_id) FROM %s.%s", c.cfg.Clickhouse.DB, c.seriesTbl) + query = fmt.Sprintf("SELECT toInt64(__mgmt_id) FROM %s.%s", c.cfg.Clickhouse.DB, c.seriesTbl) } util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs *sql.Rows - var mgmtID uint64 + var mgmtID int64 if rs, err = conn.Query(query); err != nil { err = errors.Wrapf(err, "") return err @@ -214,7 +214,7 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { err = errors.Wrapf(err, "") return err } - c.bmSeries.Add(mgmtID) + c.bmSeries.Add(uint64(mgmtID)) } util.Logger.Info(fmt.Sprintf("loaded %d series from %v", c.bmSeries.GetCardinality(), c.seriesTbl), zap.String("task", c.taskCfg.Name)) return @@ -273,7 +273,7 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { } } if badFirst { - err = errors.Errorf(`First columns of %s are expect to be "__series_id UInt64, __mgmt_id UInt64, labels String".`, c.seriesTbl) + err = errors.Errorf(`First columns of %s are expect to be "__series_id Int64, __mgmt_id Int64, labels String".`, c.seriesTbl) return } c.NameKey = "__name__" // prometheus uses internal "__name__" label for metric name From a1d241ffe2e520729d8484a3bc2ba5bf1dca696b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 1 Dec 2021 17:24:34 +0800 Subject: [PATCH 223/404] updated sonic --- .github/workflows/tests.yml | 2 +- go.mod | 65 +++++++++++++++++++++++++++++++++---- go.sum | 9 ++--- 3 files changed, 62 insertions(+), 14 deletions(-) diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index fd7bb26f..efb9e3f0 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -16,7 +16,7 @@ jobs: - name: Set up Go 1.x uses: actions/setup-go@v2 with: - go-version: 1.16.7 + go-version: 1.17 id: go - name: Check out code into the Go module directory diff --git a/go.mod b/go.mod index 22004b73..01a763a6 100644 --- a/go.mod +++ b/go.mod @@ -1,12 +1,12 @@ module github.com/housepower/clickhouse_sinker -go 1.14 +go 1.17 require ( github.com/ClickHouse/clickhouse-go v1.5.1 github.com/RoaringBitmap/roaring v0.9.4 github.com/Shopify/sarama v1.30.0 - github.com/bytedance/sonic v1.0.0-rc + github.com/bytedance/sonic v1.0.0-rc.2.0.20211130104717-49df11ef8309 github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 github.com/google/gops v0.3.18 @@ -17,7 +17,6 @@ require ( github.com/prometheus/client_golang v1.11.0 github.com/prometheus/common v0.26.0 github.com/segmentio/kafka-go v0.4.22 - github.com/smartystreets/goconvey v1.6.4 // indirect github.com/stretchr/testify v1.7.0 github.com/tidwall/gjson v1.10.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 @@ -25,11 +24,65 @@ require ( github.com/twmb/franz-go/plugin/kzap v1.0.0 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.0.2 + go.uber.org/zap v1.19.1 + golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba + gopkg.in/natefinch/lumberjack.v2 v2.0.0 +) + +require ( + github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/bits-and-blooms/bitset v1.2.0 // indirect + github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 // indirect + github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06 // indirect + github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 // indirect + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/eapache/go-resiliency v1.2.0 // indirect + github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 // indirect + github.com/eapache/queue v1.1.0 // indirect + github.com/go-errors/errors v1.0.1 // indirect + github.com/golang/protobuf v1.4.3 // indirect + github.com/golang/snappy v0.0.4 // indirect + github.com/hashicorp/go-uuid v1.0.2 // indirect + github.com/jcmturner/aescts/v2 v2.0.0 // indirect + github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect + github.com/jcmturner/gofork v1.0.0 // indirect + github.com/jcmturner/rpc/v2 v2.0.3 // indirect + github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af // indirect + github.com/json-iterator/go v1.1.11 // indirect + github.com/klauspost/compress v1.13.6 // indirect + github.com/klauspost/cpuid/v2 v2.0.9 // indirect + github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f // indirect + github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.1 // indirect + github.com/mschoch/smat v0.2.0 // indirect + github.com/pierrec/lz4 v2.6.1+incompatible // indirect + github.com/pierrec/lz4/v4 v4.1.8 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/prometheus/client_model v0.2.0 // indirect + github.com/prometheus/procfs v0.6.0 // indirect + github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect + github.com/smartystreets/goconvey v1.6.4 // indirect + github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 // indirect + github.com/tidwall/match v1.1.1 // indirect + github.com/tidwall/pretty v1.2.0 // indirect + github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 // indirect + github.com/twitchyliquid64/golang-asm v0.15.1 // indirect + github.com/twmb/franz-go/pkg/kmsg v0.0.0-20211104051938-70808186d5f7 // indirect + github.com/twmb/go-rbtree v1.0.0 // indirect + github.com/xdg-go/pbkdf2 v1.0.0 // indirect + github.com/xdg-go/stringprep v1.0.2 // indirect github.com/xdg/scram v1.0.3 // indirect github.com/xdg/stringprep v1.0.3 // indirect - go.uber.org/zap v1.19.1 + go.uber.org/atomic v1.9.0 // indirect + go.uber.org/multierr v1.7.0 // indirect golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 // indirect golang.org/x/net v0.0.0-20211020060615-d418f374d309 // indirect - golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba - gopkg.in/natefinch/lumberjack.v2 v2.0.0 + golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 // indirect + golang.org/x/text v0.3.7 // indirect + google.golang.org/protobuf v1.26.0-rc.1 // indirect + gopkg.in/ini.v1 v1.42.0 // indirect + gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect ) diff --git a/go.sum b/go.sum index 979573b4..1063d9f7 100644 --- a/go.sum +++ b/go.sum @@ -31,8 +31,8 @@ github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAK github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= -github.com/bytedance/sonic v1.0.0-rc h1:RMj5thTrWOelnkZQacWpG/Gm06eQY1yNZY8Ze+BxeDY= -github.com/bytedance/sonic v1.0.0-rc/go.mod h1:lqEFq2niUZaKwt5URNYvcyY6r72F21rXzHgnzXS1Fw4= +github.com/bytedance/sonic v1.0.0-rc.2.0.20211130104717-49df11ef8309 h1:n0R1DyA4S3BjKsjqMyAdUlX/UVC+91K5n6iV5V0CB94= +github.com/bytedance/sonic v1.0.0-rc.2.0.20211130104717-49df11ef8309/go.mod h1:lqEFq2niUZaKwt5URNYvcyY6r72F21rXzHgnzXS1Fw4= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -103,9 +103,7 @@ github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm4 github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/gorilla/securecookie v1.1.1 h1:miw7JPhV+b/lAHSXz4qd/nN9jRiAFV5FwjeKyCS8BvQ= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= -github.com/gorilla/sessions v1.2.1 h1:DHd3rPN5lE3Ts3D8rKkQ8x/0kqfeNmBAaiSi+o7FsgI= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= @@ -315,10 +313,8 @@ golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5y golang.org/x/crypto v0.0.0-20210920023735-84f357641f63/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 h1:7I4JAnoQBe7ZtJcBaYHi5UtiO8tQHbUSXxL+pnGRANg= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= -golang.org/x/mod v0.4.2 h1:Gz96sIWK3OalVv/I/qNygP42zyoKp3xptRVCWRFEBvo= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -381,7 +377,6 @@ golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgw golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.1.5 h1:ouewzE6p+/VEB31YYnTbEJdi8pFqKp4P4n85vwo3DHA= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= From 2bc40bb44a5c1ee457d080359c494cd1c3786a18 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 1 Dec 2021 22:21:21 +0800 Subject: [PATCH 224/404] replaced ShardingPolicy with ShardingStripe --- config/config.go | 4 +-- docs/configuration/config.md | 4 +-- docs/dev/design.md | 17 ++++-------- task/sharding.go | 50 +++++++++++++++++++----------------- 4 files changed, 35 insertions(+), 40 deletions(-) diff --git a/config/config.go b/config/config.go index 2031ea07..ae2e89bf 100644 --- a/config/config.go +++ b/config/config.go @@ -140,8 +140,8 @@ type TaskConfig struct { // ShardingKey is the column name to which sharding against ShardingKey string `json:"shardingKey,omitempty"` - // ShardingPolicy is `stripe,`(requires ShardingKey be numerical) or `hash`(requires ShardingKey be string) - ShardingPolicy string `json:"shardingPolicy,omitempty"` + // ShardingStripe take effect iff the sharding key is numerical + ShardingStripe uint64 `json:"shardingStripe,omitempty"` FlushInterval int `json:"flushInterval,omitempty"` BufferSize int `json:"bufferSize,omitempty"` diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 5cebb7a8..58ed7965 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -151,8 +151,8 @@ // shardingKey is the column name to which sharding against "shardingKey": "", - // shardingPolicy is `stripe,`(requires ShardingKey be numerical) or `hash`(requires ShardingKey be string) - "shardingPolicy": "", + // shardingStripe take effect iff the sharding key is numerical + "shardingStripe": 0, // interval of flushing the batch. Default to 5, max to 600. "flushInterval": 5, diff --git a/docs/dev/design.md b/docs/dev/design.md index 5132a739..8f18abac 100644 --- a/docs/dev/design.md +++ b/docs/dev/design.md @@ -1,7 +1,6 @@ # Architecture ## Sharding -### Sharding with kafka message offset stripe (default) clickhouse_sinker guarantee: @@ -12,28 +11,22 @@ So if you setup ClickHouse properly(ReplacingMergeTree ORDER BY (__kafak_topic, It's hard for clickhouse_sinker to guarantee exactly-once semantic without ReplacingMergeTree. Kafka consumer group load-balance cause duplicated messages if one consumer quit suddenly. +### Sharding with kafka message offset stripe (default) + The flow is: - Fetch message via kafka-go or samara, which starts internally an goroutine for each partition. - Parse messages in a global goroutine pool(pool size is customizable), fill the result to a ring according to the message's partition and offset. -- Generate a batch if messages in a ring reach a batchSize bondary, or flush timer fire. This ensures offset/batchSize be same for all messages inside a batch. -- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). Batch is routed according to `(kafka_offset/roundup(buffer_size))%clickhouse_shards`. +- Generate a batch if messages in a ring reach a batchSize bondary, or flush timer fire. For each message, the dest shard is determined by `(kafka_offset/roundup(buffer_size))%clickhouse_shards`. +- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). ### Sharding with custom key and policy -clickhouse_sinker guarantee: - -- at-least-once -- Every message is routed to the determined (per `shardingKey` and `shardingPolicy`) ClickHouse shard. - -`shardingKey` value is a column name. `shardingPolicy` value is `stripe,` or `hash`. -The hash function used internally is xxHash64. - The flow is: - Fetch message via kafka-go or samara, which starts internally an goroutine for each partition. - Parse messages in a global goroutine pool(pool size is customizable), fill the result to a ring according to the message's partition and offset. -- Shard messages in a ring if reach a batchSize bondary, or flush timer fire. There's one-to-one relationship between shard slots and ClickHouse shards. +- Shard messages in a ring if reach a batchSize bondary, or flush timer fire. For each message, if the sharding key is numerical(integer, float, time and etc.), the dest shard is determined by `(shardingKey/shardingStripe)%clickhouse_shards`, otherwise it is determined by `xxHash64(shardingKey)%clickhouse_shards`. - Generate batches for all shard slots if messages in one shard slot reach batchSize, or flush timer fire. Those batches form a `BatchGroup`. The `before` relationship could be impossilbe if messages of a partition are distributed to multiple batches. So those batches need to be committed after ALL of them have been written to clickhouse. - Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). diff --git a/task/sharding.go b/task/sharding.go index fc6f0c7a..4f182176 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -2,8 +2,6 @@ package task import ( "fmt" - "strconv" - "strings" "sync" "time" @@ -18,33 +16,37 @@ import ( ) type ShardingPolicy struct { - ckNum int //number of clickhouse instances + shards int //number of clickhouse shards colSeq int //shardingKey column seq, 0 based stripe uint64 //=0 means hash, >0 means stripe size } -func NewShardingPolicy(shardingKey, shardingPolicy string, dims []*model.ColumnWithType, ckNum int) (policy *ShardingPolicy, err error) { - policy = &ShardingPolicy{ckNum: ckNum} +func NewShardingPolicy(shardingKey string, shardingStripe uint64, dims []*model.ColumnWithType, shards int) (policy *ShardingPolicy, err error) { + policy = &ShardingPolicy{stripe: shardingStripe, shards: shards} colSeq := -1 for i, dim := range dims { if dim.Name == shardingKey { colSeq = i + switch dim.Type { + case model.Int, model.Float, model.DateTime, model.ElasticDateTime: + //numerical + if policy.stripe <= 0 { + policy.stripe = uint64(1) + } + case model.String: + //string + policy.stripe = 0 + default: + err = errors.Errorf("invalid shardingKey %s, expect its type be numerical or string", shardingKey) + return + } } } if colSeq < 0 { - err = errors.Errorf("invalid shardingKey %s", shardingKey) + err = errors.Errorf("invalid shardingKey %s, no such column", shardingKey) return } policy.colSeq = colSeq - if shardingPolicy == "hash" { - policy.stripe = 0 - } else if strings.HasPrefix(shardingPolicy, "stripe,") { - if policy.stripe, err = strconv.ParseUint(shardingPolicy[len("stripe,"):], 10, 64); err != nil { - err = errors.Wrapf(err, "invalid shardingPolicy %s", shardingPolicy) - } - } else { - err = errors.Errorf("invalid shardingPolicy %s", shardingPolicy) - } return } @@ -83,7 +85,7 @@ func (policy *ShardingPolicy) Calc(row *model.Row) (shard int, err error) { err = errors.Errorf("failed to convert %+v to integer", v) return } - shard = int((valu64 / policy.stripe) % uint64(policy.ckNum)) + shard = int((valu64 / policy.stripe) % uint64(policy.shards)) } else { var valu64 uint64 switch v := val.(type) { @@ -95,7 +97,7 @@ func (policy *ShardingPolicy) Calc(row *model.Row) (shard int, err error) { err = errors.Errorf("failed to convert %+v to string", v) return } - shard = int(valu64 % uint64(policy.ckNum)) + shard = int(valu64 % uint64(policy.shards)) } return } @@ -104,7 +106,7 @@ type Sharder struct { service *Service policy *ShardingPolicy batchSys *model.BatchSys - ckNum int + shards int mux sync.Mutex msgBuf []*model.Rows offsets map[int]int64 @@ -113,20 +115,20 @@ type Sharder struct { func NewSharder(service *Service) (sh *Sharder, err error) { var policy *ShardingPolicy - ckNum := pool.NumShard() + shards := pool.NumShard() taskCfg := service.taskCfg - if policy, err = NewShardingPolicy(taskCfg.ShardingKey, taskCfg.ShardingPolicy, service.clickhouse.Dims, ckNum); err != nil { + if policy, err = NewShardingPolicy(taskCfg.ShardingKey, taskCfg.ShardingStripe, service.clickhouse.Dims, shards); err != nil { return } sh = &Sharder{ service: service, policy: policy, batchSys: model.NewBatchSys(taskCfg, service.fnCommit), - ckNum: ckNum, - msgBuf: make([]*model.Rows, ckNum), + shards: shards, + msgBuf: make([]*model.Rows, shards), offsets: make(map[int]int64), } - for i := 0; i < ckNum; i++ { + for i := 0; i < shards; i++ { sh.msgBuf[i] = model.GetRows() } return @@ -162,7 +164,7 @@ func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOf sh.offsets[partition] = endOff - 1 statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Add(float64(msgCnt)) var maxBatchSize int - for i := 0; i < sh.ckNum; i++ { + for i := 0; i < sh.shards; i++ { batchSize := len(*sh.msgBuf[i]) if maxBatchSize < batchSize { maxBatchSize = batchSize From 3e7e7a9f3ced93a5a07b994c0af7b943b6fe3903 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 1 Dec 2021 22:32:32 +0800 Subject: [PATCH 225/404] make lint happy --- .github/workflows/lint.yml | 4 ++-- Makefile | 2 +- input/kafka_franz.go | 13 +++---------- output/clickhouse.go | 2 +- 4 files changed, 7 insertions(+), 14 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index d8aeba91..0be80a0e 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -14,13 +14,13 @@ jobs: uses: golangci/golangci-lint-action@v2 with: # Required: the version of golangci-lint is required and must be specified without patch version: we always use the latest patch version. - version: v1.40.1 + version: v1.43.0 # Optional: working directory, useful for monorepos # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true diff --git a/Makefile b/Makefile index e3c574d8..49c0e82f 100644 --- a/Makefile +++ b/Makefile @@ -25,6 +25,6 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.json diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 93e2a773..be23f50f 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -40,13 +40,7 @@ import ( ) const ( - TOK_ID_KRB_AP_REQ = 256 - GSS_API_GENERIC_TAG = 0x60 - KRB5_USER_AUTH = 1 - KRB5_KEYTAB_AUTH = 2 - GSS_API_INITIAL = 1 - GSS_API_VERIFY = 2 - GSS_API_FINISH = 3 + Krb5KeytabAuth = 2 ) var _ Inputer = (*KafkaFranz)(nil) @@ -131,7 +125,7 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn err = errors.Wrap(err, "") return } - if gssapiCfg.AuthType == KRB5_KEYTAB_AUTH { + if gssapiCfg.AuthType == Krb5KeytabAuth { if kt, err = keytab.Load(gssapiCfg.KeyTabPath); err != nil { err = errors.Wrap(err, "") return @@ -191,8 +185,7 @@ func (k *KafkaFranz) Run() { func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { // "LeaderEpoch: -1" will disable leader epoch validation - k.cl.CommitRecords(context.Background(), &kgo.Record{Topic: msg.Topic, Partition: int32(msg.Partition), Offset: msg.Offset, LeaderEpoch: -1}) - return nil + return k.cl.CommitRecords(context.Background(), &kgo.Record{Topic: msg.Topic, Partition: int32(msg.Partition), Offset: msg.Offset, LeaderEpoch: -1}) } // Stop kafka consumer and close all connections diff --git a/output/clickhouse.go b/output/clickhouse.go index 629d4c12..d316060c 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -170,7 +170,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { if err = batch.Commit(); err == nil { return } - // Note: kafka_go and sarama commit give different error when context is cancceled. + // Note: kafka_go and sarama commit throws different error for context cancellation. if errors.Is(err, context.Canceled) || errors.Is(err, io.ErrClosedPipe) { util.Logger.Warn("Batch.Commit failed due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) return From 5862277d48f3fdec0d37e234a3654749438df263 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 2 Dec 2021 00:20:10 +0800 Subject: [PATCH 226/404] fixed Sharder.PutElems --- task/sharding.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/task/sharding.go b/task/sharding.go index 4f182176..bcfa1922 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -139,7 +139,7 @@ func (sh *Sharder) Calc(row *model.Row) (int, error) { } func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOff, ringCapMask int64) { - if begOff <= endOff { + if begOff >= endOff { return } msgCnt := endOff - begOff From c53010da8ff4aeea733c63d5877aae7a5df83558 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 2 Dec 2021 09:34:35 +0800 Subject: [PATCH 227/404] calculate __mgmt_id --- model/message.go | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/model/message.go b/model/message.go index 9eace4d0..b19816b8 100644 --- a/model/message.go +++ b/model/message.go @@ -174,14 +174,11 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe var labels []string var seriesID, mgmtID int64 if idxSeriesID >= 0 { - // ETL could calculate "__series_id" so that clickhouse_sinker needn't filter out those Prometheus native labels. + // If some labels are not Prometheus native, ETL shall calculate and pass "__series_id", otherwise clickhouse_sinker use "__mgmt_id". val := metric.GetInt("__series_id", false) seriesID = val.(int64) - if seriesID == 0 { - dig = xxhash.New() - } - val = metric.GetInt("__mgmt_id", false) - mgmtID = val.(int64) + // clickhouse_sinker calculate "__mgmt_id" based on all labels. + dig = xxhash.New() } for i, dim := range dims { if idxSeriesID >= 0 && i == idxSeriesID { @@ -225,11 +222,9 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe } } if idxSeriesID >= 0 { + mgmtID = int64(dig.Sum64()) if seriesID == 0 { - seriesID = int64(dig.Sum64()) - } - if mgmtID == 0 { - mgmtID = seriesID + seriesID = mgmtID } (*row)[idxSeriesID] = seriesID (*row)[idxSeriesID+1] = mgmtID From 9cf8e4821d49f83a75ba884a6450c35fd3b5c6c3 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 2 Dec 2021 14:57:01 +0800 Subject: [PATCH 228/404] column name shall not be empty string --- task/task.go | 1 + 1 file changed, 1 insertion(+) diff --git a/task/task.go b/task/task.go index ed297cbf..de9b0ae9 100644 --- a/task/task.go +++ b/task/task.go @@ -131,6 +131,7 @@ func (service *Service) Init() (err error) { for _, dim := range taskCfg.ExcludeColumns { service.knownKeys.Store(dim, nil) } + service.knownKeys.Store("", nil) // column name shall not be empty string service.newKeys = sync.Map{} atomic.StoreInt32(&service.cntNewKeys, 0) } From 051032a63409de37508df99a44e1c713d8552027 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 17 Dec 2021 23:57:22 +0800 Subject: [PATCH 229/404] fix typo --- docs/dev/design.md | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/dev/design.md b/docs/dev/design.md index 8f18abac..44a69d36 100644 --- a/docs/dev/design.md +++ b/docs/dev/design.md @@ -9,39 +9,39 @@ clickhouse_sinker guarantee: So if you setup ClickHouse properly(ReplacingMergeTree ORDER BY (__kafak_topic, __kafka_partition, __kafka_offset)), you could get exactly-once semantic. -It's hard for clickhouse_sinker to guarantee exactly-once semantic without ReplacingMergeTree. Kafka consumer group load-balance cause duplicated messages if one consumer quit suddenly. +It's hard for clickhouse_sinker to guarantee exactly-once semantic without ReplacingMergeTree. Kafka consumer group load-balance cause duplicated messages if one consumer crash suddenly. ### Sharding with kafka message offset stripe (default) The flow is: -- Fetch message via kafka-go or samara, which starts internally an goroutine for each partition. -- Parse messages in a global goroutine pool(pool size is customizable), fill the result to a ring according to the message's partition and offset. -- Generate a batch if messages in a ring reach a batchSize bondary, or flush timer fire. For each message, the dest shard is determined by `(kafka_offset/roundup(buffer_size))%clickhouse_shards`. -- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). +- Fetch message via Franz, Sarama, or kafka-go, which starts internally a goroutine for each partition. +- Parse messages in a global goroutine pool(pool size is customizable), fill the result into a ring according to the message's partition and offset. +- Generate a batch when messages in a ring reach a batchSize boundary or flush timer fire. For each message, the dest shard is determined by `(kafka_offset/roundup(buffer_size))%clickhouse_shards`. +- Write batch to ClickHouse in a global goroutine pool(pool size is fixed according to the number of tasks and Clickhouse shards). -### Sharding with custom key and policy +### Sharding with custom key The flow is: -- Fetch message via kafka-go or samara, which starts internally an goroutine for each partition. -- Parse messages in a global goroutine pool(pool size is customizable), fill the result to a ring according to the message's partition and offset. -- Shard messages in a ring if reach a batchSize bondary, or flush timer fire. For each message, if the sharding key is numerical(integer, float, time and etc.), the dest shard is determined by `(shardingKey/shardingStripe)%clickhouse_shards`, otherwise it is determined by `xxHash64(shardingKey)%clickhouse_shards`. -- Generate batches for all shard slots if messages in one shard slot reach batchSize, or flush timer fire. Those batches form a `BatchGroup`. The `before` relationship could be impossilbe if messages of a partition are distributed to multiple batches. So those batches need to be committed after ALL of them have been written to clickhouse. -- Write batchs to ClickHouse in a global goroutine pool(pool size is fixed according to number of task and clickhouse shards). +- Fetch message via kafka-go or samara, which starts internally a goroutine for each partition. +- Parse messages in a global goroutine pool(pool size is customizable), fill the result into a ring according to the message's partition and offset. +- Shard messages in a ring when messages reach a batchSize boundary or flush timer fire. For each message, if the sharding key is numerical(integer, float, time, etc.), the dest shard is determined by `(shardingKey/shardingStripe)%clickhouse_shards`, otherwise it is determined by `xxHash64(shardingKey)%clickhouse_shards`. +- Generate batches for all shard slots if messages in one shard slot reach batchSize boundary or flush timer fire. Those batches form a `BatchGroup`. The `before` relationship could be impossible if messages of a partition are distributed to multiple batches. So those batches need to be committed after ALL of them have been written to Clickhouse. +- Write batch to ClickHouse in a global goroutine pool(pool size is fixed according to the number of tasks and Clickhouse shards). ## Task scheduling The clickhouse-server configuration item `max_concurrent_queries`(default 100) is the maximum number of simultaneously processed queries related to MergeTree table. If the number of concurrent INSERT is close to `max_concurrent_queries`, the user queries(`SELECT`) could fail due to the limit. -If the clickhouse-server is big, ingesting data to >=100 MergeTree tabls via clickhouse_sinker bring pressure to the clickhouse cluster. On the other side, large number of clickhouse_sinker instances requires lot of CPU/MEM resources. +If the clickhouse-server is big, ingesting data to >=100 MergeTree tables via clickhouse_sinker bring pressure to the Clickhouse cluster. On the other side, large number of clickhouse_sinker instances requires lots of CPU/MEM resources. The solution is, clickhouse_sinker instances coordinate with each other to assign tasks among themselves. The task scheduling procedure: -- Some platform(Kubernetes, Yarn and etc.) start several clickhouse_sinker instances and may start/stop instances dynamically. Every clickhouse_sinker instance register with Nacos as a single service(CLI option `--nacos-service-name`). +- Some platform(Kubernetes, Yarn, etc.) start several clickhouse_sinker instances and may start/stop instances dynamically. Every clickhouse_sinker instance registers with Nacos as a single service(CLI option `--nacos-service-name`). - Someone publish(add/delete/modify) a list of tasks(with empty assignment) to Nacos. -- The first clickhouse_sinker(per instance's ip+port) instance(named scheduler) is responsible to generate and publish task assignment regularly. The task list and assignment consist of the whole config. The task list change, service change and task lag change will trigger another assignment. The scheduler ensure Each clickhouse_innker instance's total lag be balanced. -- Each clickhouse_sinker reload the config regularly. This may start/stop tasks. clickhouse_sinker stop tasks gracefully so that there's no message lost/duplication during task transfering. +- The first clickhouse_sinker(per instance's ip+port) instance(named scheduler) is responsible to generate and publish task assignments regularly. The task list and assignment consist of the whole config. The task list change, service change, and task lag change will trigger another assignment. The scheduler ensures Each clickhouse_innker instance's total lag be balanced. +- Each clickhouse_sinker reloads the config regularly. This may start/stop tasks. clickhouse_sinker stops tasks gracefully so that there's no message lost/duplication during task transferring. From 9d3447703219e023d8fe948c0b4d936c3dd9e71e Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 31 Dec 2021 16:50:34 +0800 Subject: [PATCH 230/404] speedup initBmSeries --- output/clickhouse.go | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index d316060c..528c4262 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -194,15 +194,14 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { } func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { - var query string + tbl := c.seriesTbl if c.cfg.Clickhouse.Cluster != "" { - query = fmt.Sprintf("SELECT toInt64(__mgmt_id) FROM %s.%s", c.cfg.Clickhouse.DB, c.distSeriesTbls[0]) - } else { - query = fmt.Sprintf("SELECT toInt64(__mgmt_id) FROM %s.%s", c.cfg.Clickhouse.DB, c.seriesTbl) + tbl = c.distSeriesTbls[0] } + query := fmt.Sprintf("SELECT toUInt64(toInt64(__mgmt_id)) AS mid FROM %s.%s ORDER BY mid", c.cfg.Clickhouse.DB, tbl) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs *sql.Rows - var mgmtID int64 + var mgmtID uint64 if rs, err = conn.Query(query); err != nil { err = errors.Wrapf(err, "") return err @@ -214,7 +213,7 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { err = errors.Wrapf(err, "") return err } - c.bmSeries.Add(uint64(mgmtID)) + c.bmSeries.Add(mgmtID) } util.Logger.Info(fmt.Sprintf("loaded %d series from %v", c.bmSeries.GetCardinality(), c.seriesTbl), zap.String("task", c.taskCfg.Name)) return From 4aa298a542b4b8564dce0384c702b02dce19fbf4 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 23 Jan 2022 09:45:05 +0800 Subject: [PATCH 231/404] log pid at startup --- cmd/clickhouse_sinker/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 5d5031c7..f33c0a6f 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -131,7 +131,7 @@ func initCmdOptions() { } func getVersion() string { - return fmt.Sprintf("version %s, commit %s, date %s, builtBy %s", version, commit, date, builtBy) + return fmt.Sprintf("version %s, commit %s, date %s, builtBy %s, pid %v", version, commit, date, builtBy, os.Getpid()) } func init() { From 97263f4317a4b141df0e76e81e7d732850113756 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 24 Jan 2022 12:33:01 +0800 Subject: [PATCH 232/404] log on Service.drain --- output/clickhouse.go | 3 +++ task/task.go | 13 +++++++++---- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 528c4262..7697a8e9 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -84,6 +84,9 @@ func (c *ClickHouse) Init() (err error) { func (c *ClickHouse) Drain() { c.mux.Lock() for c.numFlying != 0 { + util.Logger.Debug("draining flying batches", + zap.String("task", c.taskCfg.Name), + zap.Int32("pending", c.numFlying)) c.taskDone.Wait() } c.mux.Unlock() diff --git a/task/task.go b/task/task.go index de9b0ae9..7d1abd54 100644 --- a/task/task.go +++ b/task/task.go @@ -337,9 +337,9 @@ func (service *Service) put(msg *model.InputMessage) { // drain ensure we have completeted procession(discard or write&commit) for all received messages, and cleared service state. func (service *Service) drain() { - savedState := atomic.LoadUint32(&service.state) - atomic.StoreUint32(&service.state, util.StateStopped) - defer atomic.StoreUint32(&service.state, savedState) + savedState := atomic.SwapUint32(&service.state, util.StateStopped) + defer atomic.CompareAndSwapUint32(&service.state, util.StateStopped, savedState) + begin := time.Now() service.Lock() for service.numFlying != 0 { service.taskDone.Wait() @@ -354,8 +354,13 @@ func (service *Service) drain() { if service.sharder != nil { service.sharder.ForceFlush(nil) } + util.Logger.Debug("generated flying batches", + zap.String("task", service.taskCfg.Name), + zap.Duration("cost", time.Since(begin))) service.clickhouse.Drain() - util.Logger.Debug("drained flying messages", zap.String("task", service.taskCfg.Name)) + util.Logger.Debug("drained flying batches", + zap.String("task", service.taskCfg.Name), + zap.Duration("cost", time.Since(begin))) } func (service *Service) Flush(batch *model.Batch) (err error) { From b2caa694bbf3a589e388a836aa1de0c3ae9b5cf2 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 24 Jan 2022 16:33:33 +0800 Subject: [PATCH 233/404] added env local-cfg-file --- cmd/clickhouse_sinker/main.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index f33c0a6f..88608e41 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -101,6 +101,7 @@ func initCmdOptions() { util.EnvIntVar(&cmdOps.HTTPPort, "http-port") util.EnvStringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs") util.EnvIntVar(&cmdOps.PushInterval, "push-interval") + util.EnvStringVar(&cmdOps.LocalCfgFile, "local-cfg-file") util.EnvStringVar(&cmdOps.NacosAddr, "nacos-addr") util.EnvStringVar(&cmdOps.NacosUsername, "nacos-username") From 16bcf3e88d79d9fd6a21f27fa8178aaae0396fea Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 3 Feb 2022 11:44:17 +0800 Subject: [PATCH 234/404] add doc on how to run as a service --- docs/guide/run.md | 58 +++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/docs/guide/run.md b/docs/guide/run.md index 1b440356..38e7ed21 100644 --- a/docs/guide/run.md +++ b/docs/guide/run.md @@ -91,3 +91,61 @@ Let's follow up a piece of the systest script. 3 rows in set. Elapsed: 0.016 sec. ``` + +## Run as a daemon + +On systemd managed Linux OSs such as RHEL, Debian and their variants, it's doable to run `clickhouse_sinker` as a system service to achieve auto-restart, coredump management etc. + +### Create `/etc/systemd/system/sinker_metric.service` + +``` +[Unit] +Description=ck-sink-metric +Requires=network-online.target +After=network-online.target + +[Service] +Type=simple +User=eoi +LimitCORE=infinity +Environment="GOTRACEBACK=crash" +ExecStart=/data02/app/sinker/sinker/clickhouse_sinker --local-cfg-file=/data02/app/sinker/sinker/ck-sink-metric.json --log-paths=/data02/app/sinker/sinker/logs/sinker_metric.log +Restart=on-failure +RestartSec=3s +StartLimitInterval=0 + +[Install] +WantedBy=multi-user.target +``` + +Note: + +- Change pathes in `ExecStart` as necessary. +- `User=eoi` means to run service as non-root for security reason. +- `LimitCORE=infinity` for service is equivalent to `ulimit -c unlimited` for non-service. +- env `GOTRACEBACK=crash` is required for Go applications to dump core. Refers to `https://pkg.go.dev/runtime`. + +### Modify `/etc/sysctl.conf` + +```kernel.core_pattern = |/usr/lib/systemd/systemd-coredump %p %u %g %s %t``` + +Run `sysctl -p`. + +### Modify `etc/systemd/coredump.conf` + +``` +[Coredump] +ProcessSizeMax=50G +ExternalSizeMax=50G +``` + +### Manage `clickhouse-sinker` service + +- To start, `systemctl start sinker_metric` +- To stop, `systemctl stop sinker_metric` +- To view status, `systemctl status sinker_metric` + +### Manage coredumps with `coredumpctl` + +Coredumps are stored under `/var/lib/systemd/coredump`. +Refers to core(5), systemd.exec(5), systemd-coredump(8), coredump.conf(5). From 3553c0faa38e222c9e77a575fca9bd88a64e504a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 22 Feb 2022 14:37:38 +0800 Subject: [PATCH 235/404] bump franz-go to v1.3.4 --- go.mod | 10 +++++----- go.sum | 21 +++++++++++---------- 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/go.mod b/go.mod index 01a763a6..38bd2008 100644 --- a/go.mod +++ b/go.mod @@ -20,7 +20,7 @@ require ( github.com/stretchr/testify v1.7.0 github.com/tidwall/gjson v1.10.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.2.4 + github.com/twmb/franz-go v1.3.4 github.com/twmb/franz-go/plugin/kzap v1.0.0 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.0.2 @@ -59,7 +59,7 @@ require ( github.com/modern-go/reflect2 v1.0.1 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect - github.com/pierrec/lz4/v4 v4.1.8 // indirect + github.com/pierrec/lz4/v4 v4.1.11 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/procfs v0.6.0 // indirect @@ -70,7 +70,7 @@ require ( github.com/tidwall/pretty v1.2.0 // indirect github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect - github.com/twmb/franz-go/pkg/kmsg v0.0.0-20211104051938-70808186d5f7 // indirect + github.com/twmb/franz-go/pkg/kmsg v0.0.0-20220222041924-02560c7b49ca // indirect github.com/twmb/go-rbtree v1.0.0 // indirect github.com/xdg-go/pbkdf2 v1.0.0 // indirect github.com/xdg-go/stringprep v1.0.2 // indirect @@ -78,8 +78,8 @@ require ( github.com/xdg/stringprep v1.0.3 // indirect go.uber.org/atomic v1.9.0 // indirect go.uber.org/multierr v1.7.0 // indirect - golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 // indirect - golang.org/x/net v0.0.0-20211020060615-d418f374d309 // indirect + golang.org/x/crypto v0.0.0-20211117183948-ae814b36b871 // indirect + golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9 // indirect golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 // indirect golang.org/x/text v0.3.7 // indirect google.golang.org/protobuf v1.26.0-rc.1 // indirect diff --git a/go.sum b/go.sum index 1063d9f7..72429647 100644 --- a/go.sum +++ b/go.sum @@ -183,8 +183,9 @@ github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= -github.com/pierrec/lz4/v4 v4.1.8 h1:ieHkV+i2BRzngO4Wd/3HGowuZStgq6QkPsD1eolNAO4= github.com/pierrec/lz4/v4 v4.1.8/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pierrec/lz4/v4 v4.1.11 h1:LVs17FAZJFOjgmJXl9Tf13WfLUvZq7/RjfEJrnwZ9OE= +github.com/pierrec/lz4/v4 v4.1.11/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -261,12 +262,12 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0 github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.0.0/go.mod h1:cdFLk8d/5/ox88y38xgiDKP3Yo338OO0t5QbTEM2K6I= -github.com/twmb/franz-go v1.2.4 h1:APqAImlBNTpv5ukAwOl/9x2kkAVKk3ocPxwlPXkYNC4= -github.com/twmb/franz-go v1.2.4/go.mod h1:e5ZOdNswX/wv+jebWNX49yc9U7zgR18Xovj9ckk6mx8= +github.com/twmb/franz-go v1.3.4 h1:8ON6aAqm1RrfAXbXPH39JD1RU9EH5kX5t6UtQPJhIp8= +github.com/twmb/franz-go v1.3.4/go.mod h1:sSAjFZNo2ux82hEhK45VobRGImo9zwlNK0yWqz2QYIk= github.com/twmb/franz-go/pkg/kmsg v0.0.0-20210901051457-3c197a133ddd/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v0.0.0-20210914042331-106aef61b693/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/kmsg v0.0.0-20211104051938-70808186d5f7 h1:YW4mW39H53O1qouKQnlrdNwyqAi5c4P10Oig8yndDKQ= -github.com/twmb/franz-go/pkg/kmsg v0.0.0-20211104051938-70808186d5f7/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go/pkg/kmsg v0.0.0-20220222041924-02560c7b49ca h1:TtTuEMxsLQ6xwtDzov7wCcP08Zsdua/9+mB2nEtNY5w= +github.com/twmb/franz-go/pkg/kmsg v0.0.0-20220222041924-02560c7b49ca/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/plugin/kzap v1.0.0 h1:vHI3IBXBTNspBhw6OXv+t28fqnij03FxOiqp0OUzmGc= github.com/twmb/franz-go/plugin/kzap v1.0.0/go.mod h1:hBnW68adqgx2m+Us874W03VmKjqgov/WI8wfYaSoZgI= github.com/twmb/go-rbtree v1.0.0 h1:KxN7dXJ8XaZ4cvmHV1qqXTshxX3EBvX/toG5+UR49Mg= @@ -311,8 +312,8 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210920023735-84f357641f63/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 h1:7I4JAnoQBe7ZtJcBaYHi5UtiO8tQHbUSXxL+pnGRANg= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20211117183948-ae814b36b871 h1:/pEO3GD/ABYAjuakUS6xSEmmlyVS4kxBNkeA9tLJiTI= +golang.org/x/crypto v0.0.0-20211117183948-ae814b36b871/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -328,10 +329,10 @@ golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210825183410-e898025ed96a/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210913180222-943fd674d43e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210917221730-978cfadd31cf/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211020060615-d418f374d309 h1:A0lJIi+hcTR6aajJH4YqKWwohY4aW9RO7oRMcdv+HKI= -golang.org/x/net v0.0.0-20211020060615-d418f374d309/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9 h1:0qxwC5n+ttVOINCBeRHO0nq9X7uy8SDsPoi5OaCdIEI= +golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= From c679e67ec08d7347d89861f310b54d6747bfb901 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 18 Apr 2022 20:52:32 +0800 Subject: [PATCH 236/404] build kafka_gen_log and kafka_gen_metric for arm64 --- cmd/kafka_gen_log/main.go | 3 +-- cmd/kafka_gen_metric/main.go | 3 +-- util/marshal_amd64.go | 14 ++++++++++++++ util/marshal_arm64.go | 14 ++++++++++++++ 4 files changed, 30 insertions(+), 4 deletions(-) create mode 100644 util/marshal_amd64.go create mode 100644 util/marshal_arm64.go diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index ad2dcd35..b165762d 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -53,7 +53,6 @@ import ( "time" "github.com/Shopify/sarama" - "github.com/bytedance/sonic" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" @@ -279,7 +278,7 @@ func (g *LogGenerator) Run() { Xforwardfor: "", } _ = wp.Submit(func() { - if b, err = sonic.Marshal(&logObj); err != nil { + if b, err = util.JsonMarshal(&logObj); err != nil { err = errors.Wrapf(err, "") util.Logger.Fatal("got error", zap.Error(err)) } diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index d7f19cf9..aa85e318 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -45,7 +45,6 @@ import ( "time" "github.com/Shopify/sarama" - "github.com/bytedance/sonic" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" @@ -156,7 +155,7 @@ func generate() { _ = wp.Submit(func() { var b []byte - if b, err = sonic.Marshal(&metric); err != nil { + if b, err = util.JsonMarshal(&metric); err != nil { err = errors.Wrapf(err, "") util.Logger.Fatal("got error", zap.Error(err)) } diff --git a/util/marshal_amd64.go b/util/marshal_amd64.go new file mode 100644 index 00000000..a3a40383 --- /dev/null +++ b/util/marshal_amd64.go @@ -0,0 +1,14 @@ +package util + +import ( + "github.com/bytedance/sonic" + + "github.com/pkg/errors" +) + +func JsonMarshal(obj interface{}) (b []byte, err error) { + if b, err = sonic.Marshal(obj); err != nil { + err = errors.Wrapf(err, "") + } + return +} diff --git a/util/marshal_arm64.go b/util/marshal_arm64.go new file mode 100644 index 00000000..c33d60e8 --- /dev/null +++ b/util/marshal_arm64.go @@ -0,0 +1,14 @@ +package util + +import ( + "encoding/json" + + "github.com/pkg/errors" +) + +func JsonMarshal(obj interface{}) (b []byte, err error) { + if b, err = json.Marshal(obj); err != nil { + err = errors.Wrapf(err, "") + } + return +} From 4e1d9f2edc8f63b8f4b38e7fe2e4f1178ee83c06 Mon Sep 17 00:00:00 2001 From: Travis Bischel Date: Sun, 1 May 2022 18:52:58 -0600 Subject: [PATCH 237/404] bump franz-go to v1.5.0, fix kerberos breakage See the breaking changes in v1.5 in https://github.com/twmb/franz-go/blob/master/CHANGELOG.md#v150. Basically, gokrb5's license is a bit broken (missed placing the name in the copyright section), so I've split kerberos at the dependency level. This is technically a breaking change, but is fixed solely at the mod level: ``` go get github.com/twmb/franz-go/pkg/sasl/kerberos@v1.0.0 go get github.com/twmb/franz-go@v1.5.0 ``` I think thie dep bump is good, but probably worth testing if you can. --- go.mod | 17 +++++++++-------- go.sum | 35 +++++++++++++++++++++-------------- 2 files changed, 30 insertions(+), 22 deletions(-) diff --git a/go.mod b/go.mod index 38bd2008..62ee15fd 100644 --- a/go.mod +++ b/go.mod @@ -20,7 +20,8 @@ require ( github.com/stretchr/testify v1.7.0 github.com/tidwall/gjson v1.10.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.3.4 + github.com/twmb/franz-go v1.5.0 + github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 github.com/twmb/franz-go/plugin/kzap v1.0.0 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.0.2 @@ -43,14 +44,14 @@ require ( github.com/go-errors/errors v1.0.1 // indirect github.com/golang/protobuf v1.4.3 // indirect github.com/golang/snappy v0.0.4 // indirect - github.com/hashicorp/go-uuid v1.0.2 // indirect + github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/jcmturner/aescts/v2 v2.0.0 // indirect github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect github.com/jcmturner/gofork v1.0.0 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af // indirect github.com/json-iterator/go v1.1.11 // indirect - github.com/klauspost/compress v1.13.6 // indirect + github.com/klauspost/compress v1.15.2 // indirect github.com/klauspost/cpuid/v2 v2.0.9 // indirect github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f // indirect github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 // indirect @@ -59,7 +60,7 @@ require ( github.com/modern-go/reflect2 v1.0.1 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect - github.com/pierrec/lz4/v4 v4.1.11 // indirect + github.com/pierrec/lz4/v4 v4.1.14 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/procfs v0.6.0 // indirect @@ -70,7 +71,7 @@ require ( github.com/tidwall/pretty v1.2.0 // indirect github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect - github.com/twmb/franz-go/pkg/kmsg v0.0.0-20220222041924-02560c7b49ca // indirect + github.com/twmb/franz-go/pkg/kmsg v1.0.0 // indirect github.com/twmb/go-rbtree v1.0.0 // indirect github.com/xdg-go/pbkdf2 v1.0.0 // indirect github.com/xdg-go/stringprep v1.0.2 // indirect @@ -78,9 +79,9 @@ require ( github.com/xdg/stringprep v1.0.3 // indirect go.uber.org/atomic v1.9.0 // indirect go.uber.org/multierr v1.7.0 // indirect - golang.org/x/crypto v0.0.0-20211117183948-ae814b36b871 // indirect - golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9 // indirect - golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 // indirect + golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f // indirect + golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4 // indirect + golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e // indirect golang.org/x/text v0.3.7 // indirect google.golang.org/protobuf v1.26.0-rc.1 // indirect gopkg.in/ini.v1 v1.42.0 // indirect diff --git a/go.sum b/go.sum index 72429647..238f5e69 100644 --- a/go.sum +++ b/go.sum @@ -94,8 +94,9 @@ github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMyw github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.7 h1:81/ik6ipDQS2aGcBfIN5dHDB36BwrStyeAQquSYCV4o= +github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gops v0.3.18 h1:my259V+172PVFmduS2RAsq4FKH+HjKqdh7pLr17Ot8c= github.com/google/gops v0.3.18/go.mod h1:Pfp8hWGIFdV/7rY9/O/U5WgdjYQXf/GiEK4NVuVd2ZE= @@ -105,8 +106,9 @@ github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORR github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= -github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= +github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= @@ -143,8 +145,9 @@ github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2F github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.13.5/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.13.6 h1:P76CopJELS0TiO2mebmnzgWaajssP/EszplttgQxcgc= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= +github.com/klauspost/compress v1.15.2 h1:3WH+AG7s2+T8o3nrM/8u2rdqUEcQhmga7smjrT41nAw= +github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/cpuid/v2 v2.0.9 h1:lgaqFMSdTdQYdZ04uHyN2d/eKdOMyi2YLSvlQIBFYa4= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -184,8 +187,8 @@ github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4/v4 v4.1.8/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= -github.com/pierrec/lz4/v4 v4.1.11 h1:LVs17FAZJFOjgmJXl9Tf13WfLUvZq7/RjfEJrnwZ9OE= -github.com/pierrec/lz4/v4 v4.1.11/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pierrec/lz4/v4 v4.1.14 h1:+fL8AQEZtz/ijeNnpduH0bROTu0O3NZAlPjQxGn8LwE= +github.com/pierrec/lz4/v4 v4.1.14/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -262,12 +265,14 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0 github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.0.0/go.mod h1:cdFLk8d/5/ox88y38xgiDKP3Yo338OO0t5QbTEM2K6I= -github.com/twmb/franz-go v1.3.4 h1:8ON6aAqm1RrfAXbXPH39JD1RU9EH5kX5t6UtQPJhIp8= -github.com/twmb/franz-go v1.3.4/go.mod h1:sSAjFZNo2ux82hEhK45VobRGImo9zwlNK0yWqz2QYIk= +github.com/twmb/franz-go v1.5.0 h1:DC2Ov0Z0+wqqrHYkTHukzm14x2mv059MD1YL6N9AYew= +github.com/twmb/franz-go v1.5.0/go.mod h1:ZKQ5AtqBbdc783bLCay7nDc21lJnIIA8mFJYhLMF19E= github.com/twmb/franz-go/pkg/kmsg v0.0.0-20210901051457-3c197a133ddd/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v0.0.0-20210914042331-106aef61b693/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/kmsg v0.0.0-20220222041924-02560c7b49ca h1:TtTuEMxsLQ6xwtDzov7wCcP08Zsdua/9+mB2nEtNY5w= -github.com/twmb/franz-go/pkg/kmsg v0.0.0-20220222041924-02560c7b49ca/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go/pkg/kmsg v1.0.0 h1:dQdaXLDUEb+XkSUqw2/GvMEGdw1o89X2fOiFjlhztyA= +github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 h1:Wfq7o2Eu8CaPJOqu/mqM1pu9b3HuO0aWfDGcdo1cVR8= +github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0/go.mod h1:S0nXtRBGwqtOfTGQL3BDliF3hSxyUm0dqy5gEfPsQT0= github.com/twmb/franz-go/plugin/kzap v1.0.0 h1:vHI3IBXBTNspBhw6OXv+t28fqnij03FxOiqp0OUzmGc= github.com/twmb/franz-go/plugin/kzap v1.0.0/go.mod h1:hBnW68adqgx2m+Us874W03VmKjqgov/WI8wfYaSoZgI= github.com/twmb/go-rbtree v1.0.0 h1:KxN7dXJ8XaZ4cvmHV1qqXTshxX3EBvX/toG5+UR49Mg= @@ -312,8 +317,8 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210920023735-84f357641f63/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20211117183948-ae814b36b871 h1:/pEO3GD/ABYAjuakUS6xSEmmlyVS4kxBNkeA9tLJiTI= -golang.org/x/crypto v0.0.0-20211117183948-ae814b36b871/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f h1:OeJjE6G4dgCY4PIXvIRQbE8+RX+uXZyGhUy/ksMGJoc= +golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -331,8 +336,8 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20210825183410-e898025ed96a/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210917221730-978cfadd31cf/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9 h1:0qxwC5n+ttVOINCBeRHO0nq9X7uy8SDsPoi5OaCdIEI= -golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4 h1:HVyaeDAYux4pnY+D/SiwmLOR36ewZ4iGQIIrtnuCjFA= +golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -358,9 +363,11 @@ golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 h1:SrN+KX8Art/Sf4HNj6Zcz06G7VEz+7w9tdXTPOZ7+l4= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e h1:fLOSk5Q00efkSvAm+4xcoXD+RRmLmmulPn5I3Y9F2EM= +golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= From df435b7718f765421b6e4aa25347747e29386937 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 3 May 2022 22:53:01 +0800 Subject: [PATCH 238/404] let lint happy --- .github/workflows/lint.yml | 4 ++-- Makefile | 2 +- cmd/kafka_gen_log/main.go | 23 +++++++++-------------- cmd/kafka_gen_metric/main.go | 2 +- util/marshal_amd64.go | 2 +- util/marshal_arm64.go | 2 +- 6 files changed, 15 insertions(+), 20 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 0be80a0e..68e470f8 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -14,13 +14,13 @@ jobs: uses: golangci/golangci-lint-action@v2 with: # Required: the version of golangci-lint is required and must be specified without patch version: we always use the latest patch version. - version: v1.43.0 + version: v1.45.2 # Optional: working directory, useful for monorepos # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen + args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen,containedctx # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true diff --git a/Makefile b/Makefile index 49c0e82f..3be8c4b9 100644 --- a/Makefile +++ b/Makefile @@ -25,6 +25,6 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen + golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen,containedctx run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.json diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index b165762d..6ae2d8c6 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -194,22 +194,17 @@ func (g *LogGenerator) next() (err error) { g.reader = nil } g.lineno = 0 - for i := 0; i < len(g.logfiles); i++ { - // a log file may disappear, retry another log file - g.off = (g.off + 1) % len(g.logfiles) - g.fp = g.logfiles[g.off] - var reader *os.File - if reader, err = os.Open(g.fp); err == nil { - g.reader = reader - g.scanner = bufio.NewScanner(g.reader) - util.Logger.Debug(fmt.Sprintf("scanning %+v", g.fp)) - return nil - } + // a log file may disappear, retry another log file + g.off = (g.off + 1) % len(g.logfiles) + g.fp = g.logfiles[g.off] + var reader *os.File + if reader, err = os.Open(g.fp); err != nil { err = errors.Wrapf(err, "") util.Logger.Fatal("os.Open failed", zap.String("path", g.fp), zap.Error(err)) - time.Sleep(6000 * time.Second) } - err = errors.Errorf("no readable file") + g.reader = reader + g.scanner = bufio.NewScanner(g.reader) + util.Logger.Debug(fmt.Sprintf("scanning %+v", g.fp)) return } @@ -278,7 +273,7 @@ func (g *LogGenerator) Run() { Xforwardfor: "", } _ = wp.Submit(func() { - if b, err = util.JsonMarshal(&logObj); err != nil { + if b, err = util.JSONMarshal(&logObj); err != nil { err = errors.Wrapf(err, "") util.Logger.Fatal("got error", zap.Error(err)) } diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index aa85e318..52442f28 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -155,7 +155,7 @@ func generate() { _ = wp.Submit(func() { var b []byte - if b, err = util.JsonMarshal(&metric); err != nil { + if b, err = util.JSONMarshal(&metric); err != nil { err = errors.Wrapf(err, "") util.Logger.Fatal("got error", zap.Error(err)) } diff --git a/util/marshal_amd64.go b/util/marshal_amd64.go index a3a40383..d197c0de 100644 --- a/util/marshal_amd64.go +++ b/util/marshal_amd64.go @@ -6,7 +6,7 @@ import ( "github.com/pkg/errors" ) -func JsonMarshal(obj interface{}) (b []byte, err error) { +func JSONMarshal(obj interface{}) (b []byte, err error) { if b, err = sonic.Marshal(obj); err != nil { err = errors.Wrapf(err, "") } diff --git a/util/marshal_arm64.go b/util/marshal_arm64.go index c33d60e8..fc226c4f 100644 --- a/util/marshal_arm64.go +++ b/util/marshal_arm64.go @@ -6,7 +6,7 @@ import ( "github.com/pkg/errors" ) -func JsonMarshal(obj interface{}) (b []byte, err error) { +func JSONMarshal(obj interface{}) (b []byte, err error) { if b, err = json.Marshal(obj); err != nil { err = errors.Wrapf(err, "") } From c806524ae91d04bc9d9e423df47f74dd49e40549 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 4 May 2022 10:31:42 +0800 Subject: [PATCH 239/404] improved lint.yml --- .github/workflows/lint.yml | 31 +++++++++++++++++++++++++------ 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 68e470f8..0fdc00a2 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -1,29 +1,48 @@ +# https://github.com/golangci/golangci-lint-action name: lint on: push: tags: [ v* ] - branches: [ master ] + branches: + - master + - main pull_request: - branches: [ master ] + branches: + - master + - main jobs: golangci-lint: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v2 + - uses: actions/setup-go@v3 + with: + go-version: 1.17 + - uses: actions/checkout@v3 - name: golangci-lint - uses: golangci/golangci-lint-action@v2 + uses: golangci/golangci-lint-action@v3 with: - # Required: the version of golangci-lint is required and must be specified without patch version: we always use the latest patch version. + # Optional: version of golangci-lint to use in form of v1.2 or v1.2.3 or `latest` to use the latest version version: v1.45.2 # Optional: working directory, useful for monorepos # working-directory: somedir # Optional: golangci-lint command line arguments. - args: --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen,containedctx + # args: --issues-exit-code=0 + args: --timeout=3m --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen,containedctx # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true + + # Optional: if set to true then the all caching functionality will be complete disabled, + # takes precedence over all other caching options. + # skip-cache: true + + # Optional: if set to true then the action don't cache or restore ~/go/pkg. + # skip-pkg-cache: true + + # Optional: if set to true then the action don't cache or restore ~/.cache/go-build. + # skip-build-cache: true gitleaks: runs-on: ubuntu-latest steps: From c0180eb2eac14b4b284659ad818d8e13b9ba8374 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 26 May 2022 16:14:16 +0800 Subject: [PATCH 240/404] escape labels --- model/message.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/model/message.go b/model/message.go index b19816b8..19f5832d 100644 --- a/model/message.go +++ b/model/message.go @@ -215,7 +215,7 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe } // "labels" JSON excludes "le", so that "labels" can be used as group key for histogram queries. if dim.Name != nameKey && dim.Name != "le" { - labels = append(labels, fmt.Sprintf(`"%s": "%s"`, dim.Name, labelVal)) + labels = append(labels, fmt.Sprintf(`%s: %s`, strconv.Quote(dim.Name), strconv.Quote(labelVal))) } } } From 7580a1d18a9d56a4ebd54d7c32933216697cd195 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 27 May 2022 16:13:09 +0800 Subject: [PATCH 241/404] introduced LoadSeriesAtStartup --- config/config.go | 2 ++ output/clickhouse.go | 8 ++++++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/config/config.go b/config/config.go index ae2e89bf..874261fc 100644 --- a/config/config.go +++ b/config/config.go @@ -137,6 +137,8 @@ type TaskConfig struct { } // PrometheusSchema expects each message is a Prometheus metric(timestamp, value, metric name and a list of labels). PrometheusSchema bool + // whether load series at startup + LoadSeriesAtStartup bool // ShardingKey is the column name to which sharding against ShardingKey string `json:"shardingKey,omitempty"` diff --git a/output/clickhouse.go b/output/clickhouse.go index 7697a8e9..cf9c7b34 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -201,6 +201,11 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { if c.cfg.Clickhouse.Cluster != "" { tbl = c.distSeriesTbls[0] } + c.bmSeries = roaring64.New() + if !c.taskCfg.LoadSeriesAtStartup { + util.Logger.Info(fmt.Sprintf("skipped loading series from %v", tbl), zap.String("task", c.taskCfg.Name)) + return + } query := fmt.Sprintf("SELECT toUInt64(toInt64(__mgmt_id)) AS mid FROM %s.%s ORDER BY mid", c.cfg.Clickhouse.DB, tbl) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs *sql.Rows @@ -210,7 +215,6 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { return err } defer rs.Close() - c.bmSeries = roaring64.New() for rs.Next() { if err = rs.Scan(&mgmtID); err != nil { err = errors.Wrapf(err, "") @@ -218,7 +222,7 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { } c.bmSeries.Add(mgmtID) } - util.Logger.Info(fmt.Sprintf("loaded %d series from %v", c.bmSeries.GetCardinality(), c.seriesTbl), zap.String("task", c.taskCfg.Name)) + util.Logger.Info(fmt.Sprintf("loaded %d series from %v", c.bmSeries.GetCardinality(), tbl), zap.String("task", c.taskCfg.Name)) return } From 053e8eeb2b7a49fdb672d81a4dab765736690175 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 30 May 2022 16:23:35 +0800 Subject: [PATCH 242/404] KafkaFranz.CommitMessages retry cl.CommitRecords --- input/kafka_franz.go | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index be23f50f..040918a2 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -41,6 +41,8 @@ import ( const ( Krb5KeytabAuth = 2 + CommitRetries = 6 + RetryBackoff = 5 * time.Second ) var _ Inputer = (*KafkaFranz)(nil) @@ -185,7 +187,19 @@ func (k *KafkaFranz) Run() { func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { // "LeaderEpoch: -1" will disable leader epoch validation - return k.cl.CommitRecords(context.Background(), &kgo.Record{Topic: msg.Topic, Partition: int32(msg.Partition), Offset: msg.Offset, LeaderEpoch: -1}) + var err error + for i := 0; i < CommitRetries; i++ { + err = k.cl.CommitRecords(context.Background(), &kgo.Record{Topic: msg.Topic, Partition: int32(msg.Partition), Offset: msg.Offset, LeaderEpoch: -1}) + if err == nil { + break + } + err = errors.Wrap(err, "") + if i < CommitRetries-1 && !errors.Is(err, context.Canceled) { + util.Logger.Error("cl.CommitRecords failed, will retry later", zap.String("task", k.taskCfg.Name), zap.Int("try", i), zap.Error(err)) + time.Sleep(RetryBackoff) + } + } + return err } // Stop kafka consumer and close all connections From eaba9da8f2207d601dcfa61613532eaa9a1a009b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 14 Jun 2022 15:41:45 +0800 Subject: [PATCH 243/404] bump thirdparty dependencies to latest releases --- config_manager/nacos.go | 5 +- go.mod | 89 ++++--- go.sum | 540 +++++++++++++++++++++++++++++++--------- input/kafka_franz.go | 15 +- 4 files changed, 481 insertions(+), 168 deletions(-) diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 362f3282..386e58a3 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -86,11 +86,12 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) NotLoadCacheAtStart: true, LogDir: filepath.Join(clientDir, "log"), CacheDir: filepath.Join(clientDir, "cache"), - RotateTime: "1h", - MaxAge: 3, LogLevel: "debug", Username: properties["username"].(string), Password: properties["password"].(string), + LogRollingConfig: &constant.ClientLogRollingConfig{ + MaxAge: 3, + }, } ncm.configClient, err = clients.CreateConfigClient(map[string]interface{}{ diff --git a/go.mod b/go.mod index 62ee15fd..14e409b6 100644 --- a/go.mod +++ b/go.mod @@ -3,87 +3,86 @@ module github.com/housepower/clickhouse_sinker go 1.17 require ( - github.com/ClickHouse/clickhouse-go v1.5.1 - github.com/RoaringBitmap/roaring v0.9.4 - github.com/Shopify/sarama v1.30.0 - github.com/bytedance/sonic v1.0.0-rc.2.0.20211130104717-49df11ef8309 + github.com/ClickHouse/clickhouse-go v1.5.4 + github.com/RoaringBitmap/roaring v1.2.0 + github.com/Shopify/sarama v1.34.1 + github.com/bytedance/sonic v1.3.1 github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 - github.com/google/gops v0.3.18 + github.com/google/gops v0.3.23 github.com/jcmturner/gokrb5/v8 v8.4.2 - github.com/jinzhu/copier v0.3.2 - github.com/nacos-group/nacos-sdk-go v1.0.7 + github.com/jinzhu/copier v0.3.5 + github.com/nacos-group/nacos-sdk-go v1.1.1 github.com/pkg/errors v0.9.1 - github.com/prometheus/client_golang v1.11.0 - github.com/prometheus/common v0.26.0 - github.com/segmentio/kafka-go v0.4.22 - github.com/stretchr/testify v1.7.0 - github.com/tidwall/gjson v1.10.1 + github.com/prometheus/client_golang v1.12.2 + github.com/prometheus/common v0.34.0 + github.com/segmentio/kafka-go v0.4.32 + github.com/stretchr/testify v1.7.1 + github.com/tidwall/gjson v1.14.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.5.0 + github.com/twmb/franz-go v1.6.0 github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 - github.com/twmb/franz-go/plugin/kzap v1.0.0 + github.com/twmb/franz-go/plugin/kzap v1.1.0 github.com/valyala/fastjson v1.6.3 - github.com/xdg-go/scram v1.0.2 - go.uber.org/zap v1.19.1 - golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba + github.com/xdg-go/scram v1.1.1 + go.uber.org/zap v1.21.0 + golang.org/x/time v0.0.0-20220609170525-579cf78fd858 gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) require ( - github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.61.1639 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/bits-and-blooms/bitset v1.2.0 // indirect - github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 // indirect - github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06 // indirect + github.com/bits-and-blooms/bitset v1.2.2 // indirect + github.com/buger/jsonparser v1.1.1 // indirect + github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a // indirect github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/eapache/go-resiliency v1.2.0 // indirect github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 // indirect github.com/eapache/queue v1.1.0 // indirect - github.com/go-errors/errors v1.0.1 // indirect - github.com/golang/protobuf v1.4.3 // indirect + github.com/go-errors/errors v1.4.2 // indirect + github.com/golang/protobuf v1.5.2 // indirect github.com/golang/snappy v0.0.4 // indirect + github.com/hashicorp/errwrap v1.1.0 // indirect + github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/jcmturner/aescts/v2 v2.0.0 // indirect github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect github.com/jcmturner/gofork v1.0.0 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af // indirect - github.com/json-iterator/go v1.1.11 // indirect - github.com/klauspost/compress v1.15.2 // indirect - github.com/klauspost/cpuid/v2 v2.0.9 // indirect - github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f // indirect - github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 // indirect + github.com/jmespath/go-jmespath v0.4.0 // indirect + github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/compress v1.15.6 // indirect + github.com/klauspost/cpuid/v2 v2.0.13 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect - github.com/modern-go/reflect2 v1.0.1 // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect github.com/pierrec/lz4/v4 v4.1.14 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect - github.com/prometheus/procfs v0.6.0 // indirect + github.com/prometheus/procfs v0.7.3 // indirect github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect - github.com/smartystreets/goconvey v1.6.4 // indirect - github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 // indirect + github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.0 // indirect - github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect - github.com/twmb/franz-go/pkg/kmsg v1.0.0 // indirect - github.com/twmb/go-rbtree v1.0.0 // indirect + github.com/twmb/franz-go/pkg/kmsg v1.1.0 // indirect github.com/xdg-go/pbkdf2 v1.0.0 // indirect - github.com/xdg-go/stringprep v1.0.2 // indirect - github.com/xdg/scram v1.0.3 // indirect + github.com/xdg-go/stringprep v1.0.3 // indirect + github.com/xdg/scram v1.0.5 // indirect github.com/xdg/stringprep v1.0.3 // indirect go.uber.org/atomic v1.9.0 // indirect - go.uber.org/multierr v1.7.0 // indirect - golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f // indirect - golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4 // indirect - golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e // indirect + go.uber.org/multierr v1.8.0 // indirect + golang.org/x/arch v0.0.0-20220412001346-fc48f9fe4c15 // indirect + golang.org/x/crypto v0.0.0-20220525230936-793ad666bf5e // indirect + golang.org/x/net v0.0.0-20220607020251-c690dde0001d // indirect + golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f // indirect + golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d // indirect golang.org/x/text v0.3.7 // indirect - google.golang.org/protobuf v1.26.0-rc.1 // indirect - gopkg.in/ini.v1 v1.42.0 // indirect - gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect + google.golang.org/protobuf v1.28.0 // indirect + gopkg.in/ini.v1 v1.66.6 // indirect + gopkg.in/yaml.v3 v3.0.0 // indirect ) diff --git a/go.sum b/go.sum index 238f5e69..2dce3ade 100644 --- a/go.sum +++ b/go.sum @@ -1,45 +1,87 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= +cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= +cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= +cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= +cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= +cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= +cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= +cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= +cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= +cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= +cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= +cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= +cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= +cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= +cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= +cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= +cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= +cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= +cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/clickhouse-go v1.5.1 h1:I8zVFZTz80crCs0FFEBJooIxsPcV0xfthzK1YrkpJTc= -github.com/ClickHouse/clickhouse-go v1.5.1/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= +github.com/ClickHouse/clickhouse-go v1.5.4/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/RoaringBitmap/roaring v0.9.4 h1:ckvZSX5gwCRaJYBNe7syNawCU5oruY9gQmjXlp4riwo= -github.com/RoaringBitmap/roaring v0.9.4/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= -github.com/Shopify/sarama v1.30.0 h1:TOZL6r37xJBDEMLx4yjB77jxbZYXPaDow08TSK6vIL0= -github.com/Shopify/sarama v1.30.0/go.mod h1:zujlQQx1kzHsh4jfV1USnptCQrHAEZ2Hk8fTKCulPVs= -github.com/Shopify/toxiproxy/v2 v2.1.6-0.20210914104332-15ea381dcdae h1:ePgznFqEG1v3AjMklnK8H7BSc++FDSo7xfK9K7Af+0Y= -github.com/Shopify/toxiproxy/v2 v2.1.6-0.20210914104332-15ea381dcdae/go.mod h1:/cvHQkZ1fst0EmZnA5dFtiQdWCNCFYzb+uE2vqVgvx0= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +github.com/RoaringBitmap/roaring v1.2.0 h1:qayex3YgtOmzev8slia4A0jPGsn2o2bnqKDcRpyRUiI= +github.com/RoaringBitmap/roaring v1.2.0/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= +github.com/Shopify/sarama v1.34.1 h1:pVCQO7BMAK3s1jWhgi5v1W6lwZ6Veiekfc2vsgRS06Y= +github.com/Shopify/sarama v1.34.1/go.mod h1:NZSNswsnStpq8TUdFaqnpXm2Do6KRzTIjdBdVlL1YRM= +github.com/Shopify/toxiproxy/v2 v2.4.0 h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64= +github.com/Shopify/toxiproxy/v2 v2.4.0/go.mod h1:3ilnjng821bkozDRxNoo64oI/DKqM+rOyJzb564+bvg= +github.com/StackExchange/wmi v1.2.1/go.mod h1:rcmrprowKIVzvc+NUiLncP2uuArMWLCbu9SBzvHz7e8= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= -github.com/aliyun/alibaba-cloud-sdk-go v1.61.18 h1:zOVTBdCKFd9JbCKz9/nt+FovbjPFmb7mUnp8nH9fQBA= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= +github.com/aliyun/alibaba-cloud-sdk-go v1.61.1639 h1:zP12a7XLwrPnTJd+BwR0yzK5TNSRvlzSp+KYtTnvgo4= +github.com/aliyun/alibaba-cloud-sdk-go v1.61.1639/go.mod h1:RcDobYh8k5VP6TNybz9m++gL3ijVI5wueVr0EM10VsU= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bits-and-blooms/bitset v1.2.0 h1:Kn4yilvwNtMACtf1eYDlG8H77R07mZSPbMjLyS07ChA= github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= +github.com/bits-and-blooms/bitset v1.2.2 h1:J5gbX05GpMdBjCvQ9MteIg2KKDExr7DrgK+Yc15FvIk= +github.com/bits-and-blooms/bitset v1.2.2/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAKk= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= -github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= -github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= -github.com/bytedance/sonic v1.0.0-rc.2.0.20211130104717-49df11ef8309 h1:n0R1DyA4S3BjKsjqMyAdUlX/UVC+91K5n6iV5V0CB94= -github.com/bytedance/sonic v1.0.0-rc.2.0.20211130104717-49df11ef8309/go.mod h1:lqEFq2niUZaKwt5URNYvcyY6r72F21rXzHgnzXS1Fw4= +github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= +github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= +github.com/bytedance/sonic v1.3.1 h1:pIYjcbBCV4M+tDCcSXnOL9/OGvfHo9bsWWmFNy/WPCI= +github.com/bytedance/sonic v1.3.1/go.mod h1:V973WhNhGmvHxW6nQmsHEfHaoU9F3zTF+93rH03hcUQ= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06 h1:1sDoSuDPWzhkdzNVxCxtIaKiAe96ESVPv8coGwc1gZ4= github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= +github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a h1:lmGPzuocwDxoPAMr9h16zoJY/USZR9jIh99nrmKk1uI= +github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -51,64 +93,110 @@ github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8 github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/fagongzi/goetty v1.7.0 h1:Z0uoEVqP4uQSQW+HR3bg5GGwmisZpJQ1sK/ab9HK7q0= github.com/fagongzi/goetty v1.7.0/go.mod h1:lLUyHhtjlOqatxVXgyLocwoI2o359JzLE7EWRGZiGw4= -github.com/fastly/go-utils v0.0.0-20180712184237-d95a45783239 h1:Ghm4eQYC0nEPnSJdVkTrXpu9KtoVCSo1hg7mtI7G9KU= -github.com/fastly/go-utils v0.0.0-20180712184237-d95a45783239/go.mod h1:Gdwt2ce0yfBxPvZrHkprdPPTTS3N5rwmLE8T22KBXlw= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= -github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= -github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= -github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= +github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= +github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= +github.com/go-kit/log v0.2.0/go.mod h1:NwTd00d/i8cPZ3xOwwiv2PO5MOcx78fFErGNcVmBjv0= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= +github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= +github.com/go-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-ole/go-ole v1.2.6-0.20210915003542-8b1f7f90f6b1/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/goccy/go-json v0.7.2 h1:MY1gMmtCxRpaI8YGpeHCvXUb+FVIo09pnjqF9Rhh274= -github.com/goccy/go-json v0.7.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.9.4 h1:L8MLKG2mvVXiQu07qB6hmfqeSYQdOnqPot2GhsIwIaI= +github.com/goccy/go-json v0.9.4/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= -github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= +github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.4 h1:l75CXGRSwbaYNpl/Z2X1XIIAMSCquvXgpVZDhwEIJsc= +github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.4.3 h1:JjCZWpVbqXDqFVmTfYWEVTMIYrL/NPdPSCHPJ0T/raM= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.7 h1:81/ik6ipDQS2aGcBfIN5dHDB36BwrStyeAQquSYCV4o= github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/gops v0.3.18 h1:my259V+172PVFmduS2RAsq4FKH+HjKqdh7pLr17Ot8c= -github.com/google/gops v0.3.18/go.mod h1:Pfp8hWGIFdV/7rY9/O/U5WgdjYQXf/GiEK4NVuVd2ZE= +github.com/google/gops v0.3.23 h1:OjsHRINl5FiIyTc8jivIg4UN0GY6Nh32SL8KRbl8GQo= +github.com/google/gops v0.3.23/go.mod h1:7diIdLsqpCihPSX3fQagksT/Ku/y4RL9LHTlKyEUDl8= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= +github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= +github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= @@ -121,51 +209,48 @@ github.com/jcmturner/gokrb5/v8 v8.4.2 h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJz github.com/jcmturner/gokrb5/v8 v8.4.2/go.mod h1:sb+Xq/fTY5yktf/VxLsE3wlfPqQjp0aWNYyvBVK62bc= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869 h1:IPJ3dvxmJ4uczJe5YQdrYB16oTJlGSC/OyZDqUk9xX4= -github.com/jehiah/go-strftime v0.0.0-20171201141054-1d33003b3869/go.mod h1:cJ6Cj7dQo+O6GJNiMx+Pa94qKj+TG8ONdKHgMNIyyag= -github.com/jinzhu/copier v0.3.2 h1:QdBOCbaouLDYaIPFfi1bKv5F5tPpeTwXe4sD0jqtz5w= -github.com/jinzhu/copier v0.3.2/go.mod h1:24xnZezI2Yqac9J61UC6/dG/k76ttpq0DdJI3QmUvro= -github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= +github.com/jinzhu/copier v0.3.5 h1:GlvfUwHk62RokgqVNvYsku0TATCF7bAHVwEXoBh3iJg= +github.com/jinzhu/copier v0.3.5/go.mod h1:DfbEm0FYsaqBcKcFuvmOZb218JkPGtvSHsKg8S8hyyg= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= +github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= +github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= +github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= +github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= -github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= -github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.11 h1:uVUAXhF2To8cbw/3xN3pxj6kk7TYKs98NIrTqPlMWAQ= github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= +github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.13.5/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.15.2 h1:3WH+AG7s2+T8o3nrM/8u2rdqUEcQhmga7smjrT41nAw= +github.com/klauspost/compress v1.14.2/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/cpuid/v2 v2.0.9 h1:lgaqFMSdTdQYdZ04uHyN2d/eKdOMyi2YLSvlQIBFYa4= +github.com/klauspost/compress v1.15.4/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= +github.com/klauspost/compress v1.15.6 h1:6D9PcO8QWu0JyaQ2zUMmu16T1T+zjjEpP91guRsvDfY= +github.com/klauspost/compress v1.15.6/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= +github.com/klauspost/cpuid/v2 v2.0.13 h1:1XxvOiqXZ8SULZUKim/wncr3wZ38H4yCuVDvKdK9OGs= +github.com/klauspost/cpuid/v2 v2.0.13/go.mod h1:g2LTdtYhdyuGPqyWyv7qRAmj1WBqxuObKfj5c0PQa7c= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= +github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/lestrrat/go-envload v0.0.0-20180220120943-6ed08b54a570 h1:0iQektZGS248WXmGIYOwRXSQhD4qn3icjMpuxwO7qlo= -github.com/lestrrat/go-envload v0.0.0-20180220120943-6ed08b54a570/go.mod h1:BLt8L9ld7wVsvEWQbuLrUZnCMnUmLZ+CGDzKtclrTlE= -github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f h1:sgUSP4zdTUZYZgAGGtN5Lxk92rK+JUFOwf+FT99EEI4= -github.com/lestrrat/go-file-rotatelogs v0.0.0-20180223000712-d3151e2a480f/go.mod h1:UGmTpUd3rjbtfIpwAPrcfmGf/Z1HS95TATB+m57TPB8= -github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042 h1:Bvq8AziQ5jFF4BHGAEDSqwPW1NJS3XshxbRCxtjFAZc= -github.com/lestrrat/go-strftime v0.0.0-20180220042222-ba3bf9c1d042/go.mod h1:TPpsiPUEh0zFL1Snz4crhMlBe60PYxRHr5oFF3rRYg0= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= @@ -174,19 +259,18 @@ github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/mschoch/smat v0.2.0 h1:8imxQsjDm8yFEAVBe7azKmKSgzSkZXDuKkSq9374khM= github.com/mschoch/smat v0.2.0/go.mod h1:kc9mz7DoBKqDyiRL7VZN8KvXQMWeTaVnttLRXOlotKw= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nacos-group/nacos-sdk-go v1.0.7 h1:Am1tJFe7GUTNCREKsZ5ok0H2OspHDRmRcsxn7DiSwhA= -github.com/nacos-group/nacos-sdk-go v1.0.7/go.mod h1:hlAPn3UdzlxIlSILAyOXKxjFSvDJ9oLzTJ9hLAK1KzA= +github.com/nacos-group/nacos-sdk-go v1.1.1 h1:beczWcOoTaVBMgCgikqvZflrN5Xbw7pWAWpxl+VJGIA= +github.com/nacos-group/nacos-sdk-go v1.1.1/go.mod h1:UHOtQNQY/qpk2dhg6gDq8u5+/CEIc3+lWmrmxEzX0/g= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= -github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= -github.com/pierrec/lz4/v4 v4.1.8/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.14 h1:+fL8AQEZtz/ijeNnpduH0bROTu0O3NZAlPjQxGn8LwE= github.com/pierrec/lz4/v4 v4.1.14/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -199,43 +283,49 @@ github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXP github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.11.0 h1:HNkLOAEQMIDv/K+04rukrLx6ch7msSRwf3/SASFAGtQ= github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= +github.com/prometheus/client_golang v1.12.1/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= +github.com/prometheus/client_golang v1.12.2 h1:51L9cDoUHVrXx4zWYlcLQIZ+d+VXHgqnYKkIuq4g/34= +github.com/prometheus/client_golang v1.12.2/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= +github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= +github.com/prometheus/common v0.34.0 h1:RBmGO9d/FVjqHT0yUGQwBJhkwKV+wPCn7KGpvfab0uE= +github.com/prometheus/common v0.34.0/go.mod h1:gB3sOl7P0TvJabZpLY5uQMpUqRCPPCyRLCZYc7JZTNE= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/prometheus/procfs v0.7.3 h1:4jVXhlkAyzOScmCkXBTOLRLTz8EeU+eyjrwB/EPq0VU= +github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= +github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/segmentio/kafka-go v0.4.22 h1:F4k2OTm9Y4+zliuoXgNKJZTktE0miQioZZzofsjhRdk= -github.com/segmentio/kafka-go v0.4.22/go.mod h1:XzMcoMjSzDGHcIwpWUI7GB43iKZ2fTVmryPSGLf/MPg= -github.com/shirou/gopsutil/v3 v3.21.2/go.mod h1:ghfMypLDrFSWN2c9cDYFLHyynQ+QUht0cv/18ZqVczw= +github.com/segmentio/kafka-go v0.4.32 h1:Ohr+9E+kDv/Ld2UPJN9hnKZRd2qgiqCmI8v2e1qlfLM= +github.com/segmentio/kafka-go v0.4.32/go.mod h1:JAPPIiY3MQIwVHj64CWOP0LsFFfQ7H0w69kuoxnMIS0= +github.com/shirou/gopsutil/v3 v3.21.9/go.mod h1:YWp/H8Qs5fVmf17v7JNZzA0mPJ+mS2e9JdiUF9LlKzQ= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71 h1:2MR0pKUzlP3SGgj5NYJe/zRYDwOu9ku6YHy+Iw7l5DM= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= +github.com/streadway/amqp v1.0.0 h1:kuuDrUJFZL1QYL9hUNuCxNObNzB0bV/ZG5jV3RWAQgo= +github.com/streadway/amqp v1.0.0/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= @@ -243,163 +333,368 @@ github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UV github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/tebeka/strftime v0.1.3 h1:5HQXOqWKYRFfNyBMNVc9z5+QzuBtIXy03psIhtdJYto= -github.com/tebeka/strftime v0.1.3/go.mod h1:7wJm3dZlpr4l/oVK0t1HYIc4rMzQ2XJlOMIUJUJH6XQ= -github.com/tidwall/gjson v1.8.0/go.mod h1:5/xDoumyyDNerp2U36lyolv46b3uF/9Bu6OfyQ9GImk= -github.com/tidwall/gjson v1.10.1 h1:Midn39zaqkz9SPTBDuEe0RdW9T4T+MpBOAC10e9/qFE= -github.com/tidwall/gjson v1.10.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= -github.com/tidwall/match v1.0.3/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= +github.com/stretchr/testify v1.7.1 h1:5TQK59W5E3v0r2duFAb7P95B6hEeOyEnHRa8MjYSMTY= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/tidwall/gjson v1.12.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= +github.com/tidwall/gjson v1.13.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= +github.com/tidwall/gjson v1.14.1 h1:iymTbGkQBhveq21bEvAQ81I0LEBork8BFe1CUZXdyuo= +github.com/tidwall/gjson v1.14.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= -github.com/tidwall/pretty v1.1.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= -github.com/tklauser/go-sysconf v0.3.4/go.mod h1:Cl2c8ZRWfHD5IrfHo9VN+FX9kCFjIOyVklgXycLB6ek= -github.com/tklauser/numcpus v0.2.1/go.mod h1:9aU+wOc6WjUIZEwWMP62PL/41d65P+iks1gBkr4QyP8= -github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3 h1:kF/7m/ZU+0D4Jj5eZ41Zm3IH/J8OElK1Qtd7tVKAwLk= -github.com/toolkits/concurrent v0.0.0-20150624120057-a4371d70e3e3/go.mod h1:QDlpd3qS71vYtakd2hmdpqhJ9nwv6mD6A30bQ1BPBFE= +github.com/tidwall/sjson v1.2.4 h1:cuiLzLnaMeBhRmEv00Lpk3tkYrcxpmbU81tAY4Dw0tc= +github.com/tidwall/sjson v1.2.4/go.mod h1:098SZ494YoMWPmMO6ct4dcFnqxwj9r/gF0Etp19pSNM= +github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= +github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= -github.com/twmb/franz-go v1.0.0/go.mod h1:cdFLk8d/5/ox88y38xgiDKP3Yo338OO0t5QbTEM2K6I= -github.com/twmb/franz-go v1.5.0 h1:DC2Ov0Z0+wqqrHYkTHukzm14x2mv059MD1YL6N9AYew= github.com/twmb/franz-go v1.5.0/go.mod h1:ZKQ5AtqBbdc783bLCay7nDc21lJnIIA8mFJYhLMF19E= -github.com/twmb/franz-go/pkg/kmsg v0.0.0-20210901051457-3c197a133ddd/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/kmsg v0.0.0-20210914042331-106aef61b693/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/kmsg v1.0.0 h1:dQdaXLDUEb+XkSUqw2/GvMEGdw1o89X2fOiFjlhztyA= +github.com/twmb/franz-go v1.5.3/go.mod h1:eqHYpAuvlTArOdZ1XtPYyOQ1uUb40CSZwbpL3ccjibI= +github.com/twmb/franz-go v1.6.0 h1:yri7YsVBe/k1LKcoZSLILgUI3U14e82qtD9i4VOcs9c= +github.com/twmb/franz-go v1.6.0/go.mod h1:xdMwpUIQL/JDKKwerc5qJQG8TU1SNIddfjKJJyqRJIg= github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go/pkg/kmsg v1.1.0 h1:csckTxG48q7Tem7ZwMxe2jAb0ehDNglxZccGnpqe4RU= +github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 h1:Wfq7o2Eu8CaPJOqu/mqM1pu9b3HuO0aWfDGcdo1cVR8= github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0/go.mod h1:S0nXtRBGwqtOfTGQL3BDliF3hSxyUm0dqy5gEfPsQT0= -github.com/twmb/franz-go/plugin/kzap v1.0.0 h1:vHI3IBXBTNspBhw6OXv+t28fqnij03FxOiqp0OUzmGc= -github.com/twmb/franz-go/plugin/kzap v1.0.0/go.mod h1:hBnW68adqgx2m+Us874W03VmKjqgov/WI8wfYaSoZgI= -github.com/twmb/go-rbtree v1.0.0 h1:KxN7dXJ8XaZ4cvmHV1qqXTshxX3EBvX/toG5+UR49Mg= +github.com/twmb/franz-go/plugin/kzap v1.1.0 h1:HaHgfcXYQi9QYRJ5ORdD/qwDjQKV85ZOeeKPwd46nyQ= +github.com/twmb/franz-go/plugin/kzap v1.1.0/go.mod h1:TUlWYqucIyz6U7xLo++gkHBDiJmed5FpYe42rCL2YG0= github.com/twmb/go-rbtree v1.0.0/go.mod h1:UlIAI8gu3KRPkXSobZnmJfVwCJgEhD/liWzT5ppzIyc= github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= github.com/valyala/fastjson v1.6.3/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= -github.com/xdg-go/scram v1.0.2 h1:akYIkZ28e6A96dkWNJQu3nmCzH3YfwMPQExUYDaRv7w= -github.com/xdg-go/scram v1.0.2/go.mod h1:1WAq6h33pAW+iRreB34OORO2Nf7qel3VV3fjBj+hCSs= -github.com/xdg-go/stringprep v1.0.2 h1:6iq84/ryjjeRmMJwxutI51F2GIPlP5BfTvXHeYjyhBc= -github.com/xdg-go/stringprep v1.0.2/go.mod h1:8F9zXuvzgwmyT5DUm4GUfZGDdT3W+LCvS6+da4O5kxM= +github.com/xdg-go/scram v1.1.1 h1:VOMT+81stJgXW3CpHyqHN3AXDYIMsx56mEFrB37Mb/E= +github.com/xdg-go/scram v1.1.1/go.mod h1:RaEWvsqvNKKvBPvcKeFjrG2cJqOkHTiyTpzz23ni57g= +github.com/xdg-go/stringprep v1.0.3 h1:kdwGpVNwPFtjs98xCGkHjQtGKh86rDcRZN17QEMCOIs= +github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgklLGvcBnW8= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= -github.com/xdg/scram v1.0.3 h1:nTadYh2Fs4BK2xdldEa2g5bbaZp0/+1nJMMPtPxS/to= -github.com/xdg/scram v1.0.3/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= +github.com/xdg/scram v1.0.5 h1:TuS0RFmt5Is5qm9Tm2SoD89OPqe4IRiFtyFY4iwWXsw= +github.com/xdg/scram v1.0.5/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= github.com/xdg/stringprep v1.0.3 h1:cmL5Enob4W83ti/ZHuZLuKD/xqJfus4fVPwE+/BDm+4= github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= -github.com/xlab/treeprint v1.0.0/go.mod h1:IoImgRak9i3zJyuxOKUP1v4UZd1tMoKkq/Cimt1uhCg= +github.com/xlab/treeprint v1.1.0/go.mod h1:gj5Gd3gPdKtR1ikdDK6fnFLdmIS0X30kTTuNd/WEJu0= +github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.11-0.20210813005559-691160354723 h1:sHOAIxRGBp443oHZIPB+HsUGaksVCXVQENPxwTfQdH4= -go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= +go.uber.org/goleak v1.1.11 h1:wy28qYRKZgnJTxGxvye5/wgWr1EKjmUDGYox5mGlRlI= +go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/multierr v1.7.0 h1:zaiO/rmgFjbmCXdSYJWQcdvOCsthmdaHfr3Gm2Kx4Ec= -go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= +go.uber.org/multierr v1.8.0 h1:dg6GjLku4EH+249NNmoIciG9N/jURbDG+pFlTkhzIC8= +go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.19.1 h1:ue41HOKd1vGURxrmeKIgELGb3jPW9DMUDGtsinblHwI= -go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= +go.uber.org/zap v1.21.0 h1:WefMeulhovoZ2sYXz7st6K0sLj7bBhpiFaud4r4zST8= +go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= +golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.0.0-20220412001346-fc48f9fe4c15 h1:GVfVkciLYxn5mY5EncwAe0SXUn9Rm81rRkZ0TTmn/cU= +golang.org/x/arch v0.0.0-20220412001346-fc48f9fe4c15/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190506204251-e1dfcc566284/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20210920023735-84f357641f63/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f h1:OeJjE6G4dgCY4PIXvIRQbE8+RX+uXZyGhUy/ksMGJoc= +golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.0.0-20220518034528-6f7dac969898/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.0.0-20220525230936-793ad666bf5e h1:T8NU3HyQ8ClP4SEE+KbFlg6n0NhuTsN4MyznaarGsZM= +golang.org/x/crypto v0.0.0-20220525230936-793ad666bf5e/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210825183410-e898025ed96a/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210917221730-978cfadd31cf/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4 h1:HVyaeDAYux4pnY+D/SiwmLOR36ewZ4iGQIIrtnuCjFA= +golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220520000938-2e3eb7b945c2/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220607020251-c690dde0001d h1:4SFsTMi4UahlKoloni7L4eYzhFRifURQLw+yv0QDCx8= +golang.org/x/net v0.0.0-20220607020251-c690dde0001d/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f h1:Ax0t5p6N38Ga0dThY21weqDEyz2oklo4IvDkpigvkD8= +golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200501052902-10377860bb8e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e h1:fLOSk5Q00efkSvAm+4xcoXD+RRmLmmulPn5I3Y9F2EM= +golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210902050250-f475640dd07b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d h1:Zu/JngovGLVi6t2J3nmAf3AoTDwuzw85YZ3b9o4yU7s= +golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba h1:O8mE0/t419eoIwhTFpKVkHiTs/Igowgfkj25AcZrtiE= -golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20220609170525-579cf78fd858 h1:Dpdu/EMxGMFgq0CeYMh4fazTD2vtlZRYE7wyynxJb9U= +golang.org/x/time v0.0.0-20220609170525-579cf78fd858/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= +google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= +google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= +google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= +google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= +google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.26.0-rc.1 h1:7QnIQpGRHE5RnLKnESfDoxm2dTapTZua5a0kS0A+VXQ= +google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= +google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.28.0 h1:w43yiav+6bVFTBQFZX0r7ipe9JQ1QsbMgHwbBziscLw= +google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -407,8 +702,10 @@ gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/ini.v1 v1.42.0 h1:7N3gPTt50s8GuLortA00n8AqRTk75qOP98+mTPpgzRk= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/ini.v1 v1.66.2/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/ini.v1 v1.66.6 h1:LATuAqN/shcYAOkv3wl2L4rkaKqkcgTBQjOyYDvcPKI= +gopkg.in/ini.v1 v1.66.6/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= @@ -418,10 +715,23 @@ gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20220512140231-539c8e751b99/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0 h1:hjy8E9ON/egN1tAYqKb61G10WtihqetD4sz2H+8nIeA= +gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/goversion v1.2.0/go.mod h1:Eih9y/uIBS3ulggl7KNJ09xGSLcuNaLgmvvqa07sgfo= +rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= +rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 040918a2..820a41d3 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -160,14 +160,17 @@ func (k *KafkaFranz) Run() { if fetches == nil || fetches.IsClientClosed() { break } - var hasError bool + var beCanceled bool fetches.EachError(func(_ string, _ int32, err error) { - err = errors.Wrap(err, "") - util.Logger.Error("kgo.Client.PollFetchs() failed", zap.Error(err)) - hasError = true + if errors.Is(err, context.Canceled) { + beCanceled = true + } else { + err = errors.Wrap(err, "") + util.Logger.Info("kgo.Client.PollFetchs() failed", zap.Error(err)) + } }) - if hasError { - continue + if beCanceled { + break } fetches.EachRecord(func(rec *kgo.Record) { msg := &model.InputMessage{ From 5bd5fb6befbd0aadbf5080b57f0e2d814c2ecb06 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 22 Jun 2022 22:10:22 +0800 Subject: [PATCH 244/404] introduced PromLabelsBlackList --- config/config.go | 13 +++++++++++++ model/message.go | 5 +++-- task/task.go | 6 +++++- 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/config/config.go b/config/config.go index 874261fc..e6f5ef6f 100644 --- a/config/config.go +++ b/config/config.go @@ -137,6 +137,8 @@ type TaskConfig struct { } // PrometheusSchema expects each message is a Prometheus metric(timestamp, value, metric name and a list of labels). PrometheusSchema bool + // fields match PromLabelsBlackList are not considered as labels. Requires PrometheusSchema be true. + PromLabelsBlackList string // the regexp of black list // whether load series at startup LoadSeriesAtStartup bool @@ -282,6 +284,8 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { if taskCfg.PrometheusSchema { taskCfg.DynamicSchema.Enable = true taskCfg.AutoSchema = true + } else { + taskCfg.PromLabelsBlackList = "" } if taskCfg.DynamicSchema.Enable { if taskCfg.Parser != "fastjson" && taskCfg.Parser != "gjson" { @@ -298,6 +302,9 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { return } } + } else { + taskCfg.DynamicSchema.WhiteList = "" + taskCfg.DynamicSchema.BlackList = "" } if taskCfg.DynamicSchema.WhiteList != "" { if _, err = regexp.Compile(taskCfg.DynamicSchema.WhiteList); err != nil { @@ -311,6 +318,12 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { return } } + if taskCfg.PromLabelsBlackList != "" { + if _, err = regexp.Compile(taskCfg.PromLabelsBlackList); err != nil { + err = errors.Wrapf(err, "PromLabelsBlackList %s is invalid regexp", taskCfg.PromLabelsBlackList) + return + } + } return } diff --git a/model/message.go b/model/message.go index 19f5832d..4d7354dd 100644 --- a/model/message.go +++ b/model/message.go @@ -3,6 +3,7 @@ package model import ( "container/list" "fmt" + "regexp" "strconv" "strings" "sync" @@ -168,7 +169,7 @@ func PutRow(r *Row) { rowPool.Put(r) } -func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSeriesID int, nameKey string) (row *Row) { +func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSeriesID int, nameKey string, lblBlkList *regexp.Regexp) (row *Row) { row = GetRow() var dig *xxhash.Digest var labels []string @@ -214,7 +215,7 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe _, _ = dig.WriteString(labelVal) } // "labels" JSON excludes "le", so that "labels" can be used as group key for histogram queries. - if dim.Name != nameKey && dim.Name != "le" { + if dim.Name != nameKey && dim.Name != "le" && (lblBlkList == nil || !lblBlkList.MatchString(dim.Name)) { labels = append(labels, fmt.Sprintf(`%s: %s`, strconv.Quote(dim.Name), strconv.Quote(labelVal))) } } diff --git a/task/task.go b/task/task.go index 7d1abd54..907588b7 100644 --- a/task/task.go +++ b/task/task.go @@ -46,6 +46,7 @@ type Service struct { taskCfg *config.TaskConfig whiteList *regexp.Regexp blackList *regexp.Regexp + lblBlkList *regexp.Regexp dims []*model.ColumnWithType idxSerID int @@ -86,6 +87,9 @@ func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) (service *Se if taskCfg.DynamicSchema.BlackList != "" { service.blackList = regexp.MustCompile(taskCfg.DynamicSchema.BlackList) } + if taskCfg.PromLabelsBlackList != "" { + service.lblBlkList = regexp.MustCompile(taskCfg.PromLabelsBlackList) + } return } @@ -291,7 +295,7 @@ func (service *Service) put(msg *model.InputMessage) { msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - row = model.MetricToRow(metric, msg, service.dims, service.idxSerID, service.nameKey) + row = model.MetricToRow(metric, msg, service.dims, service.idxSerID, service.nameKey, service.lblBlkList) if taskCfg.DynamicSchema.Enable { foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys, service.whiteList, service.blackList) } From ccf3b5bbda7bed4fe528a99f12a4753d93c71f2c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 23 Jun 2022 10:10:54 +0800 Subject: [PATCH 245/404] removed non-OSS gitleaks --- .github/workflows/lint.yml | 8 -------- 1 file changed, 8 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 0fdc00a2..d316501a 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -43,11 +43,3 @@ jobs: # Optional: if set to true then the action don't cache or restore ~/.cache/go-build. # skip-build-cache: true - gitleaks: - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v2 - with: - fetch-depth: '0' - - name: gitleaks-action - uses: zricethezav/gitleaks-action@master From d155e30ded861504ab00d4300da0e85667260db7 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 28 Jun 2022 17:32:45 +0800 Subject: [PATCH 246/404] treat KEEPER_EXCEPTION as retriable --- output/clickhouse.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index cf9c7b34..29f58f1b 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -45,7 +45,8 @@ var ( // https://github.com/ClickHouse/ClickHouse/issues/24036 // src/Common/ErrorCodes.cpp // src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp - replicaSpecificErrorCodes = []int32{242, 319, 1000} //TABLE_IS_READ_ONLY, UNKNOWN_STATUS_OF_INSERT, POCO_EXCEPTION + // ZooKeeper issues(https://issues.apache.org/jira/browse/ZOOKEEPER-4410) can cause ClickHouse exeception: "Code": 999, "Message": "Cannot allocate block number..." + replicaSpecificErrorCodes = []int32{242, 319, 999, 1000} //TABLE_IS_READ_ONLY, UNKNOWN_STATUS_OF_INSERT, KEEPER_EXCEPTION, POCO_EXCEPTION ) // ClickHouse is an output service consumers from kafka messages From 14a56207a7405673beb9a5b237473c83ac7cdced Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 30 Jun 2022 15:23:23 +0800 Subject: [PATCH 247/404] adjusted loglevel of batch creation --- task/ring.go | 2 +- task/sharding.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/task/ring.go b/task/ring.go index b4ace8e9..d643967d 100644 --- a/task/ring.go +++ b/task/ring.go @@ -205,7 +205,7 @@ func (ring *Ring) genBatchOrShard() { batch.RealSize = len(*batch.Rows) if batch.RealSize > 0 { - util.Logger.Debug(fmt.Sprintf("going to flush a batch for topic %v patittion %d, offset [%d,%d), messages %d, parse errors: %d", + util.Logger.Info(fmt.Sprintf("created a batch for topic %v patittion %d, offset [%d,%d), messages %d, parse errors: %d", taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, batch.RealSize, parseErrs), zap.String("task", taskCfg.Name)) diff --git a/task/sharding.go b/task/sharding.go index bcfa1922..0e3f1f6f 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -204,7 +204,7 @@ func (sh *Sharder) doFlush(_ interface{}) { } } if msgCnt > 0 { - util.Logger.Debug(fmt.Sprintf("going to flush batch group for topic %v, offsets %+v, messages %d", taskCfg.Topic, sh.offsets, msgCnt), zap.String("task", taskCfg.Name)) + util.Logger.Info(fmt.Sprintf("created a batch group for topic %v, offsets %+v, messages %d", taskCfg.Topic, sh.offsets, msgCnt), zap.String("task", taskCfg.Name)) sh.batchSys.CreateBatchGroupMulti(batches, sh.offsets) sh.offsets = make(map[int]int64) // ALL batches in a group shall be populated before sending any one to next stage. From dc11462e0603f0c85630a6a103fc9bfeab8350ec Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 30 Jun 2022 16:16:13 +0800 Subject: [PATCH 248/404] bump roaring, clickhouse-server and go --- .github/workflows/lint.yml | 2 +- .github/workflows/tests.yml | 2 +- docker-compose.yml | 2 +- go.mod | 10 +++++----- go.sum | 15 +++++++++------ 5 files changed, 17 insertions(+), 14 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index d316501a..6ce7bd37 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -16,7 +16,7 @@ jobs: steps: - uses: actions/setup-go@v3 with: - go-version: 1.17 + go-version: 1.18 - uses: actions/checkout@v3 - name: golangci-lint uses: golangci/golangci-lint-action@v3 diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index efb9e3f0..e5608767 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -16,7 +16,7 @@ jobs: - name: Set up Go 1.x uses: actions/setup-go@v2 with: - go-version: 1.17 + go-version: 1.18 id: go - name: Check out code into the Go module directory diff --git a/docker-compose.yml b/docker-compose.yml index a2337a63..5f591103 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -33,7 +33,7 @@ services: security_opt: - label:disable clickhouse: - image: yandex/clickhouse-server:20.8.5.45 + image: clickhouse/clickhouse-server:22.3 restart: always ports: - "8123:8123" diff --git a/go.mod b/go.mod index 14e409b6..0414e6c0 100644 --- a/go.mod +++ b/go.mod @@ -1,10 +1,10 @@ module github.com/housepower/clickhouse_sinker -go 1.17 +go 1.18 require ( github.com/ClickHouse/clickhouse-go v1.5.4 - github.com/RoaringBitmap/roaring v1.2.0 + github.com/RoaringBitmap/roaring v1.2.1 github.com/Shopify/sarama v1.34.1 github.com/bytedance/sonic v1.3.1 github.com/cespare/xxhash/v2 v2.1.2 @@ -17,7 +17,7 @@ require ( github.com/prometheus/client_golang v1.12.2 github.com/prometheus/common v0.34.0 github.com/segmentio/kafka-go v0.4.32 - github.com/stretchr/testify v1.7.1 + github.com/stretchr/testify v1.7.5 github.com/tidwall/gjson v1.14.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/twmb/franz-go v1.6.0 @@ -60,7 +60,7 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect - github.com/pierrec/lz4/v4 v4.1.14 // indirect + github.com/pierrec/lz4/v4 v4.1.15 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/procfs v0.7.3 // indirect @@ -84,5 +84,5 @@ require ( golang.org/x/text v0.3.7 // indirect google.golang.org/protobuf v1.28.0 // indirect gopkg.in/ini.v1 v1.66.6 // indirect - gopkg.in/yaml.v3 v3.0.0 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index 2dce3ade..bf6d0dc7 100644 --- a/go.sum +++ b/go.sum @@ -38,8 +38,8 @@ github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV github.com/ClickHouse/clickhouse-go v1.5.4/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/RoaringBitmap/roaring v1.2.0 h1:qayex3YgtOmzev8slia4A0jPGsn2o2bnqKDcRpyRUiI= -github.com/RoaringBitmap/roaring v1.2.0/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= +github.com/RoaringBitmap/roaring v1.2.1 h1:58/LJlg/81wfEHd5L9qsHduznOIhyv4qb1yWcSvVq9A= +github.com/RoaringBitmap/roaring v1.2.1/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= github.com/Shopify/sarama v1.34.1 h1:pVCQO7BMAK3s1jWhgi5v1W6lwZ6Veiekfc2vsgRS06Y= github.com/Shopify/sarama v1.34.1/go.mod h1:NZSNswsnStpq8TUdFaqnpXm2Do6KRzTIjdBdVlL1YRM= github.com/Shopify/toxiproxy/v2 v2.4.0 h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64= @@ -271,8 +271,9 @@ github.com/nacos-group/nacos-sdk-go v1.1.1/go.mod h1:UHOtQNQY/qpk2dhg6gDq8u5+/CE github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= -github.com/pierrec/lz4/v4 v4.1.14 h1:+fL8AQEZtz/ijeNnpduH0bROTu0O3NZAlPjQxGn8LwE= github.com/pierrec/lz4/v4 v4.1.14/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= +github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -328,14 +329,16 @@ github.com/streadway/amqp v1.0.0 h1:kuuDrUJFZL1QYL9hUNuCxNObNzB0bV/ZG5jV3RWAQgo= github.com/streadway/amqp v1.0.0/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.1 h1:5TQK59W5E3v0r2duFAb7P95B6hEeOyEnHRa8MjYSMTY= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.5 h1:s5PTfem8p8EbKQOctVV53k6jCJt3UX4IEJzwh+C324Q= +github.com/stretchr/testify v1.7.5/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/tidwall/gjson v1.12.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/gjson v1.13.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/gjson v1.14.1 h1:iymTbGkQBhveq21bEvAQ81I0LEBork8BFe1CUZXdyuo= @@ -551,7 +554,6 @@ golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210902050250-f475640dd07b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d h1:Zu/JngovGLVi6t2J3nmAf3AoTDwuzw85YZ3b9o4yU7s= golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -721,8 +723,9 @@ gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20220512140231-539c8e751b99/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0 h1:hjy8E9ON/egN1tAYqKb61G10WtihqetD4sz2H+8nIeA= gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= From b492fc8bde1d6cab16a0fe3d3f15a0c2397e6de6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 30 Jun 2022 16:37:41 +0800 Subject: [PATCH 249/404] bump clickhouse_go v2.2.0 --- cmd/clickhouse_sinker/main.go | 2 +- cmd/nacos_publish_config/main.go | 2 +- go.mod | 9 ++++--- go.sum | 45 ++++++++++++++++++++++++++++---- output/clickhouse_util.go | 2 +- 5 files changed, 49 insertions(+), 11 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 88608e41..e2f1e9fa 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -40,7 +40,7 @@ import ( "github.com/housepower/clickhouse_sinker/util" "go.uber.org/zap" - _ "github.com/ClickHouse/clickhouse-go" + _ "github.com/ClickHouse/clickhouse-go/v2" "github.com/prometheus/client_golang/prometheus/promhttp" ) diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 74b3c3ad..e1986898 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -22,7 +22,7 @@ import ( "reflect" "time" - _ "github.com/ClickHouse/clickhouse-go" + _ "github.com/ClickHouse/clickhouse-go/v2" "github.com/housepower/clickhouse_sinker/config" cm "github.com/housepower/clickhouse_sinker/config_manager" "github.com/housepower/clickhouse_sinker/util" diff --git a/go.mod b/go.mod index 0414e6c0..a9861bce 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module github.com/housepower/clickhouse_sinker go 1.18 require ( - github.com/ClickHouse/clickhouse-go v1.5.4 + github.com/ClickHouse/clickhouse-go/v2 v2.2.0 github.com/RoaringBitmap/roaring v1.2.1 github.com/Shopify/sarama v1.34.1 github.com/bytedance/sonic v1.3.1 @@ -36,7 +36,6 @@ require ( github.com/bits-and-blooms/bitset v1.2.2 // indirect github.com/buger/jsonparser v1.1.1 // indirect github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a // indirect - github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/eapache/go-resiliency v1.2.0 // indirect github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 // indirect @@ -44,6 +43,7 @@ require ( github.com/go-errors/errors v1.4.2 // indirect github.com/golang/protobuf v1.5.2 // indirect github.com/golang/snappy v0.0.4 // indirect + github.com/google/uuid v1.3.0 // indirect github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/hashicorp/go-uuid v1.0.3 // indirect @@ -59,12 +59,13 @@ require ( github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect - github.com/pierrec/lz4 v2.6.1+incompatible // indirect + github.com/paulmach/orb v0.7.1 // indirect github.com/pierrec/lz4/v4 v4.1.15 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/procfs v0.7.3 // indirect github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect + github.com/shopspring/decimal v1.3.1 // indirect github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.0 // indirect @@ -74,6 +75,8 @@ require ( github.com/xdg-go/stringprep v1.0.3 // indirect github.com/xdg/scram v1.0.5 // indirect github.com/xdg/stringprep v1.0.3 // indirect + go.opentelemetry.io/otel v1.7.0 // indirect + go.opentelemetry.io/otel/trace v1.7.0 // indirect go.uber.org/atomic v1.9.0 // indirect go.uber.org/multierr v1.8.0 // indirect golang.org/x/arch v0.0.0-20220412001346-fc48f9fe4c15 // indirect diff --git a/go.sum b/go.sum index bf6d0dc7..e0c10782 100644 --- a/go.sum +++ b/go.sum @@ -34,8 +34,9 @@ dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7 github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= github.com/ClickHouse/clickhouse-go v1.5.4/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= +github.com/ClickHouse/clickhouse-go/v2 v2.2.0 h1:dj00TDKY+xwuTJdbpspCSmTLFyWzRJerTHwaBxut1C0= +github.com/ClickHouse/clickhouse-go/v2 v2.2.0/go.mod h1:8f2XZUi7XoeU+uPIytSi1cvx8fmJxi7vIgqpvYTF1+o= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.2.1 h1:58/LJlg/81wfEHd5L9qsHduznOIhyv4qb1yWcSvVq9A= @@ -44,6 +45,7 @@ github.com/Shopify/sarama v1.34.1 h1:pVCQO7BMAK3s1jWhgi5v1W6lwZ6Veiekfc2vsgRS06Y github.com/Shopify/sarama v1.34.1/go.mod h1:NZSNswsnStpq8TUdFaqnpXm2Do6KRzTIjdBdVlL1YRM= github.com/Shopify/toxiproxy/v2 v2.4.0 h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64= github.com/Shopify/toxiproxy/v2 v2.4.0/go.mod h1:3ilnjng821bkozDRxNoo64oI/DKqM+rOyJzb564+bvg= +github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/StackExchange/wmi v1.2.1/go.mod h1:rcmrprowKIVzvc+NUiLncP2uuArMWLCbu9SBzvHz7e8= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= @@ -62,7 +64,6 @@ github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6r github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.2.2 h1:J5gbX05GpMdBjCvQ9MteIg2KKDExr7DrgK+Yc15FvIk= github.com/bits-and-blooms/bitset v1.2.2/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= -github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAKk= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= @@ -79,7 +80,6 @@ github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWR github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= @@ -115,13 +115,19 @@ github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9 github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= +github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= +github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.2.6-0.20210915003542-8b1f7f90f6b1/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/goccy/go-json v0.9.4 h1:L8MLKG2mvVXiQu07qB6hmfqeSYQdOnqPot2GhsIwIaI= github.com/goccy/go-json v0.9.4/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -180,12 +186,16 @@ github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hf github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= +github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gorilla/handlers v1.4.2/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/OAirnOIQ= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= +github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= @@ -231,6 +241,7 @@ github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfV github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= +github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.14.2/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= @@ -255,6 +266,7 @@ github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/mkevac/debugcharts v0.0.0-20191222103121-ae1c48aa8615/go.mod h1:Ad7oeElCZqA1Ufj0U9/liOF4BtVepxRcTvr2ey7zTvM= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -268,9 +280,10 @@ github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRW github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nacos-group/nacos-sdk-go v1.1.1 h1:beczWcOoTaVBMgCgikqvZflrN5Xbw7pWAWpxl+VJGIA= github.com/nacos-group/nacos-sdk-go v1.1.1/go.mod h1:UHOtQNQY/qpk2dhg6gDq8u5+/CEIc3+lWmrmxEzX0/g= +github.com/paulmach/orb v0.7.1 h1:Zha++Z5OX/l168sqHK3k4z18LDvr+YAO/VjK0ReQ9rU= +github.com/paulmach/orb v0.7.1/go.mod h1:FWRlTgl88VI1RBx/MkrwWDRhQ96ctqMCh8boXhmqB/A= +github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= -github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= -github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4/v4 v4.1.14/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -316,7 +329,12 @@ github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTE github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/segmentio/kafka-go v0.4.32 h1:Ohr+9E+kDv/Ld2UPJN9hnKZRd2qgiqCmI8v2e1qlfLM= github.com/segmentio/kafka-go v0.4.32/go.mod h1:JAPPIiY3MQIwVHj64CWOP0LsFFfQ7H0w69kuoxnMIS0= +github.com/shirou/gopsutil v2.19.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shirou/gopsutil v3.21.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil/v3 v3.21.9/go.mod h1:YWp/H8Qs5fVmf17v7JNZzA0mPJ+mS2e9JdiUF9LlKzQ= +github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= +github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= +github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= @@ -350,7 +368,9 @@ github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhso github.com/tidwall/sjson v1.2.4 h1:cuiLzLnaMeBhRmEv00Lpk3tkYrcxpmbU81tAY4Dw0tc= github.com/tidwall/sjson v1.2.4/go.mod h1:098SZ494YoMWPmMO6ct4dcFnqxwj9r/gF0Etp19pSNM= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= +github.com/tklauser/go-sysconf v0.3.10/go.mod h1:C8XykCvCb+Gn0oNCWPIlcb0RuglQTYaQ2hGm7jmxEFk= github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= +github.com/tklauser/numcpus v0.4.0/go.mod h1:1+UI3pD8NW14VMwdgJNJ1ESk2UnwhAnz5hMwiKKqXCQ= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= @@ -386,12 +406,18 @@ github.com/xlab/treeprint v1.1.0/go.mod h1:gj5Gd3gPdKtR1ikdDK6fnFLdmIS0X30kTTuNd github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opentelemetry.io/otel v1.7.0 h1:Z2lA3Tdch0iDcrhJXDIlC94XE+bxok1F9B+4Lz/lGsM= +go.opentelemetry.io/otel v1.7.0/go.mod h1:5BdUoMIz5WEs0vt0CUEMtSSaTSHBBVwrhnz7+nrD5xk= +go.opentelemetry.io/otel/trace v1.7.0 h1:O37Iogk1lEkMRXewVtZ1BBTVn5JEp8GrJvP92bJqC6o= +go.opentelemetry.io/otel/trace v1.7.0/go.mod h1:fzLSB9nqR2eXzxPXb2JW9IKE+ScyXA48yyE4TNvoHqU= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= @@ -481,6 +507,7 @@ golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/ golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= @@ -505,6 +532,7 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f h1:Ax0t5p6N38Ga0dThY21weqDEyz2oklo4IvDkpigvkD8= @@ -526,6 +554,7 @@ golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191220220014-0732a990476f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -543,6 +572,7 @@ golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -554,6 +584,8 @@ golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210902050250-f475640dd07b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220128215802-99c3d69c2c27/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220429233432-b5fbb4746d32/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d h1:Zu/JngovGLVi6t2J3nmAf3AoTDwuzw85YZ3b9o4yU7s= golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -611,9 +643,11 @@ golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -695,6 +729,7 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0 h1:w43yiav+6bVFTBQFZX0r7ipe9JQ1QsbMgHwbBziscLw= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 20a5b107..637ffb7f 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -4,7 +4,7 @@ import ( "database/sql" "fmt" - "github.com/ClickHouse/clickhouse-go" + "github.com/ClickHouse/clickhouse-go/v2" "github.com/RoaringBitmap/roaring" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" From f71e0fb92a9b321154cf605943df5517dfdebcee Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 30 Jun 2022 17:24:29 +0800 Subject: [PATCH 250/404] reworked Decimal support --- docker/test_auto_schema.json | 2 +- docker/test_dynamic_schema.json | 2 +- docker/test_fixed_schema.json | 2 +- go.mod | 2 +- model/metric.go | 1 + model/value.go | 12 ++++++++++-- parser/csv.go | 29 +++++++++++++++++++++++++++++ parser/fastjson.go | 28 ++++++++++++++++++++++++++++ parser/gjson.go | 29 +++++++++++++++++++++++++++++ pool/conn.go | 5 ++--- 10 files changed, 103 insertions(+), 9 deletions(-) diff --git a/docker/test_auto_schema.json b/docker/test_auto_schema.json index 34dffbd6..3ee0e3a0 100644 --- a/docker/test_auto_schema.json +++ b/docker/test_auto_schema.json @@ -28,5 +28,5 @@ ], "bufferSize": 50000 }, - "logLevel": "debug" + "logLevel": "info" } \ No newline at end of file diff --git a/docker/test_dynamic_schema.json b/docker/test_dynamic_schema.json index d7d564d5..4758a3a2 100644 --- a/docker/test_dynamic_schema.json +++ b/docker/test_dynamic_schema.json @@ -32,5 +32,5 @@ }, "bufferSize": 50000 }, - "logLevel": "debug" + "logLevel": "info" } \ No newline at end of file diff --git a/docker/test_fixed_schema.json b/docker/test_fixed_schema.json index 425ed942..c18e2c02 100644 --- a/docker/test_fixed_schema.json +++ b/docker/test_fixed_schema.json @@ -38,5 +38,5 @@ ], "bufferSize": 50000 }, - "logLevel": "debug" + "logLevel": "info" } \ No newline at end of file diff --git a/go.mod b/go.mod index a9861bce..5ccb6042 100644 --- a/go.mod +++ b/go.mod @@ -17,6 +17,7 @@ require ( github.com/prometheus/client_golang v1.12.2 github.com/prometheus/common v0.34.0 github.com/segmentio/kafka-go v0.4.32 + github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.7.5 github.com/tidwall/gjson v1.14.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 @@ -65,7 +66,6 @@ require ( github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/procfs v0.7.3 // indirect github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect - github.com/shopspring/decimal v1.3.1 // indirect github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.0 // indirect diff --git a/model/metric.go b/model/metric.go index 9a17d9d8..5f78d81d 100644 --- a/model/metric.go +++ b/model/metric.go @@ -24,6 +24,7 @@ import ( type Metric interface { GetInt(key string, nullable bool) (val interface{}) GetFloat(key string, nullable bool) (val interface{}) + GetDecimal(key string, nullable bool) (val interface{}) GetString(key string, nullable bool) (val interface{}) GetDateTime(key string, nullable bool) (val interface{}) GetElasticDateTime(key string, nullable bool) (val interface{}) diff --git a/model/value.go b/model/value.go index 9afab399..7417ba19 100644 --- a/model/value.go +++ b/model/value.go @@ -25,11 +25,13 @@ const ( Unknown = iota Int Float + Decimal String DateTime ElasticDateTime IntArray FloatArray + DecimalArray StringArray DateTimeArray ) @@ -49,6 +51,8 @@ func GetTypeName(typ int) (name string) { name = "Int" case Float: name = "Float" + case Decimal: + name = "Decimal" case String: name = "String" case DateTime: @@ -77,6 +81,8 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { val = metric.GetInt(name, cwt.Nullable) case Float: val = metric.GetFloat(name, cwt.Nullable) + case Decimal: + val = metric.GetDecimal(name, cwt.Nullable) case String: val = metric.GetString(name, cwt.Nullable) case DateTime: @@ -87,6 +93,8 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { val = metric.GetArray(name, Int) case FloatArray: val = metric.GetArray(name, Float) + case DecimalArray: + val = metric.GetArray(name, Decimal) case StringArray: val = metric.GetArray(name, String) case DateTimeArray: @@ -113,9 +121,9 @@ func WhichType(typ string) (dataType int, nullable bool) { dataType = DateTimeArray nullable = false } else if strings.HasPrefix(typ, "Decimal") { - dataType = Float + dataType = Decimal } else if strings.HasPrefix(typ, "Array(Decimal") { - dataType = FloatArray + dataType = DecimalArray nullable = false } else if strings.HasPrefix(typ, "FixedString") { dataType = String diff --git a/parser/csv.go b/parser/csv.go index 1442becd..fe4e49b2 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -26,6 +26,7 @@ import ( "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" + "github.com/shopspring/decimal" "github.com/tidwall/gjson" "github.com/valyala/fastjson/fastfloat" ) @@ -93,6 +94,21 @@ func (c *CsvMetric) GetFloat(key string, nullable bool) (val interface{}) { return } +// GetDecimal returns the value as decimal +func (c *CsvMetric) GetDecimal(key string, nullable bool) (val interface{}) { + var idx int + var ok bool + if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { + if nullable { + return + } + val = decimal.NewFromInt(0) + return + } + val, _ = decimal.NewFromString(c.values[idx]) + return +} + // GetInt returns int func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}) { var idx int @@ -186,6 +202,19 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { results = append(results, v) } val = results + case model.Decimal: + results := make([]decimal.Decimal, 0, len(array)) + for _, e := range array { + var v float64 + switch e.Type { + case gjson.Number: + v = e.Num + default: + v = float64(0.0) + } + results = append(results, decimal.NewFromFloat(v)) + } + val = results case model.String: results := make([]string, 0, len(array)) for _, e := range array { diff --git a/parser/fastjson.go b/parser/fastjson.go index eddda10a..1b91fd81 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -24,6 +24,7 @@ import ( "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" + "github.com/shopspring/decimal" "github.com/valyala/fastjson" "go.uber.org/zap" ) @@ -84,6 +85,20 @@ func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { return } +func (c *FastjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) { + v := c.value.Get(key) + if !fjCompatibleFloat(v) { + val = getDefaultDecimal(nullable) + return + } + if val2, err := v.Float64(); err != nil { + val = getDefaultDecimal(nullable) + } else { + val = decimal.NewFromFloat(val2) + } + return +} + func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}) { v := c.value.Get(key) if !fjCompatibleInt(v) { @@ -166,6 +181,11 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { v, _ := e.Float64() val = append(val.([]float64), v) } + case model.Decimal: + for _, e := range array { + v, _ := e.Float64() + val = append(val.([]decimal.Decimal), decimal.NewFromFloat(v)) + } case model.String: for _, e := range array { var s string @@ -291,6 +311,14 @@ func getDefaultFloat(nullable bool) (val interface{}) { return } +func getDefaultDecimal(nullable bool) (val interface{}) { + if nullable { + return + } + val = decimal.NewFromInt(0) + return +} + func getDefaultDateTime(nullable bool) (val interface{}) { if nullable { return diff --git a/parser/gjson.go b/parser/gjson.go index 7f87420e..5de64aba 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -21,6 +21,7 @@ import ( "sync" "time" + "github.com/shopspring/decimal" "github.com/tidwall/gjson" "go.uber.org/zap" @@ -79,6 +80,21 @@ func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { return } +func (c *GjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) { + r := gjson.Get(c.raw, key) + if !gjCompatibleFloat(r) { + val = getDefaultDecimal(nullable) + return + } + switch r.Type { + case gjson.Number: + val = decimal.NewFromFloat(r.Num) + default: + val = getDefaultDecimal(nullable) + } + return +} + func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) if !gjCompatibleInt(r) { @@ -168,6 +184,19 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { results = append(results, f) } val = results + case model.Decimal: + results := make([]decimal.Decimal, 0, len(array)) + for _, e := range array { + var f float64 + switch e.Type { + case gjson.Number: + f = e.Num + default: + f = float64(0.0) + } + results = append(results, decimal.NewFromFloat(f)) + } + val = results case model.String: results := make([]string, 0, len(array)) for _, e := range array { diff --git a/pool/conn.go b/pool/conn.go index 634ea81a..5f53a42b 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -26,7 +26,6 @@ import ( "sync" "time" - "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/health" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" @@ -131,8 +130,8 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara freeClusterConn() // Each shard has a *sql.DB which connects to one replica inside the shard. // "alt_hosts" tolerates replica single-point-failure. However more flexable switching is needed for some cases for example https://github.com/ClickHouse/ClickHouse/issues/24036. - dsnSuffix = fmt.Sprintf("?database=%s&username=%s&password=%s&block_size=%d", - url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password), 2*config.MaxBufferSize) + dsnSuffix = fmt.Sprintf("?database=%s&username=%s&password=%s", + url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password)) if dsnParams != "" { dsnSuffix += "&" + dsnParams } From 053e61fdfbf9d8a4f5d8f3a083b035ae0f6a9558 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 10 May 2022 12:04:12 +0800 Subject: [PATCH 251/404] added Bool support --- docs/dev/introduction.md | 1 + model/metric.go | 1 + model/value.go | 13 ++++++++++ parser/csv.go | 23 +++++++++++++++++- parser/fastjson.go | 52 ++++++++++++++++++++++++++++++---------- parser/gjson.go | 43 +++++++++++++++++++++++++-------- parser/parser.go | 2 ++ parser/parser_test.go | 52 ++++++++++++++++++++++++++++++++++++---- pool/conn.go | 2 +- 9 files changed, 161 insertions(+), 28 deletions(-) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 4954f0d6..b15c9874 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -45,6 +45,7 @@ Note: | ClickHouse data type | default value | compatible Json data type | valid range | |:--------------------:|:-------------:|:-----------------------------------:|:-------------------------------------:| +| Bool | false | Bool | false, true | | Int8, Int16, ... | 0 | Bool, Number | Int8 [-128,127], ... | | Float32, Float64 | 0.0 | Number | Float32 [-MaxFloat32,MaxFloat32], ... | | Decimal, ... | 0.0 | Number | [decimal-value-ranges](https://clickhouse.tech/docs/en/sql-reference/data-types/decimal/#decimal-value-ranges) | diff --git a/model/metric.go b/model/metric.go index 5f78d81d..a270b18d 100644 --- a/model/metric.go +++ b/model/metric.go @@ -22,6 +22,7 @@ import ( // Metric interface for metric collection type Metric interface { + GetBool(key string, nullable bool) (val interface{}) GetInt(key string, nullable bool) (val interface{}) GetFloat(key string, nullable bool) (val interface{}) GetDecimal(key string, nullable bool) (val interface{}) diff --git a/model/value.go b/model/value.go index 7417ba19..e8d7754f 100644 --- a/model/value.go +++ b/model/value.go @@ -23,12 +23,14 @@ import ( const ( Unknown = iota + Bool Int Float Decimal String DateTime ElasticDateTime + BoolArray IntArray FloatArray DecimalArray @@ -47,6 +49,8 @@ var ( func GetTypeName(typ int) (name string) { switch typ { + case Bool: + name = "Bool" case Int: name = "Int" case Float: @@ -59,6 +63,8 @@ func GetTypeName(typ int) (name string) { name = "DateTime" case ElasticDateTime: name = "ElasticDateTime" + case BoolArray: + name = "BoolArray" case IntArray: name = "IntArray" case FloatArray: @@ -77,6 +83,8 @@ func GetTypeName(typ int) (name string) { func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { name := cwt.SourceName switch cwt.Type { + case Bool: + val = metric.GetBool(name, cwt.Nullable) case Int: val = metric.GetInt(name, cwt.Nullable) case Float: @@ -89,6 +97,8 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { val = metric.GetDateTime(name, cwt.Nullable) case ElasticDateTime: val = metric.GetElasticDateTime(name, cwt.Nullable) + case BoolArray: + val = metric.GetArray(name, Bool) case IntArray: val = metric.GetArray(name, Int) case FloatArray: @@ -144,6 +154,7 @@ func WhichType(typ string) (dataType int, nullable bool) { func init() { primTypeInfo := make(map[string]TypeInfo) typeInfo = make(map[string]TypeInfo) + primTypeInfo["Bool"] = TypeInfo{Type: Bool, Nullable: false} for _, t := range []string{"UInt8", "UInt16", "UInt32", "UInt64", "Int8", "Int16", "Int32", "Int64"} { primTypeInfo[t] = TypeInfo{Type: Int, Nullable: false} @@ -164,6 +175,8 @@ func init() { typeInfo[nullK] = TypeInfo{Type: v.Type, Nullable: true} arrK := fmt.Sprintf("Array(%s)", k) switch v.Type { + case Bool: + typeInfo[arrK] = TypeInfo{Type: BoolArray, Nullable: false} case Int: typeInfo[arrK] = TypeInfo{Type: IntArray, Nullable: false} case Float: diff --git a/parser/csv.go b/parser/csv.go index fe4e49b2..c79b8be3 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -109,7 +109,21 @@ func (c *CsvMetric) GetDecimal(key string, nullable bool) (val interface{}) { return } -// GetInt returns int +func (c *CsvMetric) GetBool(key string, nullable bool) (val interface{}) { + var idx int + var ok bool + if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "" || c.values[idx] == "null" { + if nullable { + return + } + val = false + return + } + val = (c.values[idx] == "true") + return +} + + func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}) { var idx int var ok bool @@ -172,6 +186,13 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { return } switch typ { + case model.Bool: + results := make([]bool, 0, len(array)) + for _, e := range array { + v := (e.Type == gjson.True) + results = append(results, v) + } + val = results case model.Int: results := make([]int64, 0, len(array)) for _, e := range array { diff --git a/parser/fastjson.go b/parser/fastjson.go index 1b91fd81..323a6bf1 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -85,6 +85,16 @@ func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { return } +func (c *FastjsonMetric) GetBool(key string, nullable bool) (val interface{}) { + v := c.value.Get(key) + if !fjCompatibleBool(v) { + val = getDefaultBool(nullable) + return + } + val = (v.Type() == fastjson.TypeTrue) + return +} + func (c *FastjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) { v := c.value.Get(key) if !fjCompatibleFloat(v) { @@ -166,6 +176,11 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { } array, _ := v.Array() switch typ { + case model.Bool: + for _, e := range array { + v := (e != nil && e.Type() == fastjson.TypeTrue) + val = append(val.([]bool), v) + } case model.Int: for _, e := range array { var v int64 @@ -256,16 +271,23 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys, newKeys *sync.Map, white, black * return } -func fjCompatibleInt(v *fastjson.Value) (ok bool) { +func fjCompatibleBool(v *fastjson.Value) (ok bool) { if v == nil { return } switch v.Type() { - case fastjson.TypeTrue: - ok = true - case fastjson.TypeFalse: + case fastjson.TypeTrue, fastjson.TypeFalse: ok = true - case fastjson.TypeNumber: + } + return +} + +func fjCompatibleInt(v *fastjson.Value) (ok bool) { + if v == nil { + return + } + switch v.Type() { + case fastjson.TypeTrue, fastjson.TypeFalse, fastjson.TypeNumber: ok = true } return @@ -287,14 +309,20 @@ func fjCompatibleDateTime(v *fastjson.Value) (ok bool) { return } switch v.Type() { - case fastjson.TypeNumber: - ok = true - case fastjson.TypeString: + case fastjson.TypeNumber, fastjson.TypeString: ok = true } return } +func getDefaultBool(nullable bool) (val interface{}) { + if nullable { + return + } + val = false + return +} + func getDefaultInt(nullable bool) (val interface{}) { if nullable { return @@ -331,10 +359,8 @@ func fjDetectType(v *fastjson.Value) (typ int) { switch v.Type() { case fastjson.TypeNull: typ = model.Unknown - case fastjson.TypeTrue: - typ = model.Int - case fastjson.TypeFalse: - typ = model.Int + case fastjson.TypeTrue, fastjson.TypeFalse: + typ = model.Bool case fastjson.TypeNumber: typ = model.Float if _, err := v.Int64(); err == nil { @@ -351,6 +377,8 @@ func fjDetectType(v *fastjson.Value) (typ int) { if arr, err := v.Array(); err == nil && len(arr) > 0 { typ2 := fjDetectType(arr[0]) switch typ2 { + case model.Bool: + typ = model.BoolArray case model.Int: typ = model.IntArray case model.Float: diff --git a/parser/gjson.go b/parser/gjson.go index 5de64aba..57506ef3 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -80,6 +80,16 @@ func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { return } +func (c *GjsonMetric) GetBool(key string, nullable bool) (val interface{}) { + r := gjson.Get(c.raw, key) + if !gjCompatibleBool(r) { + val = getDefaultBool(nullable) + return + } + val = (r.Type == gjson.True) + return +} + func (c *GjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) if !gjCompatibleFloat(r) { @@ -154,6 +164,13 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { } array := r.Array() switch typ { + case model.Bool: + results := make([]bool, 0, len(array)) + for _, e := range array { + v := (e.Exists() && e.Type == gjson.True) + results = append(results, v) + } + val = results case model.Int: results := make([]int64, 0, len(array)) for _, e := range array { @@ -258,16 +275,24 @@ func (c *GjsonMetric) GetNewKeys(knownKeys, newKeys *sync.Map, white, black *reg return } -func gjCompatibleInt(r gjson.Result) (ok bool) { +func gjCompatibleBool(r gjson.Result) (ok bool) { if !r.Exists() { return } switch r.Type { - case gjson.True: - ok = true - case gjson.False: + case gjson.True, gjson.False: ok = true - case gjson.Number: + default: + } + return +} + +func gjCompatibleInt(r gjson.Result) (ok bool) { + if !r.Exists() { + return + } + switch r.Type { + case gjson.True, gjson.False, gjson.Number: ok = true default: } @@ -291,9 +316,7 @@ func gjCompatibleDateTime(r gjson.Result) (ok bool) { return } switch r.Type { - case gjson.Number: - ok = true - case gjson.String: + case gjson.Number, gjson.String: ok = true default: } @@ -304,7 +327,7 @@ func gjDetectType(v gjson.Result) (typ int) { typ = model.Unknown switch v.Type { case gjson.True, gjson.False: - typ = model.Int + typ = model.Bool case gjson.Number: typ = model.Float if float64(v.Int()) == v.Num { @@ -322,7 +345,7 @@ func gjDetectType(v gjson.Result) (typ int) { if array := v.Array(); len(array) != 0 { switch array[0].Type { case gjson.True, gjson.False: - typ = model.IntArray + typ = model.BoolArray case gjson.Number: typ = model.FloatArray if float64(array[0].Int()) == array[0].Num { diff --git a/parser/parser.go b/parser/parser.go index 30d18364..011e3319 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -169,6 +169,8 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time, err error) { func makeArray(typ int) (val interface{}) { switch typ { + case model.Bool: + val = []bool{} case model.Int: val = []int64{} case model.Float: diff --git a/parser/parser_test.go b/parser/parser_test.go index 41059364..3a11593f 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -70,8 +70,8 @@ var jsonSample = []byte(`{ var jsonSchema = map[string]string{ "null": "Unknown", - "bool_true": "Int", - "bool_false": "Int", + "bool_true": "Bool", + "bool_false": "Bool", "num_int": "Int", "num_float": "Float", "str": "String", @@ -86,7 +86,7 @@ var jsonSchema = map[string]string{ "obj": "String", "array_empty": "Unknown", "array_null": "Unknown", - "array_bool": "IntArray", + "array_bool": "BoolArray", "array_num_int_1": "IntArray", "array_num_int_2": "IntArray", "array_num_float": "FloatArray", @@ -220,11 +220,13 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { for j := range testCases { var v interface{} desc := fmt.Sprintf(`%s.%s("%s", %s)`, name, method, testCases[j].Field, strconv.FormatBool(testCases[j].Nullable)) - if name == "csv" && (sliceContains([]string{"GetInt", "GetFloat", "GetDateTime", "GetElasticDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || testCases[j].Nullable) { + if name == "csv" && (sliceContains([]string{"GetBool", "GetInt", "GetFloat", "GetDateTime", "GetElasticDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || testCases[j].Nullable) { skipped = append(skipped, desc) continue } switch method { + case "GetBool": + v = metric.GetBool(testCases[j].Field, testCases[j].Nullable) case "GetInt": v = metric.GetInt(testCases[j].Field, testCases[j].Nullable) case "GetFloat": @@ -246,6 +248,38 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { } } +func TestParserBool(t *testing.T) { + testCases := []SimpleCase{ + // nullable: false + {"not_exist", false, false}, + {"null", false, false}, + {"bool_true", false, true}, + {"bool_false", false, false}, + {"num_int", false, false}, + {"num_float", false, false}, + {"str", false, false}, + {"str_int", false, false}, + {"str_float", false, false}, + {"str_date_1", false, false}, + {"obj", false, false}, + {"array_empty", false, false}, + // nullable: true + {"not_exist", true, nil}, + {"null", true, nil}, + {"bool_true", true, true}, + {"bool_false", true, false}, + {"num_int", true, nil}, + {"num_float", true, nil}, + {"str", true, nil}, + {"str_int", true, nil}, + {"str_float", true, nil}, + {"str_date_1", true, nil}, + {"obj", true, nil}, + {"array_empty", true, nil}, + } + doTestSimple(t, "GetBool", testCases) +} + func TestParserInt(t *testing.T) { testCases := []SimpleCase{ // nullable: false @@ -442,51 +476,61 @@ func TestParserArray(t *testing.T) { {"str_date_1", model.DateTime, []time.Time{}}, {"obj", model.String, []string{}}, + {"array_empty", model.Bool, []bool{}}, {"array_empty", model.Int, []int64{}}, {"array_empty", model.Float, []float64{}}, {"array_empty", model.String, []string{}}, {"array_empty", model.DateTime, []time.Time{}}, + {"array_null", model.Bool, []bool{false}}, {"array_null", model.Int, []int64{0}}, {"array_null", model.Float, []float64{0.0}}, {"array_null", model.String, []string{""}}, {"array_null", model.DateTime, []time.Time{Epoch}}, + {"array_bool", model.Bool, []bool{true, false}}, {"array_bool", model.Int, []int64{1, 0}}, {"array_bool", model.Float, []float64{0.0, 0.0}}, {"array_bool", model.String, []string{"true", "false"}}, {"array_bool", model.DateTime, []time.Time{Epoch, Epoch}}, + {"array_num_int_1", model.Bool, []bool{false, false, false, false, false, false, false, false, false}}, {"array_num_int_1", model.Int, []int64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 0, 0}}, {"array_num_int_1", model.Float, []float64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 18446744073709551615, 18446744073709551616}}, {"array_num_int_1", model.String, []string{"0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"}}, {"array_num_int_1", model.DateTime, []time.Time{Epoch, UnixFloat(255, timeUnit), UnixFloat(256, timeUnit), UnixFloat(65535, timeUnit), UnixFloat(65536, timeUnit), UnixFloat(4294967295, timeUnit), UnixFloat(4294967296, timeUnit), Epoch, Epoch}}, + {"array_num_int_2", model.Bool, []bool{false, false, false, false, false, false, false, false, false, false, false, false, false, false, false}}, {"array_num_int_2", model.Int, []int64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, {"array_num_int_2", model.Float, []float64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, {"array_num_int_2", model.String, []string{"-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"}}, {"array_num_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, UnixFloat(127, timeUnit), UnixFloat(128, timeUnit), UnixFloat(32767, timeUnit), UnixFloat(32768, timeUnit), UnixFloat(2147483647, timeUnit), UnixFloat(2147483648, timeUnit), UnixFloat(9223372036854775807, timeUnit)}}, + {"array_num_float", model.Bool, []bool{false, false, false, false, false, false, false}}, {"array_num_float", model.Int, []int64{0, 0, 0, 0, 0, 0, 0}}, {"array_num_float", model.Float, []float64{4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308, math.Inf(-1), math.Inf(1)}}, {"array_num_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"}}, {"array_num_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, UnixFloat(3.40282346638528859811704183484516925440e+38, timeUnit), UnixFloat(1.797693134862315708145274237317043567981e+308, timeUnit), UnixFloat(math.Inf(-1), timeUnit), UnixFloat(math.Inf(1), timeUnit)}}, + {"array_str", model.Bool, []bool{false, false, false}}, {"array_str", model.Int, []int64{0, 0, 0}}, {"array_str", model.Float, []float64{0.0, 0.0, 0.0}}, {"array_str", model.String, []string{"aa", "bb", "cc"}}, {"array_str", model.DateTime, []time.Time{Epoch, Epoch, Epoch}}, + {"array_str_int_1", model.Bool, []bool{false, false, false, false, false, false, false, false, false}}, {"array_str_int_1", model.Int, []int64{0, 0, 0, 0, 0, 0, 0, 0, 0}}, {"array_str_int_1", model.Float, []float64{0, 0, 0, 0, 0, 0, 0, 0, 0}}, {"array_str_int_1", model.String, []string{"0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"}}, {"array_str_int_1", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch}}, + {"array_str_int_2", model.Bool, []bool{false, false, false, false, false, false, false, false, false, false, false, false, false, false, false}}, {"array_str_int_2", model.Int, []int64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}}, {"array_str_int_2", model.Float, []float64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}}, {"array_str_int_2", model.String, []string{"-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"}}, {"array_str_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch}}, + {"array_str_float", model.Bool, []bool{false, false, false, false, false, false, false}}, {"array_str_float", model.Int, []int64{0, 0, 0, 0, 0, 0, 0}}, {"array_str_float", model.Float, []float64{0, 0, 0, 0, 0, 0, 0}}, {"array_str_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"}}, diff --git a/pool/conn.go b/pool/conn.go index 5f53a42b..6313cc10 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -129,7 +129,7 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara defer lock.Unlock() freeClusterConn() // Each shard has a *sql.DB which connects to one replica inside the shard. - // "alt_hosts" tolerates replica single-point-failure. However more flexable switching is needed for some cases for example https://github.com/ClickHouse/ClickHouse/issues/24036. + // "hosts" tolerates replica single-point-failure. However more flexable switching is needed for some cases for example https://github.com/ClickHouse/ClickHouse/issues/24036. dsnSuffix = fmt.Sprintf("?database=%s&username=%s&password=%s", url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password)) if dsnParams != "" { From 41700626a80b529bf307ba254fe140d430d5de82 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 30 Jun 2022 22:07:44 +0800 Subject: [PATCH 252/404] make lint happy --- .github/workflows/lint.yml | 4 +-- .golangci.yml | 70 ------------------------------------ Makefile | 2 +- cmd/kafka_gen_metric/main.go | 2 +- parser/csv.go | 1 - util/workerpool_test.go | 2 +- 6 files changed, 5 insertions(+), 76 deletions(-) delete mode 100644 .golangci.yml diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 6ce7bd37..ea3b0dfe 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -22,14 +22,14 @@ jobs: uses: golangci/golangci-lint-action@v3 with: # Optional: version of golangci-lint to use in form of v1.2 or v1.2.3 or `latest` to use the latest version - version: v1.45.2 + version: v1.46.2 # Optional: working directory, useful for monorepos # working-directory: somedir # Optional: golangci-lint command line arguments. # args: --issues-exit-code=0 - args: --timeout=3m --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen,containedctx + args: --timeout=3m # Optional: show only new issues if it's a pull request. The default value is `false`. # only-new-issues: true diff --git a/.golangci.yml b/.golangci.yml deleted file mode 100644 index 91298ba4..00000000 --- a/.golangci.yml +++ /dev/null @@ -1,70 +0,0 @@ -run: - concurrency: 4 - timeout: 1m - issues-exit-code: 1 - tests: true - skip-dirs-use-default: true - -output: - format: colored-line-number - print-issued-lines: true - print-linter-name: true - -linters-settings: - errcheck: - check-type-assertions: false - check-blank: false - - funlen: - lines: 512 - statements: 50 - lll: - line-length: 180 - -issues: - exclude-rules: - # Exclude some linters from running on tests files. - - path: _test\.go - linters: - - gocyclo - - errcheck - - dupl - - lll - - gosec - # Exclude known linters from partially hard-vendored code, - # which is impossible to exclude via "nolint" comments. - - path: internal/hmac/ - text: "weak cryptographic primitive" - linters: - - gosec - # Exclude some staticcheck messages - - linters: - - staticcheck - text: "SA9003:" - # Exclude lll issues for long lines with go:generate - - linters: - - lll - source: "^//go:generate " - -linters: - enable-all: true - disable: - - dupl - - wsl - - gochecknoglobals - - goconst - - gocognit - - maligned - - godox - - gochecknoinits - - gosec - - gomnd - - prealloc - - nlreturn - - testpackage - - nestif - - exhaustive - - godot - - gci - - gocritic - - gofumpt diff --git a/Makefile b/Makefile index 3be8c4b9..2ac18758 100644 --- a/Makefile +++ b/Makefile @@ -25,6 +25,6 @@ benchtest: pre systest: build bash go.test.sh lint: - golangci-lint run --issues-exit-code=0 --disable=nakedret,exhaustivestruct,wrapcheck,paralleltest,rowserrcheck,cyclop,scopelint,nilerr,interfacer,funlen,golint,revive,tagliatelle,forcetypeassert,ireturn,varnamelen,containedctx + golangci-lint run --timeout=3m run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.json diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index 52442f28..91aa7bf4 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -150,7 +150,7 @@ func generate() { ShiftTag: int32(rand.Intn(65535)), SeasonTag: int32(rand.Intn(65535)), SpikeTag: int32(rand.Intn(65535)), - IsMissing: int32(rand.Intn(1)), + IsMissing: int32(rand.Intn(2)), } _ = wp.Submit(func() { diff --git a/parser/csv.go b/parser/csv.go index c79b8be3..8251ed8d 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -123,7 +123,6 @@ func (c *CsvMetric) GetBool(key string, nullable bool) (val interface{}) { return } - func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}) { var idx int var ok bool diff --git a/util/workerpool_test.go b/util/workerpool_test.go index f1260100..3f418b13 100644 --- a/util/workerpool_test.go +++ b/util/workerpool_test.go @@ -13,7 +13,7 @@ func TestWorkerPool(t *testing.T) { rspChan := make(chan string, len(requests)) for _, r := range requests { r := r - wp.Submit(func() { + _ = wp.Submit(func() { rspChan <- r }) } From 20998a15644c826443c0c4e1203e89ad91dd64d4 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 30 Jun 2022 23:22:10 +0800 Subject: [PATCH 253/404] Use franz-go to calculate lags, close #155 --- config_manager/lags.go | 149 ++++++++++++++--------------------------- go.mod | 1 + go.sum | 2 + input/kafka_franz.go | 34 ++++++---- 4 files changed, 73 insertions(+), 113 deletions(-) diff --git a/config_manager/lags.go b/config_manager/lags.go index 09959946..44fa1eb5 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -1,14 +1,13 @@ package rcm import ( - "strings" + "context" - "github.com/Shopify/sarama" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/input" - "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" - "go.uber.org/zap" + "github.com/twmb/franz-go/pkg/kadm" + "github.com/twmb/franz-go/pkg/kgo" ) type StateLag struct { @@ -18,51 +17,35 @@ type StateLag struct { // GetTaskStateAndLags get state and lag of all tasks. func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err error) { - var client sarama.Client - var adminClient sarama.ClusterAdmin - var closer func() - if client, adminClient, closer, err = newClient(cfg); err != nil { + var cl *kgo.Client + var adm *kadm.Client + if cl, adm, err = newClient(cfg); err != nil { return } - defer closer() + defer adm.Close() + defer cl.Close() stateLags = make(map[string]StateLag) for _, taskCfg := range cfg.Tasks { + var state string var totalLags int64 - if totalLags, err = getLag(client, adminClient, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { + if state, totalLags, err = getStateAndLag(adm, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { return } - stateLags[taskCfg.Name] = StateLag{Lag: totalLags} - } - - // Get consumer groups' state - groups := make([]string, len(cfg.Tasks)) - for i, taskCfg := range cfg.Tasks { - groups[i] = taskCfg.ConsumerGroup - } - var gd []*sarama.GroupDescription - if gd, err = adminClient.DescribeConsumerGroups(groups); err != nil { - err = errors.Wrapf(err, "failed to describe consumer groups") - return - } - for i, taskCfg := range cfg.Tasks { - if stateLag, ok := stateLags[taskCfg.Name]; ok { - stateLag.State = gd[i].State - stateLags[taskCfg.Name] = stateLag - } + stateLags[taskCfg.Name] = StateLag{State: state, Lag: totalLags} } return } // GetTaskStateAndLag get state and lag of a task. func GetTaskStateAndLag(cfg *config.Config, taskName string) (stateLag StateLag, err error) { - var client sarama.Client - var adminClient sarama.ClusterAdmin - var closer func() - if client, adminClient, closer, err = newClient(cfg); err != nil { + var cl *kgo.Client + var adm *kadm.Client + if cl, adm, err = newClient(cfg); err != nil { return } - defer closer() + defer adm.Close() + defer cl.Close() var taskCfg *config.TaskConfig for _, tskCfg := range cfg.Tasks { @@ -75,89 +58,55 @@ func GetTaskStateAndLag(cfg *config.Config, taskName string) (stateLag StateLag, err = errors.Errorf("task %q doesn't exist", taskName) return } - if stateLag.Lag, err = getLag(client, adminClient, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { + if stateLag.State, stateLag.Lag, err = getStateAndLag(adm, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { return } - - // Get consumer group' state - var gd []*sarama.GroupDescription - if gd, err = adminClient.DescribeConsumerGroups([]string{taskCfg.ConsumerGroup}); err != nil { - err = errors.Wrapf(err, "failed to describe consumer groups") - return - } - stateLag.State = gd[0].State return } -func newClient(cfg *config.Config) (client sarama.Client, adminClient sarama.ClusterAdmin, closer func(), err error) { - var sarCfg *sarama.Config - if sarCfg, err = input.GetSaramaConfig(&cfg.Kafka); err != nil { - return - } - if adminClient, err = sarama.NewClusterAdmin(strings.Split(cfg.Kafka.Brokers, ","), sarCfg); err != nil { - err = errors.Wrapf(err, "sarama.NewClusterAdmin failed") +func newClient(cfg *config.Config) (cl *kgo.Client, adm *kadm.Client, err error) { + var opts []kgo.Opt + if opts, err = input.GetFranzConfig(&cfg.Kafka); err != nil { return } - if client, err = sarama.NewClient(strings.Split(cfg.Kafka.Brokers, ","), sarCfg); err != nil { - err = errors.Wrapf(err, "sarama.NewClient failed") + if cl, err = kgo.NewClient(opts...); err != nil { + err = errors.Wrap(err, "") return } - closer = func() { - if err2 := client.Close(); err2 != nil { - util.Logger.Error("failed to close Kafka client", zap.Error(err2)) - } - if err2 := adminClient.Close(); err2 != nil { - util.Logger.Error("failed to close Kafka admin client", zap.Error(err2)) - } - } + adm = kadm.NewClient(cl) return } -// getLag is inspired by https://github.com/cloudhut/kminion/blob/1ffd02ba94a5edc26d4f11e57191ed3479d8a111/prometheus/collect_consumer_group_lags.go -func getLag(client sarama.Client, adminClient sarama.ClusterAdmin, topic, group string) (totalLags int64, err error) { - // Get number of partitions - var partitions int - var topicsMeta []*sarama.TopicMetadata - if topicsMeta, err = adminClient.DescribeTopics([]string{topic}); err != nil { - err = errors.Wrapf(err, "failed to describe topic %q", topic) +// getStateAndLag is inspired by https://github.com/cloudhut/kminion/blob/1ffd02ba94a5edc26d4f11e57191ed3479d8a111/prometheus/collect_consumer_group_lags.go +func getStateAndLag(adm *kadm.Client, topic, group string) (state string, totalLags int64, err error) { + ctx := context.Background() + var ok bool + var descGroups kadm.DescribedGroups + var descGroup kadm.DescribedGroup + if descGroups, err = adm.DescribeGroups(ctx, group); err != nil { + err = errors.Wrapf(err, "") return } - partitions = len(topicsMeta[0].Partitions) - - // Get partitions' oldest and newest offset - var oldestOffsets, newestOffsets []int64 - var oldestOffset, newestOffset int64 - for partition := 0; partition < partitions; partition++ { - oldestOffset, err = client.GetOffset(topic, int32(partition), sarama.OffsetOldest) - if err != nil { - err = errors.Wrapf(err, "failed to get oldest offset for topic %q partition %q", topic, partition) - return - } - newestOffset, err = client.GetOffset(topic, int32(partition), sarama.OffsetNewest) - if err != nil { - err = errors.Wrapf(err, "failed to get newest offset for topic %q partition %q", topic, partition) - return - } - oldestOffsets = append(oldestOffsets, oldestOffset) - newestOffsets = append(newestOffsets, newestOffset) + if descGroup, ok = descGroups[group]; ok { + state = descGroup.State + } else { + state = "NA" } - - // Get consumer group' offset, calculate lag - pidList := make([]int32, partitions) - for partition := 0; partition < partitions; partition++ { - pidList[partition] = int32(partition) + var commit kadm.OffsetResponses + if commit, err = adm.FetchOffsets(ctx, group); err != nil { + err = errors.Wrapf(err, "") + return } - var rep *sarama.OffsetFetchResponse - if rep, err = adminClient.ListConsumerGroupOffsets(group, map[string][]int32{topic: pidList}); err != nil { - for partition := 0; partition < partitions; partition++ { - totalLags += newestOffsets[partition] - oldestOffsets[partition] + 1 - } - } else { - for partition := 0; partition < partitions; partition++ { - block := rep.GetBlock(topic, int32(partition)) - lag := newestOffsets[partition] - block.Offset - 1 - if lag > 0 { - totalLags += lag + var offsets kadm.ListedOffsets + if offsets, err = adm.ListEndOffsets(ctx, topic); err != nil { + err = errors.Wrapf(err, "") + return + } + grpLag := kadm.CalculateGroupLag(descGroup, commit, offsets) + if topLag, ok := grpLag[topic]; ok { + for _, grpMemberLag := range topLag { + if grpMemberLag.Lag >= 0 { + totalLags += grpMemberLag.Lag } } } diff --git a/go.mod b/go.mod index 5ccb6042..a73f502a 100644 --- a/go.mod +++ b/go.mod @@ -22,6 +22,7 @@ require ( github.com/tidwall/gjson v1.14.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/twmb/franz-go v1.6.0 + github.com/twmb/franz-go/pkg/kadm v1.1.1 github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 github.com/twmb/franz-go/plugin/kzap v1.1.0 github.com/valyala/fastjson v1.6.3 diff --git a/go.sum b/go.sum index e0c10782..40f5b9e0 100644 --- a/go.sum +++ b/go.sum @@ -379,6 +379,8 @@ github.com/twmb/franz-go v1.5.0/go.mod h1:ZKQ5AtqBbdc783bLCay7nDc21lJnIIA8mFJYhL github.com/twmb/franz-go v1.5.3/go.mod h1:eqHYpAuvlTArOdZ1XtPYyOQ1uUb40CSZwbpL3ccjibI= github.com/twmb/franz-go v1.6.0 h1:yri7YsVBe/k1LKcoZSLILgUI3U14e82qtD9i4VOcs9c= github.com/twmb/franz-go v1.6.0/go.mod h1:xdMwpUIQL/JDKKwerc5qJQG8TU1SNIddfjKJJyqRJIg= +github.com/twmb/franz-go/pkg/kadm v1.1.1 h1:tqcJt9ChdqiY+Vi3F13z8/XRAJozbJNj0/7VPnrdTQA= +github.com/twmb/franz-go/pkg/kadm v1.1.1/go.mod h1:Ly8COloKx7pbwBdlP4qTYKdEVcNvk7D3+in3ujv3x/M= github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.1.0 h1:csckTxG48q7Tem7ZwMxe2jAb0ehDNglxZccGnpqe4RU= github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 820a41d3..ed67973a 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -73,22 +73,35 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn k.putFn = putFn k.cleanupFn = cleanupFn kfkCfg := &cfg.Kafka - - opts := []kgo.Opt{ - kgo.SeedBrokers(strings.Split(kfkCfg.Brokers, ",")...), + var opts []kgo.Opt + if opts, err = GetFranzConfig(kfkCfg); err != nil { + return + } + opts = append(opts, kgo.ConsumeTopics(taskCfg.Topic), kgo.ConsumerGroup(taskCfg.ConsumerGroup), + kgo.OnPartitionsRevoked(k.onPartitionRevoked)) + if !taskCfg.Earliest { + opts = append(opts, kgo.ConsumeResetOffset(kgo.NewOffset().AtEnd())) + } + + if k.cl, err = kgo.NewClient(opts...); err != nil { + err = errors.Wrap(err, "") + return + } + return nil +} + +func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { + opts = []kgo.Opt{ + kgo.SeedBrokers(strings.Split(kfkCfg.Brokers, ",")...), kgo.DisableAutoCommit(), - kgo.OnPartitionsRevoked(k.onPartitionRevoked), kgo.MaxConcurrentFetches(3), kgo.FetchMaxBytes(1 << 27), //134 MB kgo.BrokerMaxReadBytes(1 << 27), //134 MB //kgo.MetadataMaxAge(...) corresponds to sarama.Config.Metadata.RefreshFrequency kgo.WithLogger(kzap.New(util.Logger)), } - if !taskCfg.Earliest { - opts = append(opts, kgo.ConsumeResetOffset(kgo.NewOffset().AtEnd())) - } if kfkCfg.TLS.Enable { var tlsCfg *tls.Config if tlsCfg, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.EndpIdentAlgo == ""); err != nil { @@ -143,12 +156,7 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn opts = append(opts, kgo.SASL(mch)) } } - - if k.cl, err = kgo.NewClient(opts...); err != nil { - err = errors.Wrap(err, "") - return - } - return nil + return } // kafka main loop From d354aa4c59a95a1b5d1e325d53f9a927e9dcbc9e Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 1 Jul 2022 13:28:32 +0800 Subject: [PATCH 254/404] use clickhouse-go/v2 write native --- output/clickhouse.go | 42 ++++++++--------- output/clickhouse_util.go | 67 ++++++++++++--------------- pool/conn.go | 95 +++++++++++++++++---------------------- 3 files changed, 88 insertions(+), 116 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 29f58f1b..157df221 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -17,7 +17,6 @@ package output import ( "context" - "database/sql" "fmt" "io" "math" @@ -27,6 +26,9 @@ import ( "sync" "time" + "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + + "github.com/ClickHouse/clickhouse-go/v2" "github.com/RoaringBitmap/roaring/roaring64" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" @@ -111,7 +113,7 @@ func (c *ClickHouse) Send(batch *model.Batch) { }) } -func (c *ClickHouse) writeSeries(rows model.Rows, conn *sql.DB) (err error) { +func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err error) { var seriesRows model.Rows c.mux.Lock() for _, row := range rows { @@ -138,7 +140,7 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( if len(*batch.Rows) == 0 { return } - var conn *sql.DB + var conn clickhouse.Conn if conn, *dbVer, err = sc.NextGoodReplica(*dbVer); err != nil { return } @@ -197,7 +199,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { } } -func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { +func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { tbl := c.seriesTbl if c.cfg.Clickhouse.Cluster != "" { tbl = c.distSeriesTbls[0] @@ -209,9 +211,9 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { } query := fmt.Sprintf("SELECT toUInt64(toInt64(__mgmt_id)) AS mid FROM %s.%s ORDER BY mid", c.cfg.Clickhouse.DB, tbl) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) - var rs *sql.Rows + var rs driver.Rows var mgmtID uint64 - if rs, err = conn.Query(query); err != nil { + if rs, err = conn.Query(context.Background(), query); err != nil { err = errors.Wrapf(err, "") return err } @@ -227,7 +229,7 @@ func (c *ClickHouse) initBmSeries(conn *sql.DB) (err error) { return } -func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { +func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { if !c.taskCfg.PrometheusSchema { c.IdxSerID = -1 return @@ -295,16 +297,13 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { // Generate SQL for series INSERT serDimsQuoted := make([]string, len(seriesDims)) - params := make([]string, len(seriesDims)) for i, serDim := range seriesDims { serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) - params[i] = "?" } - c.promSerSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s) VALUES (%s)", + c.promSerSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", c.cfg.Clickhouse.DB, c.seriesTbl, - strings.Join(serDimsQuoted, ","), - strings.Join(params, ",")) + strings.Join(serDimsQuoted, ",")) // Check distributed series table if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { @@ -326,7 +325,7 @@ func (c *ClickHouse) initSeriesSchema(conn *sql.DB) (err error) { func (c *ClickHouse) initSchema() (err error) { sc := pool.GetShardConn(0) - var conn *sql.DB + var conn clickhouse.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } @@ -355,16 +354,13 @@ func (c *ClickHouse) initSchema() (err error) { numDims = c.IdxSerID + 1 } quotedDms := make([]string, numDims) - params := make([]string, numDims) for i := 0; i < numDims; i++ { quotedDms[i] = fmt.Sprintf("`%s`", c.Dims[i].Name) - params[i] = "?" } - c.prepareSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s) VALUES (%s)", + c.prepareSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", c.cfg.Clickhouse.DB, c.taskCfg.TableName, - strings.Join(quotedDms, ","), - strings.Join(params, ",")) + strings.Join(quotedDms, ",")) util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.taskCfg.Name)) // Check distributed metric table @@ -447,13 +443,13 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } sort.Strings(queries) sc := pool.GetShardConn(0) - var conn *sql.DB + var conn clickhouse.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } for _, query := range queries { util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) - if _, err = conn.Exec(query); err != nil { + if err = conn.Exec(context.Background(), query); err != nil { err = errors.Wrapf(err, query) return } @@ -477,7 +473,7 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { taskCfg := c.taskCfg chCfg := &c.cfg.Clickhouse sc := pool.GetShardConn(0) - var conn *sql.DB + var conn clickhouse.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } @@ -485,8 +481,8 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { chCfg.DB, chCfg.Cluster, chCfg.DB, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) - var rows *sql.Rows - if rows, err = conn.Query(query); err != nil { + var rows driver.Rows + if rows, err = conn.Query(context.Background(), query); err != nil { err = errors.Wrapf(err, "") return } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 637ffb7f..90cbcc66 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -1,10 +1,11 @@ package output import ( - "database/sql" + "context" "fmt" "github.com/ClickHouse/clickhouse-go/v2" + "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/RoaringBitmap/roaring" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" @@ -16,7 +17,7 @@ import ( func shouldReconnect(err error, sc *pool.ShardConn) bool { var exp *clickhouse.Exception if errors.As(err, &exp) { - util.Logger.Error("this is an exception from clickhouse-server", zap.String("dsn", sc.GetDsn()), zap.Reflect("exception", exp)) + util.Logger.Error("this is an exception from clickhouse-server", zap.String("replica", sc.GetReplica()), zap.Reflect("exception", exp)) var replicaSpecific bool for _, ec := range replicaSpecificErrorCodes { if ec == exp.Code { @@ -29,74 +30,62 @@ func shouldReconnect(err error, sc *pool.ShardConn) bool { return true } -func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn *sql.DB) (numBad int, err error) { - var stmt *sql.Stmt - var tx *sql.Tx +func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn clickhouse.Conn) (numBad int, err error) { var errExec error - if tx, err = conn.Begin(); err != nil { - err = errors.Wrapf(err, "conn.Begin %s", prepareSQL) + var batch driver.Batch + if batch, err = conn.PrepareBatch(context.Background(), prepareSQL); err != nil { + err = errors.Wrapf(err, "clickhouse.Conn.PrepareBatch %s", prepareSQL) return } - if stmt, err = tx.Prepare(prepareSQL); err != nil { - err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) - _ = tx.Rollback() - return - } - defer stmt.Close() + defer func() { + _ = batch.Abort() + }() var bmBad *roaring.Bitmap for i, row := range rows { - if _, err = stmt.Exec((*row)[idxBegin:idxEnd]...); err != nil { + if err = batch.Append((*row)[idxBegin:idxEnd]...); err != nil { if bmBad == nil { - errExec = errors.Wrapf(err, "stmt.Exec") + errExec = errors.Wrapf(err, "driver.Batch.Append") bmBad = roaring.NewBitmap() } bmBad.AddInt(i) } } if errExec != nil { - stmt.Close() - _ = tx.Rollback() + _ = batch.Abort() numBad = int(bmBad.GetCardinality()) util.Logger.Warn(fmt.Sprintf("writeRows skipped %d rows of %d due to invalid content", numBad, len(rows)), zap.Error(errExec)) // write rows again, skip bad ones - if tx, err = conn.Begin(); err != nil { - err = errors.Wrapf(err, "conn.Begin %s", prepareSQL) - return - } - if stmt, err = tx.Prepare(prepareSQL); err != nil { - err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) - _ = tx.Rollback() + if batch, err = conn.PrepareBatch(context.Background(), prepareSQL); err != nil { + err = errors.Wrapf(err, "clickhouse.Conn.PrepareBatch %s", prepareSQL) return } - defer stmt.Close() + defer func() { + _ = batch.Abort() + }() for i, row := range rows { if !bmBad.ContainsInt(i) { - if _, err = stmt.Exec((*row)[idxBegin:idxEnd]...); err != nil { + if err = batch.Append((*row)[idxBegin:idxEnd]...); err != nil { err = errors.Wrapf(err, "stmt.Exec") break } } } - if err != nil { - _ = tx.Rollback() - return - } - if err = tx.Commit(); err != nil { - err = errors.Wrapf(err, "tx.Commit") + if err = batch.Send(); err != nil { + err = errors.Wrapf(err, "driver.Batch.Send") return } return } - if err = tx.Commit(); err != nil { - err = errors.Wrapf(err, "tx.Commit") + if err = batch.Send(); err != nil { + err = errors.Wrapf(err, "driver.Batch.Send") return } return } -func getDims(database, table string, excludedColumns []string, conn *sql.DB) (dims []*model.ColumnWithType, err error) { - var rs *sql.Rows - if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, database, table)); err != nil { +func getDims(database, table string, excludedColumns []string, conn clickhouse.Conn) (dims []*model.ColumnWithType, err error) { + var rs driver.Rows + if rs, err = conn.Query(context.Background(), fmt.Sprintf(selectSQLTemplate, database, table)); err != nil { err = errors.Wrapf(err, "") return } @@ -122,7 +111,7 @@ func getDims(database, table string, excludedColumns []string, conn *sql.DB) (di return } -func recreateDistTbls(cluster, database, table string, distTbls []string, conn *sql.DB) (err error) { +func recreateDistTbls(cluster, database, table string, distTbls []string, conn clickhouse.Conn) (err error) { var queries []string for _, distTbl := range distTbls { queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS `%s`.`%s` ON CLUSTER `%s`", database, distTbl, cluster)) @@ -132,7 +121,7 @@ func recreateDistTbls(cluster, database, table string, distTbls []string, conn * } for _, query := range queries { util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) - if _, err = conn.Exec(query); err != nil { + if err = conn.Exec(context.Background(), query); err != nil { err = errors.Wrapf(err, "") return } diff --git a/pool/conn.go b/pool/conn.go index 6313cc10..ceb1c6da 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -19,42 +19,42 @@ package pool // Clickhouse connection pool import ( - "database/sql" + "crypto/tls" "fmt" "net/url" - "strconv" "sync" "time" - "github.com/housepower/clickhouse_sinker/health" + "github.com/ClickHouse/clickhouse-go/v2" "github.com/housepower/clickhouse_sinker/util" "github.com/pkg/errors" - "github.com/troian/healthcheck" "go.uber.org/zap" ) var ( lock sync.Mutex clusterConn []*ShardConn - dsnSuffix string ) // ShardConn a datastructure for storing the clickhouse connection type ShardConn struct { - lock sync.Mutex - db *sql.DB - dbVer int - dsn string - replicas []string //ip:port list of replicas - maxOpenConns int - nextRep int //index of next replica + lock sync.Mutex + db clickhouse.Conn + dbVer int + opts clickhouse.Options + replicas []string //ip:port list of replicas + nextRep int //index of next replica } -// Close closes the current replica connection -func (sc *ShardConn) GetDsn() string { +// GetReplica returns the replica to which db connects +func (sc *ShardConn) GetReplica() (replica string) { sc.lock.Lock() defer sc.lock.Unlock() - return sc.dsn + if sc.db != nil { + curRep := (len(sc.replicas) + sc.nextRep - 1) % len(sc.replicas) + replica = sc.replicas[curRep] + } + return } // Close closes the current replica connection @@ -64,14 +64,11 @@ func (sc *ShardConn) Close() { if sc.db != nil { sc.db.Close() sc.db = nil - if err := health.Health.RemoveReadinessCheck(sc.dsn); err != nil { - util.Logger.Error("health.Health.RemoveReadinessCheck failed", zap.String("dsn", sc.dsn), zap.Error(err)) - } } } // NextGoodReplica connects to next good replica -func (sc *ShardConn) NextGoodReplica(failedVer int) (db *sql.DB, dbVer int, err error) { +func (sc *ShardConn) NextGoodReplica(failedVer int) (db clickhouse.Conn, dbVer int, err error) { sc.lock.Lock() defer sc.lock.Unlock() if sc.db != nil { @@ -84,60 +81,34 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db *sql.DB, dbVer int, err // conn4 = NextGood(ts2) will close the good connection and break users. return sc.db, sc.dbVer, nil } - if err := health.Health.RemoveReadinessCheck(sc.dsn); err != nil { - util.Logger.Warn("health.Health.RemoveReadinessCheck failed", zap.String("dsn", sc.dsn), zap.Error(err)) - } sc.db.Close() sc.db = nil } savedNextRep := sc.nextRep // try all replicas, including the current one for i := 0; i < len(sc.replicas); i++ { - sc.dsn = fmt.Sprintf("tcp://%s", sc.replicas[sc.nextRep]) + dsnSuffix + replica := sc.replicas[sc.nextRep] + sc.opts.Addr = []string{replica} sc.nextRep = (sc.nextRep + 1) % len(sc.replicas) - sqlDB, err := sql.Open("clickhouse", sc.dsn) + sc.db, err = clickhouse.Open(&sc.opts) if err != nil { - util.Logger.Warn("sql.Open failed", zap.String("dsn", sc.dsn), zap.Error(err)) + util.Logger.Warn("clickhouse.Open failed", zap.String("replica", replica), zap.Error(err)) continue } - // According to sql.Open doc, "Open may just validate its arguments without creating a connection - // to the database. To verify that the data source name is valid, call Ping." - if err := sqlDB.Ping(); err != nil { - util.Logger.Warn("sqlDB.Ping failed", zap.String("dsn", sc.dsn), zap.Error(err)) - continue - } - - // WARN: - // If the number of concurrent INSERT is close to clickhouse max_concurrent_queries(default 100), user queries could fail due to the limit. - sqlDB.SetMaxOpenConns(sc.maxOpenConns) - sqlDB.SetMaxIdleConns(0) - sqlDB.SetConnMaxIdleTime(10 * time.Second) - sc.db = sqlDB sc.dbVer++ - util.Logger.Info("sql.Open and sqlDB.Ping succeeded", zap.Int("dbVer", sc.dbVer), zap.String("dsn", sc.dsn)) - if err = health.Health.AddReadinessCheck(sc.dsn, healthcheck.DatabasePingCheck(sqlDB, 30*time.Second)); err != nil { - util.Logger.Warn("health.Health.AddReadinessCheck failed", zap.String("dsn", sc.dsn), zap.Error(err)) - } + util.Logger.Info("clickhouse.Open succeeded", zap.Int("dbVer", sc.dbVer), zap.String("replica", replica)) return sc.db, sc.dbVer, nil } err = errors.Errorf("no good replica among replicas %v since %d", sc.replicas, savedNextRep) return nil, sc.dbVer, err } +// Each shard has a clickhouse.Conn which connects to one replica inside the shard. +// We need more control than replica single-point-failure. func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool, maxOpenConns int) (err error) { lock.Lock() defer lock.Unlock() freeClusterConn() - // Each shard has a *sql.DB which connects to one replica inside the shard. - // "hosts" tolerates replica single-point-failure. However more flexable switching is needed for some cases for example https://github.com/ClickHouse/ClickHouse/issues/24036. - dsnSuffix = fmt.Sprintf("?database=%s&username=%s&password=%s", - url.QueryEscape(db), url.QueryEscape(username), url.QueryEscape(password)) - if dsnParams != "" { - dsnSuffix += "&" + dsnParams - } - if secure { - dsnSuffix += "&secure=true&skip_verify=" + strconv.FormatBool(skipVerify) - } for _, replicas := range hosts { numReplicas := len(replicas) @@ -149,8 +120,24 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) } sc := &ShardConn{ - replicas: replicaAddrs, - maxOpenConns: maxOpenConns, + replicas: replicaAddrs, + opts: clickhouse.Options{ + Auth: clickhouse.Auth{ + Database: url.QueryEscape(db), + Username: url.QueryEscape(username), + Password: url.QueryEscape(password), + }, + //Debug: true, + DialTimeout: time.Second, + MaxOpenConns: maxOpenConns, + MaxIdleConns: 1, + ConnMaxLifetime: time.Hour, + }, + } + if secure { + tlsConfig := &tls.Config{} + tlsConfig.InsecureSkipVerify = skipVerify + sc.opts.TLS = tlsConfig } if _, _, err = sc.NextGoodReplica(0); err != nil { return From 019fa9e30e903c3051e1568f145d3e1ce48adcb5 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 1 Jul 2022 15:21:39 +0800 Subject: [PATCH 255/404] replaced github.com/pkg/errors with github.com/thanos-io/thanos/pkg/errors, close #156 --- cmd/kafka_gen_log/main.go | 4 ++-- cmd/kafka_gen_metric/main.go | 2 +- config/config.go | 10 +++++----- config_manager/lags.go | 6 +++--- config_manager/nacos.go | 2 +- go.mod | 5 +++-- go.sum | 11 +++++++++-- input/kafka_franz.go | 12 ++++++------ input/kafka_go.go | 6 +++--- input/kafka_sarama.go | 4 ++-- output/clickhouse.go | 14 +++++++------- output/clickhouse_util.go | 2 +- parser/csv.go | 6 +++--- parser/fastjson.go | 2 +- parser/parser.go | 4 ++-- pool/conn.go | 4 ++-- statistics/statistics.go | 4 ++-- task/ring.go | 4 ++-- task/sharding.go | 12 ++++++------ task/task.go | 6 +++--- util/common.go | 2 +- util/marshal_amd64.go | 2 +- util/marshal_arm64.go | 2 +- 23 files changed, 67 insertions(+), 59 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 6ae2d8c6..3fe7e357 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -55,7 +55,7 @@ import ( "github.com/Shopify/sarama" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" ) @@ -174,7 +174,7 @@ func (g *LogGenerator) Init() error { } } if g.logfiles == nil || len(g.logfiles) == 0 { - err := errors.Errorf("There is no files under %v match pattern %v", LogfileDir, LogfilePattern) + err := errors.Newf("There is no files under %v match pattern %v", LogfileDir, LogfilePattern) return err } sort.Strings(g.logfiles) diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index 91aa7bf4..bd671a93 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -47,7 +47,7 @@ import ( "github.com/Shopify/sarama" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" ) diff --git a/config/config.go b/config/config.go index e6f5ef6f..ddb50b76 100644 --- a/config/config.go +++ b/config/config.go @@ -23,7 +23,7 @@ import ( "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" ) // Config struct used for different configurations use @@ -189,7 +189,7 @@ func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { // normallize and validate configuration func (cfg *Config) Normallize() (err error) { if len(cfg.Clickhouse.Hosts) == 0 || cfg.Kafka.Brokers == "" { - err = errors.Errorf("invalid configuration") + err = errors.Newf("invalid configuration") return } if cfg.Kafka.Version == "" { @@ -218,7 +218,7 @@ func (cfg *Config) Normallize() (err error) { switch cfg.Kafka.Sasl.Mechanism { case "PLAIN", "SCRAM-SHA-256", "SCRAM-SHA-512", "GSSAPI": default: - err = errors.Errorf("kafka SASL mechanism %s is unsupported", cfg.Kafka.Sasl.Mechanism) + err = errors.Newf("kafka SASL mechanism %s is unsupported", cfg.Kafka.Sasl.Mechanism) return } } @@ -289,7 +289,7 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { } if taskCfg.DynamicSchema.Enable { if taskCfg.Parser != "fastjson" && taskCfg.Parser != "gjson" { - err = errors.Errorf("Parser %s doesn't support DynamicSchema", taskCfg.Parser) + err = errors.Newf("Parser %s doesn't support DynamicSchema", taskCfg.Parser) return } if cfg.Clickhouse.Cluster == "" { @@ -298,7 +298,7 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { numHosts += len(shard) } if numHosts > 1 { - err = errors.Errorf("Need to set cluster name when DynamicSchema is enabled and number of hosts is more than one") + err = errors.Newf("Need to set cluster name when DynamicSchema is enabled and number of hosts is more than one") return } } diff --git a/config_manager/lags.go b/config_manager/lags.go index 44fa1eb5..d298c84f 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -5,7 +5,7 @@ import ( "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/input" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "github.com/twmb/franz-go/pkg/kadm" "github.com/twmb/franz-go/pkg/kgo" ) @@ -55,7 +55,7 @@ func GetTaskStateAndLag(cfg *config.Config, taskName string) (stateLag StateLag, } } if taskCfg == nil { - err = errors.Errorf("task %q doesn't exist", taskName) + err = errors.Newf("task %q doesn't exist", taskName) return } if stateLag.State, stateLag.Lag, err = getStateAndLag(adm, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { @@ -70,7 +70,7 @@ func newClient(cfg *config.Config) (cl *kgo.Client, adm *kadm.Client, err error) return } if cl, err = kgo.NewClient(opts...); err != nil { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") return } adm = kadm.NewClient(cl) diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 386e58a3..3f87038d 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -21,7 +21,7 @@ import ( "github.com/nacos-group/nacos-sdk-go/common/constant" "github.com/nacos-group/nacos-sdk-go/model" "github.com/nacos-group/nacos-sdk-go/vo" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" ) diff --git a/go.mod b/go.mod index a73f502a..a678cda6 100644 --- a/go.mod +++ b/go.mod @@ -13,12 +13,12 @@ require ( github.com/jcmturner/gokrb5/v8 v8.4.2 github.com/jinzhu/copier v0.3.5 github.com/nacos-group/nacos-sdk-go v1.1.1 - github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.12.2 github.com/prometheus/common v0.34.0 github.com/segmentio/kafka-go v0.4.32 github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.7.5 + github.com/thanos-io/thanos v0.27.0-rc.0 github.com/tidwall/gjson v1.14.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/twmb/franz-go v1.6.0 @@ -57,12 +57,13 @@ require ( github.com/json-iterator/go v1.1.12 // indirect github.com/klauspost/compress v1.15.6 // indirect github.com/klauspost/cpuid/v2 v2.0.13 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/paulmach/orb v0.7.1 // indirect github.com/pierrec/lz4/v4 v4.1.15 // indirect + github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/procfs v0.7.3 // indirect diff --git a/go.sum b/go.sum index 40f5b9e0..d5280b55 100644 --- a/go.sum +++ b/go.sum @@ -87,6 +87,7 @@ github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ3 github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dennwc/varint v1.0.0 h1:kGNFFSSw8ToIy3obO/kKr8U9GZYUAxQEVuix4zfDWzE= github.com/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q= github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= @@ -110,10 +111,12 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2 github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= +github.com/go-kit/log v0.2.0 h1:7i2K3eKTos3Vc0enKCfnVcgHh2olr/MyfboYq7cAcFw= github.com/go-kit/log v0.2.0/go.mod h1:NwTd00d/i8cPZ3xOwwiv2PO5MOcx78fFErGNcVmBjv0= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-logfmt/logfmt v0.5.1 h1:otpy5pqBCBZ1ng9RQ0dPu4PN7ba75Y/aA+UpowDyNVA= github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -264,8 +267,9 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= -github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI= +github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mkevac/debugcharts v0.0.0-20191222103121-ae1c48aa8615/go.mod h1:Ad7oeElCZqA1Ufj0U9/liOF4BtVepxRcTvr2ey7zTvM= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= @@ -321,6 +325,7 @@ github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4O github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/procfs v0.7.3 h1:4jVXhlkAyzOScmCkXBTOLRLTz8EeU+eyjrwB/EPq0VU= github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/prometheus/prometheus v1.8.2-0.20220308163432-03831554a519 h1:MSR9EaMnwV1nsmhQ2eB1Txwebyk/5ulUGcgcYi3ofL4= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= @@ -357,6 +362,8 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.5 h1:s5PTfem8p8EbKQOctVV53k6jCJt3UX4IEJzwh+C324Q= github.com/stretchr/testify v1.7.5/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/thanos-io/thanos v0.27.0-rc.0 h1:USgjag/539MYqPRr4dUcoYHmVjFOrPe17NgTHzcEB0c= +github.com/thanos-io/thanos v0.27.0-rc.0/go.mod h1:i0KCpF5fWAWBNZXBkuzGh0603YlMrbhLJdwOO5W21YI= github.com/tidwall/gjson v1.12.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/gjson v1.13.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/gjson v1.14.1 h1:iymTbGkQBhveq21bEvAQ81I0LEBork8BFe1CUZXdyuo= @@ -424,8 +431,8 @@ go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.11 h1:wy28qYRKZgnJTxGxvye5/wgWr1EKjmUDGYox5mGlRlI= go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= +go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/multierr v1.8.0 h1:dg6GjLku4EH+249NNmoIciG9N/jURbDG+pFlTkhzIC8= diff --git a/input/kafka_franz.go b/input/kafka_franz.go index ed67973a..258c2f5e 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -25,7 +25,7 @@ import ( krb5client "github.com/jcmturner/gokrb5/v8/client" krb5config "github.com/jcmturner/gokrb5/v8/config" "github.com/jcmturner/gokrb5/v8/keytab" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "github.com/twmb/franz-go/pkg/kgo" "github.com/twmb/franz-go/pkg/sasl" "github.com/twmb/franz-go/pkg/sasl/kerberos" @@ -86,7 +86,7 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn } if k.cl, err = kgo.NewClient(opts...); err != nil { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") return } return nil @@ -137,12 +137,12 @@ func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { var krbCfg *krb5config.Config var kt *keytab.Keytab if krbCfg, err = krb5config.Load(gssapiCfg.KerberosConfigPath); err != nil { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") return } if gssapiCfg.AuthType == Krb5KeytabAuth { if kt, err = keytab.Load(gssapiCfg.KeyTabPath); err != nil { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") return } auth.Client = krb5client.NewWithKeytab(gssapiCfg.Username, gssapiCfg.Realm, kt, krbCfg, krb5client.DisablePAFXFAST(gssapiCfg.DisablePAFXFAST)) @@ -173,7 +173,7 @@ func (k *KafkaFranz) Run() { if errors.Is(err, context.Canceled) { beCanceled = true } else { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") util.Logger.Info("kgo.Client.PollFetchs() failed", zap.Error(err)) } }) @@ -204,7 +204,7 @@ func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { if err == nil { break } - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") if i < CommitRetries-1 && !errors.Is(err, context.Canceled) { util.Logger.Error("cl.CommitRecords failed, will retry later", zap.String("task", k.taskCfg.Name), zap.Int("try", i), zap.Error(err)) time.Sleep(RetryBackoff) diff --git a/input/kafka_go.go b/input/kafka_go.go index 36e986b7..87f5f897 100644 --- a/input/kafka_go.go +++ b/input/kafka_go.go @@ -23,10 +23,10 @@ import ( "sync" "time" - "github.com/pkg/errors" "github.com/segmentio/kafka-go" "github.com/segmentio/kafka-go/sasl/plain" "github.com/segmentio/kafka-go/sasl/scram" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" "github.com/housepower/clickhouse_sinker/config" @@ -120,7 +120,7 @@ func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn fun return } default: - return errors.Errorf("kafka-go doesn't support SASL/%s authentication", kfkCfg.Sasl.Mechanism) + return errors.Newf("kafka-go doesn't support SASL/%s authentication", kfkCfg.Sasl.Mechanism) } } if dialer != nil { @@ -143,7 +143,7 @@ LOOP_KAFKA_GO: break LOOP_KAFKA_GO } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.taskCfg.Name).Inc() - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") util.Logger.Error("kafka.Reader.FetchMessage failed", zap.String("task", k.taskCfg.Name), zap.Error(err)) continue } diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go index 706418ad..28507ca4 100644 --- a/input/kafka_sarama.go +++ b/input/kafka_sarama.go @@ -25,7 +25,7 @@ import ( "time" "github.com/Shopify/sarama" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "github.com/xdg-go/scram" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -169,7 +169,7 @@ LOOP_SARAMA: break LOOP_SARAMA } else { statistics.ConsumeMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") util.Logger.Error("sarama.ConsumerGroup.Consume failed", zap.String("task", k.taskCfg.Name), zap.Error(err)) continue } diff --git a/output/clickhouse.go b/output/clickhouse.go index 157df221..eb113705 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -35,12 +35,12 @@ import ( "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" ) var ( - ErrTblNotExist = errors.Errorf("table doesn't exist") + ErrTblNotExist = errors.Newf("table doesn't exist") selectSQLTemplate = `select name, type, default_kind from system.columns where database = '%s' and table = '%s'` lowCardinalityRegexp = regexp.MustCompile(`LowCardinality\((.+)\)`) @@ -248,7 +248,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { } } if dimSerID == nil { - err = errors.Errorf("Metric table %s shall have column `__series_id UInt64`.", c.taskCfg.TableName) + err = errors.Newf("Metric table %s shall have column `__series_id UInt64`.", c.taskCfg.TableName) return } c.IdxSerID = len(c.Dims) @@ -282,7 +282,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { } } if badFirst { - err = errors.Errorf(`First columns of %s are expect to be "__series_id Int64, __mgmt_id Int64, labels String".`, c.seriesTbl) + err = errors.Newf(`First columns of %s are expect to be "__series_id Int64, __mgmt_id Int64, labels String".`, c.seriesTbl) return } c.NameKey = "__name__" // prometheus uses internal "__name__" label for metric name @@ -311,7 +311,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { return } if c.distSeriesTbls == nil { - err = errors.Errorf("Please create distributed table for %s.", c.seriesTbl) + err = errors.Newf("Please create distributed table for %s.", c.seriesTbl) return } } @@ -369,7 +369,7 @@ func (c *ClickHouse) initSchema() (err error) { return } if c.distMetricTbls == nil { - err = errors.Errorf("Please create distributed table for %s.", c.taskCfg.TableName) + err = errors.Newf("Please create distributed table for %s.", c.taskCfg.TableName) return } } @@ -422,7 +422,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { case model.DateTimeArray: strVal = "Array(DateTime64(3))" default: - err = errors.Errorf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) + err = errors.Newf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) return false } if c.taskCfg.PrometheusSchema { diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 90cbcc66..df7896a2 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -10,7 +10,7 @@ import ( "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" ) diff --git a/parser/csv.go b/parser/csv.go index 8251ed8d..02e4018c 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -25,8 +25,8 @@ import ( "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" "github.com/shopspring/decimal" + "github.com/thanos-io/thanos/pkg/errors" "github.com/tidwall/gjson" "github.com/valyala/fastjson/fastfloat" ) @@ -47,11 +47,11 @@ func (p *CsvParser) Parse(bs []byte) (metric model.Metric, err error) { } var value []string if value, err = r.Read(); err != nil { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") return } if len(value) != len(p.pp.csvFormat) { - err = errors.Errorf("csv value doesn't match the format") + err = errors.Newf("csv value doesn't match the format") return } metric = &CsvMetric{p.pp, value} diff --git a/parser/fastjson.go b/parser/fastjson.go index 323a6bf1..dcf45422 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -23,8 +23,8 @@ import ( "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" "github.com/shopspring/decimal" + "github.com/thanos-io/thanos/pkg/errors" "github.com/valyala/fastjson" "go.uber.org/zap" ) diff --git a/parser/parser.go b/parser/parser.go index 011e3319..8234ba14 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -22,7 +22,7 @@ import ( "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" ) var ( @@ -62,7 +62,7 @@ var ( "Mon Jan 02, 2006", } Epoch = time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC) - ErrParseDateTime = errors.Errorf("value doesn't contain DateTime") + ErrParseDateTime = errors.Newf("value doesn't contain DateTime") ) // Parse is the Parser interface diff --git a/pool/conn.go b/pool/conn.go index ceb1c6da..125dd3d5 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -27,7 +27,7 @@ import ( "github.com/ClickHouse/clickhouse-go/v2" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" ) @@ -99,7 +99,7 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db clickhouse.Conn, dbVer i util.Logger.Info("clickhouse.Open succeeded", zap.Int("dbVer", sc.dbVer), zap.String("replica", replica)) return sc.db, sc.dbVer, nil } - err = errors.Errorf("no good replica among replicas %v since %d", sc.replicas, savedNextRep) + err = errors.Newf("no good replica among replicas %v since %d", sc.replicas, savedNextRep) return nil, sc.dbVer, err } diff --git a/statistics/statistics.go b/statistics/statistics.go index 136b8a17..461675d6 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -20,11 +20,11 @@ import ( "time" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/collectors" "github.com/prometheus/client_golang/prometheus/push" "github.com/prometheus/common/expfmt" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" ) @@ -189,7 +189,7 @@ func NewPusher(addrs []string, interval int, selfAddr string) *Pusher { } var ( - errPgwEmpty = errors.New("invalid configuration for pusher") + errPgwEmpty = errors.Newf("invalid configuration for pusher") ) func (p *Pusher) Init() error { diff --git a/task/ring.go b/task/ring.go index d643967d..9a4d75d0 100644 --- a/task/ring.go +++ b/task/ring.go @@ -7,7 +7,7 @@ import ( "time" "github.com/fagongzi/goetty" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" "github.com/housepower/clickhouse_sinker/model" @@ -160,7 +160,7 @@ func (ring *Ring) scheduleForchBatchOrShard() { if errors.Is(err, goetty.ErrSystemStopped) { util.Logger.Warn("Ring.ForceBatchOrShard scheduling timer to a stopped timer wheel", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) } else { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") util.Logger.Fatal("scheduling timer filed", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) } } diff --git a/task/sharding.go b/task/sharding.go index 0e3f1f6f..0d1a0eeb 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -11,7 +11,7 @@ import ( "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" ) @@ -37,13 +37,13 @@ func NewShardingPolicy(shardingKey string, shardingStripe uint64, dims []*model. //string policy.stripe = 0 default: - err = errors.Errorf("invalid shardingKey %s, expect its type be numerical or string", shardingKey) + err = errors.Newf("invalid shardingKey %s, expect its type be numerical or string", shardingKey) return } } } if colSeq < 0 { - err = errors.Errorf("invalid shardingKey %s, no such column", shardingKey) + err = errors.Newf("invalid shardingKey %s, no such column", shardingKey) return } policy.colSeq = colSeq @@ -82,7 +82,7 @@ func (policy *ShardingPolicy) Calc(row *model.Row) (shard int, err error) { case time.Time: valu64 = uint64(v.Unix()) default: - err = errors.Errorf("failed to convert %+v to integer", v) + err = errors.Newf("failed to convert %+v to integer", v) return } shard = int((valu64 / policy.stripe) % uint64(policy.shards)) @@ -94,7 +94,7 @@ func (policy *ShardingPolicy) Calc(row *model.Row) (shard int, err error) { case string: valu64 = xxhash.Sum64String(v) default: - err = errors.Errorf("failed to convert %+v to string", v) + err = errors.Newf("failed to convert %+v to string", v) return } shard = int(valu64 % uint64(policy.shards)) @@ -220,7 +220,7 @@ func (sh *Sharder) doFlush(_ interface{}) { if errors.Is(err, goetty.ErrSystemStopped) { util.Logger.Info("Sharder.doFlush scheduling timer to a stopped timer wheel") } else { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) } } diff --git a/task/task.go b/task/task.go index 907588b7..208fc29f 100644 --- a/task/task.go +++ b/task/task.go @@ -31,7 +31,7 @@ import ( "github.com/housepower/clickhouse_sinker/parser" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" "golang.org/x/time/rate" ) @@ -155,7 +155,7 @@ func (service *Service) Run() { if errors.Is(err, goetty.ErrSystemStopped) { util.Logger.Info("Service.Run scheduling timer to a stopped timer wheel") } else { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) } } @@ -323,7 +323,7 @@ func (service *Service) put(msg *model.InputMessage) { if errors.Is(err, goetty.ErrSystemStopped) { util.Logger.Info("Service.put scheduling timer to a stopped timer wheel") } else { - err = errors.Wrap(err, "") + err = errors.Wrapf(err, "") util.Logger.Fatal("scheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } diff --git a/util/common.go b/util/common.go index 90866616..c668b810 100644 --- a/util/common.go +++ b/util/common.go @@ -36,7 +36,7 @@ import ( "gopkg.in/natefinch/lumberjack.v2" "github.com/fagongzi/goetty" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" ) var ( diff --git a/util/marshal_amd64.go b/util/marshal_amd64.go index d197c0de..46446889 100644 --- a/util/marshal_amd64.go +++ b/util/marshal_amd64.go @@ -3,7 +3,7 @@ package util import ( "github.com/bytedance/sonic" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" ) func JSONMarshal(obj interface{}) (b []byte, err error) { diff --git a/util/marshal_arm64.go b/util/marshal_arm64.go index fc226c4f..e3c681a2 100644 --- a/util/marshal_arm64.go +++ b/util/marshal_arm64.go @@ -3,7 +3,7 @@ package util import ( "encoding/json" - "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/errors" ) func JSONMarshal(obj interface{}) (b []byte, err error) { From 20fa8e08f25e4710a0a7410456ace98a52f05902 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 1 Jul 2022 18:56:55 +0800 Subject: [PATCH 256/404] added kafka_gen_prom --- Makefile | 2 + cmd/kafka_gen_log/main.go | 2 +- cmd/kafka_gen_metric/main.go | 4 +- cmd/kafka_gen_prom/main.go | 212 +++++++++++++++++++++++++++++++++++ 4 files changed, 217 insertions(+), 3 deletions(-) create mode 100644 cmd/kafka_gen_prom/main.go diff --git a/Makefile b/Makefile index 2ac18758..e791d959 100644 --- a/Makefile +++ b/Makefile @@ -13,11 +13,13 @@ build: pre $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o nacos_publish_config cmd/nacos_publish_config/main.go $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o kafka_gen_log cmd/kafka_gen_log/main.go $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o kafka_gen_metric cmd/kafka_gen_metric/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o kafka_gen_prom cmd/kafka_gen_prom/main.go debug: pre $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o clickhouse_sinker cmd/clickhouse_sinker/main.go $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o nacos_publish_config cmd/nacos_publish_config/main.go $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o kafka_gen_log cmd/kafka_gen_log/main.go $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o kafka_gen_metric cmd/kafka_gen_metric/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o kafka_gen_prom cmd/kafka_gen_prom/main.go unittest: pre go test -v ./... benchtest: pre diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 3fe7e357..cfcf4e31 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -295,7 +295,7 @@ func main() { usage := fmt.Sprintf(`Usage of %s %s kakfa_brokers topic log_file_dir log_file_pattern This util read log from given paths, fill some fields with random content, serialize and send to kafka. -kakfa_brokers: for example, 192.168.102.114:9092,192.168.102.115:9092 +kakfa_brokers: for example, 192.168.110.8:9092,192.168.110.12:9092,192.168.110.16:9092 topic: for example, apache_access_log log_file_dir: log file directory, for example, /var/log log_file_pattern: file name pattern, for example, '^secure.*$'`, os.Args[0], os.Args[0]) diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index bd671a93..2771eebc 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -125,9 +125,9 @@ func generate() { for day := 0; ; day++ { tsDay := rounded.Add(time.Duration(24*day) * time.Hour) for step := 0; step < 24*60*60; step++ { + timestamp := tsDay.Add(time.Duration(step) * time.Second) for bus := 0; bus < BusinessNum; bus++ { for ins := 0; ins < InstanceNum; ins++ { - timestamp := tsDay.Add(time.Duration(step) * time.Second) metric := Metric{ Time: timestamp, ItemGUID: fmt.Sprintf("bus%03d_ins%03d", bus, ins), @@ -179,7 +179,7 @@ func main() { usage := fmt.Sprintf(`Usage of %s %s kakfa_brokers topic This util fill some fields with random content, serialize and send to kafka. -kakfa_brokers: for example, 192.168.102.114:9092,192.168.102.115:9092 +kakfa_brokers: for example, 192.168.110.8:9092,192.168.110.12:9092,192.168.110.16:9092 topic: for example, sensor_dt_result_online`, os.Args[0], os.Args[0]) util.Logger.Info(usage) os.Exit(0) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go new file mode 100644 index 00000000..d97ddecb --- /dev/null +++ b/cmd/kafka_gen_prom/main.go @@ -0,0 +1,212 @@ +package main + +/* +CREATE TABLE prom_extend ON CLUSTER abc ( + timestamp DateTime, + value Float64, + __name__ String +) ENGINE=ReplicatedMergeTree() +PARTITION BY toYYYYMMDD(timestamp) +ORDER BY (__name__, timestamp); + +CREATE TABLE dist_prom_extend ON CLUSTER abc AS prom_extend ENGINE = Distributed(abc, default, prom_extend); + +*/ + +import ( + "bytes" + "context" + "flag" + "fmt" + "math/rand" + "os" + "os/signal" + "strings" + "sync/atomic" + "syscall" + "time" + + "github.com/Shopify/sarama" + "github.com/google/gops/agent" + "github.com/housepower/clickhouse_sinker/util" + "github.com/thanos-io/thanos/pkg/errors" + "go.uber.org/zap" +) + +const ( + Alpha = "abcdefghijklmnopqrstuvwxyz" + NumMetrics = 1000000 + NumKeys = 10 + NumAllKeys = 1000 +) + +var ( + KafkaBrokers string + KafkaTopic string + gLines int64 + gSize int64 + metrics []PromMetric +) + +type Labels map[string]string + +type Datapoint struct { + Timestamp time.Time + Value float64 + Name string `json:"__name__"` + Labels +} + +// I need every label be present at the top level. +func (dp Datapoint) MarshalJSON() ([]byte, error) { + var bbuf bytes.Buffer + bbuf.WriteString(fmt.Sprintf(`{"timestamp":"%s", "value":%f,"__name__":"%s"`, dp.Timestamp.Format(time.RFC3339), dp.Value, dp.Name)) + for key, val := range dp.Labels { + bbuf.WriteString(fmt.Sprintf(`,"%s":"%s"`, key, val)) + } + bbuf.WriteByte('}') + return bbuf.Bytes(), nil +} + +type PromMetric struct { + Name string + LabelKeys []string +} + +func randValue() (val string) { + mod := rand.Intn(2) + var leng, maxN int + if mod == 0 { //10^5=100000 + leng = 5 + maxN = 10 + } else { //3^10 = 59049 + leng = 10 + maxN = 3 + } + b := make([]byte, leng) + for i := 0; i < leng; i++ { + b[i] = Alpha[rand.Intn(maxN+1)] + } + val = string(b) + return +} + +func initMetrics() { + metrics = make([]PromMetric, NumMetrics) + for i := 0; i < NumMetrics; i++ { + m := PromMetric{ + Name: fmt.Sprintf("metric-%08d", i), + LabelKeys: make([]string, NumKeys), + } + for j := 0; j < NumKeys; j++ { + key := fmt.Sprintf("key-%06d", rand.Intn(NumAllKeys+1)) + m.LabelKeys[j] = key + } + metrics[i] = m + } +} + +func generate() { + initMetrics() + toRound := time.Now().Add(time.Duration(-30*24) * time.Hour) + // refers to time.Time.Truncate + rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) + + wp := util.NewWorkerPool(10, 10000) + config := sarama.NewConfig() + config.Version = sarama.V2_1_0_0 + w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) + if err != nil { + util.Logger.Error("sarama.NewAsyncProducer failed", zap.Error(err)) + } + defer w.Close() + chInput := w.Input() + + for day := 0; ; day++ { + tsDay := rounded.Add(time.Duration(24*day) * time.Hour) + for step := 0; step < 24*60*60; step++ { + timestamp := tsDay.Add(time.Duration(step) * time.Second) + for i := 0; i < NumMetrics; i++ { + dp := Datapoint{ + Timestamp: timestamp, + Value: rand.Float64(), + Name: metrics[i].Name, + Labels: make(Labels), + } + for _, key := range metrics[i].LabelKeys { + dp.Labels[key] = randValue() + } + + _ = wp.Submit(func() { + var b []byte + if b, err = dp.MarshalJSON(); err != nil { + err = errors.Wrapf(err, "") + util.Logger.Fatal("got error", zap.Error(err)) + } + chInput <- &sarama.ProducerMessage{ + Topic: KafkaTopic, + Key: sarama.StringEncoder(dp.Name), + Value: sarama.ByteEncoder(b), + } + atomic.AddInt64(&gLines, int64(1)) + atomic.AddInt64(&gSize, int64(len(b))) + }) + } + } + } +} + +func main() { + util.InitLogger([]string{"stdout"}) + flag.Usage = func() { + usage := fmt.Sprintf(`Usage of %s + %s kakfa_brokers topic +This util fill some fields with random content, serialize and send to kafka. +kakfa_brokers: for example, 192.168.110.8:9092,192.168.110.12:9092,192.168.110.16:9092 +topic: for example, prom_extend`, os.Args[0], os.Args[0]) + util.Logger.Info(usage) + os.Exit(0) + } + flag.Parse() + args := flag.Args() + if len(args) != 2 { + flag.Usage() + } + KafkaBrokers = args[0] + KafkaTopic = args[1] + util.Logger.Info("CLI options", + zap.String("KafkaBrokers", KafkaBrokers), + zap.String("KafkaTopic", KafkaTopic), + zap.Int("NumMetrics", NumMetrics), + zap.Int("NumKeys", NumKeys), + zap.Int("NumAllKeys", NumAllKeys), + ) + + if err := agent.Listen(agent.Options{}); err != nil { + util.Logger.Fatal("got error", zap.Error(err)) + } + + var prevLines, prevSize int64 + ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) + go generate() + + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() +LOOP: + for { + select { + case <-ctx.Done(): + util.Logger.Info("quit due to context been canceled") + break LOOP + case <-ticker.C: + var speedLine, speedSize int64 + if gLines != 0 { + speedLine = (gLines - prevLines) / int64(10) + speedSize = (gSize - prevSize) / int64(10) + } + prevLines = gLines + prevSize = gSize + util.Logger.Info("status", zap.Int64("lines", gLines), zap.Int64("bytes", gSize), zap.Int64("speed(lines/s)", speedLine), zap.Int64("speed(bytes/s)", speedSize)) + } + } +} From e29f04eb56a0296ec5c9a85fc412955096ba7592 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 1 Jul 2022 23:03:47 +0800 Subject: [PATCH 257/404] kafka_gen_prom uses franz instead of sarama --- cmd/kafka_gen_prom/main.go | 38 +++++++++++++++++++++++--------------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index d97ddecb..15a01472 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -2,9 +2,9 @@ package main /* CREATE TABLE prom_extend ON CLUSTER abc ( - timestamp DateTime, - value Float64, - __name__ String + timestamp DateTime, + value Float64, + __name__ String ) ENGINE=ReplicatedMergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY (__name__, timestamp); @@ -26,10 +26,10 @@ import ( "syscall" "time" - "github.com/Shopify/sarama" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" + "github.com/twmb/franz-go/pkg/kgo" "go.uber.org/zap" ) @@ -113,14 +113,22 @@ func generate() { rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) wp := util.NewWorkerPool(10, 10000) - config := sarama.NewConfig() - config.Version = sarama.V2_1_0_0 - w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) - if err != nil { - util.Logger.Error("sarama.NewAsyncProducer failed", zap.Error(err)) + opts := []kgo.Opt{ + kgo.SeedBrokers(strings.Split(KafkaBrokers, ",")...), + } + var err error + var cl *kgo.Client + if cl, err = kgo.NewClient(opts...); err != nil { + util.Logger.Fatal("kgo.NewClient failed", zap.Error(err)) + } + defer cl.Close() + + ctx := context.Background() + produceCb := func(rec *kgo.Record, err error) { + if err != nil { + util.Logger.Fatal("kgo.Client.Produce failed", zap.Error(err)) + } } - defer w.Close() - chInput := w.Input() for day := 0; ; day++ { tsDay := rounded.Add(time.Duration(24*day) * time.Hour) @@ -143,11 +151,11 @@ func generate() { err = errors.Wrapf(err, "") util.Logger.Fatal("got error", zap.Error(err)) } - chInput <- &sarama.ProducerMessage{ + cl.Produce(ctx, &kgo.Record{ Topic: KafkaTopic, - Key: sarama.StringEncoder(dp.Name), - Value: sarama.ByteEncoder(b), - } + Key: []byte(dp.Name), + Value: b, + }, produceCb) atomic.AddInt64(&gLines, int64(1)) atomic.AddInt64(&gSize, int64(len(b))) }) From a6d4efdf884947dffef4ca7a98ea64dd8c15b4a0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 2 Jul 2022 11:06:16 +0800 Subject: [PATCH 258/404] introduced WritingDurations --- output/clickhouse.go | 4 ++++ statistics/statistics.go | 19 ++++++++++--------- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index eb113705..666336d2 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -124,6 +124,7 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err err } c.mux.Unlock() if len(seriesRows) != 0 { + begin := time.Now() var numBad int if numBad, err = writeRows(c.promSerSQL, seriesRows, c.IdxSerID, len(c.Dims), conn); err != nil { return @@ -131,6 +132,7 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err err if numBad != 0 { statistics.ParseMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(numBad)) } + statistics.WritingDurations.WithLabelValues(c.taskCfg.Name, c.seriesTbl).Observe(time.Since(begin).Seconds()) } return } @@ -153,10 +155,12 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( return } } + begin := time.Now() var numBad int if numBad, err = writeRows(c.prepareSQL, *batch.Rows, 0, numDims, conn); err != nil { return } + statistics.WritingDurations.WithLabelValues(c.taskCfg.Name, c.taskCfg.TableName).Observe(time.Since(begin).Seconds()) if numBad != 0 { statistics.ParseMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(numBad)) } diff --git a/statistics/statistics.go b/statistics/statistics.go index 461675d6..c1853db8 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -109,13 +109,6 @@ var ( }, []string{"task", "topic", "partition"}, ) - ClickhouseReconnectTotal = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Name: prefix + "clickhouse_reconnect_total", - Help: "total num of ClickHouse reconnects", - }, - []string{"task"}, - ) RingMsgs = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Name: prefix + "ring_msgs", @@ -144,6 +137,14 @@ var ( }, []string{"task"}, ) + WritingDurations = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Name: prefix + "writing_durations", + Help: "writing durations", + Buckets: []float64{1.0, 2.0, 4.0, 8.0, 16.0, 32.0, 64.0, 128.0}, + }, + []string{"task", "table"}, + ) ) func init() { @@ -158,11 +159,11 @@ func init() { prometheus.MustRegister(FlushMsgsTotal) prometheus.MustRegister(FlushMsgsErrorTotal) prometheus.MustRegister(ConsumeOffsets) - prometheus.MustRegister(ClickhouseReconnectTotal) prometheus.MustRegister(RingMsgs) prometheus.MustRegister(ShardMsgs) prometheus.MustRegister(ParsingPoolBacklog) prometheus.MustRegister(WritingPoolBacklog) + prometheus.MustRegister(WritingDurations) prometheus.MustRegister(collectors.NewBuildInfoCollector()) } @@ -253,11 +254,11 @@ func (p *Pusher) reconnect() { Collector(FlushMsgsTotal). Collector(FlushMsgsErrorTotal). Collector(ConsumeOffsets). - Collector(ClickhouseReconnectTotal). Collector(RingMsgs). Collector(ShardMsgs). Collector(ParsingPoolBacklog). Collector(WritingPoolBacklog). + Collector(WritingDurations). Grouping("instance", p.instance).Format(expfmt.FmtText) p.inUseAddr = nextAddr } From bc140b5bdc0f2eebfb7d318a0b9f21cc2a8c88e7 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 3 Jul 2022 21:44:15 +0800 Subject: [PATCH 259/404] kafka_gen_log and kafka_gen_metric use franz --- cmd/kafka_gen_log/main.go | 36 ++++++++++++++++++++++-------------- cmd/kafka_gen_metric/main.go | 36 ++++++++++++++++++++++-------------- cmd/kafka_gen_prom/main.go | 4 ++-- 3 files changed, 46 insertions(+), 30 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index cfcf4e31..9e0d6b1c 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -52,10 +52,10 @@ import ( "syscall" "time" - "github.com/Shopify/sarama" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" + "github.com/twmb/franz-go/pkg/kgo" "go.uber.org/zap" ) @@ -229,14 +229,24 @@ func (g *LogGenerator) Run() { rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) wp := util.NewWorkerPool(10, 10000) - config := sarama.NewConfig() - config.Version = sarama.V2_1_0_0 - w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) - if err != nil { - util.Logger.Fatal("sarama.NewAsyncProducer failed", zap.Error(err)) + opts := []kgo.Opt{ + kgo.SeedBrokers(strings.Split(KafkaBrokers, ",")...), + } + var err error + var cl *kgo.Client + if cl, err = kgo.NewClient(opts...); err != nil { + util.Logger.Fatal("kgo.NewClient failed", zap.Error(err)) + } + defer cl.Close() + + ctx := context.Background() + produceCb := func(rec *kgo.Record, err error) { + if err != nil { + util.Logger.Fatal("kgo.Client.Produce failed", zap.Error(err)) + } + atomic.AddInt64(&g.lines, int64(1)) + atomic.AddInt64(&g.size, int64(len(rec.Value))) } - defer w.Close() - chInput := w.Input() var b []byte for day := 0; ; day++ { @@ -277,13 +287,11 @@ func (g *LogGenerator) Run() { err = errors.Wrapf(err, "") util.Logger.Fatal("got error", zap.Error(err)) } - chInput <- &sarama.ProducerMessage{ + cl.Produce(ctx, &kgo.Record{ Topic: KafkaTopic, - Key: sarama.StringEncoder(logObj.Hostname), - Value: sarama.ByteEncoder(b), - } - atomic.AddInt64(&g.lines, int64(1)) - atomic.AddInt64(&g.size, int64(len(b))) + Key: []byte(logObj.Hostname), + Value: b, + }, produceCb) }) } } diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index 2771eebc..18372d3a 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -44,10 +44,10 @@ import ( "syscall" "time" - "github.com/Shopify/sarama" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" + "github.com/twmb/franz-go/pkg/kgo" "go.uber.org/zap" ) @@ -113,14 +113,24 @@ func generate() { rounded := time.Date(toRound.Year(), toRound.Month(), toRound.Day(), 0, 0, 0, 0, toRound.Location()) wp := util.NewWorkerPool(10, 10000) - config := sarama.NewConfig() - config.Version = sarama.V2_1_0_0 - w, err := sarama.NewAsyncProducer(strings.Split(KafkaBrokers, ","), config) - if err != nil { - util.Logger.Error("sarama.NewAsyncProducer failed", zap.Error(err)) + opts := []kgo.Opt{ + kgo.SeedBrokers(strings.Split(KafkaBrokers, ",")...), + } + var err error + var cl *kgo.Client + if cl, err = kgo.NewClient(opts...); err != nil { + util.Logger.Fatal("kgo.NewClient failed", zap.Error(err)) + } + defer cl.Close() + + ctx := context.Background() + produceCb := func(rec *kgo.Record, err error) { + if err != nil { + util.Logger.Fatal("kgo.Client.Produce failed", zap.Error(err)) + } + atomic.AddInt64(&gLines, int64(1)) + atomic.AddInt64(&gSize, int64(len(rec.Value))) } - defer w.Close() - chInput := w.Input() for day := 0; ; day++ { tsDay := rounded.Add(time.Duration(24*day) * time.Hour) @@ -159,13 +169,11 @@ func generate() { err = errors.Wrapf(err, "") util.Logger.Fatal("got error", zap.Error(err)) } - chInput <- &sarama.ProducerMessage{ + cl.Produce(ctx, &kgo.Record{ Topic: KafkaTopic, - Key: sarama.StringEncoder(metric.ItemGUID), - Value: sarama.ByteEncoder(b), - } - atomic.AddInt64(&gLines, int64(1)) - atomic.AddInt64(&gSize, int64(len(b))) + Key: []byte(metric.ItemGUID), + Value: b, + }, produceCb) }) } } diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index 15a01472..7e3ac5bb 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -128,6 +128,8 @@ func generate() { if err != nil { util.Logger.Fatal("kgo.Client.Produce failed", zap.Error(err)) } + atomic.AddInt64(&gLines, int64(1)) + atomic.AddInt64(&gSize, int64(len(rec.Value))) } for day := 0; ; day++ { @@ -156,8 +158,6 @@ func generate() { Key: []byte(dp.Name), Value: b, }, produceCb) - atomic.AddInt64(&gLines, int64(1)) - atomic.AddInt64(&gSize, int64(len(b))) }) } } From 53f4e1c40f33c3614f25ce94779a1b51a1f58cde Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 12 Jul 2022 17:55:35 +0800 Subject: [PATCH 260/404] don't escape username and password --- pool/conn.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/pool/conn.go b/pool/conn.go index 125dd3d5..13bd8e47 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -21,7 +21,6 @@ package pool import ( "crypto/tls" "fmt" - "net/url" "sync" "time" @@ -123,9 +122,9 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara replicas: replicaAddrs, opts: clickhouse.Options{ Auth: clickhouse.Auth{ - Database: url.QueryEscape(db), - Username: url.QueryEscape(username), - Password: url.QueryEscape(password), + Database: db, + Username: username, + Password: password, }, //Debug: true, DialTimeout: time.Second, From 4257d324bd0f9c905b7774607e60167851023020 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 13 Jul 2022 17:55:37 +0800 Subject: [PATCH 261/404] replaced roaring64 with map --- output/clickhouse.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 666336d2..58052cdb 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -29,7 +29,6 @@ import ( "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/ClickHouse/clickhouse-go/v2" - "github.com/RoaringBitmap/roaring/roaring64" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" @@ -65,7 +64,7 @@ type ClickHouse struct { distMetricTbls []string distSeriesTbls []string - bmSeries *roaring64.Bitmap + bmSeries map[uint64]bool numFlying int32 mux sync.Mutex taskDone *sync.Cond @@ -117,9 +116,10 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err err var seriesRows model.Rows c.mux.Lock() for _, row := range rows { - mgmtID := (*row)[c.IdxSerID+1].(int64) - if c.bmSeries.CheckedAdd(uint64(mgmtID)) { + mgmtID := uint64((*row)[c.IdxSerID+1].(int64)) + if _, found := c.bmSeries[mgmtID]; !found { seriesRows = append(seriesRows, row) + c.bmSeries[mgmtID] = true } } c.mux.Unlock() @@ -208,7 +208,7 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { if c.cfg.Clickhouse.Cluster != "" { tbl = c.distSeriesTbls[0] } - c.bmSeries = roaring64.New() + c.bmSeries = make(map[uint64]bool) if !c.taskCfg.LoadSeriesAtStartup { util.Logger.Info(fmt.Sprintf("skipped loading series from %v", tbl), zap.String("task", c.taskCfg.Name)) return @@ -227,9 +227,9 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { err = errors.Wrapf(err, "") return err } - c.bmSeries.Add(mgmtID) + c.bmSeries[mgmtID] = true } - util.Logger.Info(fmt.Sprintf("loaded %d series from %v", c.bmSeries.GetCardinality(), tbl), zap.String("task", c.taskCfg.Name)) + util.Logger.Info(fmt.Sprintf("loaded %d series from %v", len(c.bmSeries), tbl), zap.String("task", c.taskCfg.Name)) return } From 3fe318aeb97cd1f4dd4f60c0212e84ae27ae14ad Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 20 Jul 2022 09:59:27 +0800 Subject: [PATCH 262/404] generic code to work around https://github.com/ClickHouse/clickhouse-go/issues/574 --- README.md | 2 +- cmd/kafka_gen_prom/main.go | 3 + docs/dev/introduction.md | 1 - go.mod | 1 + go.sum | 5 +- go.test.sh | 4 +- model/message.go | 2 +- model/metric.go | 18 ++- model/value.go | 194 ++++++++++++------------- output/clickhouse.go | 21 +-- output/clickhouse_util.go | 4 +- parser/csv.go | 185 ++++++++++++++---------- parser/fastjson.go | 261 +++++++++++++++++++++++----------- parser/gjson.go | 283 +++++++++++++++++++++++++------------ parser/parser.go | 20 --- parser/parser_test.go | 154 +++++++++----------- task/sharding.go | 9 +- task/task.go | 3 +- 18 files changed, 684 insertions(+), 486 deletions(-) diff --git a/README.md b/README.md index db9c2254..d492cb2d 100644 --- a/README.md +++ b/README.md @@ -3,7 +3,7 @@ [![Build Status](https://travis-ci.com/housepower/clickhouse_sinker.svg?branch=master)](https://travis-ci.com/housepower/clickhouse_sinker) [![Go Report Card](https://goreportcard.com/badge/github.com/housepower/clickhouse_sinker)](https://goreportcard.com/report/github.com/housepower/clickhouse_sinker) -clickhouse_sinker is a sinker program that transfer kafka message into [ClickHouse](https://clickhouse.yandex/). +clickhouse_sinker is a sinker program that transfer kafka message into [ClickHouse](https://github.com/clickhouse/clickhouse). [Get Started](https://housepower.github.io/clickhouse_sinker/) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index 7e3ac5bb..5f83c417 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -1,6 +1,9 @@ package main /* +https://github.com/ClickHouse/ClickHouse/issues/38878 +performance of inserting to sparse wide table is bad + CREATE TABLE prom_extend ON CLUSTER abc ( timestamp DateTime, value Float64, diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index b15c9874..c4526393 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -35,7 +35,6 @@ Refers to [design](./design.md) for how it works. - [x] Enum - [x] Array(T), where T is one of above basic types - [x] Nullable(T), where T is one of above basic types -- [x] [ElasticDateTime](https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html) => Int64 (2019-12-16T12:10:30Z => 1576498230) Note: diff --git a/go.mod b/go.mod index a678cda6..10bada1e 100644 --- a/go.mod +++ b/go.mod @@ -28,6 +28,7 @@ require ( github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.1.1 go.uber.org/zap v1.21.0 + golang.org/x/exp v0.0.0-20220713135740-79cabaa25d75 golang.org/x/time v0.0.0-20220609170525-579cf78fd858 gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) diff --git a/go.sum b/go.sum index d5280b55..8ac00d02 100644 --- a/go.sum +++ b/go.sum @@ -174,8 +174,8 @@ github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.7 h1:81/ik6ipDQS2aGcBfIN5dHDB36BwrStyeAQquSYCV4o= github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= +github.com/google/go-cmp v0.5.8 h1:e6P7q2lk1O+qJJb4BtCQXlK8vWEO8V1ZeuEdJNOqZyg= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gops v0.3.23 h1:OjsHRINl5FiIyTc8jivIg4UN0GY6Nh32SL8KRbl8GQo= github.com/google/gops v0.3.23/go.mod h1:7diIdLsqpCihPSX3fQagksT/Ku/y4RL9LHTlKyEUDl8= @@ -467,6 +467,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/exp v0.0.0-20220713135740-79cabaa25d75 h1:x03zeu7B2B11ySp+daztnwM5oBJ/8wGUSqrwcw9L0RA= +golang.org/x/exp v0.0.0-20220713135740-79cabaa25d75/go.mod h1:Kr81I6Kryrl9sr8s2FK3vxD90NdsKWRuOIl2O4CvYbA= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -661,7 +663,6 @@ golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= diff --git a/go.test.sh b/go.test.sh index 05599e91..475c9f3f 100755 --- a/go.test.sh +++ b/go.test.sh @@ -6,7 +6,7 @@ curl "localhost:8123" -d 'CREATE TABLE test_fixed_schema ( time DateTime, name String, - value Float64, + value Float32, price Decimal32(3) DEFAULT(9.9) ) ENGINE = MergeTree @@ -63,7 +63,7 @@ echo "Got test_auto_schema count => $count" schema=`curl "localhost:8123" -d 'DESC test_dynamic_schema' 2>/dev/null | grep newkey | sort | tr -d '\t' | tr '\n' ','` echo "Got test_dynamic_schema schema => $schema" -[ $schema = "newkey01Nullable(Int64),newkey02Nullable(Float64),newkey03Nullable(String),newkey04Nullable(DateTime64(3)),newkey05Nullable(String),newkey06Array(Int64),newkey07Array(Float64),newkey08Array(String),newkey09Array(DateTime64(3)),newkey10Array(String)," ] || exit 1 +[ $schema = "newkey01Nullable(Int64),newkey02Nullable(Float64),newkey03Nullable(String),newkey04Nullable(DateTime64(3))," ] || exit 1 count=`curl "localhost:8123" -d 'SELECT count() FROM test_dynamic_schema'` echo "Got test_dynamic_schema count => $count" [ $count -eq 100000 ] || exit 1 diff --git a/model/message.go b/model/message.go index 4d7354dd..6e727033 100644 --- a/model/message.go +++ b/model/message.go @@ -176,7 +176,7 @@ func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSe var seriesID, mgmtID int64 if idxSeriesID >= 0 { // If some labels are not Prometheus native, ETL shall calculate and pass "__series_id", otherwise clickhouse_sinker use "__mgmt_id". - val := metric.GetInt("__series_id", false) + val := metric.GetInt64("__series_id", false) seriesID = val.(int64) // clickhouse_sinker calculate "__mgmt_id" based on all labels. dig = xxhash.New() diff --git a/model/metric.go b/model/metric.go index a270b18d..8de2116d 100644 --- a/model/metric.go +++ b/model/metric.go @@ -23,14 +23,21 @@ import ( // Metric interface for metric collection type Metric interface { GetBool(key string, nullable bool) (val interface{}) - GetInt(key string, nullable bool) (val interface{}) - GetFloat(key string, nullable bool) (val interface{}) + GetInt8(key string, nullable bool) (val interface{}) + GetInt16(key string, nullable bool) (val interface{}) + GetInt32(key string, nullable bool) (val interface{}) + GetInt64(key string, nullable bool) (val interface{}) + GetUint8(key string, nullable bool) (val interface{}) + GetUint16(key string, nullable bool) (val interface{}) + GetUint32(key string, nullable bool) (val interface{}) + GetUint64(key string, nullable bool) (val interface{}) + GetFloat32(key string, nullable bool) (val interface{}) + GetFloat64(key string, nullable bool) (val interface{}) GetDecimal(key string, nullable bool) (val interface{}) - GetString(key string, nullable bool) (val interface{}) GetDateTime(key string, nullable bool) (val interface{}) - GetElasticDateTime(key string, nullable bool) (val interface{}) + GetString(key string, nullable bool) (val interface{}) GetArray(key string, t int) (val interface{}) - GetNewKeys(knownKeys, newKeys *sync.Map, white, black *regexp.Regexp) bool + GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, black *regexp.Regexp, partition int, offset int64) bool } // DimMetrics @@ -44,5 +51,6 @@ type ColumnWithType struct { Name string Type int Nullable bool + Array bool SourceName string } diff --git a/model/value.go b/model/value.go index e8d7754f..34171b76 100644 --- a/model/value.go +++ b/model/value.go @@ -24,23 +24,25 @@ import ( const ( Unknown = iota Bool - Int - Float + Int8 + Int16 + Int32 + Int64 + Uint8 + Uint16 + Uint32 + Uint64 + Float32 + Float64 Decimal - String DateTime - ElasticDateTime - BoolArray - IntArray - FloatArray - DecimalArray - StringArray - DateTimeArray + String ) type TypeInfo struct { Type int Nullable bool + Array bool } var ( @@ -51,140 +53,124 @@ func GetTypeName(typ int) (name string) { switch typ { case Bool: name = "Bool" - case Int: - name = "Int" - case Float: - name = "Float" + case Int8: + name = "Int8" + case Int16: + name = "Int16" + case Int32: + name = "Int32" + case Int64: + name = "Int64" + case Uint8: + name = "Uint8" + case Uint16: + name = "Uint16" + case Uint32: + name = "Uint32" + case Uint64: + name = "Uint64" + case Float32: + name = "Float32" + case Float64: + name = "Float64" case Decimal: name = "Decimal" - case String: - name = "String" case DateTime: name = "DateTime" - case ElasticDateTime: - name = "ElasticDateTime" - case BoolArray: - name = "BoolArray" - case IntArray: - name = "IntArray" - case FloatArray: - name = "FloatArray" - case StringArray: - name = "StringArray" - case DateTimeArray: - name = "DateTimeArray" + case String: + name = "String" default: name = "Unknown" } return } -// There are only three cases for the value type of metric, (float64, string, map [string] interface {}) func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { name := cwt.SourceName - switch cwt.Type { - case Bool: - val = metric.GetBool(name, cwt.Nullable) - case Int: - val = metric.GetInt(name, cwt.Nullable) - case Float: - val = metric.GetFloat(name, cwt.Nullable) - case Decimal: - val = metric.GetDecimal(name, cwt.Nullable) - case String: - val = metric.GetString(name, cwt.Nullable) - case DateTime: - val = metric.GetDateTime(name, cwt.Nullable) - case ElasticDateTime: - val = metric.GetElasticDateTime(name, cwt.Nullable) - case BoolArray: - val = metric.GetArray(name, Bool) - case IntArray: - val = metric.GetArray(name, Int) - case FloatArray: - val = metric.GetArray(name, Float) - case DecimalArray: - val = metric.GetArray(name, Decimal) - case StringArray: - val = metric.GetArray(name, String) - case DateTimeArray: - val = metric.GetArray(name, DateTime) - default: - util.Logger.Fatal("LOGIC ERROR: reached switch default condition") + if cwt.Array { + val = metric.GetArray(name, cwt.Type) + } else { + switch cwt.Type { + case Bool: + val = metric.GetBool(name, cwt.Nullable) + case Int8: + val = metric.GetInt8(name, cwt.Nullable) + case Int16: + val = metric.GetInt16(name, cwt.Nullable) + case Int32: + val = metric.GetInt32(name, cwt.Nullable) + case Int64: + val = metric.GetInt64(name, cwt.Nullable) + case Uint8: + val = metric.GetUint8(name, cwt.Nullable) + case Uint16: + val = metric.GetUint16(name, cwt.Nullable) + case Uint32: + val = metric.GetUint32(name, cwt.Nullable) + case Uint64: + val = metric.GetUint64(name, cwt.Nullable) + case Float32: + val = metric.GetFloat32(name, cwt.Nullable) + case Float64: + val = metric.GetFloat64(name, cwt.Nullable) + case Decimal: + val = metric.GetDecimal(name, cwt.Nullable) + case DateTime: + val = metric.GetDateTime(name, cwt.Nullable) + case String: + val = metric.GetString(name, cwt.Nullable) + default: + util.Logger.Fatal("LOGIC ERROR: reached switch default condition") + } } return } -func WhichType(typ string) (dataType int, nullable bool) { +func WhichType(typ string) (dataType int, nullable bool, array bool) { ti, ok := typeInfo[typ] if ok { - dataType, nullable = ti.Type, ti.Nullable + dataType, nullable, array = ti.Type, ti.Nullable, ti.Array return } + origTyp := typ nullable = strings.HasPrefix(typ, "Nullable(") + array = strings.HasPrefix(typ, "Array(") if nullable { typ = typ[len("Nullable(") : len(typ)-1] + } else if array { + typ = typ[len("Array(") : len(typ)-1] } if strings.HasPrefix(typ, "DateTime64") { dataType = DateTime - } else if strings.HasPrefix(typ, "Array(DateTime64") { - dataType = DateTimeArray - nullable = false } else if strings.HasPrefix(typ, "Decimal") { dataType = Decimal - } else if strings.HasPrefix(typ, "Array(Decimal") { - dataType = DecimalArray - nullable = false } else if strings.HasPrefix(typ, "FixedString") { dataType = String - } else if strings.HasPrefix(typ, "Array(FixedString") { - dataType = StringArray - nullable = false } else if strings.HasPrefix(typ, "Enum8(") { dataType = String } else if strings.HasPrefix(typ, "Enum16(") { dataType = String } else { - util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported ClickHouse data type %v", typ)) + util.Logger.Fatal(fmt.Sprintf("ClickHouse column type %v is not inside supported ones: %v", origTyp, typeInfo)) } - typeInfo[typ] = TypeInfo{Type: dataType, Nullable: nullable} + typeInfo[origTyp] = TypeInfo{Type: dataType, Nullable: nullable, Array: array} return } func init() { - primTypeInfo := make(map[string]TypeInfo) typeInfo = make(map[string]TypeInfo) - primTypeInfo["Bool"] = TypeInfo{Type: Bool, Nullable: false} - for _, t := range []string{"UInt8", "UInt16", "UInt32", "UInt64", "Int8", - "Int16", "Int32", "Int64"} { - primTypeInfo[t] = TypeInfo{Type: Int, Nullable: false} - } - for _, t := range []string{"Float32", "Float64"} { - primTypeInfo[t] = TypeInfo{Type: Float, Nullable: false} - } - for _, t := range []string{"String", "UUID"} { - primTypeInfo[t] = TypeInfo{Type: String, Nullable: false} - } - for _, t := range []string{"Date", "DateTime"} { - primTypeInfo[t] = TypeInfo{Type: DateTime, Nullable: false} - } - primTypeInfo["ElasticDateTime"] = TypeInfo{Type: ElasticDateTime, Nullable: false} - for k, v := range primTypeInfo { - typeInfo[k] = v - nullK := fmt.Sprintf("Nullable(%s)", k) - typeInfo[nullK] = TypeInfo{Type: v.Type, Nullable: true} - arrK := fmt.Sprintf("Array(%s)", k) - switch v.Type { - case Bool: - typeInfo[arrK] = TypeInfo{Type: BoolArray, Nullable: false} - case Int: - typeInfo[arrK] = TypeInfo{Type: IntArray, Nullable: false} - case Float: - typeInfo[arrK] = TypeInfo{Type: FloatArray, Nullable: false} - case String: - typeInfo[arrK] = TypeInfo{Type: StringArray, Nullable: false} - case DateTime: - typeInfo[arrK] = TypeInfo{Type: DateTimeArray, Nullable: false} - } + for _, t := range []int{Bool, Int8, Int16, Int32, Int64, Uint8, Uint16, Uint32, Uint64, Float32, Float64, DateTime, String} { + tn := GetTypeName(t) + typeInfo[tn] = TypeInfo{Type: t} + nullTn := fmt.Sprintf("Nullable(%s)", tn) + typeInfo[nullTn] = TypeInfo{Type: t, Nullable: true} + arrTn := fmt.Sprintf("Array(%s)", tn) + typeInfo[arrTn] = TypeInfo{Type: t, Array: true} } + typeInfo["UUID"] = TypeInfo{Type: String} + typeInfo["Nullable(UUID)"] = TypeInfo{Type: String, Nullable: true} + typeInfo["Array(UUID)"] = TypeInfo{Type: String, Array: true} + typeInfo["Date"] = TypeInfo{Type: DateTime} + typeInfo["Nullable(Date)"] = TypeInfo{Type: DateTime, Nullable: true} + typeInfo["Array(Date)"] = TypeInfo{Type: DateTime, Array: true} } diff --git a/output/clickhouse.go b/output/clickhouse.go index 58052cdb..7576bb6a 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -242,7 +242,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { var dimSerID *model.ColumnWithType for i := 0; i < len(c.Dims); { dim := c.Dims[i] - if dim.Name == "__series_id" && dim.Type == model.Int { + if dim.Name == "__series_id" && dim.Type == model.Int64 { dimSerID = dim c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) } else if dim.Type == model.String { @@ -261,8 +261,8 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { // Add string columns from series table c.seriesTbl = c.taskCfg.TableName + "_series" expSeriesDims := []*model.ColumnWithType{ - {Name: "__series_id", Type: model.Int}, - {Name: "__mgmt_id", Type: model.Int}, + {Name: "__series_id", Type: model.Int64}, + {Name: "__mgmt_id", Type: model.Int64}, {Name: "labels", Type: model.String}, } var seriesDims []*model.ColumnWithType @@ -340,11 +340,12 @@ func (c *ClickHouse) initSchema() (err error) { } else { c.Dims = make([]*model.ColumnWithType, 0) for _, dim := range c.taskCfg.Dims { - tp, nullable := model.WhichType(dim.Type) + tp, nullable, array := model.WhichType(dim.Type) c.Dims = append(c.Dims, &model.ColumnWithType{ Name: dim.Name, Type: tp, Nullable: nullable, + Array: array, SourceName: dim.SourceName, }) } @@ -409,22 +410,14 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { intVal := value.(int) var strVal string switch intVal { - case model.Int: + case model.Int64: strVal = "Nullable(Int64)" - case model.Float: + case model.Float64: strVal = "Nullable(Float64)" case model.String: strVal = "Nullable(String)" case model.DateTime: strVal = "Nullable(DateTime64(3))" - case model.IntArray: - strVal = "Array(Int64)" - case model.FloatArray: - strVal = "Array(Float64)" - case model.StringArray: - strVal = "Array(String)" - case model.DateTimeArray: - strVal = "Array(DateTime64(3))" default: err = errors.Newf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) return false diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index df7896a2..58cd75cf 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -100,8 +100,8 @@ func getDims(database, table string, excludedColumns []string, conn clickhouse.C } typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") if !util.StringContains(excludedColumns, name) && defaultKind != "MATERIALIZED" { - tp, nullable := model.WhichType(typ) - dims = append(dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, SourceName: util.GetSourceName(name)}) + tp, nullable, array := model.WhichType(typ) + dims = append(dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, Array: array, SourceName: util.GetSourceName(name)}) } } if len(dims) == 0 { diff --git a/parser/csv.go b/parser/csv.go index 02e4018c..ba860f96 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -29,6 +29,7 @@ import ( "github.com/thanos-io/thanos/pkg/errors" "github.com/tidwall/gjson" "github.com/valyala/fastjson/fastfloat" + "golang.org/x/exp/constraints" ) var _ Parser = (*CsvParser)(nil) @@ -79,65 +80,123 @@ func (c *CsvMetric) GetString(key string, nullable bool) (val interface{}) { return } -// GetFloat returns the value as float -func (c *CsvMetric) GetFloat(key string, nullable bool) (val interface{}) { +// GetDecimal returns the value as decimal +func (c *CsvMetric) GetDecimal(key string, nullable bool) (val interface{}) { var idx int var ok bool if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { if nullable { return } - val = float64(0.0) + val = decimal.NewFromInt(0) return } - val = fastfloat.ParseBestEffort(c.values[idx]) + val, _ = decimal.NewFromString(c.values[idx]) return } -// GetDecimal returns the value as decimal -func (c *CsvMetric) GetDecimal(key string, nullable bool) (val interface{}) { +func (c *CsvMetric) GetBool(key string, nullable bool) (val interface{}) { var idx int var ok bool - if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { + if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "" || c.values[idx] == "null" { if nullable { return } - val = decimal.NewFromInt(0) + val = false return } - val, _ = decimal.NewFromString(c.values[idx]) + val = (c.values[idx] == "true") return } -func (c *CsvMetric) GetBool(key string, nullable bool) (val interface{}) { +func (c *CsvMetric) GetInt8(key string, nullable bool) (val interface{}) { + return CsvGetInt[int8](c, key, nullable) +} + +func (c *CsvMetric) GetInt16(key string, nullable bool) (val interface{}) { + return CsvGetInt[int16](c, key, nullable) +} + +func (c *CsvMetric) GetInt32(key string, nullable bool) (val interface{}) { + return CsvGetInt[int32](c, key, nullable) +} + +func (c *CsvMetric) GetInt64(key string, nullable bool) (val interface{}) { + return CsvGetInt[int64](c, key, nullable) +} + +func (c *CsvMetric) GetUint8(key string, nullable bool) (val interface{}) { + return CsvGetUint[uint8](c, key, nullable) +} + +func (c *CsvMetric) GetUint16(key string, nullable bool) (val interface{}) { + return CsvGetUint[uint16](c, key, nullable) +} + +func (c *CsvMetric) GetUint32(key string, nullable bool) (val interface{}) { + return CsvGetUint[uint32](c, key, nullable) +} + +func (c *CsvMetric) GetUint64(key string, nullable bool) (val interface{}) { + return CsvGetUint[uint64](c, key, nullable) +} + +func (c *CsvMetric) GetFloat32(key string, nullable bool) (val interface{}) { + return CsvGetFloat[float32](c, key, nullable) +} + +func (c *CsvMetric) GetFloat64(key string, nullable bool) (val interface{}) { + return CsvGetFloat[float64](c, key, nullable) +} + +func CsvGetInt[T constraints.Signed](c *CsvMetric, key string, nullable bool) (val interface{}) { var idx int var ok bool - if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "" || c.values[idx] == "null" { + if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { if nullable { return } - val = false + val = T(0) return } - val = (c.values[idx] == "true") + if s := c.values[idx]; s == "true" { + val = T(1) + } else { + val = T(fastfloat.ParseInt64BestEffort(s)) + } return } -func (c *CsvMetric) GetInt(key string, nullable bool) (val interface{}) { +func CsvGetUint[T constraints.Unsigned](c *CsvMetric, key string, nullable bool) (val interface{}) { var idx int var ok bool if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { if nullable { return } - val = int64(0) + val = T(0) return } if s := c.values[idx]; s == "true" { - val = int64(1) + val = T(1) } else { - val = fastfloat.ParseInt64BestEffort(s) + val = T(fastfloat.ParseUint64BestEffort(s)) + } + return +} + +// GetFloat returns the value as float +func CsvGetFloat[T constraints.Float](c *CsvMetric, key string, nullable bool) (val interface{}) { + var idx int + var ok bool + if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { + if nullable { + return + } + val = float64(0.0) + return } + val = T(fastfloat.ParseBestEffort(c.values[idx])) return } @@ -163,91 +222,69 @@ func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}) { return } -func (c *CsvMetric) GetElasticDateTime(key string, nullable bool) (val interface{}) { - t := c.GetDateTime(key, nullable) - if t != nil { - val = t.(time.Time).Unix() - } - return -} - // GetArray parse an CSV encoded array func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { s := c.GetString(key, false) str, _ := s.(string) - if str == "" || str[0] != '[' { - val = makeArray(typ) - return - } - array := gjson.Parse(str).Array() - if len(array) == 0 { - val = makeArray(typ) - return + var array []gjson.Result + r := gjson.Parse(str) + if r.IsArray() { + array = r.Array() } switch typ { case model.Bool: results := make([]bool, 0, len(array)) for _, e := range array { - v := (e.Type == gjson.True) - results = append(results, v) - } - val = results - case model.Int: - results := make([]int64, 0, len(array)) - for _, e := range array { - var v int64 - switch e.Type { - case gjson.True: - v = int64(1) - case gjson.Number: - if v = e.Int(); float64(v) != e.Num { - v = int64(0) - } - default: - v = int64(0) - } - results = append(results, v) - } - val = results - case model.Float: - results := make([]float64, 0, len(array)) - for _, e := range array { - var v float64 - switch e.Type { - case gjson.Number: - v = e.Num - default: - v = float64(0.0) - } + v := (e.Exists() && e.Type == gjson.True) results = append(results, v) } val = results + case model.Int8: + val = GjsonIntArray[int8](array) + case model.Int16: + val = GjsonIntArray[int16](array) + case model.Int32: + val = GjsonIntArray[int32](array) + case model.Int64: + val = GjsonIntArray[int64](array) + case model.Uint8: + val = GjsonUintArray[uint8](array) + case model.Uint16: + val = GjsonUintArray[uint16](array) + case model.Uint32: + val = GjsonUintArray[uint32](array) + case model.Uint64: + val = GjsonUintArray[uint64](array) + case model.Float32: + val = GjsonFloatArray[float32](array) + case model.Float64: + val = GjsonFloatArray[float64](array) case model.Decimal: results := make([]decimal.Decimal, 0, len(array)) for _, e := range array { - var v float64 + var f float64 switch e.Type { case gjson.Number: - v = e.Num + f = e.Num default: - v = float64(0.0) + f = float64(0.0) } - results = append(results, decimal.NewFromFloat(v)) + results = append(results, decimal.NewFromFloat(f)) } val = results case model.String: results := make([]string, 0, len(array)) for _, e := range array { - var v string + var s string switch e.Type { case gjson.Null: - v = "" + s = "" case gjson.String: - v = e.Str + s = e.Str default: - v = e.Raw + s = e.Raw } - results = append(results, v) + results = append(results, s) } val = results case model.DateTime: @@ -274,6 +311,6 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { return } -func (c *CsvMetric) GetNewKeys(knownKeys, newKeys *sync.Map, white, black *regexp.Regexp) bool { +func (c *CsvMetric) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, black *regexp.Regexp, partition int, offset int64) bool { return false } diff --git a/parser/fastjson.go b/parser/fastjson.go index dcf45422..2a98bdf3 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -21,6 +21,8 @@ import ( "sync" "time" + "golang.org/x/exp/constraints" + "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" "github.com/shopspring/decimal" @@ -71,20 +73,6 @@ func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}) return } -func (c *FastjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { - v := c.value.Get(key) - if !fjCompatibleFloat(v) { - val = getDefaultFloat(nullable) - return - } - if val2, err := v.Float64(); err != nil { - val = getDefaultFloat(nullable) - } else { - val = val2 - } - return -} - func (c *FastjsonMetric) GetBool(key string, nullable bool) (val interface{}) { v := c.value.Get(key) if !fjCompatibleBool(v) { @@ -109,27 +97,102 @@ func (c *FastjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) return } -func (c *FastjsonMetric) GetInt(key string, nullable bool) (val interface{}) { +func (c *FastjsonMetric) GetInt8(key string, nullable bool) (val interface{}) { + return FastjsonGetInt[int8](c, key, nullable) +} + +func (c *FastjsonMetric) GetInt16(key string, nullable bool) (val interface{}) { + return FastjsonGetInt[int16](c, key, nullable) +} + +func (c *FastjsonMetric) GetInt32(key string, nullable bool) (val interface{}) { + return FastjsonGetInt[int32](c, key, nullable) +} + +func (c *FastjsonMetric) GetInt64(key string, nullable bool) (val interface{}) { + return FastjsonGetInt[int64](c, key, nullable) +} + +func (c *FastjsonMetric) GetUint8(key string, nullable bool) (val interface{}) { + return FastjsonGetUint[uint8](c, key, nullable) +} + +func (c *FastjsonMetric) GetUint16(key string, nullable bool) (val interface{}) { + return FastjsonGetUint[uint16](c, key, nullable) +} + +func (c *FastjsonMetric) GetUint32(key string, nullable bool) (val interface{}) { + return FastjsonGetUint[uint32](c, key, nullable) +} + +func (c *FastjsonMetric) GetUint64(key string, nullable bool) (val interface{}) { + return FastjsonGetUint[uint64](c, key, nullable) +} + +func (c *FastjsonMetric) GetFloat32(key string, nullable bool) (val interface{}) { + return FastjsonGetFloat[float32](c, key, nullable) +} + +func (c *FastjsonMetric) GetFloat64(key string, nullable bool) (val interface{}) { + return FastjsonGetFloat[float64](c, key, nullable) +} + +func FastjsonGetInt[T constraints.Signed](c *FastjsonMetric, key string, nullable bool) (val interface{}) { v := c.value.Get(key) if !fjCompatibleInt(v) { - val = getDefaultInt(nullable) + val = getDefaultInt[T](nullable) return } switch v.Type() { case fastjson.TypeTrue: - val = int64(1) + val = T(1) case fastjson.TypeFalse: - val = int64(0) + val = T(0) default: if val2, err := v.Int64(); err != nil { - val = getDefaultInt(nullable) + val = getDefaultInt[T](nullable) } else { - val = val2 + val = T(val2) } } return } +func FastjsonGetUint[T constraints.Unsigned](c *FastjsonMetric, key string, nullable bool) (val interface{}) { + v := c.value.Get(key) + if !fjCompatibleInt(v) { + val = getDefaultInt[T](nullable) + return + } + switch v.Type() { + case fastjson.TypeTrue: + val = T(1) + case fastjson.TypeFalse: + val = T(0) + default: + if val2, err := v.Uint64(); err != nil { + val = getDefaultInt[T](nullable) + } else { + val = T(val2) + } + } + return +} + +func FastjsonGetFloat[T constraints.Float](c *FastjsonMetric, key string, nullable bool) (val interface{}) { + v := c.value.Get(key) + if !fjCompatibleFloat(v) { + val = getDefaultFloat[T](nullable) + return + } + if val2, err := v.Float64(); err != nil { + val = getDefaultFloat[T](nullable) + } else { + val = T(val2) + } + return +} + func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { v := c.value.Get(key) if !fjCompatibleDateTime(v) { @@ -160,48 +223,48 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} return } -func (c *FastjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}) { - t := c.GetDateTime(key, nullable) - if t != nil { - val = t.(time.Time).Unix() - } - return -} - func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { - v := c.value.Get(key) - val = makeArray(typ) - if v == nil || v.Type() != fastjson.TypeArray { - return + var array []*fastjson.Value + if v := c.value.Get(key); v != nil { + array, _ = v.Array() } - array, _ := v.Array() switch typ { case model.Bool: + arr := make([]bool, 0) for _, e := range array { v := (e != nil && e.Type() == fastjson.TypeTrue) - val = append(val.([]bool), v) - } - case model.Int: - for _, e := range array { - var v int64 - if e.Type() == fastjson.TypeTrue { - v = 1 - } else { - v, _ = e.Int64() - } - val = append(val.([]int64), v) - } - case model.Float: - for _, e := range array { - v, _ := e.Float64() - val = append(val.([]float64), v) + arr = append(arr, v) } + val = arr + case model.Int8: + val = FastjsonIntArray[int8](array) + case model.Int16: + val = FastjsonIntArray[int16](array) + case model.Int32: + val = FastjsonIntArray[int32](array) + case model.Int64: + val = FastjsonIntArray[int64](array) + case model.Uint8: + val = FastjsonUintArray[uint8](array) + case model.Uint16: + val = FastjsonUintArray[uint16](array) + case model.Uint32: + val = FastjsonUintArray[uint32](array) + case model.Uint64: + val = FastjsonUintArray[uint64](array) + case model.Float32: + val = FastjsonFloatArray[float32](array) + case model.Float64: + val = FastjsonFloatArray[float64](array) case model.Decimal: + arr := make([]decimal.Decimal, 0) for _, e := range array { v, _ := e.Float64() - val = append(val.([]decimal.Decimal), decimal.NewFromFloat(v)) + arr = append(arr, decimal.NewFromFloat(v)) } + val = arr case model.String: + arr := make([]string, 0) for _, e := range array { var s string switch e.Type() { @@ -213,9 +276,11 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { default: s = e.String() } - val = append(val.([]string), s) + arr = append(arr, s) } + val = arr case model.DateTime: + arr := make([]time.Time, 0) for _, e := range array { var t time.Time switch e.Type() { @@ -237,15 +302,57 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { default: t = Epoch } - val = append(val.([]time.Time), t) + arr = append(arr, t) } + val = arr default: util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) } return } -func (c *FastjsonMetric) GetNewKeys(knownKeys, newKeys *sync.Map, white, black *regexp.Regexp) (foundNew bool) { +func FastjsonIntArray[T constraints.Signed](a []*fastjson.Value) (arr []T) { + arr = make([]T, 0) + for _, e := range a { + var val T + if e.Type() == fastjson.TypeTrue { + val = T(1) + } else { + var v int64 + v, _ = e.Int64() + val = T(v) + } + arr = append(arr, val) + } + return +} + +func FastjsonUintArray[T constraints.Unsigned](a []*fastjson.Value) (arr []T) { + arr = make([]T, 0) + for _, e := range a { + var val T + if e.Type() == fastjson.TypeTrue { + val = T(1) + } else { + var v uint64 + v, _ = e.Uint64() + val = T(v) + } + arr = append(arr, val) + } + return +} + +func FastjsonFloatArray[T constraints.Float](a []*fastjson.Value) (arr []T) { + arr = make([]T, 0) + for _, e := range a { + v, _ := e.Float64() + arr = append(arr, T(v)) + } + return +} + +func (c *FastjsonMetric) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, black *regexp.Regexp, partition int, offset int64) (foundNew bool) { var obj *fastjson.Object var err error if obj, err = c.value.Object(); err != nil { @@ -256,14 +363,14 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys, newKeys *sync.Map, white, black * if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { if (white == nil || white.MatchString(strKey)) && (black == nil || !black.MatchString(strKey)) { - if typ := fjDetectType(v); typ != model.Unknown { + if typ, arr := fjDetectType(v, 0); typ != model.Unknown && !arr { newKeys.Store(strKey, typ) foundNew = true - } else { - util.Logger.Warn("FastjsonMetric.GetNewKeys failed to detect field type", zap.String("key", strKey), zap.String("value", v.String())) + } else if _, loaded = warnKeys.LoadOrStore(strKey, nil); !loaded { + util.Logger.Warn("FastjsonMetric.GetNewKeys ignored new key due to unsupported type of dynamic column", zap.Int("partition", partition), zap.Int64("offset", offset), zap.String("key", strKey), zap.String("value", v.String())) } - } else { - util.Logger.Warn("FastjsonMetric.GetNewKeys ignored new key due to white/black list setting", zap.String("key", strKey), zap.String("value", v.String())) + } else if _, loaded = warnKeys.LoadOrStore(strKey, nil); !loaded { + util.Logger.Warn("FastjsonMetric.GetNewKeys ignored new key due to white/black list setting", zap.Int("partition", partition), zap.Int64("offset", offset), zap.String("key", strKey), zap.String("value", v.String())) knownKeys.Store(strKey, nil) } } @@ -323,19 +430,20 @@ func getDefaultBool(nullable bool) (val interface{}) { return } -func getDefaultInt(nullable bool) (val interface{}) { +func getDefaultInt[T constraints.Integer](nullable bool) (val interface{}) { if nullable { return } - val = int64(0) + var zero T + val = zero return } -func getDefaultFloat(nullable bool) (val interface{}) { +func getDefaultFloat[T constraints.Float](nullable bool) (val interface{}) { if nullable { return } - val = float64(0.0) + val = T(0.0) return } @@ -355,16 +463,20 @@ func getDefaultDateTime(nullable bool) (val interface{}) { return } -func fjDetectType(v *fastjson.Value) (typ int) { +func fjDetectType(v *fastjson.Value, depth int) (typ int, array bool) { + typ = model.Unknown + if depth > 1 { + return + } switch v.Type() { case fastjson.TypeNull: typ = model.Unknown case fastjson.TypeTrue, fastjson.TypeFalse: typ = model.Bool case fastjson.TypeNumber: - typ = model.Float + typ = model.Float64 if _, err := v.Int64(); err == nil { - typ = model.Int + typ = model.Int64 } case fastjson.TypeString: typ = model.String @@ -374,23 +486,14 @@ func fjDetectType(v *fastjson.Value) (typ int) { } } case fastjson.TypeArray: + if depth >= 1 { + return + } + array = true if arr, err := v.Array(); err == nil && len(arr) > 0 { - typ2 := fjDetectType(arr[0]) - switch typ2 { - case model.Bool: - typ = model.BoolArray - case model.Int: - typ = model.IntArray - case model.Float: - typ = model.FloatArray - case model.String: - typ = model.StringArray - case model.DateTime: - typ = model.DateTimeArray - } + typ, _ = fjDetectType(arr[0], depth+1) } default: - typ = model.String } return } diff --git a/parser/gjson.go b/parser/gjson.go index 57506ef3..2ba927ce 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -18,12 +18,14 @@ package parser import ( "fmt" "regexp" + "strconv" "sync" "time" "github.com/shopspring/decimal" "github.com/tidwall/gjson" "go.uber.org/zap" + "golang.org/x/exp/constraints" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" @@ -65,21 +67,6 @@ func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}) { return } -func (c *GjsonMetric) GetFloat(key string, nullable bool) (val interface{}) { - r := gjson.Get(c.raw, key) - if !gjCompatibleFloat(r) { - val = getDefaultFloat(nullable) - return - } - switch r.Type { - case gjson.Number: - val = r.Num - default: - val = getDefaultFloat(nullable) - } - return -} - func (c *GjsonMetric) GetBool(key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) if !gjCompatibleBool(r) { @@ -105,25 +92,103 @@ func (c *GjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) { return } -func (c *GjsonMetric) GetInt(key string, nullable bool) (val interface{}) { +func (c *GjsonMetric) GetInt8(key string, nullable bool) (val interface{}) { + return GjsonGetInt[int8](c, key, nullable) +} + +func (c *GjsonMetric) GetInt16(key string, nullable bool) (val interface{}) { + return GjsonGetInt[int16](c, key, nullable) +} + +func (c *GjsonMetric) GetInt32(key string, nullable bool) (val interface{}) { + return GjsonGetInt[int32](c, key, nullable) +} + +func (c *GjsonMetric) GetInt64(key string, nullable bool) (val interface{}) { + return GjsonGetInt[int64](c, key, nullable) +} + +func (c *GjsonMetric) GetUint8(key string, nullable bool) (val interface{}) { + return GjsonGetUint[uint8](c, key, nullable) +} + +func (c *GjsonMetric) GetUint16(key string, nullable bool) (val interface{}) { + return GjsonGetUint[uint16](c, key, nullable) +} + +func (c *GjsonMetric) GetUint32(key string, nullable bool) (val interface{}) { + return GjsonGetUint[uint32](c, key, nullable) +} + +func (c *GjsonMetric) GetUint64(key string, nullable bool) (val interface{}) { + return GjsonGetUint[uint64](c, key, nullable) +} + +func (c *GjsonMetric) GetFloat32(key string, nullable bool) (val interface{}) { + return GjsonGetFloat[float32](c, key, nullable) +} + +func (c *GjsonMetric) GetFloat64(key string, nullable bool) (val interface{}) { + return GjsonGetFloat[float64](c, key, nullable) +} + +func GjsonGetInt[T constraints.Signed](c *GjsonMetric, key string, nullable bool) (val interface{}) { r := gjson.Get(c.raw, key) if !gjCompatibleInt(r) { - val = getDefaultInt(nullable) + val = getDefaultInt[T](nullable) return } switch r.Type { case gjson.True: - val = int64(1) + val = T(1) case gjson.False: - val = int64(0) + val = T(0) case gjson.Number: if v := r.Int(); float64(v) != r.Num { - val = getDefaultInt(nullable) + val = getDefaultInt[T](nullable) + } else { + val = T(v) + } + default: + val = getDefaultInt[T](nullable) + } + return +} + +func GjsonGetUint[T constraints.Unsigned](c *GjsonMetric, key string, nullable bool) (val interface{}) { + r := gjson.Get(c.raw, key) + if !gjCompatibleInt(r) { + val = getDefaultInt[T](nullable) + return + } + switch r.Type { + case gjson.True: + val = T(1) + case gjson.False: + val = T(0) + case gjson.Number: + if v := r.Uint(); float64(v) != r.Num { + val = getDefaultInt[T](nullable) } else { - val = v + val = T(v) } default: - val = getDefaultInt(nullable) + val = getDefaultInt[T](nullable) + } + return +} + +func GjsonGetFloat[T constraints.Float](c *GjsonMetric, key string, nullable bool) (val interface{}) { + r := gjson.Get(c.raw, key) + if !gjCompatibleFloat(r) { + val = getDefaultFloat[T](nullable) + return + } + switch r.Type { + case gjson.Number: + val = T(r.Num) + default: + val = getDefaultFloat[T](nullable) } return } @@ -148,21 +213,12 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { return } -func (c *GjsonMetric) GetElasticDateTime(key string, nullable bool) (val interface{}) { - t := c.GetDateTime(key, nullable) - if t != nil { - val = t.(time.Time).Unix() - } - return -} - func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { + var array []gjson.Result r := gjson.Get(c.raw, key) - if !r.IsArray() { - val = makeArray(typ) - return + if r.IsArray() { + array = r.Array() } - array := r.Array() switch typ { case model.Bool: results := make([]bool, 0, len(array)) @@ -171,36 +227,26 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { results = append(results, v) } val = results - case model.Int: - results := make([]int64, 0, len(array)) - for _, e := range array { - var v int64 - switch e.Type { - case gjson.True: - v = int64(1) - case gjson.Number: - if v = e.Int(); float64(v) != e.Num { - v = int64(0) - } - default: - v = int64(0) - } - results = append(results, v) - } - val = results - case model.Float: - results := make([]float64, 0, len(array)) - for _, e := range array { - var f float64 - switch e.Type { - case gjson.Number: - f = e.Num - default: - f = float64(0.0) - } - results = append(results, f) - } - val = results + case model.Int8: + val = GjsonIntArray[int8](array) + case model.Int16: + val = GjsonIntArray[int16](array) + case model.Int32: + val = GjsonIntArray[int32](array) + case model.Int64: + val = GjsonIntArray[int64](array) + case model.Uint8: + val = GjsonUintArray[uint8](array) + case model.Uint16: + val = GjsonUintArray[uint16](array) + case model.Uint32: + val = GjsonUintArray[uint32](array) + case model.Uint64: + val = GjsonUintArray[uint64](array) + case model.Float32: + val = GjsonFloatArray[float32](array) + case model.Float64: + val = GjsonFloatArray[float64](array) case model.Decimal: results := make([]decimal.Decimal, 0, len(array)) for _, e := range array { @@ -253,20 +299,79 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { return } -func (c *GjsonMetric) GetNewKeys(knownKeys, newKeys *sync.Map, white, black *regexp.Regexp) (foundNew bool) { +func GjsonIntArray[T constraints.Signed](a []gjson.Result) (arr []T) { + arr = make([]T, 0, len(a)) + for _, e := range a { + var v T + switch e.Type { + case gjson.True: + v = T(1) + case gjson.Number: + var tmpv int64 + if tmpv = e.Int(); float64(tmpv) != e.Num { + v = T(0) + } else { + v = T(tmpv) + } + default: + v = T(0) + } + arr = append(arr, v) + } + return +} + +func GjsonUintArray[T constraints.Unsigned](a []gjson.Result) (arr []T) { + arr = make([]T, 0, len(a)) + for _, e := range a { + var v T + switch e.Type { + case gjson.True: + v = T(1) + case gjson.Number: + var tmpv uint64 + if tmpv = e.Uint(); float64(tmpv) != e.Num { + v = T(0) + } else { + v = T(tmpv) + } + default: + v = T(0) + } + arr = append(arr, v) + } + return +} + +func GjsonFloatArray[T constraints.Float](a []gjson.Result) (arr []T) { + arr = make([]T, 0, len(a)) + for _, e := range a { + var v T + switch e.Type { + case gjson.Number: + v = T(e.Num) + default: + v = T(0.0) + } + arr = append(arr, v) + } + return +} + +func (c *GjsonMetric) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, black *regexp.Regexp, partition int, offset int64) (foundNew bool) { gjson.Parse(c.raw).ForEach(func(k, v gjson.Result) bool { strKey := k.Str if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { if (white == nil || white.MatchString(strKey)) && (black == nil || !black.MatchString(strKey)) { - if typ := gjDetectType(v); typ != model.Unknown { + if typ, array := gjDetectType(v, 0); typ != model.Unknown && !array { newKeys.Store(strKey, typ) foundNew = true - } else { - util.Logger.Warn("GjsonMetric.GetNewKeys failed to detect field type", zap.String("key", strKey), zap.String("value", v.String())) + } else if _, loaded = warnKeys.LoadOrStore(strKey, nil); !loaded { + util.Logger.Warn("GjsonMetric.GetNewKeys failed to detect field type", zap.Int("partition", partition), zap.Int64("offset", offset), zap.String("key", strKey), zap.String("value", v.String())) } - } else { - util.Logger.Warn("GjsonMetric.GetNewKeys ignored new key due to white/black list setting", zap.String("key", strKey), zap.String("value", v.String())) + } else if _, loaded = warnKeys.LoadOrStore(strKey, nil); !loaded { + util.Logger.Warn("GjsonMetric.GetNewKeys ignored new key due to white/black list setting", zap.Int("partition", partition), zap.Int64("offset", offset), zap.String("key", strKey), zap.String("value", v.String())) knownKeys.Store(strKey, nil) } } @@ -323,15 +428,19 @@ func gjCompatibleDateTime(r gjson.Result) (ok bool) { return } -func gjDetectType(v gjson.Result) (typ int) { +func gjDetectType(v gjson.Result, depth int) (typ int, array bool) { typ = model.Unknown + if depth > 1 { + return + } switch v.Type { case gjson.True, gjson.False: typ = model.Bool case gjson.Number: - typ = model.Float - if float64(v.Int()) == v.Num { - typ = model.Int + if _, err := strconv.ParseInt(v.Raw, 10, 64); err == nil { + typ = model.Int64 + } else { + typ = model.Float64 } case gjson.String: typ = model.String @@ -339,27 +448,13 @@ func gjDetectType(v gjson.Result) (typ int) { typ = model.DateTime } case gjson.JSON: - if v.IsObject() { - typ = model.String - } else if v.IsArray() { + if v.IsArray() { + if depth >= 1 { + return + } + array = true if array := v.Array(); len(array) != 0 { - switch array[0].Type { - case gjson.True, gjson.False: - typ = model.BoolArray - case gjson.Number: - typ = model.FloatArray - if float64(array[0].Int()) == array[0].Num { - typ = model.IntArray - } - case gjson.String: - typ = model.StringArray - if _, layout := parseInLocation(array[0].Str, time.Local); layout != "" { - typ = model.DateTimeArray - } - case gjson.JSON: - typ = model.StringArray - default: - } + typ, _ = gjDetectType(array[0], depth+1) } } default: diff --git a/parser/parser.go b/parser/parser.go index 8234ba14..0c96a0b0 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -15,13 +15,11 @@ limitations under the License. package parser import ( - "fmt" "math" "sync" "time" "github.com/housepower/clickhouse_sinker/model" - "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" ) @@ -167,24 +165,6 @@ func (pp *Pool) ParseDateTime(key string, val string) (t time.Time, err error) { return } -func makeArray(typ int) (val interface{}) { - switch typ { - case model.Bool: - val = []bool{} - case model.Int: - val = []int64{} - case model.Float: - val = []float64{} - case model.String: - val = []string{} - case model.DateTime: - val = []time.Time{} - default: - util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) - } - return -} - func parseInLocation(val string, loc *time.Location) (t time.Time, layout string) { var err error var lay string diff --git a/parser/parser_test.go b/parser/parser_test.go index 3a11593f..ccfbd0d4 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -72,8 +72,8 @@ var jsonSchema = map[string]string{ "null": "Unknown", "bool_true": "Bool", "bool_false": "Bool", - "num_int": "Int", - "num_float": "Float", + "num_int": "Int64", + "num_float": "Float64", "str": "String", "str_int": "String", "str_float": "String", @@ -83,13 +83,13 @@ var jsonSchema = map[string]string{ "str_time_rfc3339_2": "DateTime", "str_time_clickhouse_1": "DateTime", "str_time_clickhouse_2": "DateTime", - "obj": "String", + "obj": "Unknown", "array_empty": "Unknown", "array_null": "Unknown", "array_bool": "BoolArray", - "array_num_int_1": "IntArray", - "array_num_int_2": "IntArray", - "array_num_float": "FloatArray", + "array_num_int_1": "Int64Array", + "array_num_int_2": "Int64Array", + "array_num_float": "Float64Array", "array_str": "StringArray", "array_str_int_1": "StringArray", "array_str_int_2": "StringArray", @@ -98,7 +98,7 @@ var jsonSchema = map[string]string{ "array_str_date_2": "DateTimeArray", "array_str_time_rfc3339": "DateTimeArray", "array_str_time_clickhouse": "DateTimeArray", - "array_obj": "StringArray", + "array_obj": "Unknown", } var csvSample = []byte(`null,true,false,123,123.321,"escaped_""ws",123,123.321,2009-07-13,13/07/2009,2009-07-13T09:07:13Z,2009-07-13T09:07:13.123+08:00,2009-07-13 09:07:13,2009-07-13 09:07:13.123,"{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",[],[null],"[true,false]","[0,255,256,65535,65536,4294967295,4294967296,18446744073709551615,18446744073709551616]","[-9223372036854775808,-2147483649,-2147483648,-32769,-32768,-129,-128,0,127,128,32767,32768,2147483647,2147483648,9223372036854775807]","[4.940656458412465441765687928682213723651e-324,1.401298464324817070923729583289916131280e-45,0.0,3.40282346638528859811704183484516925440e+38,1.797693134862315708145274237317043567981e+308]","[""aa"",""bb"",""cc""]","[""0"",""255"",""256"",""65535"",""65536"",""4294967295"",""4294967296"",""18446744073709551615"",""18446744073709551616""]","[""-9223372036854775808"",""-2147483649"",""-2147483648"",""-32769"",""-32768"",""-129"",""-128"",""0"",""127"",""128"",""32767"",""32768"",""2147483647"",""2147483648"",""9223372036854775807""]","[""4.940656458412465441765687928682213723651e-324"",""1.401298464324817070923729583289916131280e-45"",""0.0"",""3.40282346638528859811704183484516925440e+38"",""1.797693134862315708145274237317043567981e+308""]","[""2009-07-13"",""2009-07-14"",""2009-07-15""]","[""13/07/2009"",""14/07/2009"",""15/07/2009""]","[""2009-07-13T09:07:13Z"",""2009-07-13T09:07:13+08:00"",""2009-07-13T09:07:13.123Z"",""2009-07-13T09:07:13.123+08:00""]","[""2009-07-13 09:07:13"",""2009-07-13 09:07:13.123""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]"`) @@ -220,23 +220,37 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { for j := range testCases { var v interface{} desc := fmt.Sprintf(`%s.%s("%s", %s)`, name, method, testCases[j].Field, strconv.FormatBool(testCases[j].Nullable)) - if name == "csv" && (sliceContains([]string{"GetBool", "GetInt", "GetFloat", "GetDateTime", "GetElasticDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || testCases[j].Nullable) { + if name == "csv" && (sliceContains([]string{"GetBool", "GetInt64", "GetFloat64", "GetDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || testCases[j].Nullable) { skipped = append(skipped, desc) continue } switch method { case "GetBool": v = metric.GetBool(testCases[j].Field, testCases[j].Nullable) - case "GetInt": - v = metric.GetInt(testCases[j].Field, testCases[j].Nullable) - case "GetFloat": - v = metric.GetFloat(testCases[j].Field, testCases[j].Nullable) - case "GetString": - v = metric.GetString(testCases[j].Field, testCases[j].Nullable) + case "GetInt8": + v = metric.GetInt8(testCases[j].Field, testCases[j].Nullable) + case "GetInt16": + v = metric.GetInt16(testCases[j].Field, testCases[j].Nullable) + case "GetInt32": + v = metric.GetInt32(testCases[j].Field, testCases[j].Nullable) + case "GetInt64": + v = metric.GetInt64(testCases[j].Field, testCases[j].Nullable) + case "GetUint8": + v = metric.GetUint8(testCases[j].Field, testCases[j].Nullable) + case "GetUint16": + v = metric.GetUint16(testCases[j].Field, testCases[j].Nullable) + case "GetUint32": + v = metric.GetUint32(testCases[j].Field, testCases[j].Nullable) + case "GetUint64": + v = metric.GetUint64(testCases[j].Field, testCases[j].Nullable) + case "GetFloat32": + v = metric.GetFloat32(testCases[j].Field, testCases[j].Nullable) + case "GetFloat64": + v = metric.GetFloat64(testCases[j].Field, testCases[j].Nullable) case "GetDateTime": v = metric.GetDateTime(testCases[j].Field, testCases[j].Nullable) - case "GetElasticDateTime": - v = metric.GetElasticDateTime(testCases[j].Field, testCases[j].Nullable) + case "GetString": + v = metric.GetString(testCases[j].Field, testCases[j].Nullable) default: panic("error!") } @@ -309,7 +323,7 @@ func TestParserInt(t *testing.T) { {"obj", true, nil}, {"array_empty", true, nil}, } - doTestSimple(t, "GetInt", testCases) + doTestSimple(t, "GetInt64", testCases) } func TestParserFloat(t *testing.T) { @@ -341,7 +355,7 @@ func TestParserFloat(t *testing.T) { {"obj", true, nil}, {"array_empty", true, nil}, } - doTestSimple(t, "GetFloat", testCases) + doTestSimple(t, "GetFloat64", testCases) } func TestParserString(t *testing.T) { @@ -422,117 +436,77 @@ func TestParserDateTime(t *testing.T) { doTestSimple(t, "GetDateTime", testCases) } -func TestParserElasticDateTime(t *testing.T) { - testCases := []SimpleCase{ - // nullable: false - {"not_exist", false, Epoch.Unix()}, - {"null", false, Epoch.Unix()}, - {"bool_true", false, Epoch.Unix()}, - {"bool_false", false, Epoch.Unix()}, - {"num_int", false, UnixFloat(123, timeUnit).Unix()}, - {"num_float", false, UnixFloat(123.321, timeUnit).Unix()}, - {"str", false, Epoch.Unix()}, - {"str_int", false, Epoch.Unix()}, - {"str_float", false, Epoch.Unix()}, - {"str_date_1", false, bdLocalDate.Unix()}, - {"str_time_rfc3339_1", false, bdUtcSec.Unix()}, - {"str_time_rfc3339_2", false, bdShNs.Unix()}, - {"str_time_clickhouse_1", false, bdLocalSec.Unix()}, - {"str_time_clickhouse_2", false, bdLocalNs.Unix()}, - {"obj", false, Epoch.Unix()}, - {"array_empty", false, Epoch.Unix()}, - // nullable: true - {"not_exist", true, nil}, - {"null", true, nil}, - {"bool_true", true, nil}, - {"bool_false", true, nil}, - {"num_int", true, UnixFloat(123, timeUnit).Unix()}, - {"num_float", true, UnixFloat(123.321, timeUnit).Unix()}, - {"str", true, nil}, - {"str_int", true, nil}, - {"str_float", true, nil}, - {"str_date_1", true, bdLocalDate.Unix()}, - {"str_time_rfc3339_1", true, bdUtcSec.Unix()}, - {"str_time_rfc3339_2", true, bdShNs.Unix()}, - {"str_time_clickhouse_1", true, bdLocalSec.Unix()}, - {"str_time_clickhouse_2", true, bdLocalNs.Unix()}, - {"obj", true, nil}, - {"array_empty", true, nil}, - } - doTestSimple(t, "GetElasticDateTime", testCases) -} - func TestParserArray(t *testing.T) { initialize.Do(initMetrics) require.Nil(t, errInit) testCases := []ArrayCase{ - {"not_exist", model.Float, []float64{}}, - {"null", model.Float, []float64{}}, - {"num_int", model.Int, []int64{}}, - {"num_float", model.Float, []float64{}}, + {"not_exist", model.Float64, []float64{}}, + {"null", model.Float64, []float64{}}, + {"num_int", model.Int64, []int64{}}, + {"num_float", model.Float64, []float64{}}, {"str", model.String, []string{}}, {"str_int", model.String, []string{}}, {"str_date_1", model.DateTime, []time.Time{}}, {"obj", model.String, []string{}}, {"array_empty", model.Bool, []bool{}}, - {"array_empty", model.Int, []int64{}}, - {"array_empty", model.Float, []float64{}}, + {"array_empty", model.Int64, []int64{}}, + {"array_empty", model.Float64, []float64{}}, {"array_empty", model.String, []string{}}, {"array_empty", model.DateTime, []time.Time{}}, {"array_null", model.Bool, []bool{false}}, - {"array_null", model.Int, []int64{0}}, - {"array_null", model.Float, []float64{0.0}}, + {"array_null", model.Int64, []int64{0}}, + {"array_null", model.Float64, []float64{0.0}}, {"array_null", model.String, []string{""}}, {"array_null", model.DateTime, []time.Time{Epoch}}, {"array_bool", model.Bool, []bool{true, false}}, - {"array_bool", model.Int, []int64{1, 0}}, - {"array_bool", model.Float, []float64{0.0, 0.0}}, + {"array_bool", model.Int64, []int64{1, 0}}, + {"array_bool", model.Float64, []float64{0.0, 0.0}}, {"array_bool", model.String, []string{"true", "false"}}, {"array_bool", model.DateTime, []time.Time{Epoch, Epoch}}, {"array_num_int_1", model.Bool, []bool{false, false, false, false, false, false, false, false, false}}, - {"array_num_int_1", model.Int, []int64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 0, 0}}, - {"array_num_int_1", model.Float, []float64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 18446744073709551615, 18446744073709551616}}, + {"array_num_int_1", model.Int64, []int64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 0, 0}}, + {"array_num_int_1", model.Float64, []float64{0, 255, 256, 65535, 65536, 4294967295, 4294967296, 18446744073709551615, 18446744073709551616}}, {"array_num_int_1", model.String, []string{"0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"}}, {"array_num_int_1", model.DateTime, []time.Time{Epoch, UnixFloat(255, timeUnit), UnixFloat(256, timeUnit), UnixFloat(65535, timeUnit), UnixFloat(65536, timeUnit), UnixFloat(4294967295, timeUnit), UnixFloat(4294967296, timeUnit), Epoch, Epoch}}, {"array_num_int_2", model.Bool, []bool{false, false, false, false, false, false, false, false, false, false, false, false, false, false, false}}, - {"array_num_int_2", model.Int, []int64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, - {"array_num_int_2", model.Float, []float64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, + {"array_num_int_2", model.Int64, []int64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, + {"array_num_int_2", model.Float64, []float64{-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807}}, {"array_num_int_2", model.String, []string{"-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"}}, {"array_num_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, UnixFloat(127, timeUnit), UnixFloat(128, timeUnit), UnixFloat(32767, timeUnit), UnixFloat(32768, timeUnit), UnixFloat(2147483647, timeUnit), UnixFloat(2147483648, timeUnit), UnixFloat(9223372036854775807, timeUnit)}}, {"array_num_float", model.Bool, []bool{false, false, false, false, false, false, false}}, - {"array_num_float", model.Int, []int64{0, 0, 0, 0, 0, 0, 0}}, - {"array_num_float", model.Float, []float64{4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308, math.Inf(-1), math.Inf(1)}}, + {"array_num_float", model.Int64, []int64{0, 0, 0, 0, 0, 0, 0}}, + {"array_num_float", model.Float64, []float64{4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308, math.Inf(-1), math.Inf(1)}}, {"array_num_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"}}, {"array_num_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, UnixFloat(3.40282346638528859811704183484516925440e+38, timeUnit), UnixFloat(1.797693134862315708145274237317043567981e+308, timeUnit), UnixFloat(math.Inf(-1), timeUnit), UnixFloat(math.Inf(1), timeUnit)}}, {"array_str", model.Bool, []bool{false, false, false}}, - {"array_str", model.Int, []int64{0, 0, 0}}, - {"array_str", model.Float, []float64{0.0, 0.0, 0.0}}, + {"array_str", model.Int64, []int64{0, 0, 0}}, + {"array_str", model.Float64, []float64{0.0, 0.0, 0.0}}, {"array_str", model.String, []string{"aa", "bb", "cc"}}, {"array_str", model.DateTime, []time.Time{Epoch, Epoch, Epoch}}, {"array_str_int_1", model.Bool, []bool{false, false, false, false, false, false, false, false, false}}, - {"array_str_int_1", model.Int, []int64{0, 0, 0, 0, 0, 0, 0, 0, 0}}, - {"array_str_int_1", model.Float, []float64{0, 0, 0, 0, 0, 0, 0, 0, 0}}, + {"array_str_int_1", model.Int64, []int64{0, 0, 0, 0, 0, 0, 0, 0, 0}}, + {"array_str_int_1", model.Float64, []float64{0, 0, 0, 0, 0, 0, 0, 0, 0}}, {"array_str_int_1", model.String, []string{"0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"}}, {"array_str_int_1", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch}}, {"array_str_int_2", model.Bool, []bool{false, false, false, false, false, false, false, false, false, false, false, false, false, false, false}}, - {"array_str_int_2", model.Int, []int64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}}, - {"array_str_int_2", model.Float, []float64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}}, + {"array_str_int_2", model.Int64, []int64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}}, + {"array_str_int_2", model.Float64, []float64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}}, {"array_str_int_2", model.String, []string{"-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"}}, {"array_str_int_2", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch}}, {"array_str_float", model.Bool, []bool{false, false, false, false, false, false, false}}, - {"array_str_float", model.Int, []int64{0, 0, 0, 0, 0, 0, 0}}, - {"array_str_float", model.Float, []float64{0, 0, 0, 0, 0, 0, 0}}, + {"array_str_float", model.Int64, []int64{0, 0, 0, 0, 0, 0, 0}}, + {"array_str_float", model.Float64, []float64{0, 0, 0, 0, 0, 0, 0}}, {"array_str_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"}}, {"array_str_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, Epoch, Epoch, Epoch, Epoch}}, @@ -719,8 +693,13 @@ func TestFastjsonDetectSchema(t *testing.T) { if obj, err = c.value.Object(); err != nil { return } - obj.Visit(func(key []byte, v *fastjson.Value) { - act[string(key)] = model.GetTypeName(fjDetectType(v)) + obj.Visit(func(k []byte, v *fastjson.Value) { + typ, array := fjDetectType(v, 0) + tn := model.GetTypeName(typ) + if typ != model.Unknown && array { + tn += "Array" + } + act[string(k)] = tn }) require.Equal(t, jsonSchema, act) } @@ -735,7 +714,12 @@ func TestGjsonDetectSchema(t *testing.T) { c, _ := metric.(*GjsonMetric) obj := gjson.Parse(c.raw) obj.ForEach(func(k, v gjson.Result) bool { - act[k.Str] = model.GetTypeName(gjDetectType(v)) + typ, array := gjDetectType(v, 0) + tn := model.GetTypeName(typ) + if typ != model.Unknown && array { + tn += "Array" + } + act[k.Str] = tn return true }) require.Equal(t, jsonSchema, act) diff --git a/task/sharding.go b/task/sharding.go index 0d1a0eeb..21c4d729 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -11,6 +11,7 @@ import ( "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" + "github.com/shopspring/decimal" "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" ) @@ -26,9 +27,13 @@ func NewShardingPolicy(shardingKey string, shardingStripe uint64, dims []*model. colSeq := -1 for i, dim := range dims { if dim.Name == shardingKey { + if dim.Nullable || dim.Array { + err = errors.Newf("invalid shardingKey %s, expect its type be numerical or string", shardingKey) + return + } colSeq = i switch dim.Type { - case model.Int, model.Float, model.DateTime, model.ElasticDateTime: + case model.Int8, model.Int16, model.Int32, model.Int64, model.Uint8, model.Uint16, model.Uint32, model.Uint64, model.Float32, model.Float64, model.Decimal, model.DateTime: //numerical if policy.stripe <= 0 { policy.stripe = uint64(1) @@ -79,6 +84,8 @@ func (policy *ShardingPolicy) Calc(row *model.Row) (shard int, err error) { valu64 = uint64(v) case float64: valu64 = uint64(v) + case decimal.Decimal: + valu64 = uint64(v.IntPart()) case time.Time: valu64 = uint64(v.Unix()) default: diff --git a/task/task.go b/task/task.go index 208fc29f..3f0afd91 100644 --- a/task/task.go +++ b/task/task.go @@ -54,6 +54,7 @@ type Service struct { knownKeys sync.Map newKeys sync.Map + warnKeys sync.Map cntNewKeys int32 // size of newKeys tid goetty.Timeout @@ -297,7 +298,7 @@ func (service *Service) put(msg *model.InputMessage) { } else { row = model.MetricToRow(metric, msg, service.dims, service.idxSerID, service.nameKey, service.lblBlkList) if taskCfg.DynamicSchema.Enable { - foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys, service.whiteList, service.blackList) + foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys, &service.warnKeys, service.whiteList, service.blackList, msg.Partition, msg.Offset) } // Dumping message and result //util.Logger.Debug("parsed kafka message", zap.Int("partition", msg.Partition), zap.Int64("offset", msg.Offset), From 55aea1a972c899fbd88895a9bd84d6d95381eb9d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 21 Jul 2022 15:13:03 +0800 Subject: [PATCH 263/404] boundary check for integers and floats --- parser/csv.go | 76 +++++++++++++++---------- parser/fastjson.go | 99 +++++++++++++++++++------------- parser/gjson.go | 127 ++++++++++++++++++++++++------------------ parser/parser_test.go | 2 +- 4 files changed, 182 insertions(+), 122 deletions(-) diff --git a/parser/csv.go b/parser/csv.go index ba860f96..f6156bfd 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -18,6 +18,7 @@ import ( "bytes" "encoding/csv" "fmt" + "math" "regexp" "strconv" "sync" @@ -110,46 +111,46 @@ func (c *CsvMetric) GetBool(key string, nullable bool) (val interface{}) { } func (c *CsvMetric) GetInt8(key string, nullable bool) (val interface{}) { - return CsvGetInt[int8](c, key, nullable) + return CsvGetInt[int8](c, key, nullable, math.MinInt8, math.MaxInt8) } func (c *CsvMetric) GetInt16(key string, nullable bool) (val interface{}) { - return CsvGetInt[int16](c, key, nullable) + return CsvGetInt[int16](c, key, nullable, math.MinInt16, math.MaxInt16) } func (c *CsvMetric) GetInt32(key string, nullable bool) (val interface{}) { - return CsvGetInt[int32](c, key, nullable) + return CsvGetInt[int32](c, key, nullable, math.MinInt32, math.MaxInt32) } func (c *CsvMetric) GetInt64(key string, nullable bool) (val interface{}) { - return CsvGetInt[int64](c, key, nullable) + return CsvGetInt[int64](c, key, nullable, math.MinInt64, math.MaxInt64) } func (c *CsvMetric) GetUint8(key string, nullable bool) (val interface{}) { - return CsvGetUint[uint8](c, key, nullable) + return CsvGetUint[uint8](c, key, nullable, math.MaxUint8) } func (c *CsvMetric) GetUint16(key string, nullable bool) (val interface{}) { - return CsvGetUint[uint16](c, key, nullable) + return CsvGetUint[uint16](c, key, nullable, math.MaxUint16) } func (c *CsvMetric) GetUint32(key string, nullable bool) (val interface{}) { - return CsvGetUint[uint32](c, key, nullable) + return CsvGetUint[uint32](c, key, nullable, math.MaxUint32) } func (c *CsvMetric) GetUint64(key string, nullable bool) (val interface{}) { - return CsvGetUint[uint64](c, key, nullable) + return CsvGetUint[uint64](c, key, nullable, math.MaxUint64) } func (c *CsvMetric) GetFloat32(key string, nullable bool) (val interface{}) { - return CsvGetFloat[float32](c, key, nullable) + return CsvGetFloat[float32](c, key, nullable, math.MaxFloat32) } func (c *CsvMetric) GetFloat64(key string, nullable bool) (val interface{}) { - return CsvGetFloat[float64](c, key, nullable) + return CsvGetFloat[float64](c, key, nullable, math.MaxFloat64) } -func CsvGetInt[T constraints.Signed](c *CsvMetric, key string, nullable bool) (val interface{}) { +func CsvGetInt[T constraints.Signed](c *CsvMetric, key string, nullable bool, min, max int64) (val interface{}) { var idx int var ok bool if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { @@ -162,12 +163,19 @@ func CsvGetInt[T constraints.Signed](c *CsvMetric, key string, nullable bool) (v if s := c.values[idx]; s == "true" { val = T(1) } else { - val = T(fastfloat.ParseInt64BestEffort(s)) + val2 := fastfloat.ParseInt64BestEffort(s) + if val2 < min { + val = T(min) + } else if val2 > max { + val = T(max) + } else { + val = T(val2) + } } return } -func CsvGetUint[T constraints.Unsigned](c *CsvMetric, key string, nullable bool) (val interface{}) { +func CsvGetUint[T constraints.Unsigned](c *CsvMetric, key string, nullable bool, max uint64) (val interface{}) { var idx int var ok bool if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { @@ -180,13 +188,18 @@ func CsvGetUint[T constraints.Unsigned](c *CsvMetric, key string, nullable bool) if s := c.values[idx]; s == "true" { val = T(1) } else { - val = T(fastfloat.ParseUint64BestEffort(s)) + val2 := fastfloat.ParseUint64BestEffort(s) + if val2 > max { + val = T(max) + } else { + val = T(val2) + } } return } // GetFloat returns the value as float -func CsvGetFloat[T constraints.Float](c *CsvMetric, key string, nullable bool) (val interface{}) { +func CsvGetFloat[T constraints.Float](c *CsvMetric, key string, nullable bool, max float64) (val interface{}) { var idx int var ok bool if idx, ok = c.pp.csvFormat[key]; !ok || c.values[idx] == "null" { @@ -196,7 +209,12 @@ func CsvGetFloat[T constraints.Float](c *CsvMetric, key string, nullable bool) ( val = float64(0.0) return } - val = T(fastfloat.ParseBestEffort(c.values[idx])) + val2 := fastfloat.ParseBestEffort(c.values[idx]) + if val2 > max { + val = T(max) + } else { + val = T(val2) + } return } @@ -240,29 +258,29 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { } val = results case model.Int8: - val = GjsonIntArray[int8](array) + val = GjsonIntArray[int8](array, math.MinInt8, math.MaxInt8) case model.Int16: - val = GjsonIntArray[int16](array) + val = GjsonIntArray[int16](array, math.MinInt16, math.MaxInt16) case model.Int32: - val = GjsonIntArray[int32](array) + val = GjsonIntArray[int32](array, math.MinInt32, math.MaxInt32) case model.Int64: - val = GjsonIntArray[int64](array) + val = GjsonIntArray[int64](array, math.MinInt64, math.MaxInt64) case model.Uint8: - val = GjsonUintArray[uint8](array) + val = GjsonUintArray[uint8](array, math.MaxUint8) case model.Uint16: - val = GjsonUintArray[uint16](array) + val = GjsonUintArray[uint16](array, math.MaxUint16) case model.Uint32: - val = GjsonUintArray[uint32](array) + val = GjsonUintArray[uint32](array, math.MaxUint32) case model.Uint64: - val = GjsonUintArray[uint64](array) + val = GjsonUintArray[uint64](array, math.MaxUint64) case model.Float32: - val = GjsonFloatArray[float32](array) + val = GjsonFloatArray[float32](array, math.MaxFloat32) case model.Float64: - val = GjsonFloatArray[float64](array) + val = GjsonFloatArray[float64](array, math.MaxFloat64) case model.Decimal: results := make([]decimal.Decimal, 0, len(array)) + var f float64 for _, e := range array { - var f float64 switch e.Type { case gjson.Number: f = e.Num @@ -274,8 +292,8 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { val = results case model.String: results := make([]string, 0, len(array)) + var s string for _, e := range array { - var s string switch e.Type { case gjson.Null: s = "" @@ -289,8 +307,8 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { val = results case model.DateTime: results := make([]time.Time, 0, len(array)) + var t time.Time for _, e := range array { - var t time.Time switch e.Type { case gjson.Number: t = UnixFloat(e.Num, c.pp.timeUnit) diff --git a/parser/fastjson.go b/parser/fastjson.go index 2a98bdf3..32d102e4 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -17,6 +17,7 @@ package parser import ( "fmt" + "math" "regexp" "sync" "time" @@ -98,46 +99,46 @@ func (c *FastjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) } func (c *FastjsonMetric) GetInt8(key string, nullable bool) (val interface{}) { - return FastjsonGetInt[int8](c, key, nullable) + return FastjsonGetInt[int8](c, key, nullable, math.MinInt8, math.MaxInt8) } func (c *FastjsonMetric) GetInt16(key string, nullable bool) (val interface{}) { - return FastjsonGetInt[int16](c, key, nullable) + return FastjsonGetInt[int16](c, key, nullable, math.MinInt16, math.MaxInt16) } func (c *FastjsonMetric) GetInt32(key string, nullable bool) (val interface{}) { - return FastjsonGetInt[int32](c, key, nullable) + return FastjsonGetInt[int32](c, key, nullable, math.MinInt32, math.MaxInt32) } func (c *FastjsonMetric) GetInt64(key string, nullable bool) (val interface{}) { - return FastjsonGetInt[int64](c, key, nullable) + return FastjsonGetInt[int64](c, key, nullable, math.MinInt64, math.MaxInt64) } func (c *FastjsonMetric) GetUint8(key string, nullable bool) (val interface{}) { - return FastjsonGetUint[uint8](c, key, nullable) + return FastjsonGetUint[uint8](c, key, nullable, math.MaxUint8) } func (c *FastjsonMetric) GetUint16(key string, nullable bool) (val interface{}) { - return FastjsonGetUint[uint16](c, key, nullable) + return FastjsonGetUint[uint16](c, key, nullable, math.MaxUint16) } func (c *FastjsonMetric) GetUint32(key string, nullable bool) (val interface{}) { - return FastjsonGetUint[uint32](c, key, nullable) + return FastjsonGetUint[uint32](c, key, nullable, math.MaxUint32) } func (c *FastjsonMetric) GetUint64(key string, nullable bool) (val interface{}) { - return FastjsonGetUint[uint64](c, key, nullable) + return FastjsonGetUint[uint64](c, key, nullable, math.MaxUint64) } func (c *FastjsonMetric) GetFloat32(key string, nullable bool) (val interface{}) { - return FastjsonGetFloat[float32](c, key, nullable) + return FastjsonGetFloat[float32](c, key, nullable, math.MaxFloat32) } func (c *FastjsonMetric) GetFloat64(key string, nullable bool) (val interface{}) { - return FastjsonGetFloat[float64](c, key, nullable) + return FastjsonGetFloat[float64](c, key, nullable, math.MaxFloat64) } -func FastjsonGetInt[T constraints.Signed](c *FastjsonMetric, key string, nullable bool) (val interface{}) { +func FastjsonGetInt[T constraints.Signed](c *FastjsonMetric, key string, nullable bool, min, max int64) (val interface{}) { v := c.value.Get(key) if !fjCompatibleInt(v) { val = getDefaultInt[T](nullable) @@ -151,6 +152,10 @@ func FastjsonGetInt[T constraints.Signed](c *FastjsonMetric, key string, nullabl default: if val2, err := v.Int64(); err != nil { val = getDefaultInt[T](nullable) + } else if val2 < min { + val = T(min) + } else if val2 > max { + val = T(max) } else { val = T(val2) } @@ -158,7 +163,7 @@ func FastjsonGetInt[T constraints.Signed](c *FastjsonMetric, key string, nullabl return } -func FastjsonGetUint[T constraints.Unsigned](c *FastjsonMetric, key string, nullable bool) (val interface{}) { +func FastjsonGetUint[T constraints.Unsigned](c *FastjsonMetric, key string, nullable bool, max uint64) (val interface{}) { v := c.value.Get(key) if !fjCompatibleInt(v) { val = getDefaultInt[T](nullable) @@ -172,6 +177,8 @@ func FastjsonGetUint[T constraints.Unsigned](c *FastjsonMetric, key string, null default: if val2, err := v.Uint64(); err != nil { val = getDefaultInt[T](nullable) + } else if val2 > max { + val = T(max) } else { val = T(val2) } @@ -179,7 +186,7 @@ func FastjsonGetUint[T constraints.Unsigned](c *FastjsonMetric, key string, null return } -func FastjsonGetFloat[T constraints.Float](c *FastjsonMetric, key string, nullable bool) (val interface{}) { +func FastjsonGetFloat[T constraints.Float](c *FastjsonMetric, key string, nullable bool, max float64) (val interface{}) { v := c.value.Get(key) if !fjCompatibleFloat(v) { val = getDefaultFloat[T](nullable) @@ -187,6 +194,8 @@ func FastjsonGetFloat[T constraints.Float](c *FastjsonMetric, key string, nullab } if val2, err := v.Float64(); err != nil { val = getDefaultFloat[T](nullable) + } else if val2 > max { + val = T(max) } else { val = T(val2) } @@ -237,25 +246,25 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { } val = arr case model.Int8: - val = FastjsonIntArray[int8](array) + val = FastjsonIntArray[int8](array, math.MinInt8, math.MaxInt8) case model.Int16: - val = FastjsonIntArray[int16](array) + val = FastjsonIntArray[int16](array, math.MinInt16, math.MaxInt16) case model.Int32: - val = FastjsonIntArray[int32](array) + val = FastjsonIntArray[int32](array, math.MinInt32, math.MaxInt32) case model.Int64: - val = FastjsonIntArray[int64](array) + val = FastjsonIntArray[int64](array, math.MinInt64, math.MaxInt64) case model.Uint8: - val = FastjsonUintArray[uint8](array) + val = FastjsonUintArray[uint8](array, math.MaxUint8) case model.Uint16: - val = FastjsonUintArray[uint16](array) + val = FastjsonUintArray[uint16](array, math.MaxUint16) case model.Uint32: - val = FastjsonUintArray[uint32](array) + val = FastjsonUintArray[uint32](array, math.MaxUint32) case model.Uint64: - val = FastjsonUintArray[uint64](array) + val = FastjsonUintArray[uint64](array, math.MaxUint64) case model.Float32: - val = FastjsonFloatArray[float32](array) + val = FastjsonFloatArray[float32](array, math.MaxFloat32) case model.Float64: - val = FastjsonFloatArray[float64](array) + val = FastjsonFloatArray[float64](array, math.MaxFloat64) case model.Decimal: arr := make([]decimal.Decimal, 0) for _, e := range array { @@ -265,8 +274,8 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { val = arr case model.String: arr := make([]string, 0) + var s string for _, e := range array { - var s string switch e.Type() { case fastjson.TypeNull: s = "" @@ -281,8 +290,8 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { val = arr case model.DateTime: arr := make([]time.Time, 0) + var t time.Time for _, e := range array { - var t time.Time switch e.Type() { case fastjson.TypeNumber: if f, err := e.Float64(); err != nil { @@ -311,43 +320,57 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { return } -func FastjsonIntArray[T constraints.Signed](a []*fastjson.Value) (arr []T) { +func FastjsonIntArray[T constraints.Signed](a []*fastjson.Value, min, max int64) (arr []T) { arr = make([]T, 0) + var val T for _, e := range a { - var val T if e.Type() == fastjson.TypeTrue { val = T(1) } else { - var v int64 - v, _ = e.Int64() - val = T(v) + val2, _ := e.Int64() + if val2 < min { + val = T(min) + } else if val2 > max { + val = T(max) + } else { + val = T(val2) + } } arr = append(arr, val) } return } -func FastjsonUintArray[T constraints.Unsigned](a []*fastjson.Value) (arr []T) { +func FastjsonUintArray[T constraints.Unsigned](a []*fastjson.Value, max uint64) (arr []T) { arr = make([]T, 0) + var val T for _, e := range a { - var val T if e.Type() == fastjson.TypeTrue { val = T(1) } else { - var v uint64 - v, _ = e.Uint64() - val = T(v) + val2, _ := e.Uint64() + if val2 > max { + val = T(max) + } else { + val = T(val2) + } } arr = append(arr, val) } return } -func FastjsonFloatArray[T constraints.Float](a []*fastjson.Value) (arr []T) { +func FastjsonFloatArray[T constraints.Float](a []*fastjson.Value, max float64) (arr []T) { arr = make([]T, 0) + var val T for _, e := range a { - v, _ := e.Float64() - arr = append(arr, T(v)) + val2, _ := e.Float64() + if val2 > max { + val = T(max) + } else { + val = T(val2) + } + arr = append(arr, val) } return } diff --git a/parser/gjson.go b/parser/gjson.go index 2ba927ce..8029ec84 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -17,6 +17,7 @@ package parser import ( "fmt" + "math" "regexp" "strconv" "sync" @@ -93,46 +94,46 @@ func (c *GjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) { } func (c *GjsonMetric) GetInt8(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int8](c, key, nullable) + return GjsonGetInt[int8](c, key, nullable, math.MinInt8, math.MaxInt8) } func (c *GjsonMetric) GetInt16(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int16](c, key, nullable) + return GjsonGetInt[int16](c, key, nullable, math.MinInt16, math.MaxInt16) } func (c *GjsonMetric) GetInt32(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int32](c, key, nullable) + return GjsonGetInt[int32](c, key, nullable, math.MinInt32, math.MaxInt32) } func (c *GjsonMetric) GetInt64(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int64](c, key, nullable) + return GjsonGetInt[int64](c, key, nullable, math.MinInt64, math.MaxInt64) } func (c *GjsonMetric) GetUint8(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint8](c, key, nullable) + return GjsonGetUint[uint8](c, key, nullable, math.MaxUint8) } func (c *GjsonMetric) GetUint16(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint16](c, key, nullable) + return GjsonGetUint[uint16](c, key, nullable, math.MaxUint16) } func (c *GjsonMetric) GetUint32(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint32](c, key, nullable) + return GjsonGetUint[uint32](c, key, nullable, math.MaxUint32) } func (c *GjsonMetric) GetUint64(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint64](c, key, nullable) + return GjsonGetUint[uint64](c, key, nullable, math.MaxUint64) } func (c *GjsonMetric) GetFloat32(key string, nullable bool) (val interface{}) { - return GjsonGetFloat[float32](c, key, nullable) + return GjsonGetFloat[float32](c, key, nullable, math.MaxFloat32) } func (c *GjsonMetric) GetFloat64(key string, nullable bool) (val interface{}) { - return GjsonGetFloat[float64](c, key, nullable) + return GjsonGetFloat[float64](c, key, nullable, math.MaxFloat64) } -func GjsonGetInt[T constraints.Signed](c *GjsonMetric, key string, nullable bool) (val interface{}) { +func GjsonGetInt[T constraints.Signed](c *GjsonMetric, key string, nullable bool, min, max int64) (val interface{}) { r := gjson.Get(c.raw, key) if !gjCompatibleInt(r) { val = getDefaultInt[T](nullable) @@ -144,10 +145,14 @@ func GjsonGetInt[T constraints.Signed](c *GjsonMetric, key string, nullable bool case gjson.False: val = T(0) case gjson.Number: - if v := r.Int(); float64(v) != r.Num { + if val2 := r.Int(); float64(val2) != r.Num { val = getDefaultInt[T](nullable) + } else if val2 < min { + val = T(min) + } else if val2 > max { + val = T(max) } else { - val = T(v) + val = T(val2) } default: val = getDefaultInt[T](nullable) @@ -155,7 +160,7 @@ func GjsonGetInt[T constraints.Signed](c *GjsonMetric, key string, nullable bool return } -func GjsonGetUint[T constraints.Unsigned](c *GjsonMetric, key string, nullable bool) (val interface{}) { +func GjsonGetUint[T constraints.Unsigned](c *GjsonMetric, key string, nullable bool, max uint64) (val interface{}) { r := gjson.Get(c.raw, key) if !gjCompatibleInt(r) { val = getDefaultInt[T](nullable) @@ -167,10 +172,12 @@ func GjsonGetUint[T constraints.Unsigned](c *GjsonMetric, key string, nullable b case gjson.False: val = T(0) case gjson.Number: - if v := r.Uint(); float64(v) != r.Num { + if val2 := r.Uint(); float64(val2) != r.Num { val = getDefaultInt[T](nullable) + } else if val2 > max { + val = T(max) } else { - val = T(v) + val = T(val2) } default: val = getDefaultInt[T](nullable) @@ -178,7 +185,7 @@ func GjsonGetUint[T constraints.Unsigned](c *GjsonMetric, key string, nullable b return } -func GjsonGetFloat[T constraints.Float](c *GjsonMetric, key string, nullable bool) (val interface{}) { +func GjsonGetFloat[T constraints.Float](c *GjsonMetric, key string, nullable bool, max float64) (val interface{}) { r := gjson.Get(c.raw, key) if !gjCompatibleFloat(r) { val = getDefaultFloat[T](nullable) @@ -186,7 +193,11 @@ func GjsonGetFloat[T constraints.Float](c *GjsonMetric, key string, nullable boo } switch r.Type { case gjson.Number: - val = T(r.Num) + if r.Num > max { + val = T(max) + } else { + val = T(r.Num) + } default: val = getDefaultFloat[T](nullable) } @@ -228,29 +239,29 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { } val = results case model.Int8: - val = GjsonIntArray[int8](array) + val = GjsonIntArray[int8](array, math.MinInt8, math.MaxInt8) case model.Int16: - val = GjsonIntArray[int16](array) + val = GjsonIntArray[int16](array, math.MinInt16, math.MaxInt16) case model.Int32: - val = GjsonIntArray[int32](array) + val = GjsonIntArray[int32](array, math.MinInt32, math.MaxInt32) case model.Int64: - val = GjsonIntArray[int64](array) + val = GjsonIntArray[int64](array, math.MinInt64, math.MaxInt64) case model.Uint8: - val = GjsonUintArray[uint8](array) + val = GjsonUintArray[uint8](array, math.MaxUint8) case model.Uint16: - val = GjsonUintArray[uint16](array) + val = GjsonUintArray[uint16](array, math.MaxUint16) case model.Uint32: - val = GjsonUintArray[uint32](array) + val = GjsonUintArray[uint32](array, math.MaxUint32) case model.Uint64: - val = GjsonUintArray[uint64](array) + val = GjsonUintArray[uint64](array, math.MaxUint64) case model.Float32: - val = GjsonFloatArray[float32](array) + val = GjsonFloatArray[float32](array, math.MaxFloat32) case model.Float64: - val = GjsonFloatArray[float64](array) + val = GjsonFloatArray[float64](array, math.MaxFloat64) case model.Decimal: results := make([]decimal.Decimal, 0, len(array)) + var f float64 for _, e := range array { - var f float64 switch e.Type { case gjson.Number: f = e.Num @@ -262,8 +273,8 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { val = results case model.String: results := make([]string, 0, len(array)) + var s string for _, e := range array { - var s string switch e.Type { case gjson.Null: s = "" @@ -277,8 +288,8 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { val = results case model.DateTime: results := make([]time.Time, 0, len(array)) + var t time.Time for _, e := range array { - var t time.Time switch e.Type { case gjson.Number: t = UnixFloat(e.Num, c.pp.timeUnit) @@ -299,61 +310,69 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { return } -func GjsonIntArray[T constraints.Signed](a []gjson.Result) (arr []T) { +func GjsonIntArray[T constraints.Signed](a []gjson.Result, min, max int64) (arr []T) { arr = make([]T, 0, len(a)) + var val T for _, e := range a { - var v T switch e.Type { case gjson.True: - v = T(1) + val = T(1) case gjson.Number: - var tmpv int64 - if tmpv = e.Int(); float64(tmpv) != e.Num { - v = T(0) + if val2 := e.Int(); float64(val2) != e.Num { + val = T(0) + } else if val2 < min { + val = T(min) + } else if val2 > max { + val = T(max) } else { - v = T(tmpv) + val = T(val2) } default: - v = T(0) + val = T(0) } - arr = append(arr, v) + arr = append(arr, val) } return } -func GjsonUintArray[T constraints.Unsigned](a []gjson.Result) (arr []T) { +func GjsonUintArray[T constraints.Unsigned](a []gjson.Result, max uint64) (arr []T) { arr = make([]T, 0, len(a)) + var val T for _, e := range a { - var v T switch e.Type { case gjson.True: - v = T(1) + val = T(1) case gjson.Number: - var tmpv uint64 - if tmpv = e.Uint(); float64(tmpv) != e.Num { - v = T(0) + if val2 := e.Uint(); float64(val2) != e.Num { + val = T(0) + } else if val2 > max { + val = T(max) } else { - v = T(tmpv) + val = T(val2) } default: - v = T(0) + val = T(0) } - arr = append(arr, v) + arr = append(arr, val) } return } -func GjsonFloatArray[T constraints.Float](a []gjson.Result) (arr []T) { +func GjsonFloatArray[T constraints.Float](a []gjson.Result, max float64) (arr []T) { arr = make([]T, 0, len(a)) + var val T for _, e := range a { - var v T switch e.Type { case gjson.Number: - v = T(e.Num) + if e.Num > max { + val = T(max) + } else { + val = T(e.Num) + } default: - v = T(0.0) + val = T(0.0) } - arr = append(arr, v) + arr = append(arr, val) } return } diff --git a/parser/parser_test.go b/parser/parser_test.go index ccfbd0d4..04f9cf9b 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -482,7 +482,7 @@ func TestParserArray(t *testing.T) { {"array_num_float", model.Bool, []bool{false, false, false, false, false, false, false}}, {"array_num_float", model.Int64, []int64{0, 0, 0, 0, 0, 0, 0}}, - {"array_num_float", model.Float64, []float64{4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308, math.Inf(-1), math.Inf(1)}}, + {"array_num_float", model.Float64, []float64{4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308, math.Inf(-1), math.MaxFloat64}}, {"array_num_float", model.String, []string{"4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"}}, {"array_num_float", model.DateTime, []time.Time{Epoch, Epoch, Epoch, UnixFloat(3.40282346638528859811704183484516925440e+38, timeUnit), UnixFloat(1.797693134862315708145274237317043567981e+308, timeUnit), UnixFloat(math.Inf(-1), timeUnit), UnixFloat(math.Inf(1), timeUnit)}}, From fb0c86b8e6a3342d11aaa693d4b54da15d1fc38f Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 21 Jul 2022 15:31:48 +0800 Subject: [PATCH 264/404] bump dependent modules --- go.mod | 28 ++++++++++++------------ go.sum | 67 ++++++++++++++++++++++++++++++---------------------------- 2 files changed, 49 insertions(+), 46 deletions(-) diff --git a/go.mod b/go.mod index 10bada1e..06f86f90 100644 --- a/go.mod +++ b/go.mod @@ -6,25 +6,25 @@ require ( github.com/ClickHouse/clickhouse-go/v2 v2.2.0 github.com/RoaringBitmap/roaring v1.2.1 github.com/Shopify/sarama v1.34.1 - github.com/bytedance/sonic v1.3.1 + github.com/bytedance/sonic v1.3.3 github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 - github.com/google/gops v0.3.23 + github.com/google/gops v0.3.25 github.com/jcmturner/gokrb5/v8 v8.4.2 github.com/jinzhu/copier v0.3.5 github.com/nacos-group/nacos-sdk-go v1.1.1 github.com/prometheus/client_golang v1.12.2 - github.com/prometheus/common v0.34.0 + github.com/prometheus/common v0.37.0 github.com/segmentio/kafka-go v0.4.32 github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.7.5 - github.com/thanos-io/thanos v0.27.0-rc.0 + github.com/thanos-io/thanos v0.27.0 github.com/tidwall/gjson v1.14.1 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/twmb/franz-go v1.6.0 github.com/twmb/franz-go/pkg/kadm v1.1.1 github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 - github.com/twmb/franz-go/plugin/kzap v1.1.0 + github.com/twmb/franz-go/plugin/kzap v1.1.1 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.1.1 go.uber.org/zap v1.21.0 @@ -34,13 +34,13 @@ require ( ) require ( - github.com/aliyun/alibaba-cloud-sdk-go v1.61.1639 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.61.1682 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.2.2 // indirect github.com/buger/jsonparser v1.1.1 // indirect github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/eapache/go-resiliency v1.2.0 // indirect + github.com/eapache/go-resiliency v1.3.0 // indirect github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 // indirect github.com/eapache/queue v1.1.0 // indirect github.com/go-errors/errors v1.4.2 // indirect @@ -56,8 +56,8 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.15.6 // indirect - github.com/klauspost/cpuid/v2 v2.0.13 // indirect + github.com/klauspost/compress v1.15.8 // indirect + github.com/klauspost/cpuid/v2 v2.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect @@ -78,15 +78,15 @@ require ( github.com/xdg-go/stringprep v1.0.3 // indirect github.com/xdg/scram v1.0.5 // indirect github.com/xdg/stringprep v1.0.3 // indirect - go.opentelemetry.io/otel v1.7.0 // indirect - go.opentelemetry.io/otel/trace v1.7.0 // indirect + go.opentelemetry.io/otel v1.8.0 // indirect + go.opentelemetry.io/otel/trace v1.8.0 // indirect go.uber.org/atomic v1.9.0 // indirect go.uber.org/multierr v1.8.0 // indirect golang.org/x/arch v0.0.0-20220412001346-fc48f9fe4c15 // indirect - golang.org/x/crypto v0.0.0-20220525230936-793ad666bf5e // indirect - golang.org/x/net v0.0.0-20220607020251-c690dde0001d // indirect + golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d // indirect + golang.org/x/net v0.0.0-20220708220712-1185a9018129 // indirect golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f // indirect - golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d // indirect + golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8 // indirect golang.org/x/text v0.3.7 // indirect google.golang.org/protobuf v1.28.0 // indirect gopkg.in/ini.v1 v1.66.6 // indirect diff --git a/go.sum b/go.sum index 8ac00d02..f556183d 100644 --- a/go.sum +++ b/go.sum @@ -46,15 +46,14 @@ github.com/Shopify/sarama v1.34.1/go.mod h1:NZSNswsnStpq8TUdFaqnpXm2Do6KRzTIjdBd github.com/Shopify/toxiproxy/v2 v2.4.0 h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64= github.com/Shopify/toxiproxy/v2 v2.4.0/go.mod h1:3ilnjng821bkozDRxNoo64oI/DKqM+rOyJzb564+bvg= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= -github.com/StackExchange/wmi v1.2.1/go.mod h1:rcmrprowKIVzvc+NUiLncP2uuArMWLCbu9SBzvHz7e8= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.61.1639 h1:zP12a7XLwrPnTJd+BwR0yzK5TNSRvlzSp+KYtTnvgo4= -github.com/aliyun/alibaba-cloud-sdk-go v1.61.1639/go.mod h1:RcDobYh8k5VP6TNybz9m++gL3ijVI5wueVr0EM10VsU= +github.com/aliyun/alibaba-cloud-sdk-go v1.61.1682 h1:yhyOmpvnOJoO/ih+XupLW5DHFEP08Kg9ibH8ab1O0aA= +github.com/aliyun/alibaba-cloud-sdk-go v1.61.1682/go.mod h1:RcDobYh8k5VP6TNybz9m++gL3ijVI5wueVr0EM10VsU= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -67,8 +66,8 @@ github.com/bits-and-blooms/bitset v1.2.2/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edY github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= -github.com/bytedance/sonic v1.3.1 h1:pIYjcbBCV4M+tDCcSXnOL9/OGvfHo9bsWWmFNy/WPCI= -github.com/bytedance/sonic v1.3.1/go.mod h1:V973WhNhGmvHxW6nQmsHEfHaoU9F3zTF+93rH03hcUQ= +github.com/bytedance/sonic v1.3.3 h1:IYzrQ/JG0AbF8hcIZmVnArdIiKPVq2ijbKWAqBXyqX4= +github.com/bytedance/sonic v1.3.3/go.mod h1:V973WhNhGmvHxW6nQmsHEfHaoU9F3zTF+93rH03hcUQ= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= @@ -88,8 +87,9 @@ github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dennwc/varint v1.0.0 h1:kGNFFSSw8ToIy3obO/kKr8U9GZYUAxQEVuix4zfDWzE= -github.com/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q= github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= +github.com/eapache/go-resiliency v1.3.0 h1:RRL0nge+cWGlxXbUzJ7yMcq6w2XBEr19dCN6HECGaT0= +github.com/eapache/go-resiliency v1.3.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= @@ -122,8 +122,6 @@ github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbV github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= -github.com/go-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= -github.com/go-ole/go-ole v1.2.6-0.20210915003542-8b1f7f90f6b1/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= @@ -174,11 +172,12 @@ github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/go-cmp v0.5.8 h1:e6P7q2lk1O+qJJb4BtCQXlK8vWEO8V1ZeuEdJNOqZyg= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/gops v0.3.23 h1:OjsHRINl5FiIyTc8jivIg4UN0GY6Nh32SL8KRbl8GQo= -github.com/google/gops v0.3.23/go.mod h1:7diIdLsqpCihPSX3fQagksT/Ku/y4RL9LHTlKyEUDl8= +github.com/google/gops v0.3.25 h1:Pf6uw+cO6pDhc7HJ71NiG0x8dyQTeQcmg3HQFF39qVw= +github.com/google/gops v0.3.25/go.mod h1:8A7ebAm0id9K3H0uOggeRVGxszSvnlURun9mg3GdYDw= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= @@ -249,11 +248,12 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/compress v1.14.2/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.4/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.15.6 h1:6D9PcO8QWu0JyaQ2zUMmu16T1T+zjjEpP91guRsvDfY= github.com/klauspost/compress v1.15.6/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= +github.com/klauspost/compress v1.15.8 h1:JahtItbkWjf2jzm/T+qgMxkP9EMHsqEUA6vCMGmXvhA= +github.com/klauspost/compress v1.15.8/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= -github.com/klauspost/cpuid/v2 v2.0.13 h1:1XxvOiqXZ8SULZUKim/wncr3wZ38H4yCuVDvKdK9OGs= -github.com/klauspost/cpuid/v2 v2.0.13/go.mod h1:g2LTdtYhdyuGPqyWyv7qRAmj1WBqxuObKfj5c0PQa7c= +github.com/klauspost/cpuid/v2 v2.1.0 h1:eyi1Ad2aNJMW95zcSbmGg7Cg6cq3ADwLpMAP96d8rF0= +github.com/klauspost/cpuid/v2 v2.1.0/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= @@ -266,6 +266,7 @@ github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI= @@ -297,6 +298,7 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= @@ -316,8 +318,8 @@ github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt2 github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= -github.com/prometheus/common v0.34.0 h1:RBmGO9d/FVjqHT0yUGQwBJhkwKV+wPCn7KGpvfab0uE= -github.com/prometheus/common v0.34.0/go.mod h1:gB3sOl7P0TvJabZpLY5uQMpUqRCPPCyRLCZYc7JZTNE= +github.com/prometheus/common v0.37.0 h1:ccBbHCgIiT9uSoFY0vX8H3zsNR5eLt17/RQLUvn8pXE= +github.com/prometheus/common v0.37.0/go.mod h1:phzohg0JFMnBEFGxTDbfu3QyL5GI8gTQJFhYO5B3mfA= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= @@ -336,7 +338,7 @@ github.com/segmentio/kafka-go v0.4.32 h1:Ohr+9E+kDv/Ld2UPJN9hnKZRd2qgiqCmI8v2e1q github.com/segmentio/kafka-go v0.4.32/go.mod h1:JAPPIiY3MQIwVHj64CWOP0LsFFfQ7H0w69kuoxnMIS0= github.com/shirou/gopsutil v2.19.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.21.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/gopsutil/v3 v3.21.9/go.mod h1:YWp/H8Qs5fVmf17v7JNZzA0mPJ+mS2e9JdiUF9LlKzQ= +github.com/shirou/gopsutil/v3 v3.22.4/go.mod h1:D01hZJ4pVHPpCTZ3m3T2+wDF2YAGfd+H4ifUguaQzHM= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= @@ -362,8 +364,8 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.5 h1:s5PTfem8p8EbKQOctVV53k6jCJt3UX4IEJzwh+C324Q= github.com/stretchr/testify v1.7.5/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/thanos-io/thanos v0.27.0-rc.0 h1:USgjag/539MYqPRr4dUcoYHmVjFOrPe17NgTHzcEB0c= -github.com/thanos-io/thanos v0.27.0-rc.0/go.mod h1:i0KCpF5fWAWBNZXBkuzGh0603YlMrbhLJdwOO5W21YI= +github.com/thanos-io/thanos v0.27.0 h1:TWOPMkWX5Q1LHdROI6dg9lKU3x6Q95vO4o/bm8GaCfE= +github.com/thanos-io/thanos v0.27.0/go.mod h1:i0KCpF5fWAWBNZXBkuzGh0603YlMrbhLJdwOO5W21YI= github.com/tidwall/gjson v1.12.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/gjson v1.13.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/gjson v1.14.1 h1:iymTbGkQBhveq21bEvAQ81I0LEBork8BFe1CUZXdyuo= @@ -374,9 +376,7 @@ github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= github.com/tidwall/sjson v1.2.4 h1:cuiLzLnaMeBhRmEv00Lpk3tkYrcxpmbU81tAY4Dw0tc= github.com/tidwall/sjson v1.2.4/go.mod h1:098SZ494YoMWPmMO6ct4dcFnqxwj9r/gF0Etp19pSNM= -github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= github.com/tklauser/go-sysconf v0.3.10/go.mod h1:C8XykCvCb+Gn0oNCWPIlcb0RuglQTYaQ2hGm7jmxEFk= -github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= github.com/tklauser/numcpus v0.4.0/go.mod h1:1+UI3pD8NW14VMwdgJNJ1ESk2UnwhAnz5hMwiKKqXCQ= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= @@ -393,8 +393,8 @@ github.com/twmb/franz-go/pkg/kmsg v1.1.0 h1:csckTxG48q7Tem7ZwMxe2jAb0ehDNglxZccG github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 h1:Wfq7o2Eu8CaPJOqu/mqM1pu9b3HuO0aWfDGcdo1cVR8= github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0/go.mod h1:S0nXtRBGwqtOfTGQL3BDliF3hSxyUm0dqy5gEfPsQT0= -github.com/twmb/franz-go/plugin/kzap v1.1.0 h1:HaHgfcXYQi9QYRJ5ORdD/qwDjQKV85ZOeeKPwd46nyQ= -github.com/twmb/franz-go/plugin/kzap v1.1.0/go.mod h1:TUlWYqucIyz6U7xLo++gkHBDiJmed5FpYe42rCL2YG0= +github.com/twmb/franz-go/plugin/kzap v1.1.1 h1:ae8Z2JXn8y9ceZ2AFnwPm5U1A8d6cBvvZlygF46F2N4= +github.com/twmb/franz-go/plugin/kzap v1.1.1/go.mod h1:TUlWYqucIyz6U7xLo++gkHBDiJmed5FpYe42rCL2YG0= github.com/twmb/go-rbtree v1.0.0/go.mod h1:UlIAI8gu3KRPkXSobZnmJfVwCJgEhD/liWzT5ppzIyc= github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= @@ -423,10 +423,12 @@ go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opentelemetry.io/otel v1.7.0 h1:Z2lA3Tdch0iDcrhJXDIlC94XE+bxok1F9B+4Lz/lGsM= go.opentelemetry.io/otel v1.7.0/go.mod h1:5BdUoMIz5WEs0vt0CUEMtSSaTSHBBVwrhnz7+nrD5xk= -go.opentelemetry.io/otel/trace v1.7.0 h1:O37Iogk1lEkMRXewVtZ1BBTVn5JEp8GrJvP92bJqC6o= +go.opentelemetry.io/otel v1.8.0 h1:zcvBFizPbpa1q7FehvFiHbQwGzmPILebO0tyqIR5Djg= +go.opentelemetry.io/otel v1.8.0/go.mod h1:2pkj+iMj0o03Y+cW6/m8Y4WkRdYN3AvCXCnzRMp9yvM= go.opentelemetry.io/otel/trace v1.7.0/go.mod h1:fzLSB9nqR2eXzxPXb2JW9IKE+ScyXA48yyE4TNvoHqU= +go.opentelemetry.io/otel/trace v1.8.0 h1:cSy0DF9eGI5WIfNwZ1q2iUyGj00tGzP24dE1lOlHrfY= +go.opentelemetry.io/otel/trace v1.8.0/go.mod h1:0Bt3PXY8w+3pheS3hQUt+wow8b1ojPaTBoTCh2zIFI4= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= @@ -455,8 +457,8 @@ golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220518034528-6f7dac969898/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.0.0-20220525230936-793ad666bf5e h1:T8NU3HyQ8ClP4SEE+KbFlg6n0NhuTsN4MyznaarGsZM= -golang.org/x/crypto v0.0.0-20220525230936-793ad666bf5e/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d h1:sK3txAijHtOK88l68nt020reeT1ZdKLIYetKl95FzVY= +golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -526,8 +528,8 @@ golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220520000938-2e3eb7b945c2/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.0.0-20220607020251-c690dde0001d h1:4SFsTMi4UahlKoloni7L4eYzhFRifURQLw+yv0QDCx8= -golang.org/x/net v0.0.0-20220607020251-c690dde0001d/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.0.0-20220708220712-1185a9018129 h1:vucSRfWwTsoXro7P+3Cjlr6flUMtzCwzlvkxEQtHHB0= +golang.org/x/net v0.0.0-20220708220712-1185a9018129/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -585,20 +587,21 @@ golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210902050250-f475640dd07b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220128215802-99c3d69c2c27/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220429233432-b5fbb4746d32/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d h1:Zu/JngovGLVi6t2J3nmAf3AoTDwuzw85YZ3b9o4yU7s= -golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8 h1:0A+M6Uqn+Eje4kHMK80dtF3JCXC4ykBgQG4Fe06QRhQ= +golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From 86311cd01ecd3f6cdce34459f3ebd3c9f5b5d21b Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 25 Jul 2022 09:30:30 +0800 Subject: [PATCH 265/404] updated generators --- cmd/kafka_gen_log/main.go | 2 +- cmd/kafka_gen_metric/main.go | 2 +- cmd/kafka_gen_prom/main.go | 64 ++++++++++++++++++++++-------------- 3 files changed, 42 insertions(+), 26 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 9e0d6b1c..50d7eed2 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -30,7 +30,7 @@ CREATE TABLE apache_access_log ON CLUSTER abc ( xforwardfor LowCardinality(String) ) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{database}/{table}/{shard}', '{replica}') PARTITION BY toYYYYMMDD(timestamp) -ORDER BY (timestamp, `@hostname`, `@path`, `@lineno`); +ORDER BY (`@hostname`, `@path`, `@lineno`, timestamp); CREATE TABLE dist_apache_access_log ON CLUSTER abc AS apache_access_log ENGINE = Distributed(abc, default, apache_access_log); diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index 18372d3a..fb7f1380 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -26,7 +26,7 @@ CREATE TABLE sensor_dt_result_online ON CLUSTER abc ( is_missing Int32 ) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{database}/{table}/{shard}', '{replica}') PARTITION BY toYYYYMMDD(`@time`) -ORDER BY (`@time`, `@ItemGUID`, `@MetricName`); +ORDER BY (`@ItemGUID`, `@MetricName`, `@time`); CREATE TABLE dist_sensor_dt_result_online ON CLUSTER abc AS sensor_dt_result_online ENGINE = Distributed(abc, default, sensor_dt_result_online); diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index 5f83c417..40653201 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -7,17 +7,37 @@ performance of inserting to sparse wide table is bad CREATE TABLE prom_extend ON CLUSTER abc ( timestamp DateTime, value Float64, - __name__ String + __name__ String, + labels String ) ENGINE=ReplicatedMergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY (__name__, timestamp); CREATE TABLE dist_prom_extend ON CLUSTER abc AS prom_extend ENGINE = Distributed(abc, default, prom_extend); +CREATE TABLE prom_metric ON CLUSTER abc ( + __series_id Int64, + timestamp DateTime, + value Float64 +) ENGINE=ReplicatedReplacingMergeTree() +PARTITION BY toYYYYMMDD(timestamp) +ORDER BY (__series_id, timestamp); + +CREATE TABLE dist_prom_metric ON CLUSTER abc AS prom_metric ENGINE = Distributed(abc, default, prom_metric); + +CREATE TABLE prom_metric_series ON CLUSTER abc ( + __series_id Int64, + __mgmt_id Int64, + labels String, + __name__ String +) ENGINE=ReplicatedReplacingMergeTree() +ORDER BY (__name__, __series_id); + +CREATE TABLE dist_prom_metric_series ON CLUSTER abc AS prom_metric_series ENGINE = Distributed(abc, default, prom_metric_series); + */ import ( - "bytes" "context" "flag" "fmt" @@ -29,6 +49,7 @@ import ( "syscall" "time" + "github.com/bytedance/sonic" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" @@ -38,8 +59,10 @@ import ( const ( Alpha = "abcdefghijklmnopqrstuvwxyz" - NumMetrics = 1000000 - NumKeys = 10 + NumMetrics = 1000 + NumKeys = 5 + NumRunes = 10 + LenVal = 1 // 1000 * (10^1)^5 = 10^8 series NumAllKeys = 1000 ) @@ -62,13 +85,13 @@ type Datapoint struct { // I need every label be present at the top level. func (dp Datapoint) MarshalJSON() ([]byte, error) { - var bbuf bytes.Buffer - bbuf.WriteString(fmt.Sprintf(`{"timestamp":"%s", "value":%f,"__name__":"%s"`, dp.Timestamp.Format(time.RFC3339), dp.Value, dp.Name)) - for key, val := range dp.Labels { - bbuf.WriteString(fmt.Sprintf(`,"%s":"%s"`, key, val)) + labels, err := sonic.MarshalString(dp.Labels) + if err != nil { + return nil, err } - bbuf.WriteByte('}') - return bbuf.Bytes(), nil + labels2 := labels[1 : len(labels)-1] + msg := fmt.Sprintf(`{"timestamp":"%s", "value":%f,"__name__":"%s","labels":%s,%s}`, dp.Timestamp.Format(time.RFC3339), dp.Value, dp.Name, labels, labels2) + return []byte(msg), nil } type PromMetric struct { @@ -77,18 +100,9 @@ type PromMetric struct { } func randValue() (val string) { - mod := rand.Intn(2) - var leng, maxN int - if mod == 0 { //10^5=100000 - leng = 5 - maxN = 10 - } else { //3^10 = 59049 - leng = 10 - maxN = 3 - } - b := make([]byte, leng) - for i := 0; i < leng; i++ { - b[i] = Alpha[rand.Intn(maxN+1)] + b := make([]byte, LenVal) + for i := 0; i < LenVal; i++ { + b[i] = Alpha[rand.Intn(NumRunes+1)] } val = string(b) return @@ -98,11 +112,11 @@ func initMetrics() { metrics = make([]PromMetric, NumMetrics) for i := 0; i < NumMetrics; i++ { m := PromMetric{ - Name: fmt.Sprintf("metric-%08d", i), + Name: fmt.Sprintf("metric_%08d", i), LabelKeys: make([]string, NumKeys), } for j := 0; j < NumKeys; j++ { - key := fmt.Sprintf("key-%06d", rand.Intn(NumAllKeys+1)) + key := fmt.Sprintf("key_%06d", rand.Intn(NumAllKeys+1)) m.LabelKeys[j] = key } metrics[i] = m @@ -190,6 +204,8 @@ topic: for example, prom_extend`, os.Args[0], os.Args[0]) zap.String("KafkaTopic", KafkaTopic), zap.Int("NumMetrics", NumMetrics), zap.Int("NumKeys", NumKeys), + zap.Int("NumRunes", NumRunes), + zap.Int("LenVal", LenVal), zap.Int("NumAllKeys", NumAllKeys), ) From e6f83974caccddb0b06223b5358360731bb9b8b0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 26 Jul 2022 11:14:33 +0800 Subject: [PATCH 266/404] fixed supported column types --- model/value.go | 35 ++++++++++++++++++----------------- parser/csv.go | 8 ++++---- parser/fastjson.go | 8 ++++---- parser/gjson.go | 8 ++++---- task/sharding.go | 2 +- 5 files changed, 31 insertions(+), 30 deletions(-) diff --git a/model/value.go b/model/value.go index 34171b76..83e18cb7 100644 --- a/model/value.go +++ b/model/value.go @@ -28,10 +28,10 @@ const ( Int16 Int32 Int64 - Uint8 - Uint16 - Uint32 - Uint64 + UInt8 + UInt16 + UInt32 + UInt64 Float32 Float64 Decimal @@ -49,6 +49,7 @@ var ( typeInfo map[string]TypeInfo ) +// GetTypeName returns the column type in ClickHouse func GetTypeName(typ int) (name string) { switch typ { case Bool: @@ -61,14 +62,14 @@ func GetTypeName(typ int) (name string) { name = "Int32" case Int64: name = "Int64" - case Uint8: - name = "Uint8" - case Uint16: - name = "Uint16" - case Uint32: - name = "Uint32" - case Uint64: - name = "Uint64" + case UInt8: + name = "UInt8" + case UInt16: + name = "UInt16" + case UInt32: + name = "UInt32" + case UInt64: + name = "UInt64" case Float32: name = "Float32" case Float64: @@ -101,13 +102,13 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { val = metric.GetInt32(name, cwt.Nullable) case Int64: val = metric.GetInt64(name, cwt.Nullable) - case Uint8: + case UInt8: val = metric.GetUint8(name, cwt.Nullable) - case Uint16: + case UInt16: val = metric.GetUint16(name, cwt.Nullable) - case Uint32: + case UInt32: val = metric.GetUint32(name, cwt.Nullable) - case Uint64: + case UInt64: val = metric.GetUint64(name, cwt.Nullable) case Float32: val = metric.GetFloat32(name, cwt.Nullable) @@ -159,7 +160,7 @@ func WhichType(typ string) (dataType int, nullable bool, array bool) { func init() { typeInfo = make(map[string]TypeInfo) - for _, t := range []int{Bool, Int8, Int16, Int32, Int64, Uint8, Uint16, Uint32, Uint64, Float32, Float64, DateTime, String} { + for _, t := range []int{Bool, Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32, Float64, DateTime, String} { tn := GetTypeName(t) typeInfo[tn] = TypeInfo{Type: t} nullTn := fmt.Sprintf("Nullable(%s)", tn) diff --git a/parser/csv.go b/parser/csv.go index f6156bfd..cc06ea60 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -265,13 +265,13 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { val = GjsonIntArray[int32](array, math.MinInt32, math.MaxInt32) case model.Int64: val = GjsonIntArray[int64](array, math.MinInt64, math.MaxInt64) - case model.Uint8: + case model.UInt8: val = GjsonUintArray[uint8](array, math.MaxUint8) - case model.Uint16: + case model.UInt16: val = GjsonUintArray[uint16](array, math.MaxUint16) - case model.Uint32: + case model.UInt32: val = GjsonUintArray[uint32](array, math.MaxUint32) - case model.Uint64: + case model.UInt64: val = GjsonUintArray[uint64](array, math.MaxUint64) case model.Float32: val = GjsonFloatArray[float32](array, math.MaxFloat32) diff --git a/parser/fastjson.go b/parser/fastjson.go index 32d102e4..d155b407 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -253,13 +253,13 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { val = FastjsonIntArray[int32](array, math.MinInt32, math.MaxInt32) case model.Int64: val = FastjsonIntArray[int64](array, math.MinInt64, math.MaxInt64) - case model.Uint8: + case model.UInt8: val = FastjsonUintArray[uint8](array, math.MaxUint8) - case model.Uint16: + case model.UInt16: val = FastjsonUintArray[uint16](array, math.MaxUint16) - case model.Uint32: + case model.UInt32: val = FastjsonUintArray[uint32](array, math.MaxUint32) - case model.Uint64: + case model.UInt64: val = FastjsonUintArray[uint64](array, math.MaxUint64) case model.Float32: val = FastjsonFloatArray[float32](array, math.MaxFloat32) diff --git a/parser/gjson.go b/parser/gjson.go index 8029ec84..d85b81b3 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -246,13 +246,13 @@ func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { val = GjsonIntArray[int32](array, math.MinInt32, math.MaxInt32) case model.Int64: val = GjsonIntArray[int64](array, math.MinInt64, math.MaxInt64) - case model.Uint8: + case model.UInt8: val = GjsonUintArray[uint8](array, math.MaxUint8) - case model.Uint16: + case model.UInt16: val = GjsonUintArray[uint16](array, math.MaxUint16) - case model.Uint32: + case model.UInt32: val = GjsonUintArray[uint32](array, math.MaxUint32) - case model.Uint64: + case model.UInt64: val = GjsonUintArray[uint64](array, math.MaxUint64) case model.Float32: val = GjsonFloatArray[float32](array, math.MaxFloat32) diff --git a/task/sharding.go b/task/sharding.go index 21c4d729..4c80ddcd 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -33,7 +33,7 @@ func NewShardingPolicy(shardingKey string, shardingStripe uint64, dims []*model. } colSeq = i switch dim.Type { - case model.Int8, model.Int16, model.Int32, model.Int64, model.Uint8, model.Uint16, model.Uint32, model.Uint64, model.Float32, model.Float64, model.Decimal, model.DateTime: + case model.Int8, model.Int16, model.Int32, model.Int64, model.UInt8, model.UInt16, model.UInt32, model.UInt64, model.Float32, model.Float64, model.Decimal, model.DateTime: //numerical if policy.stripe <= 0 { policy.stripe = uint64(1) From 48c063b667242632c79862e11ce2cb26ac78e745 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 1 Aug 2022 10:14:06 +0800 Subject: [PATCH 267/404] more doc on storing prometheus metrics --- cmd/kafka_gen_prom/main.go | 50 +++++++++++++++++++++++++++++++++----- 1 file changed, 44 insertions(+), 6 deletions(-) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index 40653201..35119290 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -4,7 +4,8 @@ package main https://github.com/ClickHouse/ClickHouse/issues/38878 performance of inserting to sparse wide table is bad -CREATE TABLE prom_extend ON CLUSTER abc ( +-- Prometheus metric solution 1 - one wide table, each row is a datapoint and its series lables +CREATE TABLE default.prom_extend ON CLUSTER abc ( timestamp DateTime, value Float64, __name__ String, @@ -13,9 +14,10 @@ CREATE TABLE prom_extend ON CLUSTER abc ( PARTITION BY toYYYYMMDD(timestamp) ORDER BY (__name__, timestamp); -CREATE TABLE dist_prom_extend ON CLUSTER abc AS prom_extend ENGINE = Distributed(abc, default, prom_extend); +CREATE TABLE default.dist_prom_extend ON CLUSTER abc AS prom_extend ENGINE = Distributed(abc, default, prom_extend); -CREATE TABLE prom_metric ON CLUSTER abc ( +-- Prometheus metric solution 2 - seperated table for datapoints and series labels can join on series id +CREATE TABLE default.prom_metric ON CLUSTER abc ( __series_id Int64, timestamp DateTime, value Float64 @@ -23,9 +25,9 @@ CREATE TABLE prom_metric ON CLUSTER abc ( PARTITION BY toYYYYMMDD(timestamp) ORDER BY (__series_id, timestamp); -CREATE TABLE dist_prom_metric ON CLUSTER abc AS prom_metric ENGINE = Distributed(abc, default, prom_metric); +CREATE TABLE default.dist_prom_metric ON CLUSTER abc AS prom_metric ENGINE = Distributed(abc, default, prom_metric); -CREATE TABLE prom_metric_series ON CLUSTER abc ( +CREATE TABLE default.prom_metric_series ON CLUSTER abc ( __series_id Int64, __mgmt_id Int64, labels String, @@ -33,7 +35,43 @@ CREATE TABLE prom_metric_series ON CLUSTER abc ( ) ENGINE=ReplicatedReplacingMergeTree() ORDER BY (__name__, __series_id); -CREATE TABLE dist_prom_metric_series ON CLUSTER abc AS prom_metric_series ENGINE = Distributed(abc, default, prom_metric_series); +CREATE TABLE default.dist_prom_metric_series ON CLUSTER abc AS prom_metric_series ENGINE = Distributed(abc, default, prom_metric_series); + +CREATE TABLE default.prom_metric_agg ON CLUSTER abc ( + __series_id Int64, + timestamp DateTime, + max_value AggregateFunction(max, Float64), + min_value AggregateFunction(min, Float64), + avg_value AggregateFunction(avg, Float64) +) ENGINE=ReplicatedReplacingMergeTree() +PARTITION BY toYYYYMMDD(timestamp) +ORDER BY (__series_id, timestamp); + +CREATE TABLE default.dist_prom_metric_agg ON CLUSTER abc AS prom_metric_agg ENGINE = Distributed(abc, default, prom_metric_agg); + +SELECT __series_id, + toStartOfDay(timestamp) AS timestamp, + maxMerge(max_value) AS max_value, + minMerge(min_value) AS min_value, + avgMerge(avg_value) AS avg_value +FROM default.dist_prom_metric_agg +WHERE __series_id IN (-9223014754132113609, -9223015002162651005) +GROUP BY __series_id, timestamp +ORDER BY __series_id, timestamp; + +-- Activate aggregation for future datapoints by creating a materialized view +CREATE MATERIALIZED VIEW default.prom_metric_mv ON CLUSTER abc +TO prom_metric_agg +AS SELECT __series_id, + toStartOfHour(timestamp) AS timestamp, + maxState(value) AS max_value, + minState(value) AS min_value, + avgState(value) AS avg_value +FROM prom_metric +GROUP BY __series_id, timestamp; + +-- Deactivate aggregation by dropping the materialized view. You can revise and create it later as you will. +DROP TABLE default.prom_metric_mv ON CLUSTER abc SYNC; */ From 45c5b33afa3b2aa2a5de7416f6c08855cd93ddd2 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 8 Aug 2022 14:52:07 +0800 Subject: [PATCH 268/404] support default nacos port --- config_manager/nacos.go | 10 ++++++---- output/clickhouse_util.go | 2 +- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 3f87038d..f81cf63d 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -54,10 +54,12 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) serverAddrs := strings.Split(properties["serverAddrs"].(string), ",") for _, serverAddr := range serverAddrs { serverAddrFields := strings.SplitN(serverAddr, ":", 2) - var nacosPort uint64 - if nacosPort, err = strconv.ParseUint(serverAddrFields[1], 10, 64); err != nil { - err = errors.Wrapf(err, "") - return + var nacosPort uint64 = 8848 + if len(serverAddrFields) >= 2 { + if nacosPort, err = strconv.ParseUint(serverAddrFields[1], 10, 64); err != nil { + err = errors.Wrapf(err, "invalid nacos serverAddrs") + return + } } sc = append(sc, constant.ServerConfig{ IpAddr: serverAddrFields[0], diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 58cd75cf..22eb901c 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -114,7 +114,7 @@ func getDims(database, table string, excludedColumns []string, conn clickhouse.C func recreateDistTbls(cluster, database, table string, distTbls []string, conn clickhouse.Conn) (err error) { var queries []string for _, distTbl := range distTbls { - queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS `%s`.`%s` ON CLUSTER `%s`", database, distTbl, cluster)) + queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS `%s`.`%s` ON CLUSTER `%s` SYNC", database, distTbl, cluster)) queries = append(queries, fmt.Sprintf("CREATE TABLE `%s`.`%s` ON CLUSTER `%s` AS `%s`.`%s` ENGINE = Distributed(`%s`, `%s`, `%s`);", database, distTbl, cluster, database, table, cluster, database, table)) From 452015faae291b0b62ffa24108ed06d5d674acb4 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 11 Aug 2022 11:21:00 +0800 Subject: [PATCH 269/404] better error check for franz --- input/kafka_franz.go | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 258c2f5e..c1da7718 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -165,20 +165,13 @@ func (k *KafkaFranz) Run() { defer k.wgRun.Done() for { fetches := k.cl.PollFetches(k.ctx) - if fetches == nil || fetches.IsClientClosed() { + err := fetches.Err() + if fetches == nil || fetches.IsClientClosed() || errors.Is(err, context.Canceled) { break } - var beCanceled bool - fetches.EachError(func(_ string, _ int32, err error) { - if errors.Is(err, context.Canceled) { - beCanceled = true - } else { - err = errors.Wrapf(err, "") - util.Logger.Info("kgo.Client.PollFetchs() failed", zap.Error(err)) - } - }) - if beCanceled { - break + if err != nil { + err = errors.Wrapf(err, "") + util.Logger.Info("kgo.Client.PollFetchs() got an error", zap.Error(err)) } fetches.EachRecord(func(rec *kgo.Record) { msg := &model.InputMessage{ From 35f660cba500eb39136070f4987d103f31bd43b5 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 13 Aug 2022 11:20:51 +0800 Subject: [PATCH 270/404] changed to go 1.19 and bumped dependencies --- .github/workflows/lint.yml | 2 +- .github/workflows/tests.yml | 2 +- cmd/clickhouse_sinker/main.go | 10 +++ config/config.go | 4 +- config_manager/nacos.go | 24 +++---- go.mod | 58 ++++++++-------- go.sum | 120 ++++++++++++++++++---------------- util/common.go | 3 +- 8 files changed, 120 insertions(+), 103 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index ea3b0dfe..9791798c 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -16,7 +16,7 @@ jobs: steps: - uses: actions/setup-go@v3 with: - go-version: 1.18 + go-version: 1.19 - uses: actions/checkout@v3 - name: golangci-lint uses: golangci/golangci-lint-action@v3 diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index e5608767..a1d2d186 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -16,7 +16,7 @@ jobs: - name: Set up Go 1.x uses: actions/setup-go@v2 with: - go-version: 1.18 + go-version: 1.19 id: go - name: Check out code into the Go module directory diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index e2f1e9fa..e10d6b63 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -25,6 +25,7 @@ import ( "net/http" "net/http/pprof" "os" + "path/filepath" "reflect" "sort" "strings" @@ -215,6 +216,14 @@ func main() { var rcm cm.RemoteConfManager var properties map[string]interface{} + logDir := "." + logPaths := strings.Split(cmdOps.LogPaths, ",") + for _, logPath := range logPaths { + if logPath != "stdout" && logPath != "stderr" { + logDir, _ = filepath.Split(logPath) + } + } + logDir, _ = filepath.Abs(logDir) if cmdOps.NacosDataID != "" { util.Logger.Info(fmt.Sprintf("get config from nacos serverAddrs %s, namespaceId %s, group %s, dataId %s", cmdOps.NacosAddr, cmdOps.NacosNamespaceID, cmdOps.NacosGroup, cmdOps.NacosDataID)) @@ -227,6 +236,7 @@ func main() { properties["group"] = cmdOps.NacosGroup properties["dataId"] = cmdOps.NacosDataID properties["serviceName"] = cmdOps.NacosServiceName + properties["logDir"] = logDir } else { util.Logger.Info(fmt.Sprintf("get config from local file %s", cmdOps.LocalCfgFile)) } diff --git a/config/config.go b/config/config.go index ddb50b76..f2217a5a 100644 --- a/config/config.go +++ b/config/config.go @@ -17,7 +17,7 @@ package config import ( "encoding/json" - "io/ioutil" + "os" "regexp" "strings" @@ -174,7 +174,7 @@ const ( func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { cfg = &Config{} var b []byte - b, err = ioutil.ReadFile(cfgPath) + b, err = os.ReadFile(cfgPath) if err != nil { err = errors.Wrapf(err, "") return diff --git a/config_manager/nacos.go b/config_manager/nacos.go index f81cf63d..038a042c 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -23,6 +23,7 @@ import ( "github.com/nacos-group/nacos-sdk-go/vo" "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" + "gopkg.in/natefinch/lumberjack.v2" ) var _ RemoteConfManager = (*NacosConfManager)(nil) @@ -67,13 +68,8 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) }) } - var clientDir string - if v, ok := properties["clientDir"]; ok { - clientDir, _ = v.(string) - } else { - clientDir = "/tmp/nacos" - } - var namespaceID string //Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work! + var namespaceID string //Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work! + logDir := "." group := constant.DEFAULT_GROUP //Empty string doesn't work! if pop, ok := properties["namespaceId"]; ok { namespaceID, _ = pop.(string) @@ -81,18 +77,24 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) if pop, ok := properties["group"]; ok { group, _ = pop.(string) } + if pop, ok := properties["logDir"]; ok { + logDir, _ = pop.(string) + } + logDir, _ = filepath.Abs(logDir) cc := constant.ClientConfig{ NamespaceId: namespaceID, TimeoutMs: 5000, ListenInterval: 10000, NotLoadCacheAtStart: true, - LogDir: filepath.Join(clientDir, "log"), - CacheDir: filepath.Join(clientDir, "cache"), + LogDir: filepath.Join(logDir, "nacos_log"), + CacheDir: filepath.Join(logDir, "nacos_cache"), LogLevel: "debug", Username: properties["username"].(string), Password: properties["password"].(string), - LogRollingConfig: &constant.ClientLogRollingConfig{ - MaxAge: 3, + LogRollingConfig: &lumberjack.Logger{ + MaxSize: 10, // megabytes + MaxBackups: 1, + LocalTime: true, }, } diff --git a/go.mod b/go.mod index 06f86f90..a0163955 100644 --- a/go.mod +++ b/go.mod @@ -1,42 +1,42 @@ module github.com/housepower/clickhouse_sinker -go 1.18 +go 1.19 require ( github.com/ClickHouse/clickhouse-go/v2 v2.2.0 github.com/RoaringBitmap/roaring v1.2.1 - github.com/Shopify/sarama v1.34.1 - github.com/bytedance/sonic v1.3.3 + github.com/Shopify/sarama v1.36.0 + github.com/bytedance/sonic v1.3.5 github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 github.com/google/gops v0.3.25 - github.com/jcmturner/gokrb5/v8 v8.4.2 + github.com/jcmturner/gokrb5/v8 v8.4.3 github.com/jinzhu/copier v0.3.5 - github.com/nacos-group/nacos-sdk-go v1.1.1 - github.com/prometheus/client_golang v1.12.2 + github.com/nacos-group/nacos-sdk-go v1.1.2 + github.com/prometheus/client_golang v1.13.0 github.com/prometheus/common v0.37.0 - github.com/segmentio/kafka-go v0.4.32 + github.com/segmentio/kafka-go v0.4.34 github.com/shopspring/decimal v1.3.1 - github.com/stretchr/testify v1.7.5 + github.com/stretchr/testify v1.8.0 github.com/thanos-io/thanos v0.27.0 - github.com/tidwall/gjson v1.14.1 + github.com/tidwall/gjson v1.14.2 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/twmb/franz-go v1.6.0 - github.com/twmb/franz-go/pkg/kadm v1.1.1 + github.com/twmb/franz-go/pkg/kadm v1.2.0 github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 github.com/twmb/franz-go/plugin/kzap v1.1.1 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.1.1 - go.uber.org/zap v1.21.0 - golang.org/x/exp v0.0.0-20220713135740-79cabaa25d75 - golang.org/x/time v0.0.0-20220609170525-579cf78fd858 + go.uber.org/zap v1.22.0 + golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e + golang.org/x/time v0.0.0-20220722155302-e5dcc9cfc0b9 gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) require ( - github.com/aliyun/alibaba-cloud-sdk-go v1.61.1682 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/bits-and-blooms/bitset v1.2.2 // indirect + github.com/bits-and-blooms/bitset v1.3.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a // indirect github.com/davecgh/go-spew v1.1.1 // indirect @@ -52,11 +52,11 @@ require ( github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/jcmturner/aescts/v2 v2.0.0 // indirect github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect - github.com/jcmturner/gofork v1.0.0 // indirect + github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.15.8 // indirect + github.com/klauspost/compress v1.15.9 // indirect github.com/klauspost/cpuid/v2 v2.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect @@ -67,28 +67,28 @@ require ( github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect - github.com/prometheus/procfs v0.7.3 // indirect + github.com/prometheus/procfs v0.8.0 // indirect github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.0 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect - github.com/twmb/franz-go/pkg/kmsg v1.1.0 // indirect + github.com/twmb/franz-go/pkg/kmsg v1.2.0 // indirect github.com/xdg-go/pbkdf2 v1.0.0 // indirect github.com/xdg-go/stringprep v1.0.3 // indirect github.com/xdg/scram v1.0.5 // indirect github.com/xdg/stringprep v1.0.3 // indirect - go.opentelemetry.io/otel v1.8.0 // indirect - go.opentelemetry.io/otel/trace v1.8.0 // indirect - go.uber.org/atomic v1.9.0 // indirect + go.opentelemetry.io/otel v1.9.0 // indirect + go.opentelemetry.io/otel/trace v1.9.0 // indirect + go.uber.org/atomic v1.10.0 // indirect go.uber.org/multierr v1.8.0 // indirect - golang.org/x/arch v0.0.0-20220412001346-fc48f9fe4c15 // indirect - golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d // indirect - golang.org/x/net v0.0.0-20220708220712-1185a9018129 // indirect - golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f // indirect - golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8 // indirect + golang.org/x/arch v0.0.0-20220722155209-00200b7164a7 // indirect + golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa // indirect + golang.org/x/net v0.0.0-20220812174116-3211cb980234 // indirect + golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4 // indirect + golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab // indirect golang.org/x/text v0.3.7 // indirect - google.golang.org/protobuf v1.28.0 // indirect - gopkg.in/ini.v1 v1.66.6 // indirect + google.golang.org/protobuf v1.28.1 // indirect + gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index f556183d..381487b1 100644 --- a/go.sum +++ b/go.sum @@ -41,8 +41,8 @@ github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3P github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.2.1 h1:58/LJlg/81wfEHd5L9qsHduznOIhyv4qb1yWcSvVq9A= github.com/RoaringBitmap/roaring v1.2.1/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= -github.com/Shopify/sarama v1.34.1 h1:pVCQO7BMAK3s1jWhgi5v1W6lwZ6Veiekfc2vsgRS06Y= -github.com/Shopify/sarama v1.34.1/go.mod h1:NZSNswsnStpq8TUdFaqnpXm2Do6KRzTIjdBdVlL1YRM= +github.com/Shopify/sarama v1.36.0 h1:0OJs3eCcnezkWniVjwBbCJVaa0B1k7ImCRS3WN6NsSk= +github.com/Shopify/sarama v1.36.0/go.mod h1:9glG3eX83tgVYJ5aVtrjVUnEsOPqQIBGx1BWfN+X51I= github.com/Shopify/toxiproxy/v2 v2.4.0 h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64= github.com/Shopify/toxiproxy/v2 v2.4.0/go.mod h1:3ilnjng821bkozDRxNoo64oI/DKqM+rOyJzb564+bvg= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= @@ -52,8 +52,8 @@ github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRF github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.61.1682 h1:yhyOmpvnOJoO/ih+XupLW5DHFEP08Kg9ibH8ab1O0aA= -github.com/aliyun/alibaba-cloud-sdk-go v1.61.1682/go.mod h1:RcDobYh8k5VP6TNybz9m++gL3ijVI5wueVr0EM10VsU= +github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723 h1:oBwUer3XCooqTDMBJ5vR2tTBv/4Te125UzMoh+/na5c= +github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723/go.mod h1:RcDobYh8k5VP6TNybz9m++gL3ijVI5wueVr0EM10VsU= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -61,13 +61,13 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= -github.com/bits-and-blooms/bitset v1.2.2 h1:J5gbX05GpMdBjCvQ9MteIg2KKDExr7DrgK+Yc15FvIk= -github.com/bits-and-blooms/bitset v1.2.2/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= +github.com/bits-and-blooms/bitset v1.3.0 h1:h7mv5q31cthBTd7V4kLAZaIThj1e8vPGcSqpPue9KVI= +github.com/bits-and-blooms/bitset v1.3.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= -github.com/bytedance/sonic v1.3.3 h1:IYzrQ/JG0AbF8hcIZmVnArdIiKPVq2ijbKWAqBXyqX4= -github.com/bytedance/sonic v1.3.3/go.mod h1:V973WhNhGmvHxW6nQmsHEfHaoU9F3zTF+93rH03hcUQ= +github.com/bytedance/sonic v1.3.5 h1:xfBNhsG3QCC+AMCmCHxNQg0StI5IM/B9Jtwjqi5WlI0= +github.com/bytedance/sonic v1.3.5/go.mod h1:V973WhNhGmvHxW6nQmsHEfHaoU9F3zTF+93rH03hcUQ= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= @@ -87,7 +87,6 @@ github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dennwc/varint v1.0.0 h1:kGNFFSSw8ToIy3obO/kKr8U9GZYUAxQEVuix4zfDWzE= -github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= github.com/eapache/go-resiliency v1.3.0 h1:RRL0nge+cWGlxXbUzJ7yMcq6w2XBEr19dCN6HECGaT0= github.com/eapache/go-resiliency v1.3.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= @@ -213,12 +212,14 @@ github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFK github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM= -github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8= github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= +github.com/jcmturner/gofork v1.7.6 h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVETg= +github.com/jcmturner/gofork v1.7.6/go.mod h1:1622LH6i/EZqLloHfE7IeZ0uEJwMSUyQ/nDd82IeqRo= github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= -github.com/jcmturner/gokrb5/v8 v8.4.2 h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJzodkA= github.com/jcmturner/gokrb5/v8 v8.4.2/go.mod h1:sb+Xq/fTY5yktf/VxLsE3wlfPqQjp0aWNYyvBVK62bc= +github.com/jcmturner/gokrb5/v8 v8.4.3 h1:iTonLeSJOn7MVUtyMT+arAn5AKAPrkilzhGw8wE/Tq8= +github.com/jcmturner/gokrb5/v8 v8.4.3/go.mod h1:dqRwJGXznQrzw6cWmyo6kH+E7jksEQG/CyVWsJEsJO0= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/jinzhu/copier v0.3.5 h1:GlvfUwHk62RokgqVNvYsku0TATCF7bAHVwEXoBh3iJg= @@ -245,12 +246,11 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8 github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.14.2/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.4/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.15.6/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.15.8 h1:JahtItbkWjf2jzm/T+qgMxkP9EMHsqEUA6vCMGmXvhA= -github.com/klauspost/compress v1.15.8/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= +github.com/klauspost/compress v1.15.7/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= +github.com/klauspost/compress v1.15.9 h1:wKRjX6JRtDdrE9qwa4b/Cip7ACOshUI4smpCQanqjSY= +github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.1.0 h1:eyi1Ad2aNJMW95zcSbmGg7Cg6cq3ADwLpMAP96d8rF0= github.com/klauspost/cpuid/v2 v2.1.0/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= @@ -283,8 +283,8 @@ github.com/mschoch/smat v0.2.0 h1:8imxQsjDm8yFEAVBe7azKmKSgzSkZXDuKkSq9374khM= github.com/mschoch/smat v0.2.0/go.mod h1:kc9mz7DoBKqDyiRL7VZN8KvXQMWeTaVnttLRXOlotKw= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nacos-group/nacos-sdk-go v1.1.1 h1:beczWcOoTaVBMgCgikqvZflrN5Xbw7pWAWpxl+VJGIA= -github.com/nacos-group/nacos-sdk-go v1.1.1/go.mod h1:UHOtQNQY/qpk2dhg6gDq8u5+/CEIc3+lWmrmxEzX0/g= +github.com/nacos-group/nacos-sdk-go v1.1.2 h1:lWTpf5SXLetQetS7p31eGic/ncqsnn0Zbau1i3eC25Y= +github.com/nacos-group/nacos-sdk-go v1.1.2/go.mod h1:I8Vj4M8ZLpBk7EY2A8RXQE1SbfCA7b56TJBPIFTrUYE= github.com/paulmach/orb v0.7.1 h1:Zha++Z5OX/l168sqHK3k4z18LDvr+YAO/VjK0ReQ9rU= github.com/paulmach/orb v0.7.1/go.mod h1:FWRlTgl88VI1RBx/MkrwWDRhQ96ctqMCh8boXhmqB/A= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= @@ -305,8 +305,8 @@ github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeD github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_golang v1.12.1/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= -github.com/prometheus/client_golang v1.12.2 h1:51L9cDoUHVrXx4zWYlcLQIZ+d+VXHgqnYKkIuq4g/34= -github.com/prometheus/client_golang v1.12.2/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= +github.com/prometheus/client_golang v1.13.0 h1:b71QUfeo5M8gq2+evJdTPfZhYMAU0uKPkyPJ7TPsloU= +github.com/prometheus/client_golang v1.13.0/go.mod h1:vTeo+zgvILHsnnj/39Ou/1fPN5nJFOEMgftOUOmlvYQ= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -325,8 +325,9 @@ github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsT github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.7.3 h1:4jVXhlkAyzOScmCkXBTOLRLTz8EeU+eyjrwB/EPq0VU= github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/prometheus/procfs v0.8.0 h1:ODq8ZFEaYeCaZOJlZZdJA2AbQR98dSHSM1KW/You5mo= +github.com/prometheus/procfs v0.8.0/go.mod h1:z7EfXMXOkbkqb9IINtpCn86r/to3BnA0uaxHdg830/4= github.com/prometheus/prometheus v1.8.2-0.20220308163432-03831554a519 h1:MSR9EaMnwV1nsmhQ2eB1Txwebyk/5ulUGcgcYi3ofL4= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= @@ -334,8 +335,8 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/segmentio/kafka-go v0.4.32 h1:Ohr+9E+kDv/Ld2UPJN9hnKZRd2qgiqCmI8v2e1qlfLM= -github.com/segmentio/kafka-go v0.4.32/go.mod h1:JAPPIiY3MQIwVHj64CWOP0LsFFfQ7H0w69kuoxnMIS0= +github.com/segmentio/kafka-go v0.4.34 h1:Dm6YlLMiVSiwwav20KY0AoY63s661FXevwJ3CVHUERo= +github.com/segmentio/kafka-go v0.4.34/go.mod h1:GAjxBQJdQMB5zfNA21AhpaqOB2Mu+w3De4ni3Gbm8y0= github.com/shirou/gopsutil v2.19.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.21.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil/v3 v3.22.4/go.mod h1:D01hZJ4pVHPpCTZ3m3T2+wDF2YAGfd+H4ifUguaQzHM= @@ -346,6 +347,7 @@ github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeV github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= @@ -362,14 +364,15 @@ github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5 github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.5 h1:s5PTfem8p8EbKQOctVV53k6jCJt3UX4IEJzwh+C324Q= github.com/stretchr/testify v1.7.5/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/thanos-io/thanos v0.27.0 h1:TWOPMkWX5Q1LHdROI6dg9lKU3x6Q95vO4o/bm8GaCfE= github.com/thanos-io/thanos v0.27.0/go.mod h1:i0KCpF5fWAWBNZXBkuzGh0603YlMrbhLJdwOO5W21YI= github.com/tidwall/gjson v1.12.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/gjson v1.13.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= -github.com/tidwall/gjson v1.14.1 h1:iymTbGkQBhveq21bEvAQ81I0LEBork8BFe1CUZXdyuo= -github.com/tidwall/gjson v1.14.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= +github.com/tidwall/gjson v1.14.2 h1:6BBkirS0rAHjumnjHF6qgy5d2YAJ1TLIaFE2lzfOLqo= +github.com/tidwall/gjson v1.14.2/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= @@ -386,11 +389,12 @@ github.com/twmb/franz-go v1.5.0/go.mod h1:ZKQ5AtqBbdc783bLCay7nDc21lJnIIA8mFJYhL github.com/twmb/franz-go v1.5.3/go.mod h1:eqHYpAuvlTArOdZ1XtPYyOQ1uUb40CSZwbpL3ccjibI= github.com/twmb/franz-go v1.6.0 h1:yri7YsVBe/k1LKcoZSLILgUI3U14e82qtD9i4VOcs9c= github.com/twmb/franz-go v1.6.0/go.mod h1:xdMwpUIQL/JDKKwerc5qJQG8TU1SNIddfjKJJyqRJIg= -github.com/twmb/franz-go/pkg/kadm v1.1.1 h1:tqcJt9ChdqiY+Vi3F13z8/XRAJozbJNj0/7VPnrdTQA= -github.com/twmb/franz-go/pkg/kadm v1.1.1/go.mod h1:Ly8COloKx7pbwBdlP4qTYKdEVcNvk7D3+in3ujv3x/M= +github.com/twmb/franz-go/pkg/kadm v1.2.0 h1:3jtgdCi04TPGSCiaIBe2nxBNkHoCZRm/YljSFEh7BHA= +github.com/twmb/franz-go/pkg/kadm v1.2.0/go.mod h1:izleX4EttZwes7MRiHFmPWPSqDuGr4VN6p7jCIYoU0g= github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/kmsg v1.1.0 h1:csckTxG48q7Tem7ZwMxe2jAb0ehDNglxZccGnpqe4RU= github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go/pkg/kmsg v1.2.0 h1:jYWh2qFw5lDbNv5Gvu/sMKagzICxuA5L6m1W2Oe7XUo= +github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 h1:Wfq7o2Eu8CaPJOqu/mqM1pu9b3HuO0aWfDGcdo1cVR8= github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0/go.mod h1:S0nXtRBGwqtOfTGQL3BDliF3hSxyUm0dqy5gEfPsQT0= github.com/twmb/franz-go/plugin/kzap v1.1.1 h1:ae8Z2JXn8y9ceZ2AFnwPm5U1A8d6cBvvZlygF46F2N4= @@ -405,10 +409,8 @@ github.com/xdg-go/scram v1.1.1 h1:VOMT+81stJgXW3CpHyqHN3AXDYIMsx56mEFrB37Mb/E= github.com/xdg-go/scram v1.1.1/go.mod h1:RaEWvsqvNKKvBPvcKeFjrG2cJqOkHTiyTpzz23ni57g= github.com/xdg-go/stringprep v1.0.3 h1:kdwGpVNwPFtjs98xCGkHjQtGKh86rDcRZN17QEMCOIs= github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgklLGvcBnW8= -github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= github.com/xdg/scram v1.0.5 h1:TuS0RFmt5Is5qm9Tm2SoD89OPqe4IRiFtyFY4iwWXsw= github.com/xdg/scram v1.0.5/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= -github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= github.com/xdg/stringprep v1.0.3 h1:cmL5Enob4W83ti/ZHuZLuKD/xqJfus4fVPwE+/BDm+4= github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= github.com/xlab/treeprint v1.1.0/go.mod h1:gj5Gd3gPdKtR1ikdDK6fnFLdmIS0X30kTTuNd/WEJu0= @@ -424,15 +426,16 @@ go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opentelemetry.io/otel v1.7.0/go.mod h1:5BdUoMIz5WEs0vt0CUEMtSSaTSHBBVwrhnz7+nrD5xk= -go.opentelemetry.io/otel v1.8.0 h1:zcvBFizPbpa1q7FehvFiHbQwGzmPILebO0tyqIR5Djg= -go.opentelemetry.io/otel v1.8.0/go.mod h1:2pkj+iMj0o03Y+cW6/m8Y4WkRdYN3AvCXCnzRMp9yvM= +go.opentelemetry.io/otel v1.9.0 h1:8WZNQFIB2a71LnANS9JeyidJKKGOOremcUtb/OtHISw= +go.opentelemetry.io/otel v1.9.0/go.mod h1:np4EoPGzoPs3O67xUVNoPPcmSvsfOxNlNA4F4AC+0Eo= go.opentelemetry.io/otel/trace v1.7.0/go.mod h1:fzLSB9nqR2eXzxPXb2JW9IKE+ScyXA48yyE4TNvoHqU= -go.opentelemetry.io/otel/trace v1.8.0 h1:cSy0DF9eGI5WIfNwZ1q2iUyGj00tGzP24dE1lOlHrfY= -go.opentelemetry.io/otel/trace v1.8.0/go.mod h1:0Bt3PXY8w+3pheS3hQUt+wow8b1ojPaTBoTCh2zIFI4= +go.opentelemetry.io/otel/trace v1.9.0 h1:oZaCNJUjWcg60VXWee8lJKlqhPbXAPB51URuR47pQYc= +go.opentelemetry.io/otel/trace v1.9.0/go.mod h1:2737Q0MuG8q1uILYm2YYVkAyLtOofiTNGg6VODnOiPo= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= +go.uber.org/atomic v1.10.0 h1:9qC72Qh0+3MqyJbAn8YU5xVq1frD8bn3JtD2oXtafVQ= +go.uber.org/atomic v1.10.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= @@ -441,24 +444,24 @@ go.uber.org/multierr v1.8.0 h1:dg6GjLku4EH+249NNmoIciG9N/jURbDG+pFlTkhzIC8= go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.21.0 h1:WefMeulhovoZ2sYXz7st6K0sLj7bBhpiFaud4r4zST8= go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= +go.uber.org/zap v1.22.0 h1:Zcye5DUgBloQ9BaT4qc9BnjOFog5TvBSAGkJ3Nf70c0= +go.uber.org/zap v1.22.0/go.mod h1:H4siCOZOrAolnUPJEkfaSjDqyP+BDS0DdDWzwcgt3+U= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.0.0-20220412001346-fc48f9fe4c15 h1:GVfVkciLYxn5mY5EncwAe0SXUn9Rm81rRkZ0TTmn/cU= -golang.org/x/arch v0.0.0-20220412001346-fc48f9fe4c15/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.0.0-20220722155209-00200b7164a7 h1:VBQqJMNMRfQsWSiCTLgz9XjAfWlgnJAPv8nsp1HF8Tw= +golang.org/x/arch v0.0.0-20220722155209-00200b7164a7/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20190506204251-e1dfcc566284/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220518034528-6f7dac969898/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d h1:sK3txAijHtOK88l68nt020reeT1ZdKLIYetKl95FzVY= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa h1:zuSxTR4o9y82ebqCUJYNGJbGPo6sKVl54f/TVDObg1c= +golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -469,8 +472,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20220713135740-79cabaa25d75 h1:x03zeu7B2B11ySp+daztnwM5oBJ/8wGUSqrwcw9L0RA= -golang.org/x/exp v0.0.0-20220713135740-79cabaa25d75/go.mod h1:Kr81I6Kryrl9sr8s2FK3vxD90NdsKWRuOIl2O4CvYbA= +golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e h1:+WEEuIdZHnUeJJmEUjyYC2gfUMj69yZXw17EnHg/otA= +golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e/go.mod h1:Kr81I6Kryrl9sr8s2FK3vxD90NdsKWRuOIl2O4CvYbA= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -527,9 +530,11 @@ golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.0.0-20220520000938-2e3eb7b945c2/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.0.0-20220708220712-1185a9018129 h1:vucSRfWwTsoXro7P+3Cjlr6flUMtzCwzlvkxEQtHHB0= -golang.org/x/net v0.0.0-20220708220712-1185a9018129/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.0.0-20220706163947-c90051bbdb60/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= +golang.org/x/net v0.0.0-20220809184613-07c6da5e1ced/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= +golang.org/x/net v0.0.0-20220812174116-3211cb980234 h1:RDqmgfe7SvlMWoqC3xwQ2blLO3fcWcxMa3eBLRdRW7E= +golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -548,8 +553,8 @@ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f h1:Ax0t5p6N38Ga0dThY21weqDEyz2oklo4IvDkpigvkD8= -golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4 h1:uVc8UZUe6tr40fFVnUP5Oj+veunVezqYl9z7DYw9xzw= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -600,8 +605,10 @@ golang.org/x/sys v0.0.0-20220128215802-99c3d69c2c27/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220429233432-b5fbb4746d32/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8 h1:0A+M6Uqn+Eje4kHMK80dtF3JCXC4ykBgQG4Fe06QRhQ= -golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab h1:2QkjZIsXupsJbJIdSjjUOgWK3aEtzyuh2mPt3l/CkeU= +golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -615,8 +622,8 @@ golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20220609170525-579cf78fd858 h1:Dpdu/EMxGMFgq0CeYMh4fazTD2vtlZRYE7wyynxJb9U= -golang.org/x/time v0.0.0-20220609170525-579cf78fd858/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20220722155302-e5dcc9cfc0b9 h1:ftMN5LMiBFjbzleLqtoBZk7KdJwhuybIU+FckUHgoyQ= +golang.org/x/time v0.0.0-20220722155302-e5dcc9cfc0b9/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= @@ -743,8 +750,8 @@ google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlba google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.28.0 h1:w43yiav+6bVFTBQFZX0r7ipe9JQ1QsbMgHwbBziscLw= -google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w= +google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -754,8 +761,8 @@ gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EV gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.66.2/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/ini.v1 v1.66.6 h1:LATuAqN/shcYAOkv3wl2L4rkaKqkcgTBQjOyYDvcPKI= -gopkg.in/ini.v1 v1.66.6/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= +gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= @@ -770,7 +777,6 @@ gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20220512140231-539c8e751b99/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/util/common.go b/util/common.go index c668b810..5abcc881 100644 --- a/util/common.go +++ b/util/common.go @@ -20,7 +20,6 @@ import ( "crypto/tls" "crypto/x509" "fmt" - "io/ioutil" "net" "os" "os/exec" @@ -165,7 +164,7 @@ func NewTLSConfig(caCertFiles, clientCertFile, clientKeyFile string, insecureSki // Load CA cert caCertPool := x509.NewCertPool() for _, caCertFile := range strings.Split(caCertFiles, ",") { - caCert, err := ioutil.ReadFile(caCertFile) + caCert, err := os.ReadFile(caCertFile) if err != nil { err = errors.Wrapf(err, "") return &tlsConfig, err From cacfb12337ba769109286d58456ba9067771e4f0 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 13 Aug 2022 13:03:52 +0800 Subject: [PATCH 271/404] bump golangci-lint --- .github/workflows/lint.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 9791798c..53980c5e 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -22,7 +22,7 @@ jobs: uses: golangci/golangci-lint-action@v3 with: # Optional: version of golangci-lint to use in form of v1.2 or v1.2.3 or `latest` to use the latest version - version: v1.46.2 + version: v1.48.0 # Optional: working directory, useful for monorepos # working-directory: somedir From 8f31b6f5f2947ca98d7bf33b6601ba3b2aea77b2 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 15 Aug 2022 10:12:18 +0800 Subject: [PATCH 272/404] changed kafka_gen_prom schema --- cmd/kafka_gen_prom/main.go | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index 35119290..2dd346d1 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -7,7 +7,7 @@ performance of inserting to sparse wide table is bad -- Prometheus metric solution 1 - one wide table, each row is a datapoint and its series lables CREATE TABLE default.prom_extend ON CLUSTER abc ( timestamp DateTime, - value Float64, + value Float32, __name__ String, labels String ) ENGINE=ReplicatedMergeTree() @@ -19,8 +19,8 @@ CREATE TABLE default.dist_prom_extend ON CLUSTER abc AS prom_extend ENGINE = Dis -- Prometheus metric solution 2 - seperated table for datapoints and series labels can join on series id CREATE TABLE default.prom_metric ON CLUSTER abc ( __series_id Int64, - timestamp DateTime, - value Float64 + timestamp DateTime CODEC(DoubleDelta, LZ4), + value Float32 CODEC(ZSTD(15)) ) ENGINE=ReplicatedReplacingMergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY (__series_id, timestamp); @@ -39,10 +39,10 @@ CREATE TABLE default.dist_prom_metric_series ON CLUSTER abc AS prom_metric_serie CREATE TABLE default.prom_metric_agg ON CLUSTER abc ( __series_id Int64, - timestamp DateTime, - max_value AggregateFunction(max, Float64), - min_value AggregateFunction(min, Float64), - avg_value AggregateFunction(avg, Float64) + timestamp DateTime CODEC(DoubleDelta, LZ4), + max_value AggregateFunction(max, Float32), + min_value AggregateFunction(min, Float32), + avg_value AggregateFunction(avg, Float32) ) ENGINE=ReplicatedReplacingMergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY (__series_id, timestamp); @@ -116,7 +116,7 @@ type Labels map[string]string type Datapoint struct { Timestamp time.Time - Value float64 + Value float32 Name string `json:"__name__"` Labels } @@ -194,7 +194,7 @@ func generate() { for i := 0; i < NumMetrics; i++ { dp := Datapoint{ Timestamp: timestamp, - Value: rand.Float64(), + Value: rand.Float32(), Name: metrics[i].Name, Labels: make(Labels), } From 9fdbf60772c12707ff521155c477a81143792338 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 17 Aug 2022 15:03:29 +0800 Subject: [PATCH 273/404] clickhouse_sinker shall not depend on sonic. sonic doesn't work on Hygon CPU. --- Makefile | 12 ++---------- cmd/kafka_gen_log/main.go | 2 +- cmd/kafka_gen_log/marshal_amd64.go | 9 +++++++++ cmd/kafka_gen_log/marshal_arm64.go | 9 +++++++++ cmd/kafka_gen_metric/main.go | 2 +- cmd/kafka_gen_metric/marshal_amd64.go | 9 +++++++++ cmd/kafka_gen_metric/marshal_arm64.go | 9 +++++++++ util/marshal_amd64.go | 14 -------------- util/marshal_arm64.go | 14 -------------- 9 files changed, 40 insertions(+), 40 deletions(-) create mode 100644 cmd/kafka_gen_log/marshal_amd64.go create mode 100644 cmd/kafka_gen_log/marshal_arm64.go create mode 100644 cmd/kafka_gen_metric/marshal_amd64.go create mode 100644 cmd/kafka_gen_metric/marshal_arm64.go delete mode 100644 util/marshal_amd64.go delete mode 100644 util/marshal_arm64.go diff --git a/Makefile b/Makefile index e791d959..fd785f50 100644 --- a/Makefile +++ b/Makefile @@ -9,17 +9,9 @@ GOBUILD := $(GO) build $(BUILD_FLAG) pre: go mod tidy build: pre - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o clickhouse_sinker cmd/clickhouse_sinker/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o nacos_publish_config cmd/nacos_publish_config/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o kafka_gen_log cmd/kafka_gen_log/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o kafka_gen_metric cmd/kafka_gen_metric/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o kafka_gen_prom cmd/kafka_gen_prom/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o . ./... debug: pre - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o clickhouse_sinker cmd/clickhouse_sinker/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o nacos_publish_config cmd/nacos_publish_config/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o kafka_gen_log cmd/kafka_gen_log/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o kafka_gen_metric cmd/kafka_gen_metric/main.go - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o kafka_gen_prom cmd/kafka_gen_prom/main.go + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o . ./... unittest: pre go test -v ./... benchtest: pre diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 50d7eed2..1aa06dd1 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -283,7 +283,7 @@ func (g *LogGenerator) Run() { Xforwardfor: "", } _ = wp.Submit(func() { - if b, err = util.JSONMarshal(&logObj); err != nil { + if b, err = JSONMarshal(&logObj); err != nil { err = errors.Wrapf(err, "") util.Logger.Fatal("got error", zap.Error(err)) } diff --git a/cmd/kafka_gen_log/marshal_amd64.go b/cmd/kafka_gen_log/marshal_amd64.go new file mode 100644 index 00000000..76adce43 --- /dev/null +++ b/cmd/kafka_gen_log/marshal_amd64.go @@ -0,0 +1,9 @@ +package main + +import ( + "github.com/bytedance/sonic" +) + +func JSONMarshal(obj interface{}) ([]byte, error) { + return sonic.Marshal(obj) +} diff --git a/cmd/kafka_gen_log/marshal_arm64.go b/cmd/kafka_gen_log/marshal_arm64.go new file mode 100644 index 00000000..a956c795 --- /dev/null +++ b/cmd/kafka_gen_log/marshal_arm64.go @@ -0,0 +1,9 @@ +package main + +import ( + "encoding/json" +) + +func JSONMarshal(obj interface{}) ([]byte, error) { + return json.Marshal(obj) +} diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index fb7f1380..69940790 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -165,7 +165,7 @@ func generate() { _ = wp.Submit(func() { var b []byte - if b, err = util.JSONMarshal(&metric); err != nil { + if b, err = JSONMarshal(&metric); err != nil { err = errors.Wrapf(err, "") util.Logger.Fatal("got error", zap.Error(err)) } diff --git a/cmd/kafka_gen_metric/marshal_amd64.go b/cmd/kafka_gen_metric/marshal_amd64.go new file mode 100644 index 00000000..76adce43 --- /dev/null +++ b/cmd/kafka_gen_metric/marshal_amd64.go @@ -0,0 +1,9 @@ +package main + +import ( + "github.com/bytedance/sonic" +) + +func JSONMarshal(obj interface{}) ([]byte, error) { + return sonic.Marshal(obj) +} diff --git a/cmd/kafka_gen_metric/marshal_arm64.go b/cmd/kafka_gen_metric/marshal_arm64.go new file mode 100644 index 00000000..a956c795 --- /dev/null +++ b/cmd/kafka_gen_metric/marshal_arm64.go @@ -0,0 +1,9 @@ +package main + +import ( + "encoding/json" +) + +func JSONMarshal(obj interface{}) ([]byte, error) { + return json.Marshal(obj) +} diff --git a/util/marshal_amd64.go b/util/marshal_amd64.go deleted file mode 100644 index 46446889..00000000 --- a/util/marshal_amd64.go +++ /dev/null @@ -1,14 +0,0 @@ -package util - -import ( - "github.com/bytedance/sonic" - - "github.com/thanos-io/thanos/pkg/errors" -) - -func JSONMarshal(obj interface{}) (b []byte, err error) { - if b, err = sonic.Marshal(obj); err != nil { - err = errors.Wrapf(err, "") - } - return -} diff --git a/util/marshal_arm64.go b/util/marshal_arm64.go deleted file mode 100644 index e3c681a2..00000000 --- a/util/marshal_arm64.go +++ /dev/null @@ -1,14 +0,0 @@ -package util - -import ( - "encoding/json" - - "github.com/thanos-io/thanos/pkg/errors" -) - -func JSONMarshal(obj interface{}) (b []byte, err error) { - if b, err = json.Marshal(obj); err != nil { - err = errors.Wrapf(err, "") - } - return -} From 0014418c0d370e6fd99a4f23a0e6d20fa63985d1 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 23 Aug 2022 19:42:43 +0800 Subject: [PATCH 274/404] bump franz-go to v1.7.0 to fix https://github.com/twmb/franz-go/issues/182 --- go.mod | 6 +++--- go.sum | 17 ++++++----------- input/kafka_franz.go | 2 +- 3 files changed, 10 insertions(+), 15 deletions(-) diff --git a/go.mod b/go.mod index a0163955..c3e48358 100644 --- a/go.mod +++ b/go.mod @@ -21,9 +21,9 @@ require ( github.com/thanos-io/thanos v0.27.0 github.com/tidwall/gjson v1.14.2 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.6.0 + github.com/twmb/franz-go v1.7.0 github.com/twmb/franz-go/pkg/kadm v1.2.0 - github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 + github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.1 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.1.1 @@ -83,7 +83,7 @@ require ( go.uber.org/atomic v1.10.0 // indirect go.uber.org/multierr v1.8.0 // indirect golang.org/x/arch v0.0.0-20220722155209-00200b7164a7 // indirect - golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa // indirect + golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8 // indirect golang.org/x/net v0.0.0-20220812174116-3211cb980234 // indirect golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4 // indirect golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab // indirect diff --git a/go.sum b/go.sum index 381487b1..2e9c2598 100644 --- a/go.sum +++ b/go.sum @@ -212,12 +212,10 @@ github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFK github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM= -github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= github.com/jcmturner/gofork v1.7.6 h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVETg= github.com/jcmturner/gofork v1.7.6/go.mod h1:1622LH6i/EZqLloHfE7IeZ0uEJwMSUyQ/nDd82IeqRo= github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= -github.com/jcmturner/gokrb5/v8 v8.4.2/go.mod h1:sb+Xq/fTY5yktf/VxLsE3wlfPqQjp0aWNYyvBVK62bc= github.com/jcmturner/gokrb5/v8 v8.4.3 h1:iTonLeSJOn7MVUtyMT+arAn5AKAPrkilzhGw8wE/Tq8= github.com/jcmturner/gokrb5/v8 v8.4.3/go.mod h1:dqRwJGXznQrzw6cWmyo6kH+E7jksEQG/CyVWsJEsJO0= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= @@ -361,7 +359,6 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= -github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.5/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= @@ -385,21 +382,20 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNij github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= -github.com/twmb/franz-go v1.5.0/go.mod h1:ZKQ5AtqBbdc783bLCay7nDc21lJnIIA8mFJYhLMF19E= github.com/twmb/franz-go v1.5.3/go.mod h1:eqHYpAuvlTArOdZ1XtPYyOQ1uUb40CSZwbpL3ccjibI= -github.com/twmb/franz-go v1.6.0 h1:yri7YsVBe/k1LKcoZSLILgUI3U14e82qtD9i4VOcs9c= github.com/twmb/franz-go v1.6.0/go.mod h1:xdMwpUIQL/JDKKwerc5qJQG8TU1SNIddfjKJJyqRJIg= +github.com/twmb/franz-go v1.7.0 h1:h0ZKMqgdtxfPlTpnjt37fOpv/Xj8h3EWxHAQAA5Zclc= +github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= github.com/twmb/franz-go/pkg/kadm v1.2.0 h1:3jtgdCi04TPGSCiaIBe2nxBNkHoCZRm/YljSFEh7BHA= github.com/twmb/franz-go/pkg/kadm v1.2.0/go.mod h1:izleX4EttZwes7MRiHFmPWPSqDuGr4VN6p7jCIYoU0g= github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.2.0 h1:jYWh2qFw5lDbNv5Gvu/sMKagzICxuA5L6m1W2Oe7XUo= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0 h1:Wfq7o2Eu8CaPJOqu/mqM1pu9b3HuO0aWfDGcdo1cVR8= -github.com/twmb/franz-go/pkg/sasl/kerberos v1.0.0/go.mod h1:S0nXtRBGwqtOfTGQL3BDliF3hSxyUm0dqy5gEfPsQT0= +github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 h1:alKdbddkPw3rDh+AwmUEwh6HNYgTvDSFIe/GWYRR9RM= +github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0/go.mod h1:k8BoBjyUbFj34f0rRbn+Ky12sZFAPbmShrg0karAIMo= github.com/twmb/franz-go/plugin/kzap v1.1.1 h1:ae8Z2JXn8y9ceZ2AFnwPm5U1A8d6cBvvZlygF46F2N4= github.com/twmb/franz-go/plugin/kzap v1.1.1/go.mod h1:TUlWYqucIyz6U7xLo++gkHBDiJmed5FpYe42rCL2YG0= -github.com/twmb/go-rbtree v1.0.0/go.mod h1:UlIAI8gu3KRPkXSobZnmJfVwCJgEhD/liWzT5ppzIyc= github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= github.com/valyala/fastjson v1.6.3/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= @@ -456,12 +452,12 @@ golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220518034528-6f7dac969898/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa h1:zuSxTR4o9y82ebqCUJYNGJbGPo6sKVl54f/TVDObg1c= golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8 h1:GIAS/yBem/gq2MUqgNIzUHW7cJMmx3TGZOrnyYaNQ6c= +golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -529,7 +525,6 @@ golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220706163947-c90051bbdb60/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= golang.org/x/net v0.0.0-20220809184613-07c6da5e1ced/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= diff --git a/input/kafka_franz.go b/input/kafka_franz.go index c1da7718..0eb11dc0 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -150,7 +150,7 @@ func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { auth.Client = krb5client.NewWithPassword(gssapiCfg.Username, gssapiCfg.Realm, gssapiCfg.Password, krbCfg, krb5client.DisablePAFXFAST(gssapiCfg.DisablePAFXFAST)) } - mch = auth.AsMechanism() + mch = auth.AsMechanismWithClose() } if mch != nil { opts = append(opts, kgo.SASL(mch)) From 2e2e6041ecae6ceee1c3912670091b7f5d013c49 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 26 Aug 2022 10:50:11 +0800 Subject: [PATCH 275/404] bumped clickhouse-go to v2.3.0 --- go.mod | 7 ++++++- go.sum | 38 ++++++++++++-------------------------- 2 files changed, 18 insertions(+), 27 deletions(-) diff --git a/go.mod b/go.mod index c3e48358..f7ea7681 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module github.com/housepower/clickhouse_sinker go 1.19 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.2.0 + github.com/ClickHouse/clickhouse-go/v2 v2.3.0 github.com/RoaringBitmap/roaring v1.2.1 github.com/Shopify/sarama v1.36.0 github.com/bytedance/sonic v1.3.5 @@ -34,7 +34,9 @@ require ( ) require ( + github.com/ClickHouse/ch-go v0.47.3 // indirect github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723 // indirect + github.com/andybalholm/brotli v1.0.4 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.3.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect @@ -44,6 +46,8 @@ require ( github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 // indirect github.com/eapache/queue v1.1.0 // indirect github.com/go-errors/errors v1.4.2 // indirect + github.com/go-faster/city v1.0.1 // indirect + github.com/go-faster/errors v0.6.1 // indirect github.com/golang/protobuf v1.5.2 // indirect github.com/golang/snappy v0.0.4 // indirect github.com/google/uuid v1.3.0 // indirect @@ -69,6 +73,7 @@ require ( github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/procfs v0.8.0 // indirect github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect + github.com/segmentio/asm v1.2.0 // indirect github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.0 // indirect diff --git a/go.sum b/go.sum index 2e9c2598..d2190081 100644 --- a/go.sum +++ b/go.sum @@ -34,9 +34,10 @@ dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7 github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/ClickHouse/clickhouse-go v1.5.4/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= -github.com/ClickHouse/clickhouse-go/v2 v2.2.0 h1:dj00TDKY+xwuTJdbpspCSmTLFyWzRJerTHwaBxut1C0= -github.com/ClickHouse/clickhouse-go/v2 v2.2.0/go.mod h1:8f2XZUi7XoeU+uPIytSi1cvx8fmJxi7vIgqpvYTF1+o= +github.com/ClickHouse/ch-go v0.47.3 h1:bBKid8DRELKRf4/oXqrEks7Cc4DLb5Giwm9uazM6h3M= +github.com/ClickHouse/ch-go v0.47.3/go.mod h1:m3LHc5FeQ1Jjee5EEay5e7hQmSk4SuKyMfifNUz8l3g= +github.com/ClickHouse/clickhouse-go/v2 v2.3.0 h1:v0iT0yZspjjNgnLyPUa0WoGMme0Y/sNjCtOAFcyBkkA= +github.com/ClickHouse/clickhouse-go/v2 v2.3.0/go.mod h1:f2kb1LPopJdIyt0Y0vxNk9aiQCyhCmeVcyvOOaPCT4Q= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.2.1 h1:58/LJlg/81wfEHd5L9qsHduznOIhyv4qb1yWcSvVq9A= @@ -45,7 +46,6 @@ github.com/Shopify/sarama v1.36.0 h1:0OJs3eCcnezkWniVjwBbCJVaa0B1k7ImCRS3WN6NsSk github.com/Shopify/sarama v1.36.0/go.mod h1:9glG3eX83tgVYJ5aVtrjVUnEsOPqQIBGx1BWfN+X51I= github.com/Shopify/toxiproxy/v2 v2.4.0 h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64= github.com/Shopify/toxiproxy/v2 v2.4.0/go.mod h1:3ilnjng821bkozDRxNoo64oI/DKqM+rOyJzb564+bvg= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= @@ -54,6 +54,8 @@ github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk5 github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723 h1:oBwUer3XCooqTDMBJ5vR2tTBv/4Te125UzMoh+/na5c= github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723/go.mod h1:RcDobYh8k5VP6TNybz9m++gL3ijVI5wueVr0EM10VsU= +github.com/andybalholm/brotli v1.0.4 h1:V7DdXeJtZscaqfNuAdSRuRFzuiKlHSC/Zh3zl9qY3JY= +github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -63,7 +65,6 @@ github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6r github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.3.0 h1:h7mv5q31cthBTd7V4kLAZaIThj1e8vPGcSqpPue9KVI= github.com/bits-and-blooms/bitset v1.3.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= -github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.3.5 h1:xfBNhsG3QCC+AMCmCHxNQg0StI5IM/B9Jtwjqi5WlI0= @@ -79,7 +80,6 @@ github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWR github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= @@ -104,6 +104,10 @@ github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHqu github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= +github.com/go-faster/city v1.0.1 h1:4WAxSZ3V2Ws4QRDrscLEDcibJY8uf41H6AhXDrNDcGw= +github.com/go-faster/city v1.0.1/go.mod h1:jKcUJId49qdW3L1qKHH/3wPeUstCVpVSXTM6vO3VcTw= +github.com/go-faster/errors v0.6.1 h1:nNIPOBkprlKzkThvS/0YaX8Zs9KewLCOSFQS5BU06FI= +github.com/go-faster/errors v0.6.1/go.mod h1:5MGV2/2T9yvlrbhe9pD9LO5Z/2zCSq2T8j+Jpi2LAyY= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= @@ -117,12 +121,7 @@ github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logfmt/logfmt v0.5.1 h1:otpy5pqBCBZ1ng9RQ0dPu4PN7ba75Y/aA+UpowDyNVA= github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= -github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= -github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= -github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/goccy/go-json v0.9.4 h1:L8MLKG2mvVXiQu07qB6hmfqeSYQdOnqPot2GhsIwIaI= github.com/goccy/go-json v0.9.4/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= @@ -192,11 +191,9 @@ github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+ github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/gorilla/handlers v1.4.2/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/OAirnOIQ= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= -github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= @@ -227,7 +224,6 @@ github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9Y github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= -github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= @@ -263,13 +259,10 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= -github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI= github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= -github.com/mkevac/debugcharts v0.0.0-20191222103121-ae1c48aa8615/go.mod h1:Ad7oeElCZqA1Ufj0U9/liOF4BtVepxRcTvr2ey7zTvM= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -286,7 +279,6 @@ github.com/nacos-group/nacos-sdk-go v1.1.2/go.mod h1:I8Vj4M8ZLpBk7EY2A8RXQE1SbfC github.com/paulmach/orb v0.7.1 h1:Zha++Z5OX/l168sqHK3k4z18LDvr+YAO/VjK0ReQ9rU= github.com/paulmach/orb v0.7.1/go.mod h1:FWRlTgl88VI1RBx/MkrwWDRhQ96ctqMCh8boXhmqB/A= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= -github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4/v4 v4.1.14/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -333,12 +325,11 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= +github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= github.com/segmentio/kafka-go v0.4.34 h1:Dm6YlLMiVSiwwav20KY0AoY63s661FXevwJ3CVHUERo= github.com/segmentio/kafka-go v0.4.34/go.mod h1:GAjxBQJdQMB5zfNA21AhpaqOB2Mu+w3De4ni3Gbm8y0= -github.com/shirou/gopsutil v2.19.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/gopsutil v3.21.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil/v3 v3.22.4/go.mod h1:D01hZJ4pVHPpCTZ3m3T2+wDF2YAGfd+H4ifUguaQzHM= -github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= @@ -361,7 +352,6 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.5/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/thanos-io/thanos v0.27.0 h1:TWOPMkWX5Q1LHdROI6dg9lKU3x6Q95vO4o/bm8GaCfE= @@ -421,10 +411,8 @@ go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opentelemetry.io/otel v1.7.0/go.mod h1:5BdUoMIz5WEs0vt0CUEMtSSaTSHBBVwrhnz7+nrD5xk= go.opentelemetry.io/otel v1.9.0 h1:8WZNQFIB2a71LnANS9JeyidJKKGOOremcUtb/OtHISw= go.opentelemetry.io/otel v1.9.0/go.mod h1:np4EoPGzoPs3O67xUVNoPPcmSvsfOxNlNA4F4AC+0Eo= -go.opentelemetry.io/otel/trace v1.7.0/go.mod h1:fzLSB9nqR2eXzxPXb2JW9IKE+ScyXA48yyE4TNvoHqU= go.opentelemetry.io/otel/trace v1.9.0 h1:oZaCNJUjWcg60VXWee8lJKlqhPbXAPB51URuR47pQYc= go.opentelemetry.io/otel/trace v1.9.0/go.mod h1:2737Q0MuG8q1uILYm2YYVkAyLtOofiTNGg6VODnOiPo= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -567,7 +555,6 @@ golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191220220014-0732a990476f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -597,7 +584,6 @@ golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220128215802-99c3d69c2c27/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220429233432-b5fbb4746d32/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= From 4d3f02e29f753f5fe16a7a7fd418d66a10778323 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 3 Sep 2022 10:02:59 +0800 Subject: [PATCH 276/404] bump franz-go v1.7.1 --- go.mod | 4 ++-- go.sum | 7 ++++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index f7ea7681..062bc9e5 100644 --- a/go.mod +++ b/go.mod @@ -21,8 +21,8 @@ require ( github.com/thanos-io/thanos v0.27.0 github.com/tidwall/gjson v1.14.2 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.7.0 - github.com/twmb/franz-go/pkg/kadm v1.2.0 + github.com/twmb/franz-go v1.7.1 + github.com/twmb/franz-go/pkg/kadm v1.2.1 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.1 github.com/valyala/fastjson v1.6.3 diff --git a/go.sum b/go.sum index d2190081..15cef232 100644 --- a/go.sum +++ b/go.sum @@ -374,10 +374,11 @@ github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.5.3/go.mod h1:eqHYpAuvlTArOdZ1XtPYyOQ1uUb40CSZwbpL3ccjibI= github.com/twmb/franz-go v1.6.0/go.mod h1:xdMwpUIQL/JDKKwerc5qJQG8TU1SNIddfjKJJyqRJIg= -github.com/twmb/franz-go v1.7.0 h1:h0ZKMqgdtxfPlTpnjt37fOpv/Xj8h3EWxHAQAA5Zclc= github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go/pkg/kadm v1.2.0 h1:3jtgdCi04TPGSCiaIBe2nxBNkHoCZRm/YljSFEh7BHA= -github.com/twmb/franz-go/pkg/kadm v1.2.0/go.mod h1:izleX4EttZwes7MRiHFmPWPSqDuGr4VN6p7jCIYoU0g= +github.com/twmb/franz-go v1.7.1 h1:VyAByZr0GHUQ4vaql01la9RjGXMXElEVxq/JvcfKTRo= +github.com/twmb/franz-go v1.7.1/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= +github.com/twmb/franz-go/pkg/kadm v1.2.1 h1:jbPvbJgXmIREAso5WI4BU3hJLOkGDvktqWPUy/VtsSA= +github.com/twmb/franz-go/pkg/kadm v1.2.1/go.mod h1:izleX4EttZwes7MRiHFmPWPSqDuGr4VN6p7jCIYoU0g= github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.2.0 h1:jYWh2qFw5lDbNv5Gvu/sMKagzICxuA5L6m1W2Oe7XUo= From db78e57c24dc30a59640f2c7914a5e505de6d04e Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 19 Sep 2022 16:21:22 +0800 Subject: [PATCH 277/404] added JSON support --- cmd/kafka_gen_prom/main.go | 5 +++-- model/metric.go | 1 + model/value.go | 12 ++++++++--- parser/csv.go | 9 ++++++-- parser/fastjson.go | 44 ++++++++++++++++++++++++++++++++++++++ parser/gjson.go | 6 ++++++ parser/parser_test.go | 9 ++++---- 7 files changed, 75 insertions(+), 11 deletions(-) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index 2dd346d1..7039a945 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -95,12 +95,13 @@ import ( "go.uber.org/zap" ) +// number of series: NumMetrics * (NumRunes^LenVal)^NumKeys const ( Alpha = "abcdefghijklmnopqrstuvwxyz" NumMetrics = 1000 - NumKeys = 5 + NumKeys = 3 NumRunes = 10 - LenVal = 1 // 1000 * (10^1)^5 = 10^8 series + LenVal = 1 // 1000 * (10^1)^3 = 10^6 series NumAllKeys = 1000 ) diff --git a/model/metric.go b/model/metric.go index 8de2116d..e79d757a 100644 --- a/model/metric.go +++ b/model/metric.go @@ -36,6 +36,7 @@ type Metric interface { GetDecimal(key string, nullable bool) (val interface{}) GetDateTime(key string, nullable bool) (val interface{}) GetString(key string, nullable bool) (val interface{}) + GetObject(key string, nullable bool) (val interface{}) GetArray(key string, t int) (val interface{}) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, black *regexp.Regexp, partition int, offset int64) bool } diff --git a/model/value.go b/model/value.go index 83e18cb7..107a880d 100644 --- a/model/value.go +++ b/model/value.go @@ -1,10 +1,11 @@ -/*Copyright [2019] housepower +/* +Copyright [2019] housepower Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, @@ -37,6 +38,7 @@ const ( Decimal DateTime String + Object ) type TypeInfo struct { @@ -80,6 +82,8 @@ func GetTypeName(typ int) (name string) { name = "DateTime" case String: name = "String" + case Object: + name = "Object('json')" default: name = "Unknown" } @@ -120,6 +124,8 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { val = metric.GetDateTime(name, cwt.Nullable) case String: val = metric.GetString(name, cwt.Nullable) + case Object: + val = metric.GetObject(name, cwt.Nullable) default: util.Logger.Fatal("LOGIC ERROR: reached switch default condition") } @@ -160,7 +166,7 @@ func WhichType(typ string) (dataType int, nullable bool, array bool) { func init() { typeInfo = make(map[string]TypeInfo) - for _, t := range []int{Bool, Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32, Float64, DateTime, String} { + for _, t := range []int{Bool, Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32, Float64, DateTime, String, Object} { tn := GetTypeName(t) typeInfo[tn] = TypeInfo{Type: t} nullTn := fmt.Sprintf("Nullable(%s)", tn) diff --git a/parser/csv.go b/parser/csv.go index cc06ea60..68401ec8 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -1,10 +1,11 @@ -/*Copyright [2019] housepower +/* +Copyright [2019] housepower Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, @@ -240,6 +241,10 @@ func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}) { return } +func (c *CsvMetric) GetObject(key string, nullable bool) (val interface{}) { + return +} + // GetArray parse an CSV encoded array func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { s := c.GetString(key, false) diff --git a/parser/fastjson.go b/parser/fastjson.go index d155b407..6d8793cd 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -232,6 +232,12 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} return } +func (c *FastjsonMetric) GetObject(key string, nullable bool) (val interface{}) { + v := c.value.Get(key) + val = val2map(v) + return +} + func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { var array []*fastjson.Value if v := c.value.Get(key); v != nil { @@ -314,12 +320,48 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { arr = append(arr, t) } val = arr + case model.Object: + arr := make([]map[string]interface{}, 0) + for _, e := range array { + m := val2map(e) + if m != nil { + arr = append(arr, m) + } + } + val = arr default: util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) } return } +func val2map(v *fastjson.Value) (m map[string]interface{}) { + var err error + var obj *fastjson.Object + if obj, err = v.Object(); err != nil { + return + } + m = make(map[string]interface{}, obj.Len()) + obj.Visit(func(key []byte, v *fastjson.Value) { + strKey := string(key) + switch v.Type() { + case fastjson.TypeString: + var vb []byte + if vb, err = v.StringBytes(); err != nil { + return + } + m[strKey] = string(vb) + case fastjson.TypeNumber: + var f float64 + if f, err = v.Float64(); err != nil { + return + } + m[strKey] = f + } + }) + return +} + func FastjsonIntArray[T constraints.Signed](a []*fastjson.Value, min, max int64) (arr []T) { arr = make([]T, 0) var val T @@ -508,6 +550,8 @@ func fjDetectType(v *fastjson.Value, depth int) (typ int, array bool) { typ = model.DateTime } } + case fastjson.TypeObject: + typ = model.Object case fastjson.TypeArray: if depth >= 1 { return diff --git a/parser/gjson.go b/parser/gjson.go index d85b81b3..c23ee6bd 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -224,6 +224,10 @@ func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { return } +func (c *GjsonMetric) GetObject(key string, nullable bool) (val interface{}) { + return +} + func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { var array []gjson.Result r := gjson.Get(c.raw, key) @@ -475,6 +479,8 @@ func gjDetectType(v gjson.Result, depth int) (typ int, array bool) { if array := v.Array(); len(array) != 0 { typ, _ = gjDetectType(array[0], depth+1) } + } else if v.IsObject() { + typ = model.Object } default: } diff --git a/parser/parser_test.go b/parser/parser_test.go index 04f9cf9b..1fc61f01 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -1,10 +1,11 @@ -/*Copyright [2019] housepower +/* +Copyright [2019] housepower Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, @@ -83,7 +84,7 @@ var jsonSchema = map[string]string{ "str_time_rfc3339_2": "DateTime", "str_time_clickhouse_1": "DateTime", "str_time_clickhouse_2": "DateTime", - "obj": "Unknown", + "obj": "Object('json')", "array_empty": "Unknown", "array_null": "Unknown", "array_bool": "BoolArray", @@ -98,7 +99,7 @@ var jsonSchema = map[string]string{ "array_str_date_2": "DateTimeArray", "array_str_time_rfc3339": "DateTimeArray", "array_str_time_clickhouse": "DateTimeArray", - "array_obj": "Unknown", + "array_obj": "Object('json')Array", } var csvSample = []byte(`null,true,false,123,123.321,"escaped_""ws",123,123.321,2009-07-13,13/07/2009,2009-07-13T09:07:13Z,2009-07-13T09:07:13.123+08:00,2009-07-13 09:07:13,2009-07-13 09:07:13.123,"{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",[],[null],"[true,false]","[0,255,256,65535,65536,4294967295,4294967296,18446744073709551615,18446744073709551616]","[-9223372036854775808,-2147483649,-2147483648,-32769,-32768,-129,-128,0,127,128,32767,32768,2147483647,2147483648,9223372036854775807]","[4.940656458412465441765687928682213723651e-324,1.401298464324817070923729583289916131280e-45,0.0,3.40282346638528859811704183484516925440e+38,1.797693134862315708145274237317043567981e+308]","[""aa"",""bb"",""cc""]","[""0"",""255"",""256"",""65535"",""65536"",""4294967295"",""4294967296"",""18446744073709551615"",""18446744073709551616""]","[""-9223372036854775808"",""-2147483649"",""-2147483648"",""-32769"",""-32768"",""-129"",""-128"",""0"",""127"",""128"",""32767"",""32768"",""2147483647"",""2147483648"",""9223372036854775807""]","[""4.940656458412465441765687928682213723651e-324"",""1.401298464324817070923729583289916131280e-45"",""0.0"",""3.40282346638528859811704183484516925440e+38"",""1.797693134862315708145274237317043567981e+308""]","[""2009-07-13"",""2009-07-14"",""2009-07-15""]","[""13/07/2009"",""14/07/2009"",""15/07/2009""]","[""2009-07-13T09:07:13Z"",""2009-07-13T09:07:13+08:00"",""2009-07-13T09:07:13.123Z"",""2009-07-13T09:07:13.123+08:00""]","[""2009-07-13 09:07:13"",""2009-07-13 09:07:13.123""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]"`) From 3ea8c48940bf037d977ade0eff5dc0a7f64e045d Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 21 Sep 2022 11:08:04 +0800 Subject: [PATCH 278/404] pause parsing series when writing another series --- cmd/kafka_gen_prom/main.go | 18 ++++++++- model/message.go | 75 ++------------------------------------ output/clickhouse.go | 47 ++++++++++++++++-------- task/task.go | 62 ++++++++++++++++++++++++++++++- 4 files changed, 112 insertions(+), 90 deletions(-) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index 7039a945..2189055c 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -82,12 +82,14 @@ import ( "math/rand" "os" "os/signal" + "sort" "strings" "sync/atomic" "syscall" "time" "github.com/bytedance/sonic" + "github.com/cespare/xxhash/v2" "github.com/google/gops/agent" "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" @@ -119,17 +121,27 @@ type Datapoint struct { Timestamp time.Time Value float32 Name string `json:"__name__"` - Labels + Labels Labels + LabelKeys []string } // I need every label be present at the top level. func (dp Datapoint) MarshalJSON() ([]byte, error) { + var dig xxhash.Digest + for _, labelKey := range dp.LabelKeys { + _, _ = dig.WriteString("###") + _, _ = dig.WriteString(labelKey) + _, _ = dig.WriteString("###") + _, _ = dig.WriteString(dp.Labels[labelKey]) + } + mgmtID := int64(dig.Sum64()) + seriesID := mgmtID labels, err := sonic.MarshalString(dp.Labels) if err != nil { return nil, err } labels2 := labels[1 : len(labels)-1] - msg := fmt.Sprintf(`{"timestamp":"%s", "value":%f,"__name__":"%s","labels":%s,%s}`, dp.Timestamp.Format(time.RFC3339), dp.Value, dp.Name, labels, labels2) + msg := fmt.Sprintf(`{"timestamp":"%s", "value":%f,"__name__":"%s","labels":%s,%s,"__series_id":%d,"__mgmt_id":%d}`, dp.Timestamp.Format(time.RFC3339), dp.Value, dp.Name, labels, labels2, seriesID, mgmtID) return []byte(msg), nil } @@ -158,6 +170,7 @@ func initMetrics() { key := fmt.Sprintf("key_%06d", rand.Intn(NumAllKeys+1)) m.LabelKeys[j] = key } + sort.Strings(m.LabelKeys) metrics[i] = m } } @@ -198,6 +211,7 @@ func generate() { Value: rand.Float32(), Name: metrics[i].Name, Labels: make(Labels), + LabelKeys: metrics[i].LabelKeys, } for _, key := range metrics[i].LabelKeys { dp.Labels[key] = randValue() diff --git a/model/message.go b/model/message.go index 6e727033..d2fcf87c 100644 --- a/model/message.go +++ b/model/message.go @@ -2,15 +2,11 @@ package model import ( "container/list" - "fmt" - "regexp" "strconv" - "strings" "sync" "sync/atomic" "time" - "github.com/cespare/xxhash/v2" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/statistics" ) @@ -47,9 +43,9 @@ type Batch struct { Group *BatchGroup } -//BatchGroup consists of multiple batches. -//The `before` relationship could be impossible if messages of a partition are distributed to multiple batches. -//So those batches need to be committed after ALL of them have been written to clickhouse. +// BatchGroup consists of multiple batches. +// The `before` relationship could be impossible if messages of a partition are distributed to multiple batches. +// So those batches need to be committed after ALL of them have been written to clickhouse. type BatchGroup struct { Batchs []*Batch Offsets map[int]int64 @@ -168,68 +164,3 @@ func PutRow(r *Row) { *r = (*r)[:0] rowPool.Put(r) } - -func MetricToRow(metric Metric, msg *InputMessage, dims []*ColumnWithType, idxSeriesID int, nameKey string, lblBlkList *regexp.Regexp) (row *Row) { - row = GetRow() - var dig *xxhash.Digest - var labels []string - var seriesID, mgmtID int64 - if idxSeriesID >= 0 { - // If some labels are not Prometheus native, ETL shall calculate and pass "__series_id", otherwise clickhouse_sinker use "__mgmt_id". - val := metric.GetInt64("__series_id", false) - seriesID = val.(int64) - // clickhouse_sinker calculate "__mgmt_id" based on all labels. - dig = xxhash.New() - } - for i, dim := range dims { - if idxSeriesID >= 0 && i == idxSeriesID { - *row = append(*row, int64(0)) - } else if idxSeriesID >= 0 && i == idxSeriesID+1 { - *row = append(*row, int64(0)) - } else if idxSeriesID >= 0 && i == idxSeriesID+2 { - *row = append(*row, "") - } else if strings.HasPrefix(dim.Name, "__kafka") { - if strings.HasSuffix(dim.Name, "_topic") { - *row = append(*row, msg.Topic) - } else if strings.HasSuffix(dim.Name, "_partition") { - *row = append(*row, msg.Partition) - } else if strings.HasSuffix(dim.Name, "_offset") { - *row = append(*row, msg.Offset) - } else if strings.HasSuffix(dim.Name, "_key") { - *row = append(*row, string(msg.Key)) - } else if strings.HasSuffix(dim.Name, "_timestamp") { - *row = append(*row, *msg.Timestamp) - } else { - *row = append(*row, nil) - } - } else { - val := GetValueByType(metric, dim) - *row = append(*row, val) - if idxSeriesID >= 0 && dim.Type == String && val != nil { - if labelVal := val.(string); labelVal != "" { - // "__series_id" calculation envolves all Prometheus native labels (including metric name) in some order. - if dig != nil { - _, _ = dig.WriteString("###") - _, _ = dig.WriteString(dim.Name) - _, _ = dig.WriteString("###") - _, _ = dig.WriteString(labelVal) - } - // "labels" JSON excludes "le", so that "labels" can be used as group key for histogram queries. - if dim.Name != nameKey && dim.Name != "le" && (lblBlkList == nil || !lblBlkList.MatchString(dim.Name)) { - labels = append(labels, fmt.Sprintf(`%s: %s`, strconv.Quote(dim.Name), strconv.Quote(labelVal))) - } - } - } - } - } - if idxSeriesID >= 0 { - mgmtID = int64(dig.Sum64()) - if seriesID == 0 { - seriesID = mgmtID - } - (*row)[idxSeriesID] = seriesID - (*row)[idxSeriesID+1] = mgmtID - (*row)[idxSeriesID+2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) - } - return -} diff --git a/output/clickhouse.go b/output/clickhouse.go index 7576bb6a..7e2a8dcc 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -24,6 +24,7 @@ import ( "sort" "strings" "sync" + "sync/atomic" "time" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" @@ -53,6 +54,7 @@ var ( // ClickHouse is an output service consumers from kafka messages type ClickHouse struct { Dims []*model.ColumnWithType + NumDims int IdxSerID int NameKey string cfg *config.Config @@ -64,7 +66,8 @@ type ClickHouse struct { distMetricTbls []string distSeriesTbls []string - bmSeries map[uint64]bool + bmSeries map[int64]struct{} + wrSeries atomic.Int32 numFlying int32 mux sync.Mutex taskDone *sync.Cond @@ -112,23 +115,36 @@ func (c *ClickHouse) Send(batch *model.Batch) { }) } +func (c *ClickHouse) LoadOrStoreSeries(sid int64) (loaded bool) { + c.mux.Lock() + _, loaded = c.bmSeries[sid] + if !loaded { + c.bmSeries[sid] = struct{}{} + } + c.mux.Unlock() + return +} + +func (c *ClickHouse) IsWritingSeries() bool { + return c.wrSeries.Load() > 0 +} + func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err error) { var seriesRows model.Rows - c.mux.Lock() for _, row := range rows { - mgmtID := uint64((*row)[c.IdxSerID+1].(int64)) - if _, found := c.bmSeries[mgmtID]; !found { - seriesRows = append(seriesRows, row) - c.bmSeries[mgmtID] = true + if len(*row) != c.NumDims { + continue } + seriesRows = append(seriesRows, row) } - c.mux.Unlock() if len(seriesRows) != 0 { begin := time.Now() var numBad int - if numBad, err = writeRows(c.promSerSQL, seriesRows, c.IdxSerID, len(c.Dims), conn); err != nil { + c.wrSeries.Add(1) + if numBad, err = writeRows(c.promSerSQL, seriesRows, c.IdxSerID, c.NumDims, conn); err != nil { return } + c.wrSeries.Add(-1) if numBad != 0 { statistics.ParseMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(numBad)) } @@ -146,9 +162,9 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( if conn, *dbVer, err = sc.NextGoodReplica(*dbVer); err != nil { return } - //row[:c.IdxSerID] is for metric table + //row[:c.IdxSerID+1] is for metric table //row[c.IdxSerID:] is for series table - numDims := len(c.Dims) + numDims := c.NumDims if c.taskCfg.PrometheusSchema { numDims = c.IdxSerID + 1 if err = c.writeSeries(*batch.Rows, conn); err != nil { @@ -208,15 +224,15 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { if c.cfg.Clickhouse.Cluster != "" { tbl = c.distSeriesTbls[0] } - c.bmSeries = make(map[uint64]bool) + c.bmSeries = make(map[int64]struct{}) if !c.taskCfg.LoadSeriesAtStartup { util.Logger.Info(fmt.Sprintf("skipped loading series from %v", tbl), zap.String("task", c.taskCfg.Name)) return } - query := fmt.Sprintf("SELECT toUInt64(toInt64(__mgmt_id)) AS mid FROM %s.%s ORDER BY mid", c.cfg.Clickhouse.DB, tbl) + query := fmt.Sprintf("SELECT toInt64(__mgmt_id) AS mid FROM %s.%s ORDER BY mid", c.cfg.Clickhouse.DB, tbl) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs driver.Rows - var mgmtID uint64 + var mgmtID int64 if rs, err = conn.Query(context.Background(), query); err != nil { err = errors.Wrapf(err, "") return err @@ -227,7 +243,7 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { err = errors.Wrapf(err, "") return err } - c.bmSeries[mgmtID] = true + c.bmSeries[mgmtID] = struct{}{} } util.Logger.Info(fmt.Sprintf("loaded %d series from %v", len(c.bmSeries), tbl), zap.String("task", c.taskCfg.Name)) return @@ -354,7 +370,8 @@ func (c *ClickHouse) initSchema() (err error) { return } // Generate SQL for INSERT - numDims := len(c.Dims) + c.NumDims = len(c.Dims) + numDims := c.NumDims if c.taskCfg.PrometheusSchema { numDims = c.IdxSerID + 1 } diff --git a/task/task.go b/task/task.go index 3f0afd91..a2d56f81 100644 --- a/task/task.go +++ b/task/task.go @@ -19,6 +19,8 @@ import ( "fmt" "math" "regexp" + "strconv" + "strings" "sync" "sync/atomic" "time" @@ -48,6 +50,7 @@ type Service struct { blackList *regexp.Regexp lblBlkList *regexp.Regexp dims []*model.ColumnWithType + numDims int idxSerID int nameKey string @@ -105,6 +108,7 @@ func (service *Service) Init() (err error) { } service.dims = service.clickhouse.Dims + service.numDims = len(service.dims) service.idxSerID = service.clickhouse.IdxSerID service.nameKey = service.clickhouse.NameKey service.limiter1 = rate.NewLimiter(rate.Every(10*time.Second), 1) @@ -296,7 +300,7 @@ func (service *Service) put(msg *model.InputMessage) { msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - row = model.MetricToRow(metric, msg, service.dims, service.idxSerID, service.nameKey, service.lblBlkList) + row = service.metric2Row(metric, msg) if taskCfg.DynamicSchema.Enable { foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys, &service.warnKeys, service.whiteList, service.blackList, msg.Partition, msg.Offset) } @@ -419,3 +423,59 @@ func (service *Service) Stop() { service.wgRun.Wait() util.Logger.Debug("stopped task", zap.String("task", taskCfg.Name)) } + +func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) (row *model.Row) { + row = model.GetRow() + if service.idxSerID >= 0 { + var seriesID, mgmtID int64 + var labels []string + // If some labels are not Prometheus native, ETL shall calculate and pass "__series_id" and "__mgmt_id". + val := metric.GetInt64("__series_id", false) + seriesID = val.(int64) + val = metric.GetInt64("__mgmt_id", false) + mgmtID = val.(int64) + for i := 0; i < service.idxSerID; i++ { + dim := service.dims[i] + val := model.GetValueByType(metric, dim) + *row = append(*row, val) + } + *row = append(*row, seriesID) + newSeries := !service.clickhouse.IsWritingSeries() && !service.clickhouse.LoadOrStoreSeries(seriesID) + if newSeries { + *row = append(*row, mgmtID, nil) + for i := service.idxSerID + 3; i < service.numDims; i++ { + dim := service.dims[i] + val := model.GetValueByType(metric, dim) + *row = append(*row, val) + if val != nil && dim.Type == model.String && dim.Name != service.nameKey && dim.Name != "le" && (service.lblBlkList == nil || !service.lblBlkList.MatchString(dim.Name)) { + // "labels" JSON excludes "le", so that "labels" can be used as group key for histogram queries. + labelVal := val.(string) + labels = append(labels, fmt.Sprintf(`%s: %s`, strconv.Quote(dim.Name), strconv.Quote(labelVal))) + } + } + (*row)[service.idxSerID+2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) + } + } else { + for _, dim := range service.dims { + if strings.HasPrefix(dim.Name, "__kafka") { + if strings.HasSuffix(dim.Name, "_topic") { + *row = append(*row, msg.Topic) + } else if strings.HasSuffix(dim.Name, "_partition") { + *row = append(*row, msg.Partition) + } else if strings.HasSuffix(dim.Name, "_offset") { + *row = append(*row, msg.Offset) + } else if strings.HasSuffix(dim.Name, "_key") { + *row = append(*row, string(msg.Key)) + } else if strings.HasSuffix(dim.Name, "_timestamp") { + *row = append(*row, *msg.Timestamp) + } else { + *row = append(*row, nil) + } + } else { + val := model.GetValueByType(metric, dim) + *row = append(*row, val) + } + } + } + return +} From daaa9668cf698a9cdb466e691473deabbc439161 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 21 Sep 2022 16:33:53 +0800 Subject: [PATCH 279/404] introduced Clickhouse.AllowWriteSeries --- output/clickhouse.go | 24 +++++++++++++----------- task/task.go | 2 +- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 7e2a8dcc..930157af 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -24,7 +24,6 @@ import ( "sort" "strings" "sync" - "sync/atomic" "time" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" @@ -49,6 +48,7 @@ var ( // src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp // ZooKeeper issues(https://issues.apache.org/jira/browse/ZOOKEEPER-4410) can cause ClickHouse exeception: "Code": 999, "Message": "Cannot allocate block number..." replicaSpecificErrorCodes = []int32{242, 319, 999, 1000} //TABLE_IS_READ_ONLY, UNKNOWN_STATUS_OF_INSERT, KEEPER_EXCEPTION, POCO_EXCEPTION + wrSeriesQuota = 16384 ) // ClickHouse is an output service consumers from kafka messages @@ -67,7 +67,7 @@ type ClickHouse struct { distSeriesTbls []string bmSeries map[int64]struct{} - wrSeries atomic.Int32 + wrSeries int numFlying int32 mux sync.Mutex taskDone *sync.Cond @@ -115,20 +115,21 @@ func (c *ClickHouse) Send(batch *model.Batch) { }) } -func (c *ClickHouse) LoadOrStoreSeries(sid int64) (loaded bool) { +func (c *ClickHouse) AllowWriteSeries(sid int64) (allowed bool) { c.mux.Lock() - _, loaded = c.bmSeries[sid] + defer c.mux.Unlock() + if c.wrSeries >= wrSeriesQuota { + return + } + _, loaded := c.bmSeries[sid] if !loaded { c.bmSeries[sid] = struct{}{} + c.wrSeries++ + allowed = true } - c.mux.Unlock() return } -func (c *ClickHouse) IsWritingSeries() bool { - return c.wrSeries.Load() > 0 -} - func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err error) { var seriesRows model.Rows for _, row := range rows { @@ -140,11 +141,12 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err err if len(seriesRows) != 0 { begin := time.Now() var numBad int - c.wrSeries.Add(1) if numBad, err = writeRows(c.promSerSQL, seriesRows, c.IdxSerID, c.NumDims, conn); err != nil { return } - c.wrSeries.Add(-1) + c.mux.Lock() + c.wrSeries -= len(seriesRows) + c.mux.Unlock() if numBad != 0 { statistics.ParseMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(numBad)) } diff --git a/task/task.go b/task/task.go index a2d56f81..d1517535 100644 --- a/task/task.go +++ b/task/task.go @@ -440,7 +440,7 @@ func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) *row = append(*row, val) } *row = append(*row, seriesID) - newSeries := !service.clickhouse.IsWritingSeries() && !service.clickhouse.LoadOrStoreSeries(seriesID) + newSeries := service.clickhouse.AllowWriteSeries(seriesID) if newSeries { *row = append(*row, mgmtID, nil) for i := service.idxSerID + 3; i < service.numDims; i++ { From 56cb62e4313524779dc84eea730e35d7ad38b0c4 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 21 Sep 2022 22:50:27 +0800 Subject: [PATCH 280/404] fix systest --- docker-compose.yml | 2 +- go.test.sh | 4 ++-- output/clickhouse.go | 6 +++++- parser/fastjson.go | 7 ++++++- parser/gjson.go | 2 +- 5 files changed, 15 insertions(+), 6 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index 5f591103..675909d7 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -33,7 +33,7 @@ services: security_opt: - label:disable clickhouse: - image: clickhouse/clickhouse-server:22.3 + image: clickhouse/clickhouse-server:22.8 restart: always ports: - "8123:8123" diff --git a/go.test.sh b/go.test.sh index 475c9f3f..9fcdc43a 100755 --- a/go.test.sh +++ b/go.test.sh @@ -29,7 +29,7 @@ for i in `seq 1 10000`;do echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"price\" : $price }" done > a.json for i in `seq 10001 30000`;do - echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey01\" : $i }" + echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey00\" : false, \"newkey01\" : $i }" done >> a.json for i in `seq 30001 50000`;do echo "{\"time\" : \"${now}\", \"name\" : \"name$i\", \"value\" : $i, \"newkey02\" : $i.123, \"newkey03\" : \"name$i\", \"newkey04\" : \"${now}\", \"newkey05\" : {\"k1\": 1, \"k2\": 2} }" @@ -63,7 +63,7 @@ echo "Got test_auto_schema count => $count" schema=`curl "localhost:8123" -d 'DESC test_dynamic_schema' 2>/dev/null | grep newkey | sort | tr -d '\t' | tr '\n' ','` echo "Got test_dynamic_schema schema => $schema" -[ $schema = "newkey01Nullable(Int64),newkey02Nullable(Float64),newkey03Nullable(String),newkey04Nullable(DateTime64(3))," ] || exit 1 +[ $schema = "newkey00Nullable(Bool),newkey01Nullable(Int64),newkey02Nullable(Float64),newkey03Nullable(String),newkey04Nullable(DateTime64(3))," ] || exit 1 count=`curl "localhost:8123" -d 'SELECT count() FROM test_dynamic_schema'` echo "Got test_dynamic_schema count => $count" [ $count -eq 100000 ] || exit 1 diff --git a/output/clickhouse.go b/output/clickhouse.go index 930157af..1e424f7f 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -429,6 +429,8 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { intVal := value.(int) var strVal string switch intVal { + case model.Bool: + strVal = "Nullable(Bool)" case model.Int64: strVal = "Nullable(Int64)" case model.Float64: @@ -437,8 +439,10 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { strVal = "Nullable(String)" case model.DateTime: strVal = "Nullable(DateTime64(3))" + case model.Object: + strVal = model.GetTypeName(intVal) default: - err = errors.Newf("%s: BUG: unsupported column type %s", taskCfg.Name, strVal) + err = errors.Newf("%s: BUG: unsupported column type %s", taskCfg.Name, model.GetTypeName(intVal)) return false } if c.taskCfg.PrometheusSchema { diff --git a/parser/fastjson.go b/parser/fastjson.go index 6d8793cd..65a9d3b1 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -33,6 +33,7 @@ import ( ) var _ Parser = (*FastjsonParser)(nil) +var EmpytObject = make(map[string]interface{}) // FastjsonParser, parser for get data in json format type FastjsonParser struct { @@ -338,6 +339,10 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { func val2map(v *fastjson.Value) (m map[string]interface{}) { var err error var obj *fastjson.Object + m = EmpytObject + if v == nil { + return + } if obj, err = v.Object(); err != nil { return } @@ -428,7 +433,7 @@ func (c *FastjsonMetric) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, whit if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { if (white == nil || white.MatchString(strKey)) && (black == nil || !black.MatchString(strKey)) { - if typ, arr := fjDetectType(v, 0); typ != model.Unknown && !arr { + if typ, arr := fjDetectType(v, 0); typ != model.Unknown && typ != model.Object && !arr { newKeys.Store(strKey, typ) foundNew = true } else if _, loaded = warnKeys.LoadOrStore(strKey, nil); !loaded { diff --git a/parser/gjson.go b/parser/gjson.go index c23ee6bd..0d344edb 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -387,7 +387,7 @@ func (c *GjsonMetric) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { if (white == nil || white.MatchString(strKey)) && (black == nil || !black.MatchString(strKey)) { - if typ, array := gjDetectType(v, 0); typ != model.Unknown && !array { + if typ, array := gjDetectType(v, 0); typ != model.Unknown && typ != model.Object && !array { newKeys.Store(strKey, typ) foundNew = true } else if _, loaded = warnKeys.LoadOrStore(strKey, nil); !loaded { From ef2acee26e131592f216c8a2a26c2c842dad415c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Thu, 22 Sep 2022 10:55:45 +0800 Subject: [PATCH 281/404] bump sonic to v1.5.0 --- go.mod | 8 ++++---- go.sum | 22 ++++++++-------------- 2 files changed, 12 insertions(+), 18 deletions(-) diff --git a/go.mod b/go.mod index 062bc9e5..3727bb8f 100644 --- a/go.mod +++ b/go.mod @@ -6,7 +6,7 @@ require ( github.com/ClickHouse/clickhouse-go/v2 v2.3.0 github.com/RoaringBitmap/roaring v1.2.1 github.com/Shopify/sarama v1.36.0 - github.com/bytedance/sonic v1.3.5 + github.com/bytedance/sonic v1.5.0 github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 github.com/google/gops v0.3.25 @@ -61,7 +61,7 @@ require ( github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect github.com/klauspost/compress v1.15.9 // indirect - github.com/klauspost/cpuid/v2 v2.1.0 // indirect + github.com/klauspost/cpuid/v2 v2.1.1 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect @@ -87,11 +87,11 @@ require ( go.opentelemetry.io/otel/trace v1.9.0 // indirect go.uber.org/atomic v1.10.0 // indirect go.uber.org/multierr v1.8.0 // indirect - golang.org/x/arch v0.0.0-20220722155209-00200b7164a7 // indirect + golang.org/x/arch v0.0.0-20220919183040-2926576b28c0 // indirect golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8 // indirect golang.org/x/net v0.0.0-20220812174116-3211cb980234 // indirect golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4 // indirect - golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab // indirect + golang.org/x/sys v0.0.0-20220919091848-fb04ddd9f9c8 // indirect golang.org/x/text v0.3.7 // indirect google.golang.org/protobuf v1.28.1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect diff --git a/go.sum b/go.sum index 15cef232..4e5087c6 100644 --- a/go.sum +++ b/go.sum @@ -67,8 +67,8 @@ github.com/bits-and-blooms/bitset v1.3.0 h1:h7mv5q31cthBTd7V4kLAZaIThj1e8vPGcSqp github.com/bits-and-blooms/bitset v1.3.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= -github.com/bytedance/sonic v1.3.5 h1:xfBNhsG3QCC+AMCmCHxNQg0StI5IM/B9Jtwjqi5WlI0= -github.com/bytedance/sonic v1.3.5/go.mod h1:V973WhNhGmvHxW6nQmsHEfHaoU9F3zTF+93rH03hcUQ= +github.com/bytedance/sonic v1.5.0 h1:XWdTi8bwPgxIML+eNV1IwNuTROK6EUrQ65ey8yd6fRQ= +github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= @@ -123,8 +123,6 @@ github.com/go-logfmt/logfmt v0.5.1 h1:otpy5pqBCBZ1ng9RQ0dPu4PN7ba75Y/aA+UpowDyNV github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/goccy/go-json v0.9.4 h1:L8MLKG2mvVXiQu07qB6hmfqeSYQdOnqPot2GhsIwIaI= -github.com/goccy/go-json v0.9.4/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= @@ -246,8 +244,8 @@ github.com/klauspost/compress v1.15.7/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHU github.com/klauspost/compress v1.15.9 h1:wKRjX6JRtDdrE9qwa4b/Cip7ACOshUI4smpCQanqjSY= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= -github.com/klauspost/cpuid/v2 v2.1.0 h1:eyi1Ad2aNJMW95zcSbmGg7Cg6cq3ADwLpMAP96d8rF0= -github.com/klauspost/cpuid/v2 v2.1.0/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= +github.com/klauspost/cpuid/v2 v2.1.1 h1:t0wUqjowdm8ezddV5k0tLWVklVuvLJpoHeb4WBdydm0= +github.com/klauspost/cpuid/v2 v2.1.1/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= @@ -356,16 +354,12 @@ github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PK github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/thanos-io/thanos v0.27.0 h1:TWOPMkWX5Q1LHdROI6dg9lKU3x6Q95vO4o/bm8GaCfE= github.com/thanos-io/thanos v0.27.0/go.mod h1:i0KCpF5fWAWBNZXBkuzGh0603YlMrbhLJdwOO5W21YI= -github.com/tidwall/gjson v1.12.1/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= -github.com/tidwall/gjson v1.13.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/gjson v1.14.2 h1:6BBkirS0rAHjumnjHF6qgy5d2YAJ1TLIaFE2lzfOLqo= github.com/tidwall/gjson v1.14.2/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= -github.com/tidwall/sjson v1.2.4 h1:cuiLzLnaMeBhRmEv00Lpk3tkYrcxpmbU81tAY4Dw0tc= -github.com/tidwall/sjson v1.2.4/go.mod h1:098SZ494YoMWPmMO6ct4dcFnqxwj9r/gF0Etp19pSNM= github.com/tklauser/go-sysconf v0.3.10/go.mod h1:C8XykCvCb+Gn0oNCWPIlcb0RuglQTYaQ2hGm7jmxEFk= github.com/tklauser/numcpus v0.4.0/go.mod h1:1+UI3pD8NW14VMwdgJNJ1ESk2UnwhAnz5hMwiKKqXCQ= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= @@ -433,8 +427,8 @@ go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= go.uber.org/zap v1.22.0 h1:Zcye5DUgBloQ9BaT4qc9BnjOFog5TvBSAGkJ3Nf70c0= go.uber.org/zap v1.22.0/go.mod h1:H4siCOZOrAolnUPJEkfaSjDqyP+BDS0DdDWzwcgt3+U= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.0.0-20220722155209-00200b7164a7 h1:VBQqJMNMRfQsWSiCTLgz9XjAfWlgnJAPv8nsp1HF8Tw= -golang.org/x/arch v0.0.0-20220722155209-00200b7164a7/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.0.0-20220919183040-2926576b28c0 h1:Ibo6J679xStgTWXLeZWlCA0vTvCEFZkUWFWAmBX2Zpk= +golang.org/x/arch v0.0.0-20220919183040-2926576b28c0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -589,8 +583,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab h1:2QkjZIsXupsJbJIdSjjUOgWK3aEtzyuh2mPt3l/CkeU= -golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220919091848-fb04ddd9f9c8 h1:h+EGohizhe9XlX18rfpa8k8RAc5XyaeamM+0VHRd4lc= +golang.org/x/sys v0.0.0-20220919091848-fb04ddd9f9c8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From bdba334786c9df92a503298d1f76829b5c0a2811 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 8 Oct 2022 14:42:22 +0800 Subject: [PATCH 282/404] fixed fastjson dot characters in key --- config/config.go | 4 ++-- output/clickhouse.go | 4 ++-- output/clickhouse_util.go | 4 ++-- parser/parser_test.go | 9 +++++++-- util/common.go | 10 +++++++--- 5 files changed, 20 insertions(+), 11 deletions(-) diff --git a/config/config.go b/config/config.go index f2217a5a..5e24d543 100644 --- a/config/config.go +++ b/config/config.go @@ -259,7 +259,7 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { for i := range taskCfg.Dims { if taskCfg.Dims[i].SourceName == "" { - taskCfg.Dims[i].SourceName = util.GetSourceName(taskCfg.Dims[i].Name) + taskCfg.Dims[i].SourceName = util.GetSourceName(taskCfg.Parser, taskCfg.Dims[i].Name) } } @@ -327,7 +327,7 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { return } -//convert java client style configuration into sinker +// convert java client style configuration into sinker func (cfg *Config) convertKfkSecurity() { if protocol, ok := cfg.Kafka.Security["security.protocol"]; ok { if strings.Contains(protocol, "SASL") { diff --git a/output/clickhouse.go b/output/clickhouse.go index 1e424f7f..8c38c9f7 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -284,7 +284,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { {Name: "labels", Type: model.String}, } var seriesDims []*model.ColumnWithType - if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.seriesTbl, nil, conn); err != nil { + if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.seriesTbl, nil, c.taskCfg.Parser, conn); err != nil { if errors.Is(err, ErrTblNotExist) { err = errors.Wrapf(err, "Please create series table for %s.%s", c.cfg.Clickhouse.DB, c.taskCfg.TableName) return @@ -352,7 +352,7 @@ func (c *ClickHouse) initSchema() (err error) { return } if c.taskCfg.AutoSchema { - if c.Dims, err = getDims(c.cfg.Clickhouse.DB, c.taskCfg.TableName, c.taskCfg.ExcludeColumns, conn); err != nil { + if c.Dims, err = getDims(c.cfg.Clickhouse.DB, c.taskCfg.TableName, c.taskCfg.ExcludeColumns, c.taskCfg.Parser, conn); err != nil { return } } else { diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 22eb901c..14f4e02e 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -83,7 +83,7 @@ func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn cl return } -func getDims(database, table string, excludedColumns []string, conn clickhouse.Conn) (dims []*model.ColumnWithType, err error) { +func getDims(database, table string, excludedColumns []string, parser string, conn clickhouse.Conn) (dims []*model.ColumnWithType, err error) { var rs driver.Rows if rs, err = conn.Query(context.Background(), fmt.Sprintf(selectSQLTemplate, database, table)); err != nil { err = errors.Wrapf(err, "") @@ -101,7 +101,7 @@ func getDims(database, table string, excludedColumns []string, conn clickhouse.C typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") if !util.StringContains(excludedColumns, name) && defaultKind != "MATERIALIZED" { tp, nullable, array := model.WhichType(typ) - dims = append(dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, Array: array, SourceName: util.GetSourceName(name)}) + dims = append(dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, Array: array, SourceName: util.GetSourceName(parser, name)}) } } if len(dims) == 0 { diff --git a/parser/parser_test.go b/parser/parser_test.go index 1fc61f01..4eb9dd1e 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -42,6 +42,7 @@ var jsonSample = []byte(`{ "bool_false": false, "num_int": 123, "num_float": 123.321, + "app.kubernetes.io/name": "kube-state-metrics", "str": "escaped_\"ws", "str_int": "123", "str_float": "123.321", @@ -75,6 +76,7 @@ var jsonSchema = map[string]string{ "bool_false": "Bool", "num_int": "Int64", "num_float": "Float64", + "app.kubernetes.io/name": "String", "str": "String", "str_int": "String", "str_float": "String", @@ -102,7 +104,7 @@ var jsonSchema = map[string]string{ "array_obj": "Object('json')Array", } -var csvSample = []byte(`null,true,false,123,123.321,"escaped_""ws",123,123.321,2009-07-13,13/07/2009,2009-07-13T09:07:13Z,2009-07-13T09:07:13.123+08:00,2009-07-13 09:07:13,2009-07-13 09:07:13.123,"{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",[],[null],"[true,false]","[0,255,256,65535,65536,4294967295,4294967296,18446744073709551615,18446744073709551616]","[-9223372036854775808,-2147483649,-2147483648,-32769,-32768,-129,-128,0,127,128,32767,32768,2147483647,2147483648,9223372036854775807]","[4.940656458412465441765687928682213723651e-324,1.401298464324817070923729583289916131280e-45,0.0,3.40282346638528859811704183484516925440e+38,1.797693134862315708145274237317043567981e+308]","[""aa"",""bb"",""cc""]","[""0"",""255"",""256"",""65535"",""65536"",""4294967295"",""4294967296"",""18446744073709551615"",""18446744073709551616""]","[""-9223372036854775808"",""-2147483649"",""-2147483648"",""-32769"",""-32768"",""-129"",""-128"",""0"",""127"",""128"",""32767"",""32768"",""2147483647"",""2147483648"",""9223372036854775807""]","[""4.940656458412465441765687928682213723651e-324"",""1.401298464324817070923729583289916131280e-45"",""0.0"",""3.40282346638528859811704183484516925440e+38"",""1.797693134862315708145274237317043567981e+308""]","[""2009-07-13"",""2009-07-14"",""2009-07-15""]","[""13/07/2009"",""14/07/2009"",""15/07/2009""]","[""2009-07-13T09:07:13Z"",""2009-07-13T09:07:13+08:00"",""2009-07-13T09:07:13.123Z"",""2009-07-13T09:07:13.123+08:00""]","[""2009-07-13 09:07:13"",""2009-07-13 09:07:13.123""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]"`) +var csvSample = []byte(`null,true,false,123,123.321,kube-state-metrics,"escaped_""ws",123,123.321,2009-07-13,13/07/2009,2009-07-13T09:07:13Z,2009-07-13T09:07:13.123+08:00,2009-07-13 09:07:13,2009-07-13 09:07:13.123,"{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",[],[null],"[true,false]","[0,255,256,65535,65536,4294967295,4294967296,18446744073709551615,18446744073709551616]","[-9223372036854775808,-2147483649,-2147483648,-32769,-32768,-129,-128,0,127,128,32767,32768,2147483647,2147483648,9223372036854775807]","[4.940656458412465441765687928682213723651e-324,1.401298464324817070923729583289916131280e-45,0.0,3.40282346638528859811704183484516925440e+38,1.797693134862315708145274237317043567981e+308]","[""aa"",""bb"",""cc""]","[""0"",""255"",""256"",""65535"",""65536"",""4294967295"",""4294967296"",""18446744073709551615"",""18446744073709551616""]","[""-9223372036854775808"",""-2147483649"",""-2147483648"",""-32769"",""-32768"",""-129"",""-128"",""0"",""127"",""128"",""32767"",""32768"",""2147483647"",""2147483648"",""9223372036854775807""]","[""4.940656458412465441765687928682213723651e-324"",""1.401298464324817070923729583289916131280e-45"",""0.0"",""3.40282346638528859811704183484516925440e+38"",""1.797693134862315708145274237317043567981e+308""]","[""2009-07-13"",""2009-07-14"",""2009-07-15""]","[""13/07/2009"",""14/07/2009"",""15/07/2009""]","[""2009-07-13T09:07:13Z"",""2009-07-13T09:07:13+08:00"",""2009-07-13T09:07:13.123Z"",""2009-07-13T09:07:13.123+08:00""]","[""2009-07-13 09:07:13"",""2009-07-13 09:07:13.123""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]"`) var csvSchema = []string{ "null", @@ -110,6 +112,7 @@ var csvSchema = []string{ "bool_false", "num_int", "num_float", + "app.kubernetes.io/name", "str", "str_int", "str_float", @@ -221,7 +224,7 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { for j := range testCases { var v interface{} desc := fmt.Sprintf(`%s.%s("%s", %s)`, name, method, testCases[j].Field, strconv.FormatBool(testCases[j].Nullable)) - if name == "csv" && (sliceContains([]string{"GetBool", "GetInt64", "GetFloat64", "GetDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || testCases[j].Nullable) { + if (name == "csv" && (sliceContains([]string{"GetBool", "GetInt64", "GetFloat64", "GetDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || testCases[j].Nullable)) || (name == "gjson" && strings.Contains(testCases[j].Field, ".")) { skipped = append(skipped, desc) continue } @@ -368,6 +371,7 @@ func TestParserString(t *testing.T) { {"bool_false", false, "false"}, {"num_int", false, "123"}, {"num_float", false, "123.321"}, + {"app.kubernetes.io/name", false, `kube-state-metrics`}, {"str", false, `escaped_"ws`}, {"str_int", false, "123"}, {"str_float", false, "123.321"}, @@ -384,6 +388,7 @@ func TestParserString(t *testing.T) { {"bool_false", true, "false"}, {"num_int", true, "123"}, {"num_float", true, "123.321"}, + {"app.kubernetes.io/name", true, `kube-state-metrics`}, {"str", true, `escaped_"ws`}, {"str_int", true, "123"}, {"str_float", true, "123.321"}, diff --git a/util/common.go b/util/common.go index 5abcc881..e69cd38e 100644 --- a/util/common.go +++ b/util/common.go @@ -94,8 +94,12 @@ func StringContains(arr []string, str string) bool { } // GetSourceName returns the field name in message for the given ClickHouse column -func GetSourceName(name string) (sourcename string) { - sourcename = strings.Replace(name, ".", "\\.", -1) +func GetSourceName(parser, name string) (sourcename string) { + if parser == "gjson" { + sourcename = strings.Replace(name, ".", "\\.", -1) + } else { + sourcename = name + } return } @@ -107,7 +111,7 @@ func GetShift(s int) (shift uint) { } // GetOutboundIP get preferred outbound ip of this machine -//https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go +// https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go func GetOutboundIP() (ip net.IP, err error) { var conn net.Conn if conn, err = net.Dial("udp", "8.8.8.8:80"); err != nil { From 8f216db5c209d7207d0d9cbde04d47be5ed8fa09 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 9 Oct 2022 15:07:56 +0800 Subject: [PATCH 283/404] record mapping from series_id to mgmt_id, and allow write when the sireis_id is not found in mapping or mapped mgmt_id differs with the current one --- output/clickhouse.go | 20 ++++++++++---------- task/task.go | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 8c38c9f7..bf9bc414 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -66,7 +66,7 @@ type ClickHouse struct { distMetricTbls []string distSeriesTbls []string - bmSeries map[int64]struct{} + bmSeries map[int64]int64 wrSeries int numFlying int32 mux sync.Mutex @@ -115,15 +115,15 @@ func (c *ClickHouse) Send(batch *model.Batch) { }) } -func (c *ClickHouse) AllowWriteSeries(sid int64) (allowed bool) { +func (c *ClickHouse) AllowWriteSeries(sid, mid int64) (allowed bool) { c.mux.Lock() defer c.mux.Unlock() if c.wrSeries >= wrSeriesQuota { return } - _, loaded := c.bmSeries[sid] - if !loaded { - c.bmSeries[sid] = struct{}{} + mid2, loaded := c.bmSeries[sid] + if !loaded || mid != mid2 { + c.bmSeries[sid] = mid c.wrSeries++ allowed = true } @@ -226,26 +226,26 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { if c.cfg.Clickhouse.Cluster != "" { tbl = c.distSeriesTbls[0] } - c.bmSeries = make(map[int64]struct{}) + c.bmSeries = make(map[int64]int64) if !c.taskCfg.LoadSeriesAtStartup { util.Logger.Info(fmt.Sprintf("skipped loading series from %v", tbl), zap.String("task", c.taskCfg.Name)) return } - query := fmt.Sprintf("SELECT toInt64(__mgmt_id) AS mid FROM %s.%s ORDER BY mid", c.cfg.Clickhouse.DB, tbl) + query := fmt.Sprintf("SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s.%s ORDER BY sid", c.cfg.Clickhouse.DB, tbl) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs driver.Rows - var mgmtID int64 + var seriesID, mgmtID int64 if rs, err = conn.Query(context.Background(), query); err != nil { err = errors.Wrapf(err, "") return err } defer rs.Close() for rs.Next() { - if err = rs.Scan(&mgmtID); err != nil { + if err = rs.Scan(&seriesID, &mgmtID); err != nil { err = errors.Wrapf(err, "") return err } - c.bmSeries[mgmtID] = struct{}{} + c.bmSeries[seriesID] = mgmtID } util.Logger.Info(fmt.Sprintf("loaded %d series from %v", len(c.bmSeries), tbl), zap.String("task", c.taskCfg.Name)) return diff --git a/task/task.go b/task/task.go index d1517535..ef74560c 100644 --- a/task/task.go +++ b/task/task.go @@ -440,7 +440,7 @@ func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) *row = append(*row, val) } *row = append(*row, seriesID) - newSeries := service.clickhouse.AllowWriteSeries(seriesID) + newSeries := service.clickhouse.AllowWriteSeries(seriesID, mgmtID) if newSeries { *row = append(*row, mgmtID, nil) for i := service.idxSerID + 3; i < service.numDims; i++ { From a8951218a94ce17055eba560231115c003ab3244 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E4=BF=A1=E5=8D=9A?= Date: Tue, 25 Oct 2022 10:28:37 +0800 Subject: [PATCH 284/404] Sinker should not abort when facing zookeeper session expired issue --- output/clickhouse.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index bf9bc414..c62548de 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -47,7 +47,10 @@ var ( // src/Common/ErrorCodes.cpp // src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp // ZooKeeper issues(https://issues.apache.org/jira/browse/ZOOKEEPER-4410) can cause ClickHouse exeception: "Code": 999, "Message": "Cannot allocate block number..." - replicaSpecificErrorCodes = []int32{242, 319, 999, 1000} //TABLE_IS_READ_ONLY, UNKNOWN_STATUS_OF_INSERT, KEEPER_EXCEPTION, POCO_EXCEPTION + // CKServer too many parts possibly reason: https://github.com/ClickHouse/ClickHouse/issues/6720#issuecomment-526045768 + // zooKeeper Connection loss issue: https://cwiki.apache.org/confluence/display/ZOOKEEPER/FAQ#:~:text=How%20should%20I%20handle%20the%20CONNECTION_LOSS%20error%3F + // zooKeeper Session expired issue: https://cwiki.apache.org/confluence/display/ZOOKEEPER/FAQ#:~:text=How%20should%20I%20handle%20SESSION_EXPIRED%3F + replicaSpecificErrorCodes = []int32{225, 242, 252, 319, 999, 1000} //NO_ZOOKEEPER, TABLE_IS_READ_ONLY, TOO_MANY_PARTS, UNKNOWN_STATUS_OF_INSERT, KEEPER_EXCEPTION, POCO_EXCEPTION wrSeriesQuota = 16384 ) From 48706a57d207fa9d5d98e7d1323539f929c13055 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E4=BF=A1=E5=8D=9A?= Date: Wed, 9 Nov 2022 20:55:22 +0800 Subject: [PATCH 285/404] Fix sinker error 'invalid autocommit options specified when a group was not... --- cmd/clickhouse_sinker/main.go | 2 +- cmd/nacos_publish_config/main.go | 4 +-- config/config.go | 5 ++-- config_manager/lags.go | 1 + config_manager/nacos.go | 6 ++--- docker/test_auto_schema.hjson | 37 ++++++++++++++++++++++++++++ docker/test_auto_schema.json | 32 ------------------------ docker/test_dynamic_schema.hjson | 36 +++++++++++++++++++++++++++ docker/test_dynamic_schema.json | 36 --------------------------- docker/test_fixed_schema.hjson | 42 ++++++++++++++++++++++++++++++++ docker/test_fixed_schema.json | 42 -------------------------------- go.mod | 1 + go.sum | 2 ++ input/kafka_franz.go | 5 ++-- statistics/statistics.go | 14 +++++++++-- task/ring.go | 16 +++++++----- 16 files changed, 153 insertions(+), 128 deletions(-) create mode 100644 docker/test_auto_schema.hjson delete mode 100644 docker/test_auto_schema.json create mode 100644 docker/test_dynamic_schema.hjson delete mode 100644 docker/test_dynamic_schema.json create mode 100644 docker/test_fixed_schema.hjson delete mode 100644 docker/test_fixed_schema.json diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index e10d6b63..4ed773d4 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -85,7 +85,7 @@ func initCmdOptions() { HTTPPort: 0, PushGatewayAddrs: "", PushInterval: 10, - LocalCfgFile: "/etc/clickhouse_sinker.json", + LocalCfgFile: "/etc/clickhouse_sinker.hjson", NacosAddr: "127.0.0.1:8848", NacosNamespaceID: "", NacosGroup: "DEFAULT_GROUP", diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index e1986898..78f9eead 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -41,10 +41,10 @@ var ( `nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work!`) nacosGroup = flag.String("nacos-group", "DEFAULT_GROUP", `nacos group name. Empty string doesn't work!`) - nacosDataID = flag.String("nacos-dataid", "clickhouse_sinker.json", + nacosDataID = flag.String("nacos-dataid", "clickhouse_sinker.hjson", `nacos data id`) - localCfgFile = flag.String("local-cfg-file", "/etc/clickhouse_sinker.json", "local config file") + localCfgFile = flag.String("local-cfg-file", "/etc/clickhouse_sinker.hjson", "local config file") replicas = flag.Int("replicas", 1, "replicate each task to multiple ones with the same config except task name, consumer group and table name") maxOpenConns = flag.Int("max-open-conns", 0, "max open connections per shard") ) diff --git a/config/config.go b/config/config.go index 5e24d543..064bfead 100644 --- a/config/config.go +++ b/config/config.go @@ -16,11 +16,12 @@ limitations under the License. package config import ( - "encoding/json" "os" "regexp" "strings" + "github.com/hjson/hjson-go/v4" + "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" @@ -179,7 +180,7 @@ func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { err = errors.Wrapf(err, "") return } - if err = json.Unmarshal(b, cfg); err != nil { + if err = hjson.Unmarshal(b, cfg); err != nil { err = errors.Wrapf(err, "") return } diff --git a/config_manager/lags.go b/config_manager/lags.go index d298c84f..b0205b8a 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -69,6 +69,7 @@ func newClient(cfg *config.Config) (cl *kgo.Client, adm *kadm.Client, err error) if opts, err = input.GetFranzConfig(&cfg.Kafka); err != nil { return } + // franz.config.go 379 - invalid autocommit options specified when a group was not specified if cl, err = kgo.NewClient(opts...); err != nil { err = errors.Wrapf(err, "") return diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 038a042c..883b7443 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -2,7 +2,6 @@ package rcm import ( "context" - "encoding/json" "fmt" "math" "path/filepath" @@ -13,6 +12,7 @@ import ( "sync" "time" + "github.com/hjson/hjson-go/v4" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/util" "github.com/nacos-group/nacos-sdk-go/clients" @@ -137,7 +137,7 @@ func (ncm *NacosConfManager) GetConfig() (conf *config.Config, err error) { return } conf = &config.Config{} - if err = json.Unmarshal([]byte(content), conf); err != nil { + if err = hjson.Unmarshal([]byte(content), conf); err != nil { err = errors.Wrapf(err, "") return } @@ -146,7 +146,7 @@ func (ncm *NacosConfManager) GetConfig() (conf *config.Config, err error) { func (ncm *NacosConfManager) PublishConfig(conf *config.Config) (err error) { var bs []byte - if bs, err = json.Marshal(*conf); err != nil { + if bs, err = hjson.Marshal(*conf); err != nil { err = errors.Wrapf(err, "") return } diff --git a/docker/test_auto_schema.hjson b/docker/test_auto_schema.hjson new file mode 100644 index 00000000..b7d95346 --- /dev/null +++ b/docker/test_auto_schema.hjson @@ -0,0 +1,37 @@ +{ + clickhouse: { + hosts: [ + [ + 192.168.110.6 + # 192.168.110.8 + ] + [ + 192.168.110.10 + 192.168.110.12 + ] + [ + 192.168.110.14 + ] + ] + port: 19000 + db: default + username: "" + password: 123456 + retryTimes: 0 + } + kafka: { + brokers: 192.168.110.12:9092, 192.168.110.8:9092, 192.168.110.16:9092 + } + task: { + name: test_ck_query + topic: kafka_gen_ckdata22 + consumerGroup: test_ck_query + earliest: true + parser: json + autoSchema: true + tableName: test_ck_query_lowcardinality_r1 + excludeColumns: [] + bufferSize: 50000 + } + logLevel: info +} \ No newline at end of file diff --git a/docker/test_auto_schema.json b/docker/test_auto_schema.json deleted file mode 100644 index 3ee0e3a0..00000000 --- a/docker/test_auto_schema.json +++ /dev/null @@ -1,32 +0,0 @@ -{ - "clickhouse": { - "hosts": [ - [ - "127.0.0.1" - ] - ], - "port": 9000, - "db": "default", - "username": "", - "password": "", - "retryTimes": 0 - }, - "kafka": { - "brokers": "127.0.0.1:9093", - "version": "2.5.0" - }, - "task": { - "name": "test_auto_schema", - "topic": "topic1", - "consumerGroup": "test_auto_schema", - "earliest": true, - "parser": "json", - "autoSchema": true, - "tableName": "test_auto_schema", - "excludeColumns": [ - "day" - ], - "bufferSize": 50000 - }, - "logLevel": "info" -} \ No newline at end of file diff --git a/docker/test_dynamic_schema.hjson b/docker/test_dynamic_schema.hjson new file mode 100644 index 00000000..f10b0386 --- /dev/null +++ b/docker/test_dynamic_schema.hjson @@ -0,0 +1,36 @@ +{ + clickhouse: { + cluster: "" + hosts: [ + [ + 127.0.0.1 + ] + ] + port: 9000 + db: default + username: "" + password: "" + retryTimes: 0 + } + kafka: { + brokers: 127.0.0.1:9093 + version: 2.6.0 + } + task: { + name: test_dynamic_schema + topic: topic1 + consumerGroup: test_dynamic_schema + earliest: true + parser: json + autoSchema: true + tableName: test_dynamic_schema + excludeColumns: [ + day + ] + dynamicSchema: { + enable: true + } + bufferSize: 50000 + } + logLevel: info +} \ No newline at end of file diff --git a/docker/test_dynamic_schema.json b/docker/test_dynamic_schema.json deleted file mode 100644 index 4758a3a2..00000000 --- a/docker/test_dynamic_schema.json +++ /dev/null @@ -1,36 +0,0 @@ -{ - "clickhouse": { - "cluster": "", - "hosts": [ - [ - "127.0.0.1" - ] - ], - "port": 9000, - "db": "default", - "username": "", - "password": "", - "retryTimes": 0 - }, - "kafka": { - "brokers": "127.0.0.1:9093", - "version": "2.6.0" - }, - "task": { - "name": "test_dynamic_schema", - "topic": "topic1", - "consumerGroup": "test_dynamic_schema", - "earliest": true, - "parser": "json", - "autoSchema": true, - "tableName": "test_dynamic_schema", - "excludeColumns": [ - "day" - ], - "dynamicSchema": { - "enable": true - }, - "bufferSize": 50000 - }, - "logLevel": "info" -} \ No newline at end of file diff --git a/docker/test_fixed_schema.hjson b/docker/test_fixed_schema.hjson new file mode 100644 index 00000000..9cb0a5ab --- /dev/null +++ b/docker/test_fixed_schema.hjson @@ -0,0 +1,42 @@ +{ + clickhouse: { + hosts: [ + [ + 127.0.0.1 + ] + ] + port: 9000 + db: default + username: "" + password: "" + retryTimes: 0 + } + kafka: { + brokers: 127.0.0.1:9093 + version: 2.5.0 + } + task: { + name: test_fixed_schema + topic: topic1 + consumerGroup: test_fixed_schema + earliest: true + parser: json + tableName: test_fixed_schema + dims: [ + { + name: time + type: DateTime + } + { + name: name + type: String + } + { + name: value + type: Float32 + } + ] + bufferSize: 50000 + } + logLevel: info +} \ No newline at end of file diff --git a/docker/test_fixed_schema.json b/docker/test_fixed_schema.json deleted file mode 100644 index c18e2c02..00000000 --- a/docker/test_fixed_schema.json +++ /dev/null @@ -1,42 +0,0 @@ -{ - "clickhouse": { - "hosts": [ - [ - "127.0.0.1" - ] - ], - "port": 9000, - "db": "default", - "username": "", - "password": "", - "retryTimes": 0 - }, - "kafka": { - "brokers": "127.0.0.1:9093", - "version": "2.5.0" - }, - "task": { - "name": "test_fixed_schema", - "topic": "topic1", - "consumerGroup": "test_fixed_schema", - "earliest": true, - "parser": "json", - "tableName": "test_fixed_schema", - "dims": [ - { - "name": "time", - "type": "DateTime" - }, - { - "name": "name", - "type": "String" - }, - { - "name": "value", - "type": "Float32" - } - ], - "bufferSize": 50000 - }, - "logLevel": "info" -} \ No newline at end of file diff --git a/go.mod b/go.mod index 3727bb8f..68caadcb 100644 --- a/go.mod +++ b/go.mod @@ -10,6 +10,7 @@ require ( github.com/cespare/xxhash/v2 v2.1.2 github.com/fagongzi/goetty v1.7.0 github.com/google/gops v0.3.25 + github.com/hjson/hjson-go/v4 v4.2.0 github.com/jcmturner/gokrb5/v8 v8.4.3 github.com/jinzhu/copier v0.3.5 github.com/nacos-group/nacos-sdk-go v1.1.2 diff --git a/go.sum b/go.sum index 4e5087c6..b321b264 100644 --- a/go.sum +++ b/go.sum @@ -202,6 +202,8 @@ github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/C github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hjson/hjson-go/v4 v4.2.0 h1:GBa/BfCg/68J0dB/ztAYJtVecXpalG4nZkY4UusGZXQ= +github.com/hjson/hjson-go/v4 v4.2.0/go.mod h1:KaYt3bTw3zhBjYqnXkYywcYctk0A2nxeEFTse3rH13E= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 0eb11dc0..4f3d5b4d 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -18,6 +18,7 @@ package input import ( "context" "crypto/tls" + "strconv" "strings" "sync" "time" @@ -80,6 +81,7 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn opts = append(opts, kgo.ConsumeTopics(taskCfg.Topic), kgo.ConsumerGroup(taskCfg.ConsumerGroup), + kgo.DisableAutoCommit(), kgo.OnPartitionsRevoked(k.onPartitionRevoked)) if !taskCfg.Earliest { opts = append(opts, kgo.ConsumeResetOffset(kgo.NewOffset().AtEnd())) @@ -95,8 +97,6 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { opts = []kgo.Opt{ kgo.SeedBrokers(strings.Split(kfkCfg.Brokers, ",")...), - kgo.DisableAutoCommit(), - kgo.MaxConcurrentFetches(3), kgo.FetchMaxBytes(1 << 27), //134 MB kgo.BrokerMaxReadBytes(1 << 27), //134 MB //kgo.MetadataMaxAge(...) corresponds to sarama.Config.Metadata.RefreshFrequency @@ -173,6 +173,7 @@ func (k *KafkaFranz) Run() { err = errors.Wrapf(err, "") util.Logger.Info("kgo.Client.PollFetchs() got an error", zap.Error(err)) } + util.Logger.Debug("Records fetched", zap.String("task", k.taskCfg.Name), zap.String("records", strconv.Itoa(fetches.NumRecords()))) fetches.EachRecord(func(rec *kgo.Record) { msg := &model.InputMessage{ Topic: rec.Topic, diff --git a/statistics/statistics.go b/statistics/statistics.go index c1853db8..f6da2be7 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -1,10 +1,11 @@ -/*Copyright [2019] housepower +/* +Copyright [2019] housepower Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, @@ -109,6 +110,13 @@ var ( }, []string{"task", "topic", "partition"}, ) + ParsedRingMsgs = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "parsed_ring_msgs", + Help: "num of parsed msgs in ring", + }, + []string{"task"}, + ) RingMsgs = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Name: prefix + "ring_msgs", @@ -159,6 +167,7 @@ func init() { prometheus.MustRegister(FlushMsgsTotal) prometheus.MustRegister(FlushMsgsErrorTotal) prometheus.MustRegister(ConsumeOffsets) + prometheus.MustRegister(ParsedRingMsgs) prometheus.MustRegister(RingMsgs) prometheus.MustRegister(ShardMsgs) prometheus.MustRegister(ParsingPoolBacklog) @@ -255,6 +264,7 @@ func (p *Pusher) reconnect() { Collector(FlushMsgsErrorTotal). Collector(ConsumeOffsets). Collector(RingMsgs). + Collector(ParsedRingMsgs). Collector(ShardMsgs). Collector(ParsingPoolBacklog). Collector(WritingPoolBacklog). diff --git a/task/ring.go b/task/ring.go index 9a4d75d0..d414cfee 100644 --- a/task/ring.go +++ b/task/ring.go @@ -23,7 +23,7 @@ type Ring struct { ringCapMask int64 ringGroundOff int64 //min message offset inside the ring ringCeilingOff int64 //1 + max message offset inside the ring - ringFilledOffset int64 //every message which's offset inside range [ringGroundOff, ringFilledOffset) is in the ring + ringFilledOffset int64 //messages with offset inside range [ringGroundOff, ringFilledOffset) is a consecutive slice of ring, which should be ready to next stage batchSizeShift uint //the shift of desired batch size tid goetty.Timeout idleCnt int @@ -41,7 +41,7 @@ func (ring *Ring) QuitIdle() { ring.isIdle = false ring.ringBuf = make([]model.MsgRow, ring.ringCap) util.Logger.Info(fmt.Sprintf("topic %s partition %d quit idle", ring.service.taskCfg.Topic, ring.partition), zap.String("task", ring.service.taskCfg.Name)) - ring.scheduleForchBatchOrShard() + ring.scheduleForceBatchOrShard() } } @@ -67,6 +67,7 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { ring.ringCeilingOff = msgOffset + 1 } + statistics.ParsedRingMsgs.WithLabelValues(ring.service.taskCfg.Name).Inc() pMsgRow.Row = msgRow.Row if ring.service.sharder != nil && msgRow.Row != &model.FakedRow { if msgRow.Shard, err = ring.service.sharder.Calc(msgRow.Row); err != nil { @@ -74,11 +75,13 @@ func (ring *Ring) PutElem(msgRow model.MsgRow) { } pMsgRow.Shard = msgRow.Shard } + // the consecutive ring slice may grow because of the newly inserted row for ; ring.ringFilledOffset < ring.ringCeilingOff && ring.ringBuf[ring.ringFilledOffset&(ring.ringCapMask)].Row != nil; ring.ringFilledOffset++ { } + // generate a batch to send or flush the messages to shards when slice size reached the shifted batch size if (ring.ringFilledOffset >> ring.batchSizeShift) != (ring.ringGroundOff >> ring.batchSizeShift) { ring.genBatchOrShard() - ring.scheduleForchBatchOrShard() + ring.scheduleForceBatchOrShard() } } @@ -147,13 +150,13 @@ func (ring *Ring) ForceBatchOrShard(_ interface{}) { return } } - ring.scheduleForchBatchOrShard() + ring.scheduleForceBatchOrShard() } } -// schedule ForchBatchOrShard +// schedule ForceBatchOrShard // assume ring.mux is locked -func (ring *Ring) scheduleForchBatchOrShard() { +func (ring *Ring) scheduleForceBatchOrShard() { var err error ring.tid.Stop() if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(ring.service.taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { @@ -216,6 +219,7 @@ func (ring *Ring) genBatchOrShard() { } } statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) + statistics.ParsedRingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) ring.ringGroundOff = endOff //util.Logger.Debug(fmt.Sprintf("genBatchOrShard changed ring %p ringGroundOff to %d", ring, ring.ringGroundOff)) if ring.ringFilledOffset < ring.ringGroundOff { From c1629d3a990ebd348595af770134088e6e61383a Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Tue, 25 Oct 2022 09:52:00 +0800 Subject: [PATCH 286/404] Add comments for the change of replicaSpecificErrorCodes From 496d68ff301071ec35c4bcb68b2a23b9fc1d978c Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 3 Nov 2022 15:01:55 +0800 Subject: [PATCH 287/404] Add a new metric parsed_ring_msgs --- util/common.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/common.go b/util/common.go index e69cd38e..9124b6ac 100644 --- a/util/common.go +++ b/util/common.go @@ -105,7 +105,7 @@ func GetSourceName(parser, name string) (sourcename string) { // GetShift returns the smallest `shift` which 1< Date: Wed, 9 Nov 2022 19:10:16 +0800 Subject: [PATCH 288/404] Correct the GetShift call --- util/common.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/common.go b/util/common.go index 9124b6ac..e69cd38e 100644 --- a/util/common.go +++ b/util/common.go @@ -105,7 +105,7 @@ func GetSourceName(parser, name string) (sourcename string) { // GetShift returns the smallest `shift` which 1< Date: Wed, 30 Nov 2022 12:21:14 +0800 Subject: [PATCH 289/404] Fix the Create table error 'table already exists' --- output/clickhouse_util.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 14f4e02e..82d38351 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -115,7 +115,7 @@ func recreateDistTbls(cluster, database, table string, distTbls []string, conn c var queries []string for _, distTbl := range distTbls { queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS `%s`.`%s` ON CLUSTER `%s` SYNC", database, distTbl, cluster)) - queries = append(queries, fmt.Sprintf("CREATE TABLE `%s`.`%s` ON CLUSTER `%s` AS `%s`.`%s` ENGINE = Distributed(`%s`, `%s`, `%s`);", + queries = append(queries, fmt.Sprintf("CREATE TABLE IF NOT EXISTS `%s`.`%s` ON CLUSTER `%s` AS `%s`.`%s` ENGINE = Distributed(`%s`, `%s`, `%s`);", database, distTbl, cluster, database, table, cluster, database, table)) } From 1180257dbac701e019cd776b022c05c3560b5a73 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Wed, 30 Nov 2022 12:32:28 +0800 Subject: [PATCH 290/404] Fix the 'segmentation violation' in ch-go --- go.mod | 18 +++++++++--------- go.sum | 35 ++++++++++++++++++++--------------- output/clickhouse_util.go | 9 ++------- 3 files changed, 31 insertions(+), 31 deletions(-) diff --git a/go.mod b/go.mod index 68caadcb..09d4b114 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module github.com/housepower/clickhouse_sinker go 1.19 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.3.0 + github.com/ClickHouse/clickhouse-go/v2 v2.4.3 github.com/RoaringBitmap/roaring v1.2.1 github.com/Shopify/sarama v1.36.0 github.com/bytedance/sonic v1.5.0 @@ -18,7 +18,7 @@ require ( github.com/prometheus/common v0.37.0 github.com/segmentio/kafka-go v0.4.34 github.com/shopspring/decimal v1.3.1 - github.com/stretchr/testify v1.8.0 + github.com/stretchr/testify v1.8.1 github.com/thanos-io/thanos v0.27.0 github.com/tidwall/gjson v1.14.2 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 @@ -28,14 +28,14 @@ require ( github.com/twmb/franz-go/plugin/kzap v1.1.1 github.com/valyala/fastjson v1.6.3 github.com/xdg-go/scram v1.1.1 - go.uber.org/zap v1.22.0 + go.uber.org/zap v1.23.0 golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e golang.org/x/time v0.0.0-20220722155302-e5dcc9cfc0b9 gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) require ( - github.com/ClickHouse/ch-go v0.47.3 // indirect + github.com/ClickHouse/ch-go v0.50.0 // indirect github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723 // indirect github.com/andybalholm/brotli v1.0.4 // indirect github.com/beorn7/perks v1.0.1 // indirect @@ -61,14 +61,14 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.15.9 // indirect + github.com/klauspost/compress v1.15.12 // indirect github.com/klauspost/cpuid/v2 v2.1.1 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/paulmach/orb v0.7.1 // indirect - github.com/pierrec/lz4/v4 v4.1.15 // indirect + github.com/pierrec/lz4/v4 v4.1.17 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect @@ -84,14 +84,14 @@ require ( github.com/xdg-go/stringprep v1.0.3 // indirect github.com/xdg/scram v1.0.5 // indirect github.com/xdg/stringprep v1.0.3 // indirect - go.opentelemetry.io/otel v1.9.0 // indirect - go.opentelemetry.io/otel/trace v1.9.0 // indirect + go.opentelemetry.io/otel v1.11.1 // indirect + go.opentelemetry.io/otel/trace v1.11.1 // indirect go.uber.org/atomic v1.10.0 // indirect go.uber.org/multierr v1.8.0 // indirect golang.org/x/arch v0.0.0-20220919183040-2926576b28c0 // indirect golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8 // indirect golang.org/x/net v0.0.0-20220812174116-3211cb980234 // indirect - golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4 // indirect + golang.org/x/sync v0.0.0-20220923202941-7f9b1623fab7 // indirect golang.org/x/sys v0.0.0-20220919091848-fb04ddd9f9c8 // indirect golang.org/x/text v0.3.7 // indirect google.golang.org/protobuf v1.28.1 // indirect diff --git a/go.sum b/go.sum index b321b264..99ee04ea 100644 --- a/go.sum +++ b/go.sum @@ -34,10 +34,10 @@ dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7 github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/ClickHouse/ch-go v0.47.3 h1:bBKid8DRELKRf4/oXqrEks7Cc4DLb5Giwm9uazM6h3M= -github.com/ClickHouse/ch-go v0.47.3/go.mod h1:m3LHc5FeQ1Jjee5EEay5e7hQmSk4SuKyMfifNUz8l3g= -github.com/ClickHouse/clickhouse-go/v2 v2.3.0 h1:v0iT0yZspjjNgnLyPUa0WoGMme0Y/sNjCtOAFcyBkkA= -github.com/ClickHouse/clickhouse-go/v2 v2.3.0/go.mod h1:f2kb1LPopJdIyt0Y0vxNk9aiQCyhCmeVcyvOOaPCT4Q= +github.com/ClickHouse/ch-go v0.50.0 h1:f5Bv6WEVIcvhT9ez5+uqGZiggVTxAJ2+tgk1mYQCgAs= +github.com/ClickHouse/ch-go v0.50.0/go.mod h1:lCZ+zUH/OCr16xF4PIg8Q5SfaUVDQLU3b21FMXU+qlQ= +github.com/ClickHouse/clickhouse-go/v2 v2.4.3 h1:DROLjHmUPygvyelxOswE5+Yyul84gRKjIaOLq3B5l1w= +github.com/ClickHouse/clickhouse-go/v2 v2.4.3/go.mod h1:Q95k4+cA11bh5eqQvoxlNG6NCGi0rIAEDbroVIzqng8= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.2.1 h1:58/LJlg/81wfEHd5L9qsHduznOIhyv4qb1yWcSvVq9A= @@ -170,7 +170,7 @@ github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= -github.com/google/go-cmp v0.5.8 h1:e6P7q2lk1O+qJJb4BtCQXlK8vWEO8V1ZeuEdJNOqZyg= +github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gops v0.3.25 h1:Pf6uw+cO6pDhc7HJ71NiG0x8dyQTeQcmg3HQFF39qVw= github.com/google/gops v0.3.25/go.mod h1:8A7ebAm0id9K3H0uOggeRVGxszSvnlURun9mg3GdYDw= @@ -243,8 +243,9 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.4/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.7/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.15.9 h1:wKRjX6JRtDdrE9qwa4b/Cip7ACOshUI4smpCQanqjSY= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= +github.com/klauspost/compress v1.15.12 h1:YClS/PImqYbn+UILDnqxQCZ3RehC9N318SU3kElDUEM= +github.com/klauspost/compress v1.15.12/go.mod h1:QPwzmACJjUTFsnSHH934V6woptycfrDDJnH7hvFVbGM= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.1.1 h1:t0wUqjowdm8ezddV5k0tLWVklVuvLJpoHeb4WBdydm0= github.com/klauspost/cpuid/v2 v2.1.1/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= @@ -280,8 +281,9 @@ github.com/paulmach/orb v0.7.1 h1:Zha++Z5OX/l168sqHK3k4z18LDvr+YAO/VjK0ReQ9rU= github.com/paulmach/orb v0.7.1/go.mod h1:FWRlTgl88VI1RBx/MkrwWDRhQ96ctqMCh8boXhmqB/A= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= github.com/pierrec/lz4/v4 v4.1.14/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= -github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pierrec/lz4/v4 v4.1.17 h1:kV4Ip+/hUBC+8T6+2EgburRtkE9ef4nbY3f4dFhGjMc= +github.com/pierrec/lz4/v4 v4.1.17/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -346,14 +348,16 @@ github.com/streadway/amqp v1.0.0/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1Sd github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/thanos-io/thanos v0.27.0 h1:TWOPMkWX5Q1LHdROI6dg9lKU3x6Q95vO4o/bm8GaCfE= github.com/thanos-io/thanos v0.27.0/go.mod h1:i0KCpF5fWAWBNZXBkuzGh0603YlMrbhLJdwOO5W21YI= github.com/tidwall/gjson v1.14.2 h1:6BBkirS0rAHjumnjHF6qgy5d2YAJ1TLIaFE2lzfOLqo= @@ -408,10 +412,10 @@ go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opentelemetry.io/otel v1.9.0 h1:8WZNQFIB2a71LnANS9JeyidJKKGOOremcUtb/OtHISw= -go.opentelemetry.io/otel v1.9.0/go.mod h1:np4EoPGzoPs3O67xUVNoPPcmSvsfOxNlNA4F4AC+0Eo= -go.opentelemetry.io/otel/trace v1.9.0 h1:oZaCNJUjWcg60VXWee8lJKlqhPbXAPB51URuR47pQYc= -go.opentelemetry.io/otel/trace v1.9.0/go.mod h1:2737Q0MuG8q1uILYm2YYVkAyLtOofiTNGg6VODnOiPo= +go.opentelemetry.io/otel v1.11.1 h1:4WLLAmcfkmDk2ukNXJyq3/kiz/3UzCaYq6PskJsaou4= +go.opentelemetry.io/otel v1.11.1/go.mod h1:1nNhXBbWSD0nsL38H6btgnFN2k4i0sNLHNNMZMSbUGE= +go.opentelemetry.io/otel/trace v1.11.1 h1:ofxdnzsNrGBYXbP7t7zpUK281+go5rF7dvdIZXF8gdQ= +go.opentelemetry.io/otel/trace v1.11.1/go.mod h1:f/Q9G7vzk5u91PhbmKbg1Qn0rzH1LJ4vbPHFGkTPtOk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -426,8 +430,8 @@ go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95a go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= -go.uber.org/zap v1.22.0 h1:Zcye5DUgBloQ9BaT4qc9BnjOFog5TvBSAGkJ3Nf70c0= -go.uber.org/zap v1.22.0/go.mod h1:H4siCOZOrAolnUPJEkfaSjDqyP+BDS0DdDWzwcgt3+U= +go.uber.org/zap v1.23.0 h1:OjGQ5KQDEUawVHxNwQgPpiypGHOxo2mNZsOqTak4fFY= +go.uber.org/zap v1.23.0/go.mod h1:D+nX8jyLsMHMYrln8A0rJjFt/T/9/bGgIhAqxv5URuY= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/arch v0.0.0-20220919183040-2926576b28c0 h1:Ibo6J679xStgTWXLeZWlCA0vTvCEFZkUWFWAmBX2Zpk= golang.org/x/arch v0.0.0-20220919183040-2926576b28c0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= @@ -533,8 +537,9 @@ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4 h1:uVc8UZUe6tr40fFVnUP5Oj+veunVezqYl9z7DYw9xzw= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220923202941-7f9b1623fab7 h1:ZrnxWX62AgTKOSagEqxvb3ffipvEDX2pl7E1TdqLqIc= +golang.org/x/sync v0.0.0-20220923202941-7f9b1623fab7/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 82d38351..a372c117 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -37,9 +37,6 @@ func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn cl err = errors.Wrapf(err, "clickhouse.Conn.PrepareBatch %s", prepareSQL) return } - defer func() { - _ = batch.Abort() - }() var bmBad *roaring.Bitmap for i, row := range rows { if err = batch.Append((*row)[idxBegin:idxEnd]...); err != nil { @@ -59,25 +56,23 @@ func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn cl err = errors.Wrapf(err, "clickhouse.Conn.PrepareBatch %s", prepareSQL) return } - defer func() { - _ = batch.Abort() - }() for i, row := range rows { if !bmBad.ContainsInt(i) { if err = batch.Append((*row)[idxBegin:idxEnd]...); err != nil { - err = errors.Wrapf(err, "stmt.Exec") break } } } if err = batch.Send(); err != nil { err = errors.Wrapf(err, "driver.Batch.Send") + batch.Abort() return } return } if err = batch.Send(); err != nil { err = errors.Wrapf(err, "driver.Batch.Send") + batch.Abort() return } return From 6bc3dc0551eb79fb771e3a790dcf1a2b59db1650 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 5 Dec 2022 10:26:45 +0800 Subject: [PATCH 291/404] added stats on AllowWriteSeries --- output/clickhouse.go | 4 ++++ statistics/statistics.go | 21 +++++++++++++++++++++ 2 files changed, 25 insertions(+) diff --git a/output/clickhouse.go b/output/clickhouse.go index c62548de..9c1d739b 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -122,6 +122,7 @@ func (c *ClickHouse) AllowWriteSeries(sid, mid int64) (allowed bool) { c.mux.Lock() defer c.mux.Unlock() if c.wrSeries >= wrSeriesQuota { + statistics.WriteSeriesDropQuota.WithLabelValues(c.taskCfg.Name).Inc() return } mid2, loaded := c.bmSeries[sid] @@ -129,6 +130,9 @@ func (c *ClickHouse) AllowWriteSeries(sid, mid int64) (allowed bool) { c.bmSeries[sid] = mid c.wrSeries++ allowed = true + statistics.WriteSeriesAllowed.WithLabelValues(c.taskCfg.Name).Inc() + } else { + statistics.WriteSeriesDropUnchanged.WithLabelValues(c.taskCfg.Name).Inc() } return } diff --git a/statistics/statistics.go b/statistics/statistics.go index f6da2be7..0c19018b 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -153,6 +153,27 @@ var ( }, []string{"task", "table"}, ) + WriteSeriesDropQuota = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: prefix + "write_series_drop_quota", + Help: "num of disallowed write_series due to quota", + }, + []string{"task"}, + ) + WriteSeriesDropUnchanged = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: prefix + "write_series_drop_unchanged", + Help: "num of disallowed write_series due to unchanged", + }, + []string{"task"}, + ) + WriteSeriesAllowed = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: prefix + "write_series_allow", + Help: "num of allowed write_series", + }, + []string{"task"}, + ) ) func init() { From e67315825cf606066314f72555970ed86c5ba394 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 5 Dec 2022 11:44:00 +0800 Subject: [PATCH 292/404] register stats --- statistics/statistics.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/statistics/statistics.go b/statistics/statistics.go index 0c19018b..73ee7773 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -194,6 +194,9 @@ func init() { prometheus.MustRegister(ParsingPoolBacklog) prometheus.MustRegister(WritingPoolBacklog) prometheus.MustRegister(WritingDurations) + prometheus.MustRegister(WriteSeriesDropQuota) + prometheus.MustRegister(WriteSeriesDropUnchanged) + prometheus.MustRegister(WriteSeriesAllowed) prometheus.MustRegister(collectors.NewBuildInfoCollector()) } From d19f50ef022020f94f20316395bbdc3a6accf112 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 5 Dec 2022 14:39:03 +0800 Subject: [PATCH 293/404] writeSeries log --- output/clickhouse.go | 4 ++++ statistics/statistics.go | 8 ++++++++ 2 files changed, 12 insertions(+) diff --git a/output/clickhouse.go b/output/clickhouse.go index 9c1d739b..1f01ce0e 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -151,9 +151,13 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err err if numBad, err = writeRows(c.promSerSQL, seriesRows, c.IdxSerID, c.NumDims, conn); err != nil { return } + var curWrSeries int c.mux.Lock() c.wrSeries -= len(seriesRows) + curWrSeries = c.wrSeries c.mux.Unlock() + util.Logger.Info("ClickHouse.writeSeries succeeded", zap.Int("series", len(seriesRows)), zap.Int("c.wrSeries", curWrSeries), zap.String("task", c.taskCfg.Name)) + statistics.WriteSeriesSucceed.WithLabelValues(c.taskCfg.Name).Add(float64(len(seriesRows))) if numBad != 0 { statistics.ParseMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(numBad)) } diff --git a/statistics/statistics.go b/statistics/statistics.go index 73ee7773..d06afc4c 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -174,6 +174,13 @@ var ( }, []string{"task"}, ) + WriteSeriesSucceed = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: prefix + "write_series_succeed", + Help: "num of series handled by writeSeries", + }, + []string{"task"}, + ) ) func init() { @@ -197,6 +204,7 @@ func init() { prometheus.MustRegister(WriteSeriesDropQuota) prometheus.MustRegister(WriteSeriesDropUnchanged) prometheus.MustRegister(WriteSeriesAllowed) + prometheus.MustRegister(WriteSeriesSucceed) prometheus.MustRegister(collectors.NewBuildInfoCollector()) } From 3ad40da01aea3833aa99a5e45debefd674f94df9 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 5 Dec 2022 15:43:53 +0800 Subject: [PATCH 294/404] comment --- task/task.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/task/task.go b/task/task.go index ef74560c..da623743 100644 --- a/task/task.go +++ b/task/task.go @@ -439,10 +439,10 @@ func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) val := model.GetValueByType(metric, dim) *row = append(*row, val) } - *row = append(*row, seriesID) + *row = append(*row, seriesID) // __series_id newSeries := service.clickhouse.AllowWriteSeries(seriesID, mgmtID) if newSeries { - *row = append(*row, mgmtID, nil) + *row = append(*row, mgmtID, nil) // __mgmt_id, labels for i := service.idxSerID + 3; i < service.numDims; i++ { dim := service.dims[i] val := model.GetValueByType(metric, dim) From 28b628830d2fc3afa3a0936caa943d76f86a019c Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 5 Dec 2022 15:54:48 +0800 Subject: [PATCH 295/404] reset wrSeries timely --- output/clickhouse.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/output/clickhouse.go b/output/clickhouse.go index 1f01ce0e..ee474e53 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -85,6 +85,18 @@ func NewClickHouse(cfg *config.Config, taskCfg *config.TaskConfig) *ClickHouse { // Init the clickhouse intance func (c *ClickHouse) Init() (err error) { + go func() { + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() + for { + select { + case <-ticker.C: + c.mux.Lock() + c.wrSeries = 0 + c.mux.Unlock() + } + } + }() return c.initSchema() } @@ -154,6 +166,9 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err err var curWrSeries int c.mux.Lock() c.wrSeries -= len(seriesRows) + if c.wrSeries < 0 { + c.wrSeries = 0 + } curWrSeries = c.wrSeries c.mux.Unlock() util.Logger.Info("ClickHouse.writeSeries succeeded", zap.Int("series", len(seriesRows)), zap.Int("c.wrSeries", curWrSeries), zap.String("task", c.taskCfg.Name)) From 266fd14cda57f42bd0a88b6f64f3f8f2c3bf1454 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 7 Dec 2022 10:39:49 +0800 Subject: [PATCH 296/404] always allow writing new series --- output/clickhouse.go | 33 +++++++++++++++++++-------------- statistics/statistics.go | 25 +++++++++++++++++-------- 2 files changed, 36 insertions(+), 22 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index ee474e53..9afb7e03 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -133,16 +133,18 @@ func (c *ClickHouse) Send(batch *model.Batch) { func (c *ClickHouse) AllowWriteSeries(sid, mid int64) (allowed bool) { c.mux.Lock() defer c.mux.Unlock() - if c.wrSeries >= wrSeriesQuota { - statistics.WriteSeriesDropQuota.WithLabelValues(c.taskCfg.Name).Inc() - return - } mid2, loaded := c.bmSeries[sid] - if !loaded || mid != mid2 { - c.bmSeries[sid] = mid - c.wrSeries++ + if !loaded { allowed = true - statistics.WriteSeriesAllowed.WithLabelValues(c.taskCfg.Name).Inc() + statistics.WriteSeriesAllowNew.WithLabelValues(c.taskCfg.Name).Inc() + } else if mid != mid2 { + if c.wrSeries < wrSeriesQuota { + c.wrSeries++ + allowed = true + statistics.WriteSeriesAllowChanged.WithLabelValues(c.taskCfg.Name).Inc() + } else { + statistics.WriteSeriesDropQuota.WithLabelValues(c.taskCfg.Name).Inc() + } } else { statistics.WriteSeriesDropUnchanged.WithLabelValues(c.taskCfg.Name).Inc() } @@ -163,15 +165,18 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err err if numBad, err = writeRows(c.promSerSQL, seriesRows, c.IdxSerID, c.NumDims, conn); err != nil { return } - var curWrSeries int + // update c.bmSeries **after** writing series c.mux.Lock() - c.wrSeries -= len(seriesRows) - if c.wrSeries < 0 { - c.wrSeries = 0 + for _, row := range seriesRows { + sid := (*row)[c.IdxSerID].(int64) + mid := (*row)[c.IdxSerID+1].(int64) + if _, loaded := c.bmSeries[sid]; loaded { + c.wrSeries-- + } + c.bmSeries[sid] = mid } - curWrSeries = c.wrSeries c.mux.Unlock() - util.Logger.Info("ClickHouse.writeSeries succeeded", zap.Int("series", len(seriesRows)), zap.Int("c.wrSeries", curWrSeries), zap.String("task", c.taskCfg.Name)) + util.Logger.Info("ClickHouse.writeSeries succeeded", zap.Int("series", len(seriesRows)), zap.String("task", c.taskCfg.Name)) statistics.WriteSeriesSucceed.WithLabelValues(c.taskCfg.Name).Add(float64(len(seriesRows))) if numBad != 0 { statistics.ParseMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(numBad)) diff --git a/statistics/statistics.go b/statistics/statistics.go index d06afc4c..8cf82f24 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -153,6 +153,20 @@ var ( }, []string{"task", "table"}, ) + WriteSeriesAllowNew = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: prefix + "write_series_allow_new", + Help: "num of allowed new series", + }, + []string{"task"}, + ) + WriteSeriesAllowChanged = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: prefix + "write_series_allow_changed", + Help: "num of allowed changed series", + }, + []string{"task"}, + ) WriteSeriesDropQuota = prometheus.NewCounterVec( prometheus.CounterOpts{ Name: prefix + "write_series_drop_quota", @@ -167,13 +181,7 @@ var ( }, []string{"task"}, ) - WriteSeriesAllowed = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Name: prefix + "write_series_allow", - Help: "num of allowed write_series", - }, - []string{"task"}, - ) + // WriteSeriesSucceed = WriteSeriesAllowNew + WriteSeriesAllowChanged WriteSeriesSucceed = prometheus.NewCounterVec( prometheus.CounterOpts{ Name: prefix + "write_series_succeed", @@ -201,9 +209,10 @@ func init() { prometheus.MustRegister(ParsingPoolBacklog) prometheus.MustRegister(WritingPoolBacklog) prometheus.MustRegister(WritingDurations) + prometheus.MustRegister(WriteSeriesAllowNew) + prometheus.MustRegister(WriteSeriesAllowChanged) prometheus.MustRegister(WriteSeriesDropQuota) prometheus.MustRegister(WriteSeriesDropUnchanged) - prometheus.MustRegister(WriteSeriesAllowed) prometheus.MustRegister(WriteSeriesSucceed) prometheus.MustRegister(collectors.NewBuildInfoCollector()) } From 8fab1c3704cf5fc36eabfb00d17a9f416756cd85 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 8 Dec 2022 14:41:55 +0800 Subject: [PATCH 297/404] feat: Add clickhouse Map type support --- model/metric.go | 49 ++++- model/value.go | 96 ++++++---- output/clickhouse.go | 23 +-- output/clickhouse_util.go | 4 +- parser/csv.go | 12 +- parser/fastjson.go | 325 +++++++++++++++++++++++--------- parser/gjson.go | 379 +++++++++++++++++++++++--------------- parser/parser_test.go | 109 ++++++++++- task/sharding.go | 4 +- task/task.go | 2 +- 10 files changed, 702 insertions(+), 301 deletions(-) diff --git a/model/metric.go b/model/metric.go index e79d757a..c364e1c4 100644 --- a/model/metric.go +++ b/model/metric.go @@ -37,6 +37,7 @@ type Metric interface { GetDateTime(key string, nullable bool) (val interface{}) GetString(key string, nullable bool) (val interface{}) GetObject(key string, nullable bool) (val interface{}) + GetMap(key string, typeinfo *TypeInfo) (val interface{}) GetArray(key string, t int) (val interface{}) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, black *regexp.Regexp, partition int, offset int64) bool } @@ -50,8 +51,50 @@ type DimMetrics struct { // ColumnWithType type ColumnWithType struct { Name string - Type int - Nullable bool - Array bool + Type *TypeInfo SourceName string } + +// struct for ingesting a clickhouse Map type value +type OrderedMap struct { + keys []interface{} + values map[interface{}]interface{} +} + +func (om *OrderedMap) Get(key interface{}) (interface{}, bool) { + if value, present := om.values[key]; present { + return value, present + } + return nil, false +} + +func (om *OrderedMap) Put(key interface{}, value interface{}) { + if _, present := om.values[key]; present { + om.values[key] = value + return + } + om.keys = append(om.keys, key) + om.values[key] = value +} + +func (om *OrderedMap) Keys() <-chan interface{} { + ch := make(chan interface{}) + go func() { + defer close(ch) + for _, key := range om.keys { + ch <- key + } + }() + return ch +} + +func (om *OrderedMap) GetValues() map[interface{}]interface{} { + return om.values +} + +func NewOrderedMap() *OrderedMap { + om := OrderedMap{} + om.keys = []interface{}{} + om.values = map[interface{}]interface{}{} + return &om +} diff --git a/model/value.go b/model/value.go index 107a880d..09bd7b7f 100644 --- a/model/value.go +++ b/model/value.go @@ -17,6 +17,7 @@ package model import ( "fmt" + "regexp" "strings" "github.com/housepower/clickhouse_sinker/util" @@ -39,16 +40,20 @@ const ( DateTime String Object + Map ) type TypeInfo struct { Type int Nullable bool Array bool + MapKey *TypeInfo + MapValue *TypeInfo } var ( - typeInfo map[string]TypeInfo + typeInfo map[string]*TypeInfo + lowCardinalityRegexp = regexp.MustCompile(`^LowCardinality\((.+)\)`) ) // GetTypeName returns the column type in ClickHouse @@ -84,6 +89,8 @@ func GetTypeName(typ int) (name string) { name = "String" case Object: name = "Object('json')" + case Map: + name = "Map" default: name = "Unknown" } @@ -92,40 +99,42 @@ func GetTypeName(typ int) (name string) { func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { name := cwt.SourceName - if cwt.Array { - val = metric.GetArray(name, cwt.Type) + if cwt.Type.Array { + val = metric.GetArray(name, cwt.Type.Type) } else { - switch cwt.Type { + switch cwt.Type.Type { case Bool: - val = metric.GetBool(name, cwt.Nullable) + val = metric.GetBool(name, cwt.Type.Nullable) case Int8: - val = metric.GetInt8(name, cwt.Nullable) + val = metric.GetInt8(name, cwt.Type.Nullable) case Int16: - val = metric.GetInt16(name, cwt.Nullable) + val = metric.GetInt16(name, cwt.Type.Nullable) case Int32: - val = metric.GetInt32(name, cwt.Nullable) + val = metric.GetInt32(name, cwt.Type.Nullable) case Int64: - val = metric.GetInt64(name, cwt.Nullable) + val = metric.GetInt64(name, cwt.Type.Nullable) case UInt8: - val = metric.GetUint8(name, cwt.Nullable) + val = metric.GetUint8(name, cwt.Type.Nullable) case UInt16: - val = metric.GetUint16(name, cwt.Nullable) + val = metric.GetUint16(name, cwt.Type.Nullable) case UInt32: - val = metric.GetUint32(name, cwt.Nullable) + val = metric.GetUint32(name, cwt.Type.Nullable) case UInt64: - val = metric.GetUint64(name, cwt.Nullable) + val = metric.GetUint64(name, cwt.Type.Nullable) case Float32: - val = metric.GetFloat32(name, cwt.Nullable) + val = metric.GetFloat32(name, cwt.Type.Nullable) case Float64: - val = metric.GetFloat64(name, cwt.Nullable) + val = metric.GetFloat64(name, cwt.Type.Nullable) case Decimal: - val = metric.GetDecimal(name, cwt.Nullable) + val = metric.GetDecimal(name, cwt.Type.Nullable) case DateTime: - val = metric.GetDateTime(name, cwt.Nullable) + val = metric.GetDateTime(name, cwt.Type.Nullable) case String: - val = metric.GetString(name, cwt.Nullable) + val = metric.GetString(name, cwt.Type.Nullable) + case Map: + val = metric.GetMap(name, cwt.Type) case Object: - val = metric.GetObject(name, cwt.Nullable) + val = metric.GetObject(name, cwt.Type.Nullable) default: util.Logger.Fatal("LOGIC ERROR: reached switch default condition") } @@ -133,15 +142,17 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { return } -func WhichType(typ string) (dataType int, nullable bool, array bool) { +func WhichType(typ string) (ti *TypeInfo) { + typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") + ti, ok := typeInfo[typ] if ok { - dataType, nullable, array = ti.Type, ti.Nullable, ti.Array - return + return ti } origTyp := typ - nullable = strings.HasPrefix(typ, "Nullable(") - array = strings.HasPrefix(typ, "Array(") + nullable := strings.HasPrefix(typ, "Nullable(") + array := strings.HasPrefix(typ, "Array(") + var dataType int if nullable { typ = typ[len("Nullable(") : len(typ)-1] } else if array { @@ -157,27 +168,40 @@ func WhichType(typ string) (dataType int, nullable bool, array bool) { dataType = String } else if strings.HasPrefix(typ, "Enum16(") { dataType = String + } else if strings.HasPrefix(typ, "Map") { + dataType = Map + idx := strings.Index(typ, ", ") + ti = &TypeInfo{ + Type: dataType, + Nullable: nullable, + Array: array, + MapKey: WhichType(typ[len("Map("):idx]), + MapValue: WhichType(typ[idx+2 : len(typ)-1]), + } + typeInfo[origTyp] = ti + return ti } else { util.Logger.Fatal(fmt.Sprintf("ClickHouse column type %v is not inside supported ones: %v", origTyp, typeInfo)) } - typeInfo[origTyp] = TypeInfo{Type: dataType, Nullable: nullable, Array: array} - return + ti = &TypeInfo{Type: dataType, Nullable: nullable, Array: array} + typeInfo[origTyp] = ti + return ti } func init() { - typeInfo = make(map[string]TypeInfo) + typeInfo = make(map[string]*TypeInfo) for _, t := range []int{Bool, Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32, Float64, DateTime, String, Object} { tn := GetTypeName(t) - typeInfo[tn] = TypeInfo{Type: t} + typeInfo[tn] = &TypeInfo{Type: t} nullTn := fmt.Sprintf("Nullable(%s)", tn) - typeInfo[nullTn] = TypeInfo{Type: t, Nullable: true} + typeInfo[nullTn] = &TypeInfo{Type: t, Nullable: true} arrTn := fmt.Sprintf("Array(%s)", tn) - typeInfo[arrTn] = TypeInfo{Type: t, Array: true} + typeInfo[arrTn] = &TypeInfo{Type: t, Array: true} } - typeInfo["UUID"] = TypeInfo{Type: String} - typeInfo["Nullable(UUID)"] = TypeInfo{Type: String, Nullable: true} - typeInfo["Array(UUID)"] = TypeInfo{Type: String, Array: true} - typeInfo["Date"] = TypeInfo{Type: DateTime} - typeInfo["Nullable(Date)"] = TypeInfo{Type: DateTime, Nullable: true} - typeInfo["Array(Date)"] = TypeInfo{Type: DateTime, Array: true} + typeInfo["UUID"] = &TypeInfo{Type: String} + typeInfo["Nullable(UUID)"] = &TypeInfo{Type: String, Nullable: true} + typeInfo["Array(UUID)"] = &TypeInfo{Type: String, Array: true} + typeInfo["Date"] = &TypeInfo{Type: DateTime} + typeInfo["Nullable(Date)"] = &TypeInfo{Type: DateTime, Nullable: true} + typeInfo["Array(Date)"] = &TypeInfo{Type: DateTime, Array: true} } diff --git a/output/clickhouse.go b/output/clickhouse.go index 9afb7e03..2461ad90 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -20,7 +20,6 @@ import ( "fmt" "io" "math" - "regexp" "sort" "strings" "sync" @@ -39,9 +38,8 @@ import ( ) var ( - ErrTblNotExist = errors.Newf("table doesn't exist") - selectSQLTemplate = `select name, type, default_kind from system.columns where database = '%s' and table = '%s'` - lowCardinalityRegexp = regexp.MustCompile(`LowCardinality\((.+)\)`) + ErrTblNotExist = errors.Newf("table doesn't exist") + selectSQLTemplate = `select name, type, default_kind from system.columns where database = '%s' and table = '%s'` // https://github.com/ClickHouse/ClickHouse/issues/24036 // src/Common/ErrorCodes.cpp @@ -291,10 +289,10 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { var dimSerID *model.ColumnWithType for i := 0; i < len(c.Dims); { dim := c.Dims[i] - if dim.Name == "__series_id" && dim.Type == model.Int64 { + if dim.Name == "__series_id" && dim.Type.Type == model.Int64 { dimSerID = dim c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) - } else if dim.Type == model.String { + } else if dim.Type.Type == model.String { c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) } else { i++ @@ -310,9 +308,9 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { // Add string columns from series table c.seriesTbl = c.taskCfg.TableName + "_series" expSeriesDims := []*model.ColumnWithType{ - {Name: "__series_id", Type: model.Int64}, - {Name: "__mgmt_id", Type: model.Int64}, - {Name: "labels", Type: model.String}, + {Name: "__series_id", Type: &model.TypeInfo{Type: model.Int64}}, + {Name: "__mgmt_id", Type: &model.TypeInfo{Type: model.Int64}}, + {Name: "labels", Type: &model.TypeInfo{Type: model.String}}, } var seriesDims []*model.ColumnWithType if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.seriesTbl, nil, c.taskCfg.Parser, conn); err != nil { @@ -341,7 +339,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { c.NameKey = "__name__" // prometheus uses internal "__name__" label for metric name for i := len(expSeriesDims); i < len(seriesDims); i++ { serDim := seriesDims[i] - if serDim.Type == model.String { + if serDim.Type.Type == model.String { c.NameKey = serDim.Name // opentsdb uses "metric" tag for metric name break } @@ -389,12 +387,9 @@ func (c *ClickHouse) initSchema() (err error) { } else { c.Dims = make([]*model.ColumnWithType, 0) for _, dim := range c.taskCfg.Dims { - tp, nullable, array := model.WhichType(dim.Type) c.Dims = append(c.Dims, &model.ColumnWithType{ Name: dim.Name, - Type: tp, - Nullable: nullable, - Array: array, + Type: model.WhichType(dim.Type), SourceName: dim.SourceName, }) } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index a372c117..16f691d4 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -93,10 +93,8 @@ func getDims(database, table string, excludedColumns []string, parser string, co err = errors.Wrapf(err, "") return } - typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1") if !util.StringContains(excludedColumns, name) && defaultKind != "MATERIALIZED" { - tp, nullable, array := model.WhichType(typ) - dims = append(dims, &model.ColumnWithType{Name: name, Type: tp, Nullable: nullable, Array: array, SourceName: util.GetSourceName(parser, name)}) + dims = append(dims, &model.ColumnWithType{Name: name, Type: model.WhichType(typ), SourceName: util.GetSourceName(parser, name)}) } } if len(dims) == 0 { diff --git a/parser/csv.go b/parser/csv.go index 68401ec8..3dcffeaf 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -241,10 +241,6 @@ func (c *CsvMetric) GetDateTime(key string, nullable bool) (val interface{}) { return } -func (c *CsvMetric) GetObject(key string, nullable bool) (val interface{}) { - return -} - // GetArray parse an CSV encoded array func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { s := c.GetString(key, false) @@ -334,6 +330,14 @@ func (c *CsvMetric) GetArray(key string, typ int) (val interface{}) { return } +func (c *CsvMetric) GetObject(key string, nullable bool) (val interface{}) { + return +} + +func (c *CsvMetric) GetMap(key string, typeinfo *model.TypeInfo) (val interface{}) { + return +} + func (c *CsvMetric) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, black *regexp.Regexp, partition int, offset int64) bool { return false } diff --git a/parser/fastjson.go b/parser/fastjson.go index 65a9d3b1..1c385738 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -19,6 +19,7 @@ import ( "fmt" "math" "regexp" + "strconv" "sync" "time" @@ -57,90 +58,58 @@ type FastjsonMetric struct { } func (c *FastjsonMetric) GetString(key string, nullable bool) (val interface{}) { - v := c.value.Get(key) - if v == nil || v.Type() == fastjson.TypeNull { - if nullable { - return - } - val = "" - return - } - switch v.Type() { - case fastjson.TypeString: - b, _ := v.StringBytes() - val = string(b) - default: - val = v.String() - } - return + return getString(c.value.Get(key), nullable) } -func (c *FastjsonMetric) GetBool(key string, nullable bool) (val interface{}) { - v := c.value.Get(key) - if !fjCompatibleBool(v) { - val = getDefaultBool(nullable) - return - } - val = (v.Type() == fastjson.TypeTrue) - return +func (c *FastjsonMetric) GetBool(key string, nullable bool) interface{} { + return getBool(c.value.Get(key), nullable) } func (c *FastjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) { - v := c.value.Get(key) - if !fjCompatibleFloat(v) { - val = getDefaultDecimal(nullable) - return - } - if val2, err := v.Float64(); err != nil { - val = getDefaultDecimal(nullable) - } else { - val = decimal.NewFromFloat(val2) - } - return + return getDecimal(c.value.Get(key), nullable) } func (c *FastjsonMetric) GetInt8(key string, nullable bool) (val interface{}) { - return FastjsonGetInt[int8](c, key, nullable, math.MinInt8, math.MaxInt8) + return FastjsonGetInt[int8](c.value.Get(key), nullable, math.MinInt8, math.MaxInt8) } func (c *FastjsonMetric) GetInt16(key string, nullable bool) (val interface{}) { - return FastjsonGetInt[int16](c, key, nullable, math.MinInt16, math.MaxInt16) + return FastjsonGetInt[int16](c.value.Get(key), nullable, math.MinInt16, math.MaxInt16) } func (c *FastjsonMetric) GetInt32(key string, nullable bool) (val interface{}) { - return FastjsonGetInt[int32](c, key, nullable, math.MinInt32, math.MaxInt32) + return FastjsonGetInt[int32](c.value.Get(key), nullable, math.MinInt32, math.MaxInt32) } func (c *FastjsonMetric) GetInt64(key string, nullable bool) (val interface{}) { - return FastjsonGetInt[int64](c, key, nullable, math.MinInt64, math.MaxInt64) + return FastjsonGetInt[int64](c.value.Get(key), nullable, math.MinInt64, math.MaxInt64) } func (c *FastjsonMetric) GetUint8(key string, nullable bool) (val interface{}) { - return FastjsonGetUint[uint8](c, key, nullable, math.MaxUint8) + return FastjsonGetUint[uint8](c.value.Get(key), nullable, math.MaxUint8) } func (c *FastjsonMetric) GetUint16(key string, nullable bool) (val interface{}) { - return FastjsonGetUint[uint16](c, key, nullable, math.MaxUint16) + return FastjsonGetUint[uint16](c.value.Get(key), nullable, math.MaxUint16) } func (c *FastjsonMetric) GetUint32(key string, nullable bool) (val interface{}) { - return FastjsonGetUint[uint32](c, key, nullable, math.MaxUint32) + return FastjsonGetUint[uint32](c.value.Get(key), nullable, math.MaxUint32) } func (c *FastjsonMetric) GetUint64(key string, nullable bool) (val interface{}) { - return FastjsonGetUint[uint64](c, key, nullable, math.MaxUint64) + return FastjsonGetUint[uint64](c.value.Get(key), nullable, math.MaxUint64) } func (c *FastjsonMetric) GetFloat32(key string, nullable bool) (val interface{}) { - return FastjsonGetFloat[float32](c, key, nullable, math.MaxFloat32) + return FastjsonGetFloat[float32](c.value.Get(key), nullable, math.MaxFloat32) } func (c *FastjsonMetric) GetFloat64(key string, nullable bool) (val interface{}) { - return FastjsonGetFloat[float64](c, key, nullable, math.MaxFloat64) + return FastjsonGetFloat[float64](c.value.Get(key), nullable, math.MaxFloat64) } -func FastjsonGetInt[T constraints.Signed](c *FastjsonMetric, key string, nullable bool, min, max int64) (val interface{}) { - v := c.value.Get(key) +func FastjsonGetInt[T constraints.Signed](v *fastjson.Value, nullable bool, min, max int64) (val interface{}) { if !fjCompatibleInt(v) { val = getDefaultInt[T](nullable) return @@ -164,8 +133,7 @@ func FastjsonGetInt[T constraints.Signed](c *FastjsonMetric, key string, nullabl return } -func FastjsonGetUint[T constraints.Unsigned](c *FastjsonMetric, key string, nullable bool, max uint64) (val interface{}) { - v := c.value.Get(key) +func FastjsonGetUint[T constraints.Unsigned](v *fastjson.Value, nullable bool, max uint64) (val interface{}) { if !fjCompatibleInt(v) { val = getDefaultInt[T](nullable) return @@ -187,8 +155,7 @@ func FastjsonGetUint[T constraints.Unsigned](c *FastjsonMetric, key string, null return } -func FastjsonGetFloat[T constraints.Float](c *FastjsonMetric, key string, nullable bool, max float64) (val interface{}) { - v := c.value.Get(key) +func FastjsonGetFloat[T constraints.Float](v *fastjson.Value, nullable bool, max float64) (val interface{}) { if !fjCompatibleFloat(v) { val = getDefaultFloat[T](nullable) return @@ -204,7 +171,112 @@ func FastjsonGetFloat[T constraints.Float](c *FastjsonMetric, key string, nullab } func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { + return getDateTime(c, key, c.value.Get(key), nullable) +} + +func (c *FastjsonMetric) GetObject(key string, nullable bool) (val interface{}) { v := c.value.Get(key) + val = val2map(v) + return +} + +func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { + return getArray(c, key, c.value.Get(key), typ) +} + +func (c *FastjsonMetric) GetMap(key string, typeinfo *model.TypeInfo) (val interface{}) { + return getMap(c, c.value.Get(key), typeinfo) +} + +func (c *FastjsonMetric) val2OrderedMap(v *fastjson.Value, typeinfo *model.TypeInfo) (m *model.OrderedMap) { + var err error + var obj *fastjson.Object + m = model.NewOrderedMap() + if v == nil { + return + } + if obj, err = v.Object(); err != nil { + return + } + obj.Visit(func(key []byte, v *fastjson.Value) { + rawKey := c.castMapKeyByType(key, typeinfo.MapKey) + m.Put(rawKey, c.castMapValueByType(string(key), v, typeinfo.MapValue)) + }) + return +} + +func val2map(v *fastjson.Value) (m map[string]interface{}) { + var err error + var obj *fastjson.Object + m = EmpytObject + if v == nil { + return + } + if obj, err = v.Object(); err != nil { + return + } + m = make(map[string]interface{}, obj.Len()) + obj.Visit(func(key []byte, v *fastjson.Value) { + strKey := string(key) + switch v.Type() { + case fastjson.TypeString: + var vb []byte + if vb, err = v.StringBytes(); err != nil { + return + } + m[strKey] = string(vb) + case fastjson.TypeNumber: + var f float64 + if f, err = v.Float64(); err != nil { + return + } + m[strKey] = f + } + }) + return +} + +func getString(v *fastjson.Value, nullable bool) (val interface{}) { + if v == nil || v.Type() == fastjson.TypeNull { + if nullable { + return + } + val = "" + return + } + switch v.Type() { + case fastjson.TypeString: + b, _ := v.StringBytes() + val = string(b) + default: + val = v.String() + } + return +} + +func getBool(v *fastjson.Value, nullable bool) (val interface{}) { + if !fjCompatibleBool(v) { + val = getDefaultBool(nullable) + return + } + val = (v.Type() == fastjson.TypeTrue) + return +} + +func getDecimal(v *fastjson.Value, nullable bool) (val interface{}) { + if !fjCompatibleFloat(v) { + val = getDefaultDecimal(nullable) + return + } + if val2, err := v.Float64(); err != nil { + val = getDefaultDecimal(nullable) + } else { + val = decimal.NewFromFloat(val2) + } + return +} + +func getDateTime(c *FastjsonMetric, sourcename string, v *fastjson.Value, nullable bool) (val interface{}) { if !fjCompatibleDateTime(v) { val = getDefaultDateTime(nullable) return @@ -224,7 +296,7 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} val = getDefaultDateTime(nullable) return } - if val, err = c.pp.ParseDateTime(key, string(b)); err != nil { + if val, err = c.pp.ParseDateTime(sourcename, string(b)); err != nil { val = getDefaultDateTime(nullable) } default: @@ -233,15 +305,9 @@ func (c *FastjsonMetric) GetDateTime(key string, nullable bool) (val interface{} return } -func (c *FastjsonMetric) GetObject(key string, nullable bool) (val interface{}) { - v := c.value.Get(key) - val = val2map(v) - return -} - -func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { +func getArray(c *FastjsonMetric, sourcename string, v *fastjson.Value, typ int) (val interface{}) { var array []*fastjson.Value - if v := c.value.Get(key); v != nil { + if v != nil { array, _ = v.Array() } switch typ { @@ -311,7 +377,7 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { t = Epoch } else { var err error - if t, err = c.pp.ParseDateTime(key, string(b)); err != nil { + if t, err = c.pp.ParseDateTime(sourcename, string(b)); err != nil { t = Epoch } } @@ -336,34 +402,121 @@ func (c *FastjsonMetric) GetArray(key string, typ int) (val interface{}) { return } -func val2map(v *fastjson.Value) (m map[string]interface{}) { - var err error - var obj *fastjson.Object - m = EmpytObject - if v == nil { - return +func getMap(c *FastjsonMetric, v *fastjson.Value, typeinfo *model.TypeInfo) (val interface{}) { + if v != nil && v.Type() == fastjson.TypeObject { + val = c.val2OrderedMap(v, typeinfo) } - if obj, err = v.Object(); err != nil { - return + return +} + +func (c *FastjsonMetric) castMapKeyByType(key []byte, typeinfo *model.TypeInfo) (val interface{}) { + switch typeinfo.Type { + case model.Int8: + if res, err := strconv.ParseInt(string(key), 10, 8); err == nil { + return int8(res) + } else { + util.Logger.Error("failed to parse map key", zap.Error(err)) + } + case model.Int16: + if res, err := strconv.ParseInt(string(key), 10, 16); err == nil { + return int16(res) + } else { + util.Logger.Error("failed to parse map key", zap.Error(err)) + } + case model.Int32: + if res, err := strconv.ParseInt(string(key), 10, 32); err == nil { + return int32(res) + } else { + util.Logger.Error("failed to parse map key", zap.Error(err)) + } + case model.Int64: + if res, err := strconv.ParseInt(string(key), 10, 64); err == nil { + return int64(res) + } else { + util.Logger.Error("failed to parse map key", zap.Error(err)) + } + case model.UInt8: + if res, err := strconv.ParseUint(string(key), 10, 8); err == nil { + return uint8(res) + } else { + util.Logger.Error("failed to parse map key", zap.Error(err)) + } + case model.UInt16: + if res, err := strconv.ParseUint(string(key), 10, 16); err == nil { + return uint16(res) + } else { + util.Logger.Error("failed to parse map key", zap.Error(err)) + } + case model.UInt32: + if res, err := strconv.ParseUint(string(key), 10, 32); err == nil { + return uint32(res) + } else { + util.Logger.Error("failed to parse map key", zap.Error(err)) + } + case model.UInt64: + if res, err := strconv.ParseUint(string(key), 10, 64); err == nil { + return uint64(res) + } else { + util.Logger.Error("failed to parse map key", zap.Error(err)) + } + case model.DateTime: + if res, err := c.pp.ParseDateTime(string(key), string(key)); err == nil { + return res + } else { + util.Logger.Error("failed to parse map key", zap.Error(err)) + } + val = getDefaultDateTime(typeinfo.Nullable) + case model.String: + return string(key) + default: + util.Logger.Fatal("LOGIC ERROR: reached switch default condition") } - m = make(map[string]interface{}, obj.Len()) - obj.Visit(func(key []byte, v *fastjson.Value) { - strKey := string(key) - switch v.Type() { - case fastjson.TypeString: - var vb []byte - if vb, err = v.StringBytes(); err != nil { - return - } - m[strKey] = string(vb) - case fastjson.TypeNumber: - var f float64 - if f, err = v.Float64(); err != nil { - return - } - m[strKey] = f + + return +} + +func (c *FastjsonMetric) castMapValueByType(sourcename string, value *fastjson.Value, typeinfo *model.TypeInfo) (val interface{}) { + if typeinfo.Array { + val = getArray(c, sourcename, value, typeinfo.Type) + return + } else { + switch typeinfo.Type { + case model.Bool: + val = getBool(value, typeinfo.Nullable) + case model.Int8: + val = FastjsonGetInt[int8](value, typeinfo.Nullable, math.MinInt8, math.MaxInt8) + case model.Int16: + val = FastjsonGetInt[int16](value, typeinfo.Nullable, math.MinInt16, math.MaxInt16) + case model.Int32: + val = FastjsonGetInt[int32](value, typeinfo.Nullable, math.MinInt32, math.MaxInt32) + case model.Int64: + val = FastjsonGetInt[int64](value, typeinfo.Nullable, math.MinInt64, math.MaxInt64) + case model.UInt8: + val = FastjsonGetUint[uint8](value, typeinfo.Nullable, math.MaxUint8) + case model.UInt16: + val = FastjsonGetUint[uint16](value, typeinfo.Nullable, math.MaxUint16) + case model.UInt32: + val = FastjsonGetUint[uint32](value, typeinfo.Nullable, math.MaxUint32) + case model.UInt64: + val = FastjsonGetUint[uint64](value, typeinfo.Nullable, math.MaxUint64) + case model.Float32: + val = FastjsonGetFloat[float32](value, typeinfo.Nullable, math.MaxFloat32) + case model.Float64: + val = FastjsonGetFloat[float64](value, typeinfo.Nullable, math.MaxFloat64) + case model.Decimal: + val = getDecimal(value, typeinfo.Nullable) + case model.DateTime: + val = getDateTime(c, sourcename, value, typeinfo.Nullable) + case model.String: + val = getString(value, typeinfo.Nullable) + case model.Map: + val = getMap(c, value, typeinfo) + case model.Object: + val = val2map(value) + default: + util.Logger.Fatal("LOGIC ERROR: reached switch default condition") } - }) + } return } diff --git a/parser/gjson.go b/parser/gjson.go index 0d344edb..73c4a32c 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -49,92 +49,58 @@ type GjsonMetric struct { } func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}) { - r := gjson.Get(c.raw, key) - if !r.Exists() || r.Type == gjson.Null { - if nullable { - return - } - val = "" - return - } - switch r.Type { - case gjson.Null: - val = "" - case gjson.String: - val = r.Str - default: - val = r.Raw - } - return + return getGJsonString(gjson.Get(c.raw, key), nullable) } func (c *GjsonMetric) GetBool(key string, nullable bool) (val interface{}) { - r := gjson.Get(c.raw, key) - if !gjCompatibleBool(r) { - val = getDefaultBool(nullable) - return - } - val = (r.Type == gjson.True) - return + return getGJsonBool(gjson.Get(c.raw, key), nullable) } func (c *GjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) { - r := gjson.Get(c.raw, key) - if !gjCompatibleFloat(r) { - val = getDefaultDecimal(nullable) - return - } - switch r.Type { - case gjson.Number: - val = decimal.NewFromFloat(r.Num) - default: - val = getDefaultDecimal(nullable) - } - return + return getGJsonDecimal(gjson.Get(c.raw, key), nullable) } func (c *GjsonMetric) GetInt8(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int8](c, key, nullable, math.MinInt8, math.MaxInt8) + return GjsonGetInt[int8](c, gjson.Get(c.raw, key), nullable, math.MinInt8, math.MaxInt8) } func (c *GjsonMetric) GetInt16(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int16](c, key, nullable, math.MinInt16, math.MaxInt16) + return GjsonGetInt[int16](c, gjson.Get(c.raw, key), nullable, math.MinInt16, math.MaxInt16) } func (c *GjsonMetric) GetInt32(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int32](c, key, nullable, math.MinInt32, math.MaxInt32) + return GjsonGetInt[int32](c, gjson.Get(c.raw, key), nullable, math.MinInt32, math.MaxInt32) } func (c *GjsonMetric) GetInt64(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int64](c, key, nullable, math.MinInt64, math.MaxInt64) + return GjsonGetInt[int64](c, gjson.Get(c.raw, key), nullable, math.MinInt64, math.MaxInt64) } func (c *GjsonMetric) GetUint8(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint8](c, key, nullable, math.MaxUint8) + return GjsonGetUint[uint8](c, gjson.Get(c.raw, key), nullable, math.MaxUint8) } func (c *GjsonMetric) GetUint16(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint16](c, key, nullable, math.MaxUint16) + return GjsonGetUint[uint16](c, gjson.Get(c.raw, key), nullable, math.MaxUint16) } func (c *GjsonMetric) GetUint32(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint32](c, key, nullable, math.MaxUint32) + return GjsonGetUint[uint32](c, gjson.Get(c.raw, key), nullable, math.MaxUint32) } func (c *GjsonMetric) GetUint64(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint64](c, key, nullable, math.MaxUint64) + return GjsonGetUint[uint64](c, gjson.Get(c.raw, key), nullable, math.MaxUint64) } func (c *GjsonMetric) GetFloat32(key string, nullable bool) (val interface{}) { - return GjsonGetFloat[float32](c, key, nullable, math.MaxFloat32) + return GjsonGetFloat[float32](c, gjson.Get(c.raw, key), nullable, math.MaxFloat32) } func (c *GjsonMetric) GetFloat64(key string, nullable bool) (val interface{}) { - return GjsonGetFloat[float64](c, key, nullable, math.MaxFloat64) + return GjsonGetFloat[float64](c, gjson.Get(c.raw, key), nullable, math.MaxFloat64) } -func GjsonGetInt[T constraints.Signed](c *GjsonMetric, key string, nullable bool, min, max int64) (val interface{}) { - r := gjson.Get(c.raw, key) +func GjsonGetInt[T constraints.Signed](c *GjsonMetric, r gjson.Result, nullable bool, min, max int64) (val interface{}) { if !gjCompatibleInt(r) { val = getDefaultInt[T](nullable) return @@ -160,8 +126,7 @@ func GjsonGetInt[T constraints.Signed](c *GjsonMetric, key string, nullable bool return } -func GjsonGetUint[T constraints.Unsigned](c *GjsonMetric, key string, nullable bool, max uint64) (val interface{}) { - r := gjson.Get(c.raw, key) +func GjsonGetUint[T constraints.Unsigned](c *GjsonMetric, r gjson.Result, nullable bool, max uint64) (val interface{}) { if !gjCompatibleInt(r) { val = getDefaultInt[T](nullable) return @@ -185,8 +150,7 @@ func GjsonGetUint[T constraints.Unsigned](c *GjsonMetric, key string, nullable b return } -func GjsonGetFloat[T constraints.Float](c *GjsonMetric, key string, nullable bool, max float64) (val interface{}) { - r := gjson.Get(c.raw, key) +func GjsonGetFloat[T constraints.Float](c *GjsonMetric, r gjson.Result, nullable bool, max float64) (val interface{}) { if !gjCompatibleFloat(r) { val = getDefaultFloat[T](nullable) return @@ -205,23 +169,7 @@ func GjsonGetFloat[T constraints.Float](c *GjsonMetric, key string, nullable boo } func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { - r := gjson.Get(c.raw, key) - if !gjCompatibleDateTime(r) { - val = getDefaultDateTime(nullable) - return - } - switch r.Type { - case gjson.Number: - val = UnixFloat(r.Num, c.pp.timeUnit) - case gjson.String: - var err error - if val, err = c.pp.ParseDateTime(key, r.Str); err != nil { - val = getDefaultDateTime(nullable) - } - default: - val = getDefaultDateTime(nullable) - } - return + return getGJsonDateTime(c, key, gjson.Get(c.raw, key), nullable) } func (c *GjsonMetric) GetObject(key string, nullable bool) (val interface{}) { @@ -229,88 +177,21 @@ func (c *GjsonMetric) GetObject(key string, nullable bool) (val interface{}) { } func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { - var array []gjson.Result - r := gjson.Get(c.raw, key) - if r.IsArray() { - array = r.Array() - } - switch typ { - case model.Bool: - results := make([]bool, 0, len(array)) - for _, e := range array { - v := (e.Exists() && e.Type == gjson.True) - results = append(results, v) - } - val = results - case model.Int8: - val = GjsonIntArray[int8](array, math.MinInt8, math.MaxInt8) - case model.Int16: - val = GjsonIntArray[int16](array, math.MinInt16, math.MaxInt16) - case model.Int32: - val = GjsonIntArray[int32](array, math.MinInt32, math.MaxInt32) - case model.Int64: - val = GjsonIntArray[int64](array, math.MinInt64, math.MaxInt64) - case model.UInt8: - val = GjsonUintArray[uint8](array, math.MaxUint8) - case model.UInt16: - val = GjsonUintArray[uint16](array, math.MaxUint16) - case model.UInt32: - val = GjsonUintArray[uint32](array, math.MaxUint32) - case model.UInt64: - val = GjsonUintArray[uint64](array, math.MaxUint64) - case model.Float32: - val = GjsonFloatArray[float32](array, math.MaxFloat32) - case model.Float64: - val = GjsonFloatArray[float64](array, math.MaxFloat64) - case model.Decimal: - results := make([]decimal.Decimal, 0, len(array)) - var f float64 - for _, e := range array { - switch e.Type { - case gjson.Number: - f = e.Num - default: - f = float64(0.0) - } - results = append(results, decimal.NewFromFloat(f)) - } - val = results - case model.String: - results := make([]string, 0, len(array)) - var s string - for _, e := range array { - switch e.Type { - case gjson.Null: - s = "" - case gjson.String: - s = e.Str - default: - s = e.Raw - } - results = append(results, s) - } - val = results - case model.DateTime: - results := make([]time.Time, 0, len(array)) - var t time.Time - for _, e := range array { - switch e.Type { - case gjson.Number: - t = UnixFloat(e.Num, c.pp.timeUnit) - case gjson.String: - var err error - if t, err = c.pp.ParseDateTime(key, e.Str); err != nil { - t = Epoch - } - default: - t = Epoch - } - results = append(results, t) - } - val = results - default: - util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) - } + return getGJsonArray(c, key, gjson.Get(c.raw, key), typ) +} + +func (c *GjsonMetric) GetMap(key string, typeinfo *model.TypeInfo) (val interface{}) { + return getGJsonMap(c, gjson.Get(c.raw, key), typeinfo) +} + +func (c *GjsonMetric) val2OrderedMap(v gjson.Result, typeinfo *model.TypeInfo) (m *model.OrderedMap) { + m = model.NewOrderedMap() + v.ForEach(func(k, v gjson.Result) bool { + str := k.String() + rawKey := c.castResultByType(str, k, typeinfo.MapKey) + m.Put(rawKey, c.castResultByType(str, v, typeinfo.MapValue)) + return true + }) return } @@ -486,3 +367,199 @@ func gjDetectType(v gjson.Result, depth int) (typ int, array bool) { } return } + +func (c *GjsonMetric) castResultByType(sourcename string, value gjson.Result, typeinfo *model.TypeInfo) (val interface{}) { + if typeinfo.Array { + val = getGJsonArray(c, sourcename, value, typeinfo.Type) + return + } else { + switch typeinfo.Type { + case model.Bool: + val = getGJsonBool(value, typeinfo.Nullable) + case model.Int8: + val = GjsonGetInt[int8](c, value, typeinfo.Nullable, math.MinInt8, math.MaxInt8) + case model.Int16: + val = GjsonGetInt[int16](c, value, typeinfo.Nullable, math.MinInt16, math.MaxInt16) + case model.Int32: + val = GjsonGetInt[int32](c, value, typeinfo.Nullable, math.MinInt32, math.MaxInt32) + case model.Int64: + val = GjsonGetInt[int64](c, value, typeinfo.Nullable, math.MinInt64, math.MaxInt64) + case model.UInt8: + val = GjsonGetUint[uint8](c, value, typeinfo.Nullable, math.MaxUint8) + case model.UInt16: + val = GjsonGetUint[uint16](c, value, typeinfo.Nullable, math.MaxUint16) + case model.UInt32: + val = GjsonGetUint[uint32](c, value, typeinfo.Nullable, math.MaxUint32) + case model.UInt64: + val = GjsonGetUint[uint64](c, value, typeinfo.Nullable, math.MaxUint64) + case model.Float32: + val = GjsonGetFloat[float32](c, value, typeinfo.Nullable, math.MaxFloat32) + case model.Float64: + val = GjsonGetFloat[float64](c, value, typeinfo.Nullable, math.MaxFloat64) + case model.Decimal: + val = getGJsonDecimal(value, typeinfo.Nullable) + case model.DateTime: + val = getGJsonDateTime(c, sourcename, value, typeinfo.Nullable) + case model.String: + val = getGJsonString(value, typeinfo.Nullable) + case model.Map: + val = getGJsonMap(c, value, typeinfo) + default: + util.Logger.Fatal("LOGIC ERROR: reached switch default condition") + } + } + return +} + +func getGJsonDecimal(r gjson.Result, nullable bool) (val interface{}) { + if !gjCompatibleFloat(r) { + val = getDefaultDecimal(nullable) + return + } + switch r.Type { + case gjson.Number: + val = decimal.NewFromFloat(r.Num) + default: + val = getDefaultDecimal(nullable) + } + return +} + +func getGJsonBool(r gjson.Result, nullable bool) (val interface{}) { + if !gjCompatibleBool(r) { + val = getDefaultBool(nullable) + return + } + val = (r.Type == gjson.True) + return +} + +func getGJsonString(r gjson.Result, nullable bool) (val interface{}) { + if !r.Exists() || r.Type == gjson.Null { + if nullable { + return + } + val = "" + return + } + switch r.Type { + case gjson.Null: + val = "" + case gjson.String: + val = r.Str + default: + val = r.Raw + } + return +} + +func getGJsonDateTime(c *GjsonMetric, key string, r gjson.Result, nullable bool) (val interface{}) { + if !gjCompatibleDateTime(r) { + val = getDefaultDateTime(nullable) + return + } + switch r.Type { + case gjson.Number: + val = UnixFloat(r.Num, c.pp.timeUnit) + case gjson.String: + var err error + if val, err = c.pp.ParseDateTime(key, r.Str); err != nil { + val = getDefaultDateTime(nullable) + } + default: + val = getDefaultDateTime(nullable) + } + return +} + +func getGJsonArray(c *GjsonMetric, key string, r gjson.Result, typ int) (val interface{}) { + var array []gjson.Result + if r.IsArray() { + array = r.Array() + } + switch typ { + case model.Bool: + results := make([]bool, 0, len(array)) + for _, e := range array { + v := (e.Exists() && e.Type == gjson.True) + results = append(results, v) + } + val = results + case model.Int8: + val = GjsonIntArray[int8](array, math.MinInt8, math.MaxInt8) + case model.Int16: + val = GjsonIntArray[int16](array, math.MinInt16, math.MaxInt16) + case model.Int32: + val = GjsonIntArray[int32](array, math.MinInt32, math.MaxInt32) + case model.Int64: + val = GjsonIntArray[int64](array, math.MinInt64, math.MaxInt64) + case model.UInt8: + val = GjsonUintArray[uint8](array, math.MaxUint8) + case model.UInt16: + val = GjsonUintArray[uint16](array, math.MaxUint16) + case model.UInt32: + val = GjsonUintArray[uint32](array, math.MaxUint32) + case model.UInt64: + val = GjsonUintArray[uint64](array, math.MaxUint64) + case model.Float32: + val = GjsonFloatArray[float32](array, math.MaxFloat32) + case model.Float64: + val = GjsonFloatArray[float64](array, math.MaxFloat64) + case model.Decimal: + results := make([]decimal.Decimal, 0, len(array)) + var f float64 + for _, e := range array { + switch e.Type { + case gjson.Number: + f = e.Num + default: + f = float64(0.0) + } + results = append(results, decimal.NewFromFloat(f)) + } + val = results + case model.String: + results := make([]string, 0, len(array)) + var s string + for _, e := range array { + switch e.Type { + case gjson.Null: + s = "" + case gjson.String: + s = e.Str + default: + s = e.Raw + } + results = append(results, s) + } + val = results + case model.DateTime: + results := make([]time.Time, 0, len(array)) + var t time.Time + for _, e := range array { + switch e.Type { + case gjson.Number: + t = UnixFloat(e.Num, c.pp.timeUnit) + case gjson.String: + var err error + if t, err = c.pp.ParseDateTime(key, e.Str); err != nil { + t = Epoch + } + default: + t = Epoch + } + results = append(results, t) + } + val = results + default: + util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) + } + return +} + +func getGJsonMap(c *GjsonMetric, r gjson.Result, typeinfo *model.TypeInfo) (val interface{}) { + if r.Type == gjson.JSON { + val = c.val2OrderedMap(r, typeinfo) + } + return +} diff --git a/parser/parser_test.go b/parser/parser_test.go index 4eb9dd1e..fadf26fc 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -21,6 +21,7 @@ import ( "fmt" "log" "math" + "reflect" "strconv" "strings" "sync" @@ -67,7 +68,15 @@ var jsonSample = []byte(`{ "array_str_date_2": ["13/07/2009","14/07/2009","15/07/2009"], "array_str_time_rfc3339": ["2009-07-13T09:07:13Z", "2009-07-13T09:07:13+08:00", "2009-07-13T09:07:13.123Z", "2009-07-13T09:07:13.123+08:00"], "array_str_time_clickhouse": ["2009-07-13 09:07:13", "2009-07-13 09:07:13.123"], - "array_obj": [{"i":[1,2,3],"f":[1.1,2.2,3.3]},{"s":["aa","bb","cc"],"e":[]}] + "array_obj": [{"i":[1,2,3],"f":[1.1,2.2,3.3]},{"s":["aa","bb","cc"],"e":[]}], + "map_str_str": {"i":"first", "j":"second"}, + "map_str_uint": {"i":1, "j":2}, + "map_str_int": {"i":-1, "j":-2}, + "map_str_float": {"i":3.1415, "j":9.876}, + "map_str_bool": {"i":true, "j":false}, + "map_str_date": {"i":"2008-08-08", "j":"2022-01-01"}, + "map_str_array": {"i":[1,2,3],"j":[4,5,6]}, + "map_str_map": {"i":{"i":1, "j":2}, "j":{"i":3, "j":4}} }`) var jsonSchema = map[string]string{ @@ -102,6 +111,14 @@ var jsonSchema = map[string]string{ "array_str_time_rfc3339": "DateTimeArray", "array_str_time_clickhouse": "DateTimeArray", "array_obj": "Object('json')Array", + "map_str_str": "Object('json')", + "map_str_uint": "Object('json')", + "map_str_int": "Object('json')", + "map_str_float": "Object('json')", + "map_str_bool": "Object('json')", + "map_str_date": "Object('json')", + "map_str_array": "Object('json')", + "map_str_map": "Object('json')", } var csvSample = []byte(`null,true,false,123,123.321,kube-state-metrics,"escaped_""ws",123,123.321,2009-07-13,13/07/2009,2009-07-13T09:07:13Z,2009-07-13T09:07:13.123+08:00,2009-07-13 09:07:13,2009-07-13 09:07:13.123,"{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",[],[null],"[true,false]","[0,255,256,65535,65536,4294967295,4294967296,18446744073709551615,18446744073709551616]","[-9223372036854775808,-2147483649,-2147483648,-32769,-32768,-129,-128,0,127,128,32767,32768,2147483647,2147483648,9223372036854775807]","[4.940656458412465441765687928682213723651e-324,1.401298464324817070923729583289916131280e-45,0.0,3.40282346638528859811704183484516925440e+38,1.797693134862315708145274237317043567981e+308]","[""aa"",""bb"",""cc""]","[""0"",""255"",""256"",""65535"",""65536"",""4294967295"",""4294967296"",""18446744073709551615"",""18446744073709551616""]","[""-9223372036854775808"",""-2147483649"",""-2147483648"",""-32769"",""-32768"",""-129"",""-128"",""0"",""127"",""128"",""32767"",""32768"",""2147483647"",""2147483648"",""9223372036854775807""]","[""4.940656458412465441765687928682213723651e-324"",""1.401298464324817070923729583289916131280e-45"",""0.0"",""3.40282346638528859811704183484516925440e+38"",""1.797693134862315708145274237317043567981e+308""]","[""2009-07-13"",""2009-07-14"",""2009-07-15""]","[""13/07/2009"",""14/07/2009"",""15/07/2009""]","[""2009-07-13T09:07:13Z"",""2009-07-13T09:07:13+08:00"",""2009-07-13T09:07:13.123Z"",""2009-07-13T09:07:13.123+08:00""]","[""2009-07-13 09:07:13"",""2009-07-13 09:07:13.123""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]"`) @@ -171,6 +188,12 @@ type ArrayCase struct { ExpVal interface{} } +type MapCase struct { + field string + typ *model.TypeInfo + expVal interface{} +} + type DateTimeCase struct { TS string ExpVal time.Time @@ -543,6 +566,90 @@ func TestParserArray(t *testing.T) { } } +func TestParserMap(t *testing.T) { + initialize.Do(initMetrics) + require.Nil(t, errInit) + + testCases := []MapCase{ + {"map_str_str", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.String}, MapValue: &model.TypeInfo{Type: model.String}}, map[string]string{"i": "first", "j": "second"}}, + {"map_str_uint", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.String}, MapValue: &model.TypeInfo{Type: model.UInt64}}, map[string]uint64{"i": 1, "j": 2}}, + {"map_str_int", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.String}, MapValue: &model.TypeInfo{Type: model.Int64}}, map[string]int64{"i": -1, "j": -2}}, + {"map_str_float", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.String}, MapValue: &model.TypeInfo{Type: model.Float64}}, map[string]float64{"i": 3.1415, "j": 9.876}}, + {"map_str_bool", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.String}, MapValue: &model.TypeInfo{Type: model.Bool}}, map[string]bool{"i": true, "j": false}}, + {"map_str_date", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.String}, MapValue: &model.TypeInfo{Type: model.DateTime}}, map[string]time.Time{"i": time.Date(2008, 8, 8, 0, 0, 0, 0, time.Local).UTC(), "j": time.Date(2022, 1, 1, 0, 0, 0, 0, time.Local).UTC()}}, + {"map_str_array", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.String}, MapValue: &model.TypeInfo{Type: model.UInt64, Array: true}}, map[string][]uint64{"i": {1, 2, 3}, "j": {4, 5, 6}}}, + {"map_str_map", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.String}, MapValue: &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.String}, MapValue: &model.TypeInfo{Type: model.UInt64}}}, map[string]map[string]uint64{"i": {"i": 1, "j": 2}, "j": {"i": 3, "j": 4}}}, + } + for _, name := range []string{"fastjson", "gjson"} { + metric := metrics[name] + for _, it := range testCases { + desc := fmt.Sprintf(`%s.GetMap("%s", %s)`, name, it.field, model.GetTypeName(it.typ.Type)) + + orderMap := metric.GetMap(it.field, it.typ) + compareMap(t, orderMap, it.expVal, desc) + } + } +} + +func compareMap(t *testing.T, map1 interface{}, map2 interface{}, desc string) { + oMap, ok := map1.(*model.OrderedMap) + assert.True(t, ok, desc) + map1 = oMap.GetValues() + + value1 := reflect.ValueOf(map1) + value2 := reflect.ValueOf(map2) + assert.Equal(t, value1.IsNil(), value2.IsNil()) + assert.Equal(t, value1.Len(), value2.Len()) + + if value1.Kind() == reflect.Ptr { + value1 = value1.Elem() + value2 = value2.Elem() + } + assert.Equal(t, value1.Kind(), reflect.Map, fmt.Sprintf("ToMap only accepts struct or struct pointer; got %T", value1)) + assert.Equal(t, value2.Kind(), reflect.Map, fmt.Sprintf("ToMap only accepts struct or struct pointer; got %T", value2)) + + // v1 - map[interface{}]interface{}, v2 could be any map type + var compareValueFunc func(v1, v2 reflect.Value) = func(v1, v2 reflect.Value) { + switch v2.Kind() { + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64: + assert.Equal(t, v1.Interface().(int64), v2.Int()) + case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr: + assert.Equal(t, v1.Interface().(uint64), v2.Uint()) + case reflect.String: + assert.Equal(t, v1.Interface().(string), v2.String()) + case reflect.Bool: + assert.Equal(t, v1.Interface().(bool), v2.Bool()) + case reflect.Float32, reflect.Float64: + assert.Equal(t, v1.Interface().(float64), v2.Float()) + case reflect.Map: + compareMap(t, v1.Interface(), v2.Interface(), desc) + case reflect.Array: + fallthrough + case reflect.Slice: + if v2.Len() == 0 { + return + } + // didn't find a good way to convert interface{} to []Type + switch v2.Index(0).Kind() { + case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr: + array1 := v1.Interface().([]uint64) + assert.Equal(t, len(array1), v2.Len()) + for i := 0; i < v2.Len(); i++ { + assert.Equal(t, array1[i], v2.Index(i).Uint()) + } + default: + assert.Fail(t, "uncovered array type comparison, update compareValueFunc accordingly") + } + default: + // Normal equality suffices + assert.Equal(t, v1.Interface(), v2.Interface()) + } + } + for _, key := range value2.MapKeys() { + compareValueFunc(value1.MapIndex(key), value2.MapIndex(key)) + } +} + func TestParseDateTime(t *testing.T) { // https://en.wikipedia.org/wiki/List_of_tz_database_time_zones // https://en.wikipedia.org/wiki/List_of_time_zone_abbreviations, "not part of the international time and date standard ISO 8601 and their use as sole designator for a time zone is discouraged." diff --git a/task/sharding.go b/task/sharding.go index 4c80ddcd..3f06355d 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -27,12 +27,12 @@ func NewShardingPolicy(shardingKey string, shardingStripe uint64, dims []*model. colSeq := -1 for i, dim := range dims { if dim.Name == shardingKey { - if dim.Nullable || dim.Array { + if dim.Type.Nullable || dim.Type.Array { err = errors.Newf("invalid shardingKey %s, expect its type be numerical or string", shardingKey) return } colSeq = i - switch dim.Type { + switch dim.Type.Type { case model.Int8, model.Int16, model.Int32, model.Int64, model.UInt8, model.UInt16, model.UInt32, model.UInt64, model.Float32, model.Float64, model.Decimal, model.DateTime: //numerical if policy.stripe <= 0 { diff --git a/task/task.go b/task/task.go index da623743..9b8fcc26 100644 --- a/task/task.go +++ b/task/task.go @@ -447,7 +447,7 @@ func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) dim := service.dims[i] val := model.GetValueByType(metric, dim) *row = append(*row, val) - if val != nil && dim.Type == model.String && dim.Name != service.nameKey && dim.Name != "le" && (service.lblBlkList == nil || !service.lblBlkList.MatchString(dim.Name)) { + if val != nil && dim.Type.Type == model.String && dim.Name != service.nameKey && dim.Name != "le" && (service.lblBlkList == nil || !service.lblBlkList.MatchString(dim.Name)) { // "labels" JSON excludes "le", so that "labels" can be used as group key for histogram queries. labelVal := val.(string) labels = append(labels, fmt.Sprintf(`%s: %s`, strconv.Quote(dim.Name), strconv.Quote(labelVal))) From e12a7b4e7bb7fba97dad3fb4859ca1a9ba83c1ff Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 9 Dec 2022 22:26:51 +0800 Subject: [PATCH 298/404] systest pass --- Makefile | 2 +- docker/test_auto_schema.hjson | 24 ++++++++---------------- docs/guide/run.md | 4 ++-- go.test.sh | 12 ++++++------ output/clickhouse.go | 12 +++++------- output/clickhouse_util.go | 4 ++-- 6 files changed, 24 insertions(+), 34 deletions(-) diff --git a/Makefile b/Makefile index fd785f50..acdcdacb 100644 --- a/Makefile +++ b/Makefile @@ -21,4 +21,4 @@ systest: build lint: golangci-lint run --timeout=3m run: pre - go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.json + go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.hjson diff --git a/docker/test_auto_schema.hjson b/docker/test_auto_schema.hjson index b7d95346..c4557e94 100644 --- a/docker/test_auto_schema.hjson +++ b/docker/test_auto_schema.hjson @@ -2,34 +2,26 @@ clickhouse: { hosts: [ [ - 192.168.110.6 - # 192.168.110.8 - ] - [ - 192.168.110.10 - 192.168.110.12 - ] - [ - 192.168.110.14 + 127.0.0.1 ] ] - port: 19000 + port: 9000 db: default username: "" - password: 123456 + password: "" retryTimes: 0 } kafka: { - brokers: 192.168.110.12:9092, 192.168.110.8:9092, 192.168.110.16:9092 + brokers: 127.0.0.1:9092 } task: { - name: test_ck_query - topic: kafka_gen_ckdata22 - consumerGroup: test_ck_query + name: test_auto_schema + topic: topic1 + consumerGroup: test_auto_schema earliest: true parser: json autoSchema: true - tableName: test_ck_query_lowcardinality_r1 + tableName: test_auto_schema excludeColumns: [] bufferSize: 50000 } diff --git a/docs/guide/run.md b/docs/guide/run.md index 38e7ed21..bef02631 100644 --- a/docs/guide/run.md +++ b/docs/guide/run.md @@ -10,7 +10,7 @@ Note: Ensure `clickhouse-server` and `kafka` work before running clickhouse_sink - For local file: - `clickhouse_sinker --local-cfg-file docker/test_auto_schema.json` + `clickhouse_sinker --local-cfg-file docker/test_auto_schema.hjson` - For Nacos: @@ -71,7 +71,7 @@ Let's follow up a piece of the systest script. * Run clickhouse_sinker ```bash - $ ./clickhouse_sinker --local-cfg-file docker/test_auto_schema.json + $ ./clickhouse_sinker --local-cfg-file docker/test_auto_schema.hjson ``` diff --git a/go.test.sh b/go.test.sh index 9fcdc43a..6fbef090 100755 --- a/go.test.sh +++ b/go.test.sh @@ -48,9 +48,9 @@ sudo docker cp send.sh kafka:/tmp/ sudo docker exec kafka sh /tmp/send.sh echo "start clickhouse_sinker to consume" -timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_fixed_schema.json -timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_auto_schema.json -timeout 60 ./clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.json +timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_fixed_schema.hjson +timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_auto_schema.hjson +timeout 60 ./clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.hjson echo "check result 1" count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` @@ -74,9 +74,9 @@ curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' curl "localhost:8123" -d 'TRUNCATE TABLE test_dynamic_schema' echo "publish clickhouse_sinker config" -./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema --local-cfg-file docker/test_fixed_schema.json -./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema --local-cfg-file docker/test_auto_schema.json -./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.json +./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema --local-cfg-file docker/test_fixed_schema.hjson +./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema --local-cfg-file docker/test_auto_schema.hjson +./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.hjson echo "start clickhouse_sinker to consume" sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_fixed_schema --all-topics --to-earliest diff --git a/output/clickhouse.go b/output/clickhouse.go index 2461ad90..91008e07 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -83,16 +83,14 @@ func NewClickHouse(cfg *config.Config, taskCfg *config.TaskConfig) *ClickHouse { // Init the clickhouse intance func (c *ClickHouse) Init() (err error) { + // FIXME: goroutine leak? go func() { ticker := time.NewTicker(10 * time.Second) defer ticker.Stop() - for { - select { - case <-ticker.C: - c.mux.Lock() - c.wrSeries = 0 - c.mux.Unlock() - } + for range ticker.C { + c.mux.Lock() + c.wrSeries = 0 + c.mux.Unlock() } }() return c.initSchema() diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 16f691d4..d213609a 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -65,14 +65,14 @@ func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn cl } if err = batch.Send(); err != nil { err = errors.Wrapf(err, "driver.Batch.Send") - batch.Abort() + _ = batch.Abort() return } return } if err = batch.Send(); err != nil { err = errors.Wrapf(err, "driver.Batch.Send") - batch.Abort() + _ = batch.Abort() return } return From 827b20bd8bce08fa1b10cdc477edfe24d610c3ab Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 9 Dec 2022 22:47:42 +0800 Subject: [PATCH 299/404] fix ClickHouse.Init goroutine leak --- output/clickhouse.go | 31 ++++++++++++++++--------------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 91008e07..a3313247 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -67,32 +67,24 @@ type ClickHouse struct { distMetricTbls []string distSeriesTbls []string - bmSeries map[int64]int64 - wrSeries int - numFlying int32 - mux sync.Mutex - taskDone *sync.Cond + bmSeries map[int64]int64 + wrSeries int + numFlying int32 + mux sync.Mutex + taskDone *sync.Cond + nextResetQuota time.Time } // NewClickHouse new a clickhouse instance func NewClickHouse(cfg *config.Config, taskCfg *config.TaskConfig) *ClickHouse { ck := &ClickHouse{cfg: cfg, taskCfg: taskCfg} ck.taskDone = sync.NewCond(&ck.mux) + ck.nextResetQuota = time.Now().Add(10 * time.Second) return ck } // Init the clickhouse intance func (c *ClickHouse) Init() (err error) { - // FIXME: goroutine leak? - go func() { - ticker := time.NewTicker(10 * time.Second) - defer ticker.Stop() - for range ticker.C { - c.mux.Lock() - c.wrSeries = 0 - c.mux.Unlock() - } - }() return c.initSchema() } @@ -137,6 +129,15 @@ func (c *ClickHouse) AllowWriteSeries(sid, mid int64) (allowed bool) { if c.wrSeries < wrSeriesQuota { c.wrSeries++ allowed = true + } else { + now := time.Now() + if now.After(c.nextResetQuota) { + c.nextResetQuota = now.Add(10 * time.Second) + c.wrSeries = 1 + allowed = true + } + } + if allowed { statistics.WriteSeriesAllowChanged.WithLabelValues(c.taskCfg.Name).Inc() } else { statistics.WriteSeriesDropQuota.WithLabelValues(c.taskCfg.Name).Inc() From 932f50ccdd9e5d7f32419a0c6c552c3fe886b490 Mon Sep 17 00:00:00 2001 From: Geoff Genz Date: Thu, 8 Dec 2022 16:19:33 -0700 Subject: [PATCH 300/404] Small updates to allow TLS connections for AWS MSK, etc. --- cmd/clickhouse_sinker/main.go | 4 ++-- config/config.go | 12 +++++++++--- input/kafka_franz.go | 5 +++-- pool/conn.go | 13 ++++++++----- util/common.go | 20 +++++++++++--------- 5 files changed, 33 insertions(+), 21 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 4ed773d4..2090e804 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -420,7 +420,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { // 1. Initialize clickhouse connections chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { + chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.DialTimeout); err != nil { return } @@ -456,7 +456,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 2. Initialize clickhouse connections. chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { + chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.DialTimeout); err != nil { return } diff --git a/config/config.go b/config/config.go index 064bfead..d0996bf4 100644 --- a/config/config.go +++ b/config/config.go @@ -44,7 +44,7 @@ type KafkaConfig struct { Security map[string]string TLS struct { Enable bool - CaCertFiles string // Required. It's the CA cert.pem with which Kafka brokers certs be signed. + CaCertFiles string // CA cert.pem with which Kafka brokers certs be signed. Leave empty for certificates trusted by the OS ClientCertFile string // Required for client authentication. It's client cert.pem. ClientKeyFile string // Required if and only if ClientCertFile is present. It's client key.pem. @@ -97,9 +97,10 @@ type ClickHouseConfig struct { RetryTimes int //<=0 means retry infinitely MaxOpenConns int + DialTimeout int // Connection dial timeout in seconds } -// Task configuration parameters +// TaskConfig parameters type TaskConfig struct { Name string @@ -170,6 +171,7 @@ const ( defaultLogLevel = "info" defaultKerberosConfigPath = "/etc/krb5.conf" defaultMaxOpenConns = 1 + defaultDialTimeout = 2 ) func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { @@ -187,7 +189,7 @@ func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { return } -// normallize and validate configuration +// Normalize and validate configuration func (cfg *Config) Normallize() (err error) { if len(cfg.Clickhouse.Hosts) == 0 || cfg.Kafka.Brokers == "" { err = errors.Newf("invalid configuration") @@ -230,6 +232,10 @@ func (cfg *Config) Normallize() (err error) { cfg.Clickhouse.MaxOpenConns = defaultMaxOpenConns } + if cfg.Clickhouse.DialTimeout <= 0 { + cfg.Clickhouse.DialTimeout = defaultDialTimeout + } + if cfg.Task != nil { cfg.Tasks = append(cfg.Tasks, cfg.Task) cfg.Task = nil diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 4f3d5b4d..ea85463a 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -97,8 +97,9 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { opts = []kgo.Opt{ kgo.SeedBrokers(strings.Split(kfkCfg.Brokers, ",")...), - kgo.FetchMaxBytes(1 << 27), //134 MB - kgo.BrokerMaxReadBytes(1 << 27), //134 MB + kgo.DisableAutoCommit(), + kgo.FetchMaxBytes(20971520), // 20 MB -- Larger numbers are likely to cause OOM. Should be configurable + kgo.BrokerMaxReadBytes(20971520), //kgo.MetadataMaxAge(...) corresponds to sarama.Config.Metadata.RefreshFrequency kgo.WithLogger(kzap.New(util.Logger)), } diff --git a/pool/conn.go b/pool/conn.go index 13bd8e47..6c76d3ca 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -104,7 +104,8 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db clickhouse.Conn, dbVer i // Each shard has a clickhouse.Conn which connects to one replica inside the shard. // We need more control than replica single-point-failure. -func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool, maxOpenConns int) (err error) { +func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool, + maxOpenConns int, dialTimeout int) (err error) { lock.Lock() defer lock.Unlock() freeClusterConn() @@ -113,8 +114,11 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara numReplicas := len(replicas) replicaAddrs := make([]string, numReplicas) for i, ip := range replicas { - if ips2, err := util.GetIP4Byname(ip); err == nil { - ip = ips2[0] + // Changing hostnames to IPs breaks TLS connections in many cases + if !secure { + if ips2, err := util.GetIP4Byname(ip); err == nil { + ip = ips2[0] + } } replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) } @@ -126,8 +130,7 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara Username: username, Password: password, }, - //Debug: true, - DialTimeout: time.Second, + DialTimeout: time.Second * time.Duration(dialTimeout), MaxOpenConns: maxOpenConns, MaxIdleConns: 1, ConnMaxLifetime: time.Hour, diff --git a/util/common.go b/util/common.go index e69cd38e..ca38a5b8 100644 --- a/util/common.go +++ b/util/common.go @@ -165,17 +165,19 @@ func NewTLSConfig(caCertFiles, clientCertFile, clientKeyFile string, insecureSki tlsConfig.Certificates = []tls.Certificate{cert} } - // Load CA cert - caCertPool := x509.NewCertPool() - for _, caCertFile := range strings.Split(caCertFiles, ",") { - caCert, err := os.ReadFile(caCertFile) - if err != nil { - err = errors.Wrapf(err, "") - return &tlsConfig, err + // Load CA cert if it exists. Not needed for OS trusted certs + if caCertFiles != "" { + caCertPool := x509.NewCertPool() + for _, caCertFile := range strings.Split(caCertFiles, ",") { + caCert, err := os.ReadFile(caCertFile) + if err != nil { + err = errors.Wrapf(err, "") + return &tlsConfig, err + } + caCertPool.AppendCertsFromPEM(caCert) } - caCertPool.AppendCertsFromPEM(caCert) + tlsConfig.RootCAs = caCertPool } - tlsConfig.RootCAs = caCertPool tlsConfig.InsecureSkipVerify = insecureSkipVerify return &tlsConfig, nil } From 6a5283378a01a7cfac384aab9073b9c8a0da1941 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Mon, 12 Dec 2022 10:55:09 +0800 Subject: [PATCH 301/404] Fix - invalid autocommit options specified when a group was not specified --- input/kafka_franz.go | 1 - 1 file changed, 1 deletion(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index ea85463a..923f0b68 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -97,7 +97,6 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { opts = []kgo.Opt{ kgo.SeedBrokers(strings.Split(kfkCfg.Brokers, ",")...), - kgo.DisableAutoCommit(), kgo.FetchMaxBytes(20971520), // 20 MB -- Larger numbers are likely to cause OOM. Should be configurable kgo.BrokerMaxReadBytes(20971520), //kgo.MetadataMaxAge(...) corresponds to sarama.Config.Metadata.RefreshFrequency From b452c6df2c15b8024c0d54a2f13c735d7a8e86e5 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sun, 18 Dec 2022 16:51:00 +0800 Subject: [PATCH 302/404] format hjson files --- docker/test_auto_schema.hjson | 54 ++++++++++----------- docker/test_dynamic_schema.hjson | 66 +++++++++++++------------- docker/test_fixed_schema.hjson | 80 ++++++++++++++++---------------- 3 files changed, 100 insertions(+), 100 deletions(-) diff --git a/docker/test_auto_schema.hjson b/docker/test_auto_schema.hjson index c4557e94..efc3c7c8 100644 --- a/docker/test_auto_schema.hjson +++ b/docker/test_auto_schema.hjson @@ -1,29 +1,29 @@ { - clickhouse: { - hosts: [ - [ - 127.0.0.1 - ] - ] - port: 9000 - db: default - username: "" - password: "" - retryTimes: 0 - } - kafka: { - brokers: 127.0.0.1:9092 - } - task: { - name: test_auto_schema - topic: topic1 - consumerGroup: test_auto_schema - earliest: true - parser: json - autoSchema: true - tableName: test_auto_schema - excludeColumns: [] - bufferSize: 50000 - } - logLevel: info + clickhouse: { + hosts: [ + [ + 127.0.0.1 + ] + ] + port: 9000 + db: default + username: "" + password: "" + retryTimes: 0 + } + kafka: { + brokers: 127.0.0.1:9092 + } + task: { + name: test_auto_schema + topic: topic1 + consumerGroup: test_auto_schema + earliest: true + parser: json + autoSchema: true + tableName: test_auto_schema + excludeColumns: [] + bufferSize: 50000 + } + logLevel: info } \ No newline at end of file diff --git a/docker/test_dynamic_schema.hjson b/docker/test_dynamic_schema.hjson index f10b0386..7840d82d 100644 --- a/docker/test_dynamic_schema.hjson +++ b/docker/test_dynamic_schema.hjson @@ -1,36 +1,36 @@ { - clickhouse: { - cluster: "" - hosts: [ - [ - 127.0.0.1 - ] - ] - port: 9000 - db: default - username: "" - password: "" - retryTimes: 0 - } - kafka: { - brokers: 127.0.0.1:9093 - version: 2.6.0 - } - task: { - name: test_dynamic_schema - topic: topic1 - consumerGroup: test_dynamic_schema - earliest: true - parser: json - autoSchema: true - tableName: test_dynamic_schema - excludeColumns: [ - day - ] - dynamicSchema: { - enable: true + clickhouse: { + cluster: "" + hosts: [ + [ + 127.0.0.1 + ] + ] + port: 9000 + db: default + username: "" + password: "" + retryTimes: 0 } - bufferSize: 50000 - } - logLevel: info + kafka: { + brokers: 127.0.0.1:9093 + version: 2.6.0 + } + task: { + name: test_dynamic_schema + topic: topic1 + consumerGroup: test_dynamic_schema + earliest: true + parser: json + autoSchema: true + tableName: test_dynamic_schema + excludeColumns: [ + day + ] + dynamicSchema: { + enable: true + } + bufferSize: 50000 + } + logLevel: info } \ No newline at end of file diff --git a/docker/test_fixed_schema.hjson b/docker/test_fixed_schema.hjson index 9cb0a5ab..aa5f2d6f 100644 --- a/docker/test_fixed_schema.hjson +++ b/docker/test_fixed_schema.hjson @@ -1,42 +1,42 @@ { - clickhouse: { - hosts: [ - [ - 127.0.0.1 - ] - ] - port: 9000 - db: default - username: "" - password: "" - retryTimes: 0 - } - kafka: { - brokers: 127.0.0.1:9093 - version: 2.5.0 - } - task: { - name: test_fixed_schema - topic: topic1 - consumerGroup: test_fixed_schema - earliest: true - parser: json - tableName: test_fixed_schema - dims: [ - { - name: time - type: DateTime - } - { - name: name - type: String - } - { - name: value - type: Float32 - } - ] - bufferSize: 50000 - } - logLevel: info + clickhouse: { + hosts: [ + [ + 127.0.0.1 + ] + ] + port: 9000 + db: default + username: "" + password: "" + retryTimes: 0 + } + kafka: { + brokers: 127.0.0.1:9093 + version: 2.5.0 + } + task: { + name: test_fixed_schema + topic: topic1 + consumerGroup: test_fixed_schema + earliest: true + parser: json + tableName: test_fixed_schema + dims: [ + { + name: time + type: DateTime + } + { + name: name + type: String + } + { + name: value + type: Float32 + } + ] + bufferSize: 50000 + } + logLevel: info } \ No newline at end of file From 0a4add97d44e7782154ed8a4da2575fa44f77719 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 23 Dec 2022 13:54:59 +0800 Subject: [PATCH 303/404] Group the tasks by consumerGroup property to reduce number of kafka client --- cmd/clickhouse_sinker/main.go | 332 +------------------------ config/config.go | 6 - docker/test_auto_schema.hjson | 1 + docker/test_dynamic_schema.hjson | 1 + docker/test_fixed_schema.hjson | 1 + go.mod | 28 +-- go.sum | 68 +---- input/input.go | 36 --- input/kafka_franz.go | 76 +++--- input/kafka_go.go | 188 -------------- input/kafka_sarama.go | 226 ----------------- model/message.go | 95 +------ output/clickhouse.go | 35 +-- statistics/statistics.go | 2 +- task/consumer.go | 251 +++++++++++++++++++ task/ring.go | 229 ----------------- task/sharding.go | 99 ++------ task/sinker.go | 413 +++++++++++++++++++++++++++++++ task/task.go | 351 ++++++-------------------- util/common.go | 46 ++-- 20 files changed, 861 insertions(+), 1623 deletions(-) delete mode 100644 input/input.go delete mode 100644 input/kafka_go.go delete mode 100644 input/kafka_sarama.go create mode 100644 task/consumer.go delete mode 100644 task/ring.go create mode 100644 task/sinker.go diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 2090e804..df75fa48 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -16,7 +16,6 @@ limitations under the License. package main import ( - "context" "encoding/json" "flag" "fmt" @@ -26,17 +25,10 @@ import ( "net/http/pprof" "os" "path/filepath" - "reflect" - "sort" "strings" - "sync" - "time" - "github.com/housepower/clickhouse_sinker/config" cm "github.com/housepower/clickhouse_sinker/config_manager" "github.com/housepower/clickhouse_sinker/health" - "github.com/housepower/clickhouse_sinker/pool" - "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/task" "github.com/housepower/clickhouse_sinker/util" "go.uber.org/zap" @@ -45,23 +37,6 @@ import ( "github.com/prometheus/client_golang/prometheus/promhttp" ) -type CmdOptions struct { - ShowVer bool - LogLevel string // "debug", "info", "warn", "error", "dpanic", "panic", "fatal" - LogPaths string // comma-separated paths. "stdout" means the console stdout - HTTPPort int // 0 menas a randomly OS chosen port - PushGatewayAddrs string - PushInterval int - LocalCfgFile string - NacosAddr string - NacosNamespaceID string - NacosGroup string - NacosUsername string - NacosPassword string - NacosDataID string - NacosServiceName string // participate in assignment management if not empty -} - var ( //goreleaser fill following info per https://goreleaser.com/customization/build/. version = "None" @@ -69,16 +44,16 @@ var ( date = "None" builtBy = "None" - cmdOps CmdOptions + cmdOps util.CmdOptions selfIP string httpAddr string httpMetrics = promhttp.Handler() - runner *Sinker + runner *task.Sinker ) func initCmdOptions() { // 1. Set options to default value. - cmdOps = CmdOptions{ + cmdOps = util.CmdOptions{ ShowVer: false, LogLevel: "info", LogPaths: "stdout,clickhouse_sinker.log", @@ -174,11 +149,11 @@ func main() { mux.HandleFunc("/state", func(w http.ResponseWriter, r *http.Request) { w.Header().Set("Content-Type", "application/json") - if runner != nil && runner.curCfg != nil { + if runner != nil && runner.GetCurrentConfig() != nil { var stateLags map[string]cm.StateLag var bs []byte var err error - if stateLags, err = cm.GetTaskStateAndLags(runner.curCfg); err == nil { + if stateLags, err = cm.GetTaskStateAndLags(runner.GetCurrentConfig()); err == nil { if bs, err = json.Marshal(stateLags); err == nil { _, _ = w.Write(bs) } @@ -250,7 +225,7 @@ func main() { } } } - runner = NewSinker(rcm) + runner = task.NewSinker(rcm, httpAddr, &cmdOps) return runner.Init() }, func() error { runner.Run() @@ -260,298 +235,3 @@ func main() { return nil }) } - -// Sinker object maintains number of task for each partition -type Sinker struct { - curCfg *config.Config - numCfg int - pusher *statistics.Pusher - tasks map[string]*task.Service - rcm cm.RemoteConfManager - ctx context.Context - cancel context.CancelFunc - stopped chan struct{} -} - -// NewSinker get an instance of sinker with the task list -func NewSinker(rcm cm.RemoteConfManager) *Sinker { - ctx, cancel := context.WithCancel(context.Background()) - s := &Sinker{ - tasks: make(map[string]*task.Service), - rcm: rcm, - ctx: ctx, - cancel: cancel, - stopped: make(chan struct{}), - } - return s -} - -func (s *Sinker) Init() (err error) { - return -} - -// Run is the mainloop to get and apply config -func (s *Sinker) Run() { - var err error - var newCfg *config.Config - defer func() { - s.stopped <- struct{}{} - }() - if cmdOps.PushGatewayAddrs != "" { - addrs := strings.Split(cmdOps.PushGatewayAddrs, ",") - s.pusher = statistics.NewPusher(addrs, cmdOps.PushInterval, httpAddr) - if err = s.pusher.Init(); err != nil { - return - } - go s.pusher.Run() - } - if s.rcm == nil { - if _, err = os.Stat(cmdOps.LocalCfgFile); err == nil { - if newCfg, err = config.ParseLocalCfgFile(cmdOps.LocalCfgFile); err != nil { - util.Logger.Fatal("config.ParseLocalCfgFile failed", zap.Error(err)) - return - } - } else { - util.Logger.Fatal("expect --local-cfg-file or --nacos-dataid") - return - } - if err = newCfg.Normallize(); err != nil { - util.Logger.Fatal("newCfg.Normallize failed", zap.Error(err)) - return - } - if err = s.applyConfig(newCfg); err != nil { - util.Logger.Fatal("s.applyConfig failed", zap.Error(err)) - return - } - <-s.ctx.Done() - } else { - if cmdOps.NacosServiceName != "" { - go s.rcm.Run() - } - // Golang <-time.After() is not garbage collected before expiry. - ticker := time.NewTicker(10 * time.Second) - defer ticker.Stop() - for { - select { - case <-s.ctx.Done(): - util.Logger.Info("Sinker.Run quit due to context has been canceled") - return - case <-ticker.C: - if newCfg, err = s.rcm.GetConfig(); err != nil { - util.Logger.Error("s.rcm.GetConfig failed", zap.Error(err)) - continue - } - if err = newCfg.Normallize(); err != nil { - util.Logger.Error("newCfg.Normallize failed", zap.Error(err)) - continue - } - if err = s.applyConfig(newCfg); err != nil { - util.Logger.Error("s.applyConfig failed", zap.Error(err)) - continue - } - } - } - } -} - -// Close shutdown task -func (s *Sinker) Close() { - // 1. Stop rcm - if s.rcm != nil { - s.rcm.Stop() - s.rcm = nil - } - // 2. Quit Run mainloop - s.cancel() - <-s.stopped - // 3. Stop tasks gracefully. - s.stopAllTasks() - // 4. Stop pusher - if s.pusher != nil { - s.pusher.Stop() - s.pusher = nil - } -} - -func (s *Sinker) stopAllTasks() { - var wg sync.WaitGroup - for _, tsk := range s.tasks { - wg.Add(1) - go func(tsk *task.Service) { - tsk.Stop() - wg.Done() - }(tsk) - } - wg.Wait() - for taskName := range s.tasks { - delete(s.tasks, taskName) - } - util.Logger.Info("stopped all tasks") - if util.GlobalParsingPool != nil { - util.GlobalParsingPool.StopWait() - } - util.Logger.Info("stopped parsing pool") - if util.GlobalTimerWheel != nil { - util.GlobalTimerWheel.Stop() - } - util.Logger.Info("stopped timer wheel") - if util.GlobalWritingPool != nil { - util.GlobalWritingPool.StopWait() - } - util.Logger.Info("stopped writing pool") -} - -func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { - util.SetLogLevel(newCfg.LogLevel) - if s.curCfg == nil { - // The first time invoking of applyConfig - err = s.applyFirstConfig(newCfg) - } else if !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || - !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || - !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) || - !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { - err = s.applyAnotherConfig(newCfg) - } - return -} - -func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { - util.Logger.Info("going to apply the first config", zap.Reflect("config", newCfg)) - // 1. Initialize clickhouse connections - chCfg := &newCfg.Clickhouse - if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.DialTimeout); err != nil { - return - } - - // 2. Start goroutine pools. - util.InitGlobalTimerWheel() - util.InitGlobalParsingPool() - util.InitGlobalWritingPool(len(chCfg.Hosts) * chCfg.MaxOpenConns) - - // 3. Generate, initialize and run task - for _, taskCfg := range newCfg.Tasks { - if cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(httpAddr, taskCfg.Name) { - continue - } - task := task.NewTaskService(newCfg, taskCfg) - if err = task.Init(); err != nil { - return - } - s.tasks[taskCfg.Name] = task - } - for _, task := range s.tasks { - go task.Run() - } - s.curCfg = newCfg - util.Logger.Info("applied the first config") - return -} - -func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { - util.Logger.Info("going to apply another config", zap.Int("number", s.numCfg), zap.Reflect("config", newCfg)) - if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) { - // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). - s.stopAllTasks() - // 2. Initialize clickhouse connections. - chCfg := &newCfg.Clickhouse - if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.DialTimeout); err != nil { - return - } - - // 3. Restart goroutine pools. - util.Logger.Info("restarting parsing, writing and timer pool") - util.GlobalTimerWheel = nil - util.InitGlobalTimerWheel() - util.GlobalParsingPool.Restart() - maxWorkers := len(newCfg.Clickhouse.Hosts) * newCfg.Clickhouse.MaxOpenConns - util.GlobalWritingPool.Resize(maxWorkers) - util.GlobalWritingPool.Restart() - util.Logger.Info("resized writing pool", zap.Int("maxWorkers", maxWorkers)) - - // 4. Generate, initialize and run tasks. - var tasksToStart []string - for _, taskCfg := range newCfg.Tasks { - if cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(httpAddr, taskCfg.Name) { - continue - } - task := task.NewTaskService(newCfg, taskCfg) - if err = task.Init(); err != nil { - return - } - tasksToStart = append(tasksToStart, taskCfg.Name) - s.tasks[taskCfg.Name] = task - } - for _, task := range s.tasks { - go task.Run() - } - sort.Strings(tasksToStart) - util.Logger.Info("started tasks", zap.Reflect("tasks", tasksToStart)) - } else if !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) || !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { - //1. Find tasks need to stop. - var tasksToStop []string - curCfgTasks := make(map[string]*config.TaskConfig) - newCfgTasks := make(map[string]*config.TaskConfig) - for _, taskCfg := range s.curCfg.Tasks { - curCfgTasks[taskCfg.Name] = taskCfg - } - for _, taskCfg := range newCfg.Tasks { - if cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(httpAddr, taskCfg.Name) { - continue - } - newCfgTasks[taskCfg.Name] = taskCfg - } - for taskName := range s.tasks { - curTaskCfg := curCfgTasks[taskName] - newTaskCfg, ok := newCfgTasks[taskName] - if !ok || !reflect.DeepEqual(newTaskCfg, curTaskCfg) { - tasksToStop = append(tasksToStop, taskName) - } - } - sort.Strings(tasksToStop) - // 2. Stop tasks in parallel found at the previous step. - // They must drain flying batchs as quickly as possible to allow another clickhouse_sinker - // instance take over partitions safely. - var wg sync.WaitGroup - for _, taskName := range tasksToStop { - wg.Add(1) - go func(tsk *task.Service) { - tsk.Stop() - wg.Done() - }(s.tasks[taskName]) - } - wg.Wait() - for _, taskName := range tasksToStop { - delete(s.tasks, taskName) - } - util.Logger.Info("stopped tasks", zap.Reflect("tasks", tasksToStop)) - // 3. Initailize tasks which are new or their config differ. - var tasksToStart []string - var newTasks []*task.Service - for taskName, taskCfg := range newCfgTasks { - if _, ok := s.tasks[taskName]; ok { - continue - } - task := task.NewTaskService(newCfg, taskCfg) - if err = task.Init(); err != nil { - return - } - s.tasks[taskName] = task - tasksToStart = append(tasksToStart, taskName) - newTasks = append(newTasks, task) - } - - // 4. Start new tasks. We don't do it at step 3 in order to avoid goroutine leak due to errors raised by later steps. - for _, task := range newTasks { - go task.Run() - } - sort.Strings(tasksToStart) - util.Logger.Info("started tasks", zap.Reflect("tasks", tasksToStart)) - } - // Record the new config - s.curCfg = newCfg - util.Logger.Info("applied another config", zap.Int("number", s.numCfg)) - s.numCfg++ - return -} diff --git a/config/config.go b/config/config.go index d0996bf4..6d7f7fa0 100644 --- a/config/config.go +++ b/config/config.go @@ -254,12 +254,6 @@ func (cfg *Config) Normallize() (err error) { } func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { - if taskCfg.KafkaClient == "" { - // known limitations of kafka-go: - // - The Reader API is too high-level. There's no generation cleanup callback which sarama provides. - // - Doesn't support SASL/GSSAPI(Kerberos). https://github.com/segmentio/kafka-go/issues/539 - taskCfg.KafkaClient = "franz" - } if taskCfg.Parser == "" || taskCfg.Parser == "json" { taskCfg.Parser = "fastjson" } diff --git a/docker/test_auto_schema.hjson b/docker/test_auto_schema.hjson index efc3c7c8..59d5802d 100644 --- a/docker/test_auto_schema.hjson +++ b/docker/test_auto_schema.hjson @@ -22,6 +22,7 @@ parser: json autoSchema: true tableName: test_auto_schema + shardingKey: name excludeColumns: [] bufferSize: 50000 } diff --git a/docker/test_dynamic_schema.hjson b/docker/test_dynamic_schema.hjson index 7840d82d..cec71991 100644 --- a/docker/test_dynamic_schema.hjson +++ b/docker/test_dynamic_schema.hjson @@ -30,6 +30,7 @@ dynamicSchema: { enable: true } + shardingKey: name bufferSize: 50000 } logLevel: info diff --git a/docker/test_fixed_schema.hjson b/docker/test_fixed_schema.hjson index aa5f2d6f..8b23fd24 100644 --- a/docker/test_fixed_schema.hjson +++ b/docker/test_fixed_schema.hjson @@ -36,6 +36,7 @@ type: Float32 } ] + shardingKey: name bufferSize: 50000 } logLevel: info diff --git a/go.mod b/go.mod index 09d4b114..d495b5b9 100644 --- a/go.mod +++ b/go.mod @@ -5,10 +5,8 @@ go 1.19 require ( github.com/ClickHouse/clickhouse-go/v2 v2.4.3 github.com/RoaringBitmap/roaring v1.2.1 - github.com/Shopify/sarama v1.36.0 github.com/bytedance/sonic v1.5.0 github.com/cespare/xxhash/v2 v2.1.2 - github.com/fagongzi/goetty v1.7.0 github.com/google/gops v0.3.25 github.com/hjson/hjson-go/v4 v4.2.0 github.com/jcmturner/gokrb5/v8 v8.4.3 @@ -16,18 +14,16 @@ require ( github.com/nacos-group/nacos-sdk-go v1.1.2 github.com/prometheus/client_golang v1.13.0 github.com/prometheus/common v0.37.0 - github.com/segmentio/kafka-go v0.4.34 github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.8.1 github.com/thanos-io/thanos v0.27.0 github.com/tidwall/gjson v1.14.2 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.7.1 - github.com/twmb/franz-go/pkg/kadm v1.2.1 + github.com/twmb/franz-go v1.10.4 + github.com/twmb/franz-go/pkg/kadm v1.6.0 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.1 github.com/valyala/fastjson v1.6.3 - github.com/xdg-go/scram v1.1.1 go.uber.org/zap v1.23.0 golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e golang.org/x/time v0.0.0-20220722155302-e5dcc9cfc0b9 @@ -43,17 +39,11 @@ require ( github.com/buger/jsonparser v1.1.1 // indirect github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/eapache/go-resiliency v1.3.0 // indirect - github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 // indirect - github.com/eapache/queue v1.1.0 // indirect github.com/go-errors/errors v1.4.2 // indirect github.com/go-faster/city v1.0.1 // indirect github.com/go-faster/errors v0.6.1 // indirect github.com/golang/protobuf v1.5.2 // indirect - github.com/golang/snappy v0.0.4 // indirect github.com/google/uuid v1.3.0 // indirect - github.com/hashicorp/errwrap v1.1.0 // indirect - github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/jcmturner/aescts/v2 v2.0.0 // indirect github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect @@ -63,6 +53,7 @@ require ( github.com/json-iterator/go v1.1.12 // indirect github.com/klauspost/compress v1.15.12 // indirect github.com/klauspost/cpuid/v2 v2.1.1 // indirect + github.com/kr/pretty v0.3.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect @@ -73,28 +64,23 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/procfs v0.8.0 // indirect - github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.0 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect github.com/twmb/franz-go/pkg/kmsg v1.2.0 // indirect - github.com/xdg-go/pbkdf2 v1.0.0 // indirect - github.com/xdg-go/stringprep v1.0.3 // indirect - github.com/xdg/scram v1.0.5 // indirect - github.com/xdg/stringprep v1.0.3 // indirect go.opentelemetry.io/otel v1.11.1 // indirect go.opentelemetry.io/otel/trace v1.11.1 // indirect go.uber.org/atomic v1.10.0 // indirect go.uber.org/multierr v1.8.0 // indirect golang.org/x/arch v0.0.0-20220919183040-2926576b28c0 // indirect - golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8 // indirect - golang.org/x/net v0.0.0-20220812174116-3211cb980234 // indirect + golang.org/x/crypto v0.3.0 // indirect + golang.org/x/net v0.2.0 // indirect golang.org/x/sync v0.0.0-20220923202941-7f9b1623fab7 // indirect - golang.org/x/sys v0.0.0-20220919091848-fb04ddd9f9c8 // indirect - golang.org/x/text v0.3.7 // indirect + golang.org/x/sys v0.2.0 // indirect google.golang.org/protobuf v1.28.1 // indirect + gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index 99ee04ea..bb9a300c 100644 --- a/go.sum +++ b/go.sum @@ -42,10 +42,6 @@ github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3P github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.2.1 h1:58/LJlg/81wfEHd5L9qsHduznOIhyv4qb1yWcSvVq9A= github.com/RoaringBitmap/roaring v1.2.1/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= -github.com/Shopify/sarama v1.36.0 h1:0OJs3eCcnezkWniVjwBbCJVaa0B1k7ImCRS3WN6NsSk= -github.com/Shopify/sarama v1.36.0/go.mod h1:9glG3eX83tgVYJ5aVtrjVUnEsOPqQIBGx1BWfN+X51I= -github.com/Shopify/toxiproxy/v2 v2.4.0 h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64= -github.com/Shopify/toxiproxy/v2 v2.4.0/go.mod h1:3ilnjng821bkozDRxNoo64oI/DKqM+rOyJzb564+bvg= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= @@ -81,26 +77,15 @@ github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5P github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dennwc/varint v1.0.0 h1:kGNFFSSw8ToIy3obO/kKr8U9GZYUAxQEVuix4zfDWzE= -github.com/eapache/go-resiliency v1.3.0 h1:RRL0nge+cWGlxXbUzJ7yMcq6w2XBEr19dCN6HECGaT0= -github.com/eapache/go-resiliency v1.3.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= -github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= -github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= -github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= -github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/fagongzi/goetty v1.7.0 h1:Z0uoEVqP4uQSQW+HR3bg5GGwmisZpJQ1sK/ab9HK7q0= -github.com/fagongzi/goetty v1.7.0/go.mod h1:lLUyHhtjlOqatxVXgyLocwoI2o359JzLE7EWRGZiGw4= -github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= -github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= @@ -155,8 +140,6 @@ github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= -github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= -github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= @@ -189,14 +172,8 @@ github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+ github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= -github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= -github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= -github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= @@ -242,7 +219,6 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.4/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.15.7/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.12 h1:YClS/PImqYbn+UILDnqxQCZ3RehC9N318SU3kElDUEM= github.com/klauspost/compress v1.15.12/go.mod h1:QPwzmACJjUTFsnSHH934V6woptycfrDDJnH7hvFVbGM= @@ -321,20 +297,14 @@ github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1 github.com/prometheus/procfs v0.8.0 h1:ODq8ZFEaYeCaZOJlZZdJA2AbQR98dSHSM1KW/You5mo= github.com/prometheus/procfs v0.8.0/go.mod h1:z7EfXMXOkbkqb9IINtpCn86r/to3BnA0uaxHdg830/4= github.com/prometheus/prometheus v1.8.2-0.20220308163432-03831554a519 h1:MSR9EaMnwV1nsmhQ2eB1Txwebyk/5ulUGcgcYi3ofL4= -github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= -github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= -github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= -github.com/segmentio/kafka-go v0.4.34 h1:Dm6YlLMiVSiwwav20KY0AoY63s661FXevwJ3CVHUERo= -github.com/segmentio/kafka-go v0.4.34/go.mod h1:GAjxBQJdQMB5zfNA21AhpaqOB2Mu+w3De4ni3Gbm8y0= github.com/shirou/gopsutil/v3 v3.22.4/go.mod h1:D01hZJ4pVHPpCTZ3m3T2+wDF2YAGfd+H4ifUguaQzHM= github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= @@ -373,12 +343,11 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0 github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.5.3/go.mod h1:eqHYpAuvlTArOdZ1XtPYyOQ1uUb40CSZwbpL3ccjibI= -github.com/twmb/franz-go v1.6.0/go.mod h1:xdMwpUIQL/JDKKwerc5qJQG8TU1SNIddfjKJJyqRJIg= github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go v1.7.1 h1:VyAByZr0GHUQ4vaql01la9RjGXMXElEVxq/JvcfKTRo= -github.com/twmb/franz-go v1.7.1/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go/pkg/kadm v1.2.1 h1:jbPvbJgXmIREAso5WI4BU3hJLOkGDvktqWPUy/VtsSA= -github.com/twmb/franz-go/pkg/kadm v1.2.1/go.mod h1:izleX4EttZwes7MRiHFmPWPSqDuGr4VN6p7jCIYoU0g= +github.com/twmb/franz-go v1.10.4 h1:1PGpRG0uGTSSZCBV6lAMYcuVsyReMqdNBQRd8QCzw9U= +github.com/twmb/franz-go v1.10.4/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= +github.com/twmb/franz-go/pkg/kadm v1.6.0 h1:jfbpdneFgwO8wcvkMnu670+qYmOI4A9USHR/VTsAqrA= +github.com/twmb/franz-go/pkg/kadm v1.6.0/go.mod h1:1FifItwSffE++249YqRooeEfDnKRgvki6GqTJezTVAM= github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.2.0 h1:jYWh2qFw5lDbNv5Gvu/sMKagzICxuA5L6m1W2Oe7XUo= @@ -387,19 +356,8 @@ github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 h1:alKdbddkPw3rDh+AwmUEwh6HNYg github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0/go.mod h1:k8BoBjyUbFj34f0rRbn+Ky12sZFAPbmShrg0karAIMo= github.com/twmb/franz-go/plugin/kzap v1.1.1 h1:ae8Z2JXn8y9ceZ2AFnwPm5U1A8d6cBvvZlygF46F2N4= github.com/twmb/franz-go/plugin/kzap v1.1.1/go.mod h1:TUlWYqucIyz6U7xLo++gkHBDiJmed5FpYe42rCL2YG0= -github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= github.com/valyala/fastjson v1.6.3/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= -github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= -github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= -github.com/xdg-go/scram v1.1.1 h1:VOMT+81stJgXW3CpHyqHN3AXDYIMsx56mEFrB37Mb/E= -github.com/xdg-go/scram v1.1.1/go.mod h1:RaEWvsqvNKKvBPvcKeFjrG2cJqOkHTiyTpzz23ni57g= -github.com/xdg-go/stringprep v1.0.3 h1:kdwGpVNwPFtjs98xCGkHjQtGKh86rDcRZN17QEMCOIs= -github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgklLGvcBnW8= -github.com/xdg/scram v1.0.5 h1:TuS0RFmt5Is5qm9Tm2SoD89OPqe4IRiFtyFY4iwWXsw= -github.com/xdg/scram v1.0.5/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= -github.com/xdg/stringprep v1.0.3 h1:cmL5Enob4W83ti/ZHuZLuKD/xqJfus4fVPwE+/BDm+4= -github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= github.com/xlab/treeprint v1.1.0/go.mod h1:gj5Gd3gPdKtR1ikdDK6fnFLdmIS0X30kTTuNd/WEJu0= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -442,11 +400,10 @@ golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.0.0-20220518034528-6f7dac969898/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8 h1:GIAS/yBem/gq2MUqgNIzUHW7cJMmx3TGZOrnyYaNQ6c= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.3.0 h1:a06MkbcxBrEFc0w0QIZWXrH/9cCX6KJyWbBOIwAn+7A= +golang.org/x/crypto v0.3.0/go.mod h1:hebNnKkNXi2UzZN1eVRvBB7co0a+JxK6XbPiWVs/3J4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -514,11 +471,10 @@ golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.0.0-20220706163947-c90051bbdb60/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= -golang.org/x/net v0.0.0-20220809184613-07c6da5e1ced/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= -golang.org/x/net v0.0.0-20220812174116-3211cb980234 h1:RDqmgfe7SvlMWoqC3xwQ2blLO3fcWcxMa3eBLRdRW7E= golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= +golang.org/x/net v0.2.0 h1:sZfSu1wtKLGlWI4ZZayP0ck9Y73K1ynO6gqzTdBVdPU= +golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -537,7 +493,6 @@ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220923202941-7f9b1623fab7 h1:ZrnxWX62AgTKOSagEqxvb3ffipvEDX2pl7E1TdqLqIc= golang.org/x/sync v0.0.0-20220923202941-7f9b1623fab7/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -590,8 +545,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220919091848-fb04ddd9f9c8 h1:h+EGohizhe9XlX18rfpa8k8RAc5XyaeamM+0VHRd4lc= -golang.org/x/sys v0.0.0-20220919091848-fb04ddd9f9c8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.2.0 h1:ljd4t30dBnAvMZaQCevtY0xLLD0A+bRZXbgLMLU1F/A= +golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -600,7 +555,6 @@ golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3 golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -748,10 +702,8 @@ gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= -gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= diff --git a/input/input.go b/input/input.go deleted file mode 100644 index 26b5e6bb..00000000 --- a/input/input.go +++ /dev/null @@ -1,36 +0,0 @@ -package input - -import ( - "fmt" - - "github.com/housepower/clickhouse_sinker/config" - "github.com/housepower/clickhouse_sinker/model" - "github.com/housepower/clickhouse_sinker/util" -) - -const ( - TypeKafkaGo = "kafka-go" - TypeKafkaSarama = "sarama" - TypeKafkaFranz = "franz" -) - -type Inputer interface { - Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg *model.InputMessage), cleanupFn func()) error - Run() - Stop() error - CommitMessages(message *model.InputMessage) error -} - -func NewInputer(typ string) Inputer { - switch typ { - case TypeKafkaGo: - return NewKafkaGo() - case TypeKafkaSarama: - return NewKafkaSarama() - case TypeKafkaFranz: - return NewKafkaFranz() - default: - util.Logger.Fatal(fmt.Sprintf("BUG: %s is not a supported input type", typ)) - return nil - } -} diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 923f0b68..7a6769d8 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -18,6 +18,7 @@ package input import ( "context" "crypto/tls" + "fmt" "strconv" "strings" "sync" @@ -46,18 +47,16 @@ const ( RetryBackoff = 5 * time.Second ) -var _ Inputer = (*KafkaFranz)(nil) - // KafkaFranz implements input.Inputer // refers to examples/group_consuming/main.go type KafkaFranz struct { cfg *config.Config - taskCfg *config.TaskConfig + taskCfgs []*config.TaskConfig cl *kgo.Client ctx context.Context cancel context.CancelFunc wgRun sync.WaitGroup - putFn func(msg *model.InputMessage) + fetch chan []*kgo.Record cleanupFn func() } @@ -67,23 +66,31 @@ func NewKafkaFranz() *KafkaFranz { } // Init Initialise the kafka instance with configuration -func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg *model.InputMessage), cleanupFn func()) (err error) { +func (k *KafkaFranz) Init(cfg *config.Config, taskCfgs []*config.TaskConfig, f chan []*kgo.Record, cleanupFn func()) (err error) { k.cfg = cfg - k.taskCfg = taskCfg + k.taskCfgs = taskCfgs k.ctx, k.cancel = context.WithCancel(context.Background()) - k.putFn = putFn + k.fetch = f k.cleanupFn = cleanupFn kfkCfg := &cfg.Kafka var opts []kgo.Opt if opts, err = GetFranzConfig(kfkCfg); err != nil { return } + var topics []string + for _, taskcfg := range k.taskCfgs { + topics = append(topics, taskcfg.Topic) + } opts = append(opts, - kgo.ConsumeTopics(taskCfg.Topic), - kgo.ConsumerGroup(taskCfg.ConsumerGroup), + kgo.ConsumeTopics(topics...), + kgo.ConsumerGroup(taskCfgs[0].ConsumerGroup), kgo.DisableAutoCommit(), - kgo.OnPartitionsRevoked(k.onPartitionRevoked)) - if !taskCfg.Earliest { + kgo.OnPartitionsRevoked(k.onPartitionRevoked), + kgo.RebalanceTimeout(time.Minute*2), + kgo.SessionTimeout(time.Minute*2), + kgo.RequestTimeoutOverhead(time.Minute*1), + ) + if !taskCfgs[0].Earliest { opts = append(opts, kgo.ConsumeResetOffset(kgo.NewOffset().AtEnd())) } @@ -97,8 +104,9 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { opts = []kgo.Opt{ kgo.SeedBrokers(strings.Split(kfkCfg.Brokers, ",")...), - kgo.FetchMaxBytes(20971520), // 20 MB -- Larger numbers are likely to cause OOM. Should be configurable - kgo.BrokerMaxReadBytes(20971520), + // kgo.FetchMaxBytes(), // 50 MB -- take the default config + // kgo.BrokerMaxReadBytes(), // 100 MB + kgo.FetchMaxPartitionBytes(1 << 24), // 16MB //kgo.MetadataMaxAge(...) corresponds to sarama.Config.Metadata.RefreshFrequency kgo.WithLogger(kzap.New(util.Logger)), } @@ -173,21 +181,12 @@ func (k *KafkaFranz) Run() { err = errors.Wrapf(err, "") util.Logger.Info("kgo.Client.PollFetchs() got an error", zap.Error(err)) } - util.Logger.Debug("Records fetched", zap.String("task", k.taskCfg.Name), zap.String("records", strconv.Itoa(fetches.NumRecords()))) - fetches.EachRecord(func(rec *kgo.Record) { - msg := &model.InputMessage{ - Topic: rec.Topic, - Partition: int(rec.Partition), - Key: rec.Key, - Value: rec.Value, - Offset: rec.Offset, - Timestamp: &rec.Timestamp, - } - k.putFn(msg) - }) + + util.Logger.Debug("Records fetched", zap.String("records", strconv.Itoa(fetches.NumRecords()))) + k.fetch <- fetches.Records() } k.cl.Close() // will trigger k.onPartitionRevoked - util.Logger.Info("KafkaFranz.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) + util.Logger.Info("KafkaFranz.Run quit due to context has been canceled", zap.String("consumer group", k.taskCfgs[0].ConsumerGroup)) } func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { @@ -200,7 +199,7 @@ func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { } err = errors.Wrapf(err, "") if i < CommitRetries-1 && !errors.Is(err, context.Canceled) { - util.Logger.Error("cl.CommitRecords failed, will retry later", zap.String("task", k.taskCfg.Name), zap.Int("try", i), zap.Error(err)) + util.Logger.Error("cl.CommitRecords failed, will retry later", zap.String("topic", msg.Topic), zap.Int("try", i), zap.Error(err)) time.Sleep(RetryBackoff) } } @@ -210,20 +209,37 @@ func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { // Stop kafka consumer and close all connections func (k *KafkaFranz) Stop() error { k.cancel() + + // prevent the block of k.Run + quit := make(chan struct{}) + go func() { + select { + case <-k.fetch: + case <-quit: + } + }() + k.wgRun.Wait() + select { + case quit <- struct{}{}: + default: + } return nil } // Description of this kafka consumer, which topic it reads from func (k *KafkaFranz) Description() string { - return "kafka consumer of topic " + k.taskCfg.Topic + var topics []string + for _, taskcfg := range k.taskCfgs { + topics = append(topics, taskcfg.Topic) + } + return fmt.Sprint("kafka consumer of topic ", topics) } func (k *KafkaFranz) onPartitionRevoked(_ context.Context, _ *kgo.Client, _ map[string][]int32) { begin := time.Now() k.cleanupFn() util.Logger.Info("consumer group cleanup", - zap.String("task", k.taskCfg.Name), - zap.String("consumer group", k.taskCfg.ConsumerGroup), + zap.String("consumer group", k.taskCfgs[0].ConsumerGroup), zap.Duration("cost", time.Since(begin))) } diff --git a/input/kafka_go.go b/input/kafka_go.go deleted file mode 100644 index 87f5f897..00000000 --- a/input/kafka_go.go +++ /dev/null @@ -1,188 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package input - -import ( - "context" - "crypto/tls" - "io" - "strings" - "sync" - "time" - - "github.com/segmentio/kafka-go" - "github.com/segmentio/kafka-go/sasl/plain" - "github.com/segmentio/kafka-go/sasl/scram" - "github.com/thanos-io/thanos/pkg/errors" - "go.uber.org/zap" - - "github.com/housepower/clickhouse_sinker/config" - "github.com/housepower/clickhouse_sinker/model" - "github.com/housepower/clickhouse_sinker/statistics" - "github.com/housepower/clickhouse_sinker/util" -) - -var _ Inputer = (*KafkaGo)(nil) - -// KafkaGo implements input.Inputer -type KafkaGo struct { - cfg *config.Config - taskCfg *config.TaskConfig - r *kafka.Reader - ctx context.Context - cancel context.CancelFunc - wgRun sync.WaitGroup - putFn func(msg *model.InputMessage) - cleanupFn func() -} - -type KafkaGoLogger struct { - logger *zap.Logger -} - -func (kgl *KafkaGoLogger) Printf(template string, args ...interface{}) { - kgl.logger.Sugar().Debugf(template, args) -} - -// NewKafkaGo get instance of kafka reader -func NewKafkaGo() *KafkaGo { - return &KafkaGo{} -} - -// Init Initialise the kafka instance with configuration -func (k *KafkaGo) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg *model.InputMessage), cleanupFn func()) (err error) { - k.cfg = cfg - k.taskCfg = taskCfg - kfkCfg := &cfg.Kafka - k.ctx, k.cancel = context.WithCancel(context.Background()) - k.putFn = putFn - k.cleanupFn = cleanupFn - offset := kafka.LastOffset - if k.taskCfg.Earliest { - offset = kafka.FirstOffset - } - readerCfg := &kafka.ReaderConfig{ - Brokers: strings.Split(kfkCfg.Brokers, ","), - GroupID: k.taskCfg.ConsumerGroup, - Topic: k.taskCfg.Topic, - StartOffset: offset, - MinBytes: 1024 * 1024, // sarama.Config.Consumer.Fetch.Min - MaxBytes: 100 * 1024 * 1024, // sarama.Config.MaxResponseSize - MaxWait: time.Duration(100) * time.Millisecond, // sarama.Config.Consumer.MaxWaitTime - CommitInterval: time.Second, // sarama.Config.Consumer.Offsets.AutoCommit.Interval - // PartitionWatchInterval is only used when GroupID is set and WatchPartitionChanges is set. - PartitionWatchInterval: 600 * time.Second, // sarama.Config.Metadata.RefreshFrequency - WatchPartitionChanges: true, - ErrorLogger: &KafkaGoLogger{util.Logger}, - } - var dialer *kafka.Dialer - if kfkCfg.TLS.Enable { - var tlsConfig *tls.Config - if tlsConfig, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.EndpIdentAlgo == ""); err != nil { - return - } - dialer = &kafka.Dialer{ - DualStack: true, - TLS: tlsConfig, - } - } - if kfkCfg.Sasl.Enable { - if dialer == nil { - dialer = &kafka.Dialer{DualStack: true} - } - switch kfkCfg.Sasl.Mechanism { - case "PLAIN": - dialer.SASLMechanism = plain.Mechanism{ - Username: kfkCfg.Sasl.Username, - Password: kfkCfg.Sasl.Password, - } - case "SCRAM-SHA-256": - if dialer.SASLMechanism, err = scram.Mechanism(scram.SHA256, kfkCfg.Sasl.Username, kfkCfg.Sasl.Password); err != nil { - err = errors.Wrapf(err, "") - return - } - case "SCRAM-SHA-512": - if dialer.SASLMechanism, err = scram.Mechanism(scram.SHA512, kfkCfg.Sasl.Username, kfkCfg.Sasl.Password); err != nil { - err = errors.Wrapf(err, "") - return - } - default: - return errors.Newf("kafka-go doesn't support SASL/%s authentication", kfkCfg.Sasl.Mechanism) - } - } - if dialer != nil { - readerCfg.Dialer = dialer - } - k.r = kafka.NewReader(*readerCfg) - return nil -} - -// kafka main loop -func (k *KafkaGo) Run() { - k.wgRun.Add(1) - defer k.wgRun.Done() -LOOP_KAFKA_GO: - for { - var err error - var msg kafka.Message - if msg, err = k.r.FetchMessage(k.ctx); err != nil { - if errors.Is(err, context.Canceled) || errors.Is(err, io.EOF) { - break LOOP_KAFKA_GO - } else { - statistics.ConsumeMsgsErrorTotal.WithLabelValues(k.taskCfg.Name).Inc() - err = errors.Wrapf(err, "") - util.Logger.Error("kafka.Reader.FetchMessage failed", zap.String("task", k.taskCfg.Name), zap.Error(err)) - continue - } - } - k.putFn(&model.InputMessage{ - Topic: msg.Topic, - Partition: msg.Partition, - Key: msg.Key, - Value: msg.Value, - Offset: msg.Offset, - Timestamp: &msg.Time, - }) - } - // Note: a closed kafka-go client cannot commit offsets. - k.cleanupFn() - k.r.Close() - util.Logger.Info("KafkaGo.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) -} - -func (k *KafkaGo) CommitMessages(msg *model.InputMessage) (err error) { - if err = k.r.CommitMessages(context.Background(), kafka.Message{ - Topic: msg.Topic, - Partition: msg.Partition, - Offset: msg.Offset, - }); err != nil { - err = errors.Wrapf(err, "") - return - } - return -} - -// Stop kafka consumer and close all connections -func (k *KafkaGo) Stop() error { - k.cancel() - k.wgRun.Wait() - return nil -} - -// Description of this kafka consumer, which topic it reads from -func (k *KafkaGo) Description() string { - return "kafka consumer of topic " + k.taskCfg.Topic -} diff --git a/input/kafka_sarama.go b/input/kafka_sarama.go deleted file mode 100644 index 28507ca4..00000000 --- a/input/kafka_sarama.go +++ /dev/null @@ -1,226 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package input - -import ( - "context" - "crypto/sha256" - "crypto/sha512" - "hash" - "strings" - "sync" - "time" - - "github.com/Shopify/sarama" - "github.com/thanos-io/thanos/pkg/errors" - "github.com/xdg-go/scram" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" - - "github.com/housepower/clickhouse_sinker/config" - "github.com/housepower/clickhouse_sinker/model" - "github.com/housepower/clickhouse_sinker/statistics" - "github.com/housepower/clickhouse_sinker/util" -) - -var _ Inputer = (*KafkaSarama)(nil) - -// KafkaSarama implements input.Inputer -type KafkaSarama struct { - cfg *config.Config - taskCfg *config.TaskConfig - cg sarama.ConsumerGroup - sess sarama.ConsumerGroupSession - ctx context.Context - cancel context.CancelFunc - wgRun sync.WaitGroup - putFn func(msg *model.InputMessage) - cleanupFn func() -} - -// NewKafkaSarama get instance of kafka reader -func NewKafkaSarama() *KafkaSarama { - return &KafkaSarama{} -} - -type MyConsumerGroupHandler struct { - k *KafkaSarama //point back to which kafka this handler belongs to -} - -func (h MyConsumerGroupHandler) Setup(sess sarama.ConsumerGroupSession) error { - h.k.sess = sess - return nil -} - -func (h MyConsumerGroupHandler) Cleanup(_ sarama.ConsumerGroupSession) error { - begin := time.Now() - h.k.cleanupFn() - util.Logger.Info("consumer group cleanup", - zap.String("task", h.k.taskCfg.Name), - zap.String("consumer group", h.k.taskCfg.ConsumerGroup), - zap.Int32("generation id", h.k.sess.GenerationID()), - zap.Duration("cost", time.Since(begin))) - return nil -} - -func (h MyConsumerGroupHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error { - for msg := range claim.Messages() { - h.k.putFn(&model.InputMessage{ - Topic: msg.Topic, - Partition: int(msg.Partition), - Key: msg.Key, - Value: msg.Value, - Offset: msg.Offset, - Timestamp: &msg.Timestamp, - }) - } - return nil -} - -// Init Initialise the kafka instance with configuration -func (k *KafkaSarama) Init(cfg *config.Config, taskCfg *config.TaskConfig, putFn func(msg *model.InputMessage), cleanupFn func()) (err error) { - k.cfg = cfg - k.taskCfg = taskCfg - k.ctx, k.cancel = context.WithCancel(context.Background()) - k.putFn = putFn - k.cleanupFn = cleanupFn - kfkCfg := &cfg.Kafka - sarCfg, err := GetSaramaConfig(&cfg.Kafka) - if err != nil { - return err - } - if taskCfg.Earliest { - sarCfg.Consumer.Offsets.Initial = sarama.OffsetOldest - } - cg, err := sarama.NewConsumerGroup(strings.Split(kfkCfg.Brokers, ","), taskCfg.ConsumerGroup, sarCfg) - if err != nil { - return err - } - sarama.Logger, _ = zap.NewStdLogAt(util.Logger.With(zap.String("name", "sarama")), zapcore.DebugLevel) - k.cg = cg - return nil -} - -func GetSaramaConfig(kfkCfg *config.KafkaConfig) (sarCfg *sarama.Config, err error) { - sarCfg = sarama.NewConfig() - if sarCfg.Version, err = sarama.ParseKafkaVersion(kfkCfg.Version); err != nil { - err = errors.Wrapf(err, "") - return - } - if kfkCfg.TLS.Enable { - sarCfg.Net.TLS.Enable = true - if sarCfg.Net.TLS.Config, err = util.NewTLSConfig(kfkCfg.TLS.CaCertFiles, kfkCfg.TLS.ClientCertFile, kfkCfg.TLS.ClientKeyFile, kfkCfg.TLS.EndpIdentAlgo == ""); err != nil { - return - } - } - // check for authentication - if kfkCfg.Sasl.Enable { - sarCfg.Net.SASL.Enable = true - if sarCfg.Version.IsAtLeast(sarama.V1_0_0_0) { - sarCfg.Net.SASL.Version = sarama.SASLHandshakeV1 - } - sarCfg.Net.SASL.Mechanism = (sarama.SASLMechanism)(kfkCfg.Sasl.Mechanism) - switch sarCfg.Net.SASL.Mechanism { - case "SCRAM-SHA-256": - sarCfg.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &XDGSCRAMClient{HashGeneratorFcn: SHA256} } - case "SCRAM-SHA-512": - sarCfg.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &XDGSCRAMClient{HashGeneratorFcn: SHA256} } - default: - } - sarCfg.Net.SASL.User = kfkCfg.Sasl.Username - sarCfg.Net.SASL.Password = kfkCfg.Sasl.Password - sarCfg.Net.SASL.GSSAPI = kfkCfg.Sasl.GSSAPI - } - sarCfg.ChannelBufferSize = 1024 - return -} - -// kafka main loop -func (k *KafkaSarama) Run() { - k.wgRun.Add(1) - defer k.wgRun.Done() - taskCfg := k.taskCfg -LOOP_SARAMA: - for { - select { - case <-k.ctx.Done(): - break LOOP_SARAMA - default: - } - handler := MyConsumerGroupHandler{k} - // `Consume` should be called inside an infinite loop, when a - // server-side rebalance happens, the consumer session will need to be - // recreated to get the new claims - if err := k.cg.Consume(k.ctx, []string{taskCfg.Topic}, handler); err != nil { - if errors.Is(err, context.Canceled) || errors.Is(err, sarama.ErrClosedConsumerGroup) { - break LOOP_SARAMA - } else { - statistics.ConsumeMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() - err = errors.Wrapf(err, "") - util.Logger.Error("sarama.ConsumerGroup.Consume failed", zap.String("task", k.taskCfg.Name), zap.Error(err)) - continue - } - } - } - k.cg.Close() - util.Logger.Info("KafkaSarama.Run quit due to context has been canceled", zap.String("task", k.taskCfg.Name)) -} - -func (k *KafkaSarama) CommitMessages(msg *model.InputMessage) error { - k.sess.MarkOffset(msg.Topic, int32(msg.Partition), msg.Offset+1, "") - return nil -} - -// Stop kafka consumer and close all connections -func (k *KafkaSarama) Stop() error { - k.cancel() - k.wgRun.Wait() - return nil -} - -// Description of this kafka consumer, which topic it reads from -func (k *KafkaSarama) Description() string { - return "kafka consumer of topic " + k.taskCfg.Topic -} - -// Predefined SCRAMClientGeneratorFunc, copied from https://github.com/Shopify/sarama/blob/master/examples/sasl_scram_client/scram_client.go - -var SHA256 scram.HashGeneratorFcn = func() hash.Hash { return sha256.New() } -var SHA512 scram.HashGeneratorFcn = func() hash.Hash { return sha512.New() } - -type XDGSCRAMClient struct { - *scram.Client - *scram.ClientConversation - scram.HashGeneratorFcn -} - -func (x *XDGSCRAMClient) Begin(userName, password, authzID string) (err error) { - x.Client, err = x.HashGeneratorFcn.NewClient(userName, password, authzID) - if err != nil { - return err - } - x.ClientConversation = x.Client.NewConversation() - return nil -} - -func (x *XDGSCRAMClient) Step(challenge string) (response string, err error) { - response, err = x.ClientConversation.Step(challenge) - return -} - -func (x *XDGSCRAMClient) Done() bool { - return x.ClientConversation.Done() -} diff --git a/model/message.go b/model/message.go index d2fcf87c..9cb49ac6 100644 --- a/model/message.go +++ b/model/message.go @@ -1,14 +1,8 @@ package model import ( - "container/list" - "strconv" "sync" - "sync/atomic" "time" - - "github.com/housepower/clickhouse_sinker/config" - "github.com/housepower/clickhouse_sinker/statistics" ) var ( @@ -40,101 +34,14 @@ type Batch struct { Rows *Rows BatchIdx int64 RealSize int - Group *BatchGroup -} - -// BatchGroup consists of multiple batches. -// The `before` relationship could be impossible if messages of a partition are distributed to multiple batches. -// So those batches need to be committed after ALL of them have been written to clickhouse. -type BatchGroup struct { - Batchs []*Batch - Offsets map[int]int64 - Sys *BatchSys - PendWrite int32 //how many batches in this group are pending to wirte to ClickHouse -} - -type BatchSys struct { - taskCfg *config.TaskConfig - mux sync.Mutex - groups list.List - fnCommit func(partition int, offset int64) error -} -func NewBatchSys(taskCfg *config.TaskConfig, fnCommit func(partition int, offset int64) error) *BatchSys { - return &BatchSys{taskCfg: taskCfg, fnCommit: fnCommit} -} - -func (bs *BatchSys) TryCommit() error { - bs.mux.Lock() - defer bs.mux.Unlock() - // ensure groups be committed orderly -LOOP: - for e := bs.groups.Front(); e != nil; { - grp, _ := e.Value.(*BatchGroup) - if atomic.LoadInt32(&grp.PendWrite) != 0 { - break LOOP - } - // commit the whole group - for j, off := range grp.Offsets { - if err := bs.fnCommit(j, off); err != nil { - return err - } - statistics.ConsumeOffsets.WithLabelValues(bs.taskCfg.Name, bs.taskCfg.Topic, strconv.Itoa(j)).Set(float64(off)) - } - eNext := e.Next() - bs.groups.Remove(e) - e = eNext - } - return nil -} - -func (bs *BatchSys) CreateBatchGroupSingle(batch *Batch, partition int, offset int64) { - bg := &BatchGroup{ - Sys: bs, - Batchs: []*Batch{batch}, - Offsets: make(map[int]int64), - PendWrite: 1, - } - bg.Batchs[0].Group = bg - bg.Offsets[partition] = offset - bs.mux.Lock() - bs.groups.PushBack(bg) - bs.mux.Unlock() -} - -func (bs *BatchSys) CreateBatchGroupMulti(batches []*Batch, offsets map[int]int64) { - bg := &BatchGroup{Sys: bs, PendWrite: int32(len(batches))} - bg.Batchs = append(bg.Batchs, batches...) - bg.Offsets = offsets - for _, batch := range bg.Batchs { - batch.Group = bg - } - bs.mux.Lock() - bs.groups.PushBack(bg) - bs.mux.Unlock() -} - -func NewBatch() (b *Batch) { - return &Batch{ - Rows: GetRows(), - } + Wg *sync.WaitGroup } func (b *Batch) Size() int { return len(*b.Rows) } -// Commit is not retry-able! -func (b *Batch) Commit() error { - for _, row := range *b.Rows { - PutRow(row) - } - PutRows(b.Rows) - b.Rows = nil - atomic.AddInt32(&b.Group.PendWrite, -1) - return b.Group.Sys.TryCommit() -} - func GetRows() (rs *Rows) { v := rowsPool.Get() if v == nil { diff --git a/output/clickhouse.go b/output/clickhouse.go index a3313247..6c328167 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -18,7 +18,6 @@ package output import ( "context" "fmt" - "io" "math" "sort" "strings" @@ -102,12 +101,9 @@ func (c *ClickHouse) Drain() { // Send a batch to clickhouse func (c *ClickHouse) Send(batch *model.Batch) { - c.mux.Lock() - c.numFlying++ - c.mux.Unlock() - statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Inc() - _ = util.GlobalWritingPool.Submit(func() { + if err := util.GlobalWritingPool.Submit(func() { c.loopWrite(batch) + batch.Wg.Done() c.mux.Lock() c.numFlying-- if c.numFlying == 0 { @@ -115,7 +111,14 @@ func (c *ClickHouse) Send(batch *model.Batch) { } c.mux.Unlock() statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Dec() - }) + }); err != nil { + return + } + + c.mux.Lock() + c.numFlying++ + c.mux.Unlock() + statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Inc() } func (c *ClickHouse) AllowWriteSeries(sid, mid int64) (allowed bool) { @@ -192,6 +195,8 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( if conn, *dbVer, err = sc.NextGoodReplica(*dbVer); err != nil { return } + util.Logger.Debug("writing batch", zap.String("task", c.taskCfg.Name), zap.String("replica", sc.GetReplica()), zap.Int("dbVer", *dbVer)) + //row[:c.IdxSerID+1] is for metric table //row[c.IdxSerID:] is for series table numDims := c.NumDims @@ -221,17 +226,15 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { var reconnect bool var dbVer int sc := pool.GetShardConn(batch.BatchIdx) + defer func() { + for _, row := range *batch.Rows { + model.PutRow(row) + } + model.PutRows(batch.Rows) + }() for { if err = c.write(batch, sc, &dbVer); err == nil { - if err = batch.Commit(); err == nil { - return - } - // Note: kafka_go and sarama commit throws different error for context cancellation. - if errors.Is(err, context.Canceled) || errors.Is(err, io.ErrClosedPipe) { - util.Logger.Warn("Batch.Commit failed due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) - return - } - util.Logger.Fatal("Batch.Commit failed with permanent error", zap.String("task", c.taskCfg.Name), zap.Error(err)) + return } if errors.Is(err, context.Canceled) { util.Logger.Info("ClickHouse.write failed due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) diff --git a/statistics/statistics.go b/statistics/statistics.go index 8cf82f24..42a7be77 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -45,7 +45,7 @@ var ( Name: prefix + "consumer_msgs_error_total", Help: "total num of consume errors", }, - []string{"task"}, + []string{"group"}, ) ParseMsgsErrorTotal = prometheus.NewCounterVec( prometheus.CounterOpts{ diff --git a/task/consumer.go b/task/consumer.go new file mode 100644 index 00000000..085247bd --- /dev/null +++ b/task/consumer.go @@ -0,0 +1,251 @@ +/*Copyright [2019] housepower + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package task + +import ( + "encoding/json" + "reflect" + "sync" + "sync/atomic" + "time" + + "github.com/housepower/clickhouse_sinker/config" + "github.com/housepower/clickhouse_sinker/input" + "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" + "github.com/twmb/franz-go/pkg/kgo" + "go.uber.org/zap" + + _ "github.com/ClickHouse/clickhouse-go/v2" +) + +type Commit struct { + group string + offsets []byte + wg *sync.WaitGroup + consumer *Consumer +} + +type Consumer struct { + sinker *Sinker + inputer *input.KafkaFranz + tasks sync.Map + cfgs []*config.TaskConfig + fetches chan []*kgo.Record + processWg sync.WaitGroup + stopProcess chan struct{} + state atomic.Uint32 + + recMap map[string]map[int32]int64 // committed RecMap + recMux sync.Mutex + + numFlying int32 + mux sync.Mutex + commitDone *sync.Cond +} + +const ( + MaxCountInBuf = 1 << 27 + MaxParallelism = 10 +) + +func newConsumer(s *Sinker) *Consumer { + c := &Consumer{ + sinker: s, + numFlying: 0, + stopProcess: make(chan struct{}), + fetches: make(chan []*kgo.Record), + recMap: make(map[string]map[int32]int64), + } + c.state.Store(util.StateStopped) + c.commitDone = sync.NewCond(&c.mux) + return c +} + +func (c *Consumer) start() { + c.inputer = input.NewKafkaFranz() + c.state.Store(util.StateRunning) + if err := c.inputer.Init(c.sinker.curCfg, c.cfgs, c.fetches, c.cleanupFn); err == nil { + go c.inputer.Run() + go c.processFetch() + } else { + util.Logger.Fatal("failed to init consumer", zap.String("consumer", c.cfgs[0].ConsumerGroup), zap.Error(err)) + } +} + +func (c *Consumer) stop(force bool) (err error) { + c.state.Store(util.StateStopped) + + // stop the processFetch routine, make sure no more input to the- commit chan & writing pool + c.stopProcess <- struct{}{} + c.processWg.Wait() + err = c.inputer.Stop() + return err +} + +func (c *Consumer) restart() { + if err := c.stop(false); err != nil { + util.Logger.Fatal("failed to restart consumer group", zap.String("group", c.cfgs[0].ConsumerGroup)) + } + c.start() +} + +func (c *Consumer) cleanupFn() { + // ensure the completion of writing to ck + var wg sync.WaitGroup + c.tasks.Range(func(key, value any) bool { + wg.Add(1) + go func(t *Service) { + // drain ensure we have completeted persisting all received messages + t.clickhouse.Drain() + wg.Done() + }(value.(*Service)) + return true + }) + wg.Wait() + + // ensure the completion of offset submission + c.mux.Lock() + for c.numFlying != 0 { + util.Logger.Debug("draining flying pending commits", zap.String("consumergroup", c.cfgs[0].ConsumerGroup), zap.Int32("pending", c.numFlying)) + c.commitDone.Wait() + } + c.mux.Unlock() +} + +func (c *Consumer) processFetch() { + c.processWg.Add(1) + defer c.processWg.Done() + recMap := make(map[string]map[int32]int64) + var bufLength, bufThreshold int64 + + flushFn := func() { + c.recMux.Lock() + ok := reflect.DeepEqual(recMap, c.recMap) + c.recMux.Unlock() + if ok { + return + } + var wg sync.WaitGroup + c.tasks.Range(func(key, value any) bool { + // flush to shard, ck + value.(*Service).sharder.Flush(&wg) + return true + }) + bufLength = 0 + off, err := json.Marshal(recMap) + if err != nil { + return + } + c.mux.Lock() + c.numFlying++ + c.mux.Unlock() + c.sinker.commits <- &Commit{group: c.cfgs[0].ConsumerGroup, offsets: off, wg: &wg, consumer: c} + } + + for _, it := range c.cfgs { + bufThreshold += int64(it.BufferSize) + } + bufThreshold = bufThreshold * int64(len(c.sinker.curCfg.Clickhouse.Hosts)) * 4 / 5 + if bufThreshold > MaxCountInBuf { + bufThreshold = MaxCountInBuf + } + + ticker := time.NewTicker(time.Duration(c.cfgs[0].FlushInterval) * time.Second) + defer ticker.Stop() + for { + select { + case fetch := <-c.fetches: + if c.state.Load() == util.StateStopped { + continue + } + + items, done := int64(len(fetch)), int64(-1) + var concurrency int + if concurrency = int(items/1000) + 1; concurrency > MaxParallelism { + concurrency = MaxParallelism + } + + var wg sync.WaitGroup + var err error + wg.Add(concurrency) + for i := 0; i < concurrency; i++ { + go func() { + for { + index := atomic.AddInt64(&done, 1) + if index >= items || c.state.Load() == util.StateStopped { + wg.Done() + break + } + + rec := fetch[index] + msg := &model.InputMessage{ + Topic: rec.Topic, + Partition: int(rec.Partition), + Key: rec.Key, + Value: rec.Value, + Offset: rec.Offset, + Timestamp: &rec.Timestamp, + } + tablename := "" + for _, it := range rec.Headers { + if it.Key == "__table_name" { + tablename = string(it.Value) + break + } + } + + c.tasks.Range(func(key, value any) bool { + tsk := value.(*Service) + if (tablename != "" && tsk.taskCfg.TableName == tablename) || tsk.taskCfg.Topic == rec.Topic { + bufLength++ + if e := tsk.Put(msg, flushFn); e != nil { + atomic.StoreInt64(&done, items) + err = e + return false + } + } + return true + }) + } + }() + } + wg.Wait() + + // record the latest offset in order + // assume the c.state was reset to stopped when facing error, so that further fetch won't get processed + if err == nil { + for i, it := range fetch { + if recMap[it.Topic] == nil { + recMap[it.Topic] = make(map[int32]int64) + } + recMap[it.Topic][it.Partition] = it.Offset + fetch[i] = nil + } + } + + if bufLength > bufThreshold { + flushFn() + } + case <-ticker.C: + flushFn() + case <-c.stopProcess: + flushFn() + util.Logger.Info("stopped processing loop", zap.String("group", c.cfgs[0].ConsumerGroup)) + return + } + } +} diff --git a/task/ring.go b/task/ring.go deleted file mode 100644 index d414cfee..00000000 --- a/task/ring.go +++ /dev/null @@ -1,229 +0,0 @@ -package task - -import ( - "fmt" - "sync" - "sync/atomic" - "time" - - "github.com/fagongzi/goetty" - "github.com/thanos-io/thanos/pkg/errors" - "go.uber.org/zap" - - "github.com/housepower/clickhouse_sinker/model" - "github.com/housepower/clickhouse_sinker/statistics" - "github.com/housepower/clickhouse_sinker/util" -) - -type Ring struct { - mux sync.Mutex //protect ring* - available *sync.Cond - ringBuf []model.MsgRow - ringCap int64 //message is allowed to insert into the ring if its offset in inside [ringGroundOff, ringGroundOff+ringCap) - ringCapMask int64 - ringGroundOff int64 //min message offset inside the ring - ringCeilingOff int64 //1 + max message offset inside the ring - ringFilledOffset int64 //messages with offset inside range [ringGroundOff, ringFilledOffset) is a consecutive slice of ring, which should be ready to next stage - batchSizeShift uint //the shift of desired batch size - tid goetty.Timeout - idleCnt int - isIdle bool - partition int - batchSys *model.BatchSys - - service *Service -} - -// assumes ring.mux is locked -func (ring *Ring) QuitIdle() { - if ring.isIdle { - ring.idleCnt = 0 - ring.isIdle = false - ring.ringBuf = make([]model.MsgRow, ring.ringCap) - util.Logger.Info(fmt.Sprintf("topic %s partition %d quit idle", ring.service.taskCfg.Topic, ring.partition), zap.String("task", ring.service.taskCfg.Name)) - ring.scheduleForceBatchOrShard() - } -} - -// assumes ring.mux is locked, and msg.Offset is in range [ring.ringGroundOff, ring.ringGroundOff+ring.ringCap) -func (ring *Ring) PutMsgNolock(msg *model.InputMessage) { - ring.QuitIdle() - ring.ringBuf[msg.Offset&ring.ringCapMask].Msg = msg - statistics.RingMsgs.WithLabelValues(ring.service.taskCfg.Name).Inc() -} - -func (ring *Ring) PutElem(msgRow model.MsgRow) { - var err error - taskCfg := ring.service.taskCfg - msgOffset := msgRow.Msg.Offset - pMsgRow := &ring.ringBuf[msgOffset&ring.ringCapMask] - ring.mux.Lock() - defer ring.mux.Unlock() - if msgOffset < ring.ringFilledOffset || pMsgRow.Msg != msgRow.Msg { - return - } - // assert(msgOffset < ring.ringGroundOff + ring.ringCap) - if msgOffset >= ring.ringCeilingOff { - ring.ringCeilingOff = msgOffset + 1 - } - - statistics.ParsedRingMsgs.WithLabelValues(ring.service.taskCfg.Name).Inc() - pMsgRow.Row = msgRow.Row - if ring.service.sharder != nil && msgRow.Row != &model.FakedRow { - if msgRow.Shard, err = ring.service.sharder.Calc(msgRow.Row); err != nil { - util.Logger.Fatal("shard number calculation failed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - pMsgRow.Shard = msgRow.Shard - } - // the consecutive ring slice may grow because of the newly inserted row - for ; ring.ringFilledOffset < ring.ringCeilingOff && ring.ringBuf[ring.ringFilledOffset&(ring.ringCapMask)].Row != nil; ring.ringFilledOffset++ { - } - // generate a batch to send or flush the messages to shards when slice size reached the shifted batch size - if (ring.ringFilledOffset >> ring.batchSizeShift) != (ring.ringGroundOff >> ring.batchSizeShift) { - ring.genBatchOrShard() - ring.scheduleForceBatchOrShard() - } -} - -func (ring *Ring) MakeRoom(newMsg *model.InputMessage) { - // assert(!ring.isIdle) - taskCfg := ring.service.taskCfg - ring.mux.Lock() - defer ring.mux.Unlock() - statistics.RingForceBatchAllTotal.WithLabelValues(taskCfg.Name).Inc() - ring.idleCnt = 0 - prevMsgOff := newMsg.Offset - 1 - if newMsg.Offset != ring.ringGroundOff+ring.ringCap || - ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg == nil { - var msgCnt int - for off := ring.ringGroundOff; off < ring.ringGroundOff+ring.ringCap; off++ { - msgRow := &ring.ringBuf[off&ring.ringCapMask] - if msgRow.Msg != nil { - msgCnt++ - } - msgRow.Msg = nil - msgRow.Row = nil - } - util.Logger.Info(fmt.Sprintf("Ring.MakeRoom discarded %d messages for topic %v patittion %d, offset [%d,%d)", - msgCnt, taskCfg.Topic, ring.partition, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap), - zap.String("task", taskCfg.Name)) - ring.ringGroundOff = newMsg.Offset - ring.ringFilledOffset = newMsg.Offset - ring.ringCeilingOff = newMsg.Offset - } else { - for ; prevMsgOff > ring.ringGroundOff && ring.ringBuf[(prevMsgOff-1)&ring.ringCapMask].Msg != nil; prevMsgOff-- { - } - // assert(ring.ringFilledOffset < prevMsgOff) - var msgCnt int - for off := ring.ringGroundOff; off < prevMsgOff; off++ { - msgRow := &ring.ringBuf[off&ring.ringCapMask] - if msgRow.Msg != nil { - msgCnt++ - } - msgRow.Msg = nil - msgRow.Row = nil - } - util.Logger.Info(fmt.Sprintf("Ring.MakeRoom discarded %d messages for topic %v patittion %d, offset [%d,%d)", - msgCnt, taskCfg.Topic, ring.partition, ring.ringGroundOff, prevMsgOff), - zap.String("task", taskCfg.Name)) - ring.ringGroundOff = prevMsgOff - ring.ringFilledOffset = newMsg.Offset - ring.ringCeilingOff = newMsg.Offset - } -} - -func (ring *Ring) ForceBatchOrShard(_ interface{}) { - taskCfg := ring.service.taskCfg - ring.mux.Lock() - defer ring.mux.Unlock() - if !ring.isIdle { - if ring.ringFilledOffset > ring.ringGroundOff { - ring.genBatchOrShard() - ring.idleCnt = 0 - } else if ring.ringBuf[ring.ringGroundOff&ring.ringCapMask].Msg == nil { - ring.idleCnt++ - if ring.idleCnt >= 2 { - ring.idleCnt = 0 - ring.isIdle = true - ring.ringBuf = nil - util.Logger.Info(fmt.Sprintf("topic %s partition %d became idle", taskCfg.Topic, ring.partition), zap.String("task", taskCfg.Name)) - return - } - } - ring.scheduleForceBatchOrShard() - } -} - -// schedule ForceBatchOrShard -// assume ring.mux is locked -func (ring *Ring) scheduleForceBatchOrShard() { - var err error - ring.tid.Stop() - if ring.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(ring.service.taskCfg.FlushInterval)*time.Second, ring.ForceBatchOrShard, nil); err != nil { - if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Warn("Ring.ForceBatchOrShard scheduling timer to a stopped timer wheel", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) - } else { - err = errors.Wrapf(err, "") - util.Logger.Fatal("scheduling timer filed", zap.String("task", ring.service.taskCfg.Name), zap.Error(err)) - } - } -} - -// generate a batch for messages [ring.ringGroundOff, ring.ringFilledOffset), respect batchSize boundary -// assume ring.mux is locked -func (ring *Ring) genBatchOrShard() { - taskCfg := ring.service.taskCfg - var parseErrs int - // Respect batchSize boundary - endOff := ((ring.ringGroundOff >> ring.batchSizeShift) + 1) << ring.batchSizeShift - if endOff > ring.ringFilledOffset { - endOff = ring.ringFilledOffset - } - msgCnt := endOff - ring.ringGroundOff - if atomic.LoadUint32(&ring.service.state) != util.StateRunning { - util.Logger.Info(fmt.Sprintf("Ring.genBatchOrShard discarded a batch for topic %v patittion %d, offset [%d,%d), messages %d", - taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, msgCnt), - zap.String("task", taskCfg.Name)) - for i := ring.ringGroundOff; i < endOff; i++ { - msgRow := &ring.ringBuf[i&(ring.ringCapMask)] - msgRow.Msg = nil - msgRow.Row = nil - msgRow.Shard = -1 - } - } else if ring.service.sharder != nil { - ring.service.sharder.PutElems(ring.partition, ring.ringBuf, ring.ringGroundOff, endOff, ring.ringCapMask) - } else { - batch := model.NewBatch() - for i := ring.ringGroundOff; i < endOff; i++ { - msgRow := &ring.ringBuf[i&(ring.ringCapMask)] - if msgRow.Row != &model.FakedRow { - *batch.Rows = append(*batch.Rows, msgRow.Row) - } else { - parseErrs++ - } - msgRow.Msg = nil - msgRow.Row = nil - msgRow.Shard = -1 - } - batch.RealSize = len(*batch.Rows) - - if batch.RealSize > 0 { - util.Logger.Info(fmt.Sprintf("created a batch for topic %v patittion %d, offset [%d,%d), messages %d, parse errors: %d", - taskCfg.Topic, ring.partition, ring.ringGroundOff, endOff, batch.RealSize, parseErrs), - zap.String("task", taskCfg.Name)) - - batch.BatchIdx = ring.ringGroundOff >> ring.batchSizeShift - ring.batchSys.CreateBatchGroupSingle(batch, ring.partition, endOff-1) - ring.service.Flush(batch) - statistics.RingNormalBatchsTotal.WithLabelValues(taskCfg.Name).Inc() - } - } - statistics.RingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) - statistics.ParsedRingMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) - ring.ringGroundOff = endOff - //util.Logger.Debug(fmt.Sprintf("genBatchOrShard changed ring %p ringGroundOff to %d", ring, ring.ringGroundOff)) - if ring.ringFilledOffset < ring.ringGroundOff { - ring.ringFilledOffset = ring.ringGroundOff - } - ring.available.Broadcast() -} diff --git a/task/sharding.go b/task/sharding.go index 3f06355d..5ede029e 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -6,14 +6,12 @@ import ( "time" "github.com/cespare/xxhash/v2" - "github.com/fagongzi/goetty" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" "github.com/shopspring/decimal" "github.com/thanos-io/thanos/pkg/errors" - "go.uber.org/zap" ) type ShardingPolicy struct { @@ -110,14 +108,11 @@ func (policy *ShardingPolicy) Calc(row *model.Row) (shard int, err error) { } type Sharder struct { - service *Service - policy *ShardingPolicy - batchSys *model.BatchSys - shards int - mux sync.Mutex - msgBuf []*model.Rows - offsets map[int]int64 - tid goetty.Timeout + service *Service + policy *ShardingPolicy + shards int + mux sync.Mutex + msgBuf []*model.Rows } func NewSharder(service *Service) (sh *Sharder, err error) { @@ -128,12 +123,10 @@ func NewSharder(service *Service) (sh *Sharder, err error) { return } sh = &Sharder{ - service: service, - policy: policy, - batchSys: model.NewBatchSys(taskCfg, service.fnCommit), - shards: shards, - msgBuf: make([]*model.Rows, shards), - offsets: make(map[int]int64), + service: service, + policy: policy, + shards: shards, + msgBuf: make([]*model.Rows, shards), } for i := 0; i < shards; i++ { sh.msgBuf[i] = model.GetRows() @@ -145,57 +138,18 @@ func (sh *Sharder) Calc(row *model.Row) (int, error) { return sh.policy.Calc(row) } -func (sh *Sharder) PutElems(partition int, ringBuf []model.MsgRow, begOff, endOff, ringCapMask int64) { - if begOff >= endOff { - return - } - msgCnt := endOff - begOff +func (sh *Sharder) PutElement(msgRow *model.MsgRow) { sh.mux.Lock() defer sh.mux.Unlock() - var parseErrs int - taskCfg := sh.service.taskCfg - for i := begOff; i < endOff; i++ { - msgRow := &ringBuf[i&ringCapMask] - //assert msg.Offset==i - if msgRow.Row != &model.FakedRow { - rows := sh.msgBuf[msgRow.Shard] - *rows = append(*rows, msgRow.Row) - } else { - parseErrs++ - } - msgRow.Msg = nil - msgRow.Row = nil - msgRow.Shard = -1 - } - - sh.offsets[partition] = endOff - 1 - statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Add(float64(msgCnt)) - var maxBatchSize int - for i := 0; i < sh.shards; i++ { - batchSize := len(*sh.msgBuf[i]) - if maxBatchSize < batchSize { - maxBatchSize = batchSize - } - } - util.Logger.Debug(fmt.Sprintf("sharded a batch for topic %v patittion %d, offset [%d, %d), messages %d, parse errors: %d", - taskCfg.Topic, partition, begOff, endOff, msgCnt, parseErrs), - zap.String("task", taskCfg.Name)) - if maxBatchSize >= taskCfg.BufferSize { - sh.doFlush(nil) - } + rows := sh.msgBuf[msgRow.Shard] + *rows = append(*rows, msgRow.Row) + statistics.ShardMsgs.WithLabelValues(sh.service.taskCfg.Name).Inc() } -func (sh *Sharder) ForceFlush(arg interface{}) { +func (sh *Sharder) Flush(wg *sync.WaitGroup) { sh.mux.Lock() - sh.doFlush(arg) - sh.mux.Unlock() -} - -// assmues sh.mux has been locked -func (sh *Sharder) doFlush(_ interface{}) { - var err error + defer sh.mux.Unlock() var msgCnt int - var batches []*model.Batch taskCfg := sh.service.taskCfg for i, rows := range sh.msgBuf { realSize := len(*rows) @@ -205,30 +159,15 @@ func (sh *Sharder) doFlush(_ interface{}) { Rows: rows, BatchIdx: int64(i), RealSize: realSize, + Wg: wg, } - batches = append(batches, batch) + batch.Wg.Add(1) + sh.service.clickhouse.Send(batch) sh.msgBuf[i] = model.GetRows() } } if msgCnt > 0 { - util.Logger.Info(fmt.Sprintf("created a batch group for topic %v, offsets %+v, messages %d", taskCfg.Topic, sh.offsets, msgCnt), zap.String("task", taskCfg.Name)) - sh.batchSys.CreateBatchGroupMulti(batches, sh.offsets) - sh.offsets = make(map[int]int64) - // ALL batches in a group shall be populated before sending any one to next stage. - for _, batch := range batches { - sh.service.Flush(batch) - } + util.Logger.Info(fmt.Sprintf("created a batch group for task %v, messages %d", sh.service.taskCfg.Name, msgCnt)) statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } - - // reschedule the delayed ForceFlush - sh.tid.Stop() - if sh.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, sh.ForceFlush, nil); err != nil { - if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Info("Sharder.doFlush scheduling timer to a stopped timer wheel") - } else { - err = errors.Wrapf(err, "") - util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - } } diff --git a/task/sinker.go b/task/sinker.go new file mode 100644 index 00000000..b6264368 --- /dev/null +++ b/task/sinker.go @@ -0,0 +1,413 @@ +/*Copyright [2019] housepower + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package task + +import ( + "context" + "encoding/json" + "errors" + "io" + "os" + "reflect" + "strings" + "sync" + "time" + + "github.com/housepower/clickhouse_sinker/config" + cm "github.com/housepower/clickhouse_sinker/config_manager" + "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/pool" + "github.com/housepower/clickhouse_sinker/statistics" + "github.com/housepower/clickhouse_sinker/util" + "go.uber.org/zap" + + _ "github.com/ClickHouse/clickhouse-go/v2" +) + +// Sinker object maintains number of task for each partition +type Sinker struct { + curCfg *config.Config + cmdOps *util.CmdOptions + httpAddr string + numCfg int + pusher *statistics.Pusher + rcm cm.RemoteConfManager + ctx context.Context + cancel context.CancelFunc + + consumers map[string]*Consumer + commits chan *Commit + exit chan struct{} + stopCommit chan struct{} +} + +// NewSinker get an instance of sinker with the task list +func NewSinker(rcm cm.RemoteConfManager, http string, cmd *util.CmdOptions) *Sinker { + ctx, cancel := context.WithCancel(context.Background()) + s := &Sinker{ + rcm: rcm, + ctx: ctx, + cmdOps: cmd, + cancel: cancel, + commits: make(chan *Commit, 10), + exit: make(chan struct{}), + stopCommit: make(chan struct{}), + consumers: make(map[string]*Consumer), + httpAddr: http, + } + return s +} + +func (s *Sinker) Init() (err error) { + return +} + +func (s *Sinker) GetCurrentConfig() *config.Config { + return s.curCfg +} + +// Run is the mainloop to get and apply config +func (s *Sinker) Run() { + var err error + var newCfg *config.Config + defer func() { + s.exit <- struct{}{} + }() + if s.cmdOps.PushGatewayAddrs != "" { + addrs := strings.Split(s.cmdOps.PushGatewayAddrs, ",") + s.pusher = statistics.NewPusher(addrs, s.cmdOps.PushInterval, s.httpAddr) + if err = s.pusher.Init(); err != nil { + return + } + go s.pusher.Run() + } + if s.rcm == nil { + if _, err = os.Stat(s.cmdOps.LocalCfgFile); err == nil { + if newCfg, err = config.ParseLocalCfgFile(s.cmdOps.LocalCfgFile); err != nil { + util.Logger.Fatal("config.ParseLocalCfgFile failed", zap.Error(err)) + return + } + } else { + util.Logger.Fatal("expect --local-cfg-file or --nacos-dataid") + return + } + if err = newCfg.Normallize(); err != nil { + util.Logger.Fatal("newCfg.Normallize failed", zap.Error(err)) + return + } + if err = s.applyConfig(newCfg); err != nil { + util.Logger.Fatal("s.applyConfig failed", zap.Error(err)) + return + } + <-s.ctx.Done() + } else { + if s.cmdOps.NacosServiceName != "" { + go s.rcm.Run() + } + // Golang <-time.After() is not garbage collected before expiry. + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() + for { + select { + case <-s.ctx.Done(): + util.Logger.Info("Sinker.Run quit due to context has been canceled") + return + case <-ticker.C: + if newCfg, err = s.rcm.GetConfig(); err != nil { + util.Logger.Error("s.rcm.GetConfig failed", zap.Error(err)) + continue + } + if err = newCfg.Normallize(); err != nil { + util.Logger.Error("newCfg.Normallize failed", zap.Error(err)) + continue + } + if err = s.applyConfig(newCfg); err != nil { + util.Logger.Error("s.applyConfig failed", zap.Error(err)) + continue + } + } + } + } +} + +// Close shutdown task +func (s *Sinker) Close() { + // 1. Stop rcm + if s.rcm != nil { + s.rcm.Stop() + s.rcm = nil + } + // 2. Quit Run mainloop + s.cancel() + <-s.exit + // 3. Stop tasks gracefully. + s.stopAllTasks() + // 4. Stop pusher + if s.pusher != nil { + s.pusher.Stop() + s.pusher = nil + } +} + +func (s *Sinker) stopAllTasks() { + // stop the input to prevent further kafka consumption, but will drain all the parsed messages in memory + var wg sync.WaitGroup + for _, v := range s.consumers { + if v.state.Load() == util.StateRunning { + wg.Add(1) + go func(c *Consumer) { + c.stop(false) + wg.Done() + }(v) + } + } + wg.Wait() + + util.Logger.Info("stopped all consumers") + s.stopCommit <- struct{}{} + util.Logger.Debug("stopped commit session") + + for name := range s.consumers { + delete(s.consumers, name) + } + + if util.GlobalWritingPool != nil { + util.GlobalWritingPool.StopWait() + } + util.Logger.Debug("stopped writing pool") +} + +func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { + util.SetLogLevel(newCfg.LogLevel) + if s.curCfg == nil { + // The first time invoking of applyConfig + err = s.applyFirstConfig(newCfg) + } else if !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) || + !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || + !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) || + !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { + err = s.applyAnotherConfig(newCfg) + } + return +} + +func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { + util.Logger.Info("going to apply the first config", zap.Reflect("config", newCfg)) + // 1. Initialize clickhouse connections + chCfg := &newCfg.Clickhouse + if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, + chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.DialTimeout); err != nil { + return + } + + // 2. Start goroutine pools. + util.InitGlobalWritingPool(len(chCfg.Hosts) * chCfg.MaxOpenConns) + go s.commitFn() + + // 3. Generate, initialize and run task + for _, taskCfg := range newCfg.Tasks { + if s.cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(s.httpAddr, taskCfg.Name) { + continue + } + var c *Consumer + var ok bool + if c, ok = s.consumers[taskCfg.ConsumerGroup]; ok { + if taskCfg.Earliest != c.cfgs[0].Earliest { + util.Logger.Fatal("Tasks are sharing same consumer group, but with different Earliest property specified!", + zap.String("task", taskCfg.Name), zap.String("task", c.cfgs[0].Name)) + } + } else { + c = newConsumer(s) + s.consumers[taskCfg.ConsumerGroup] = c + } + task := NewTaskService(newCfg, taskCfg, c) + if err = task.Init(); err != nil { + return + } + } + + // 4. Start fetching messages + s.curCfg = newCfg + for _, v := range s.consumers { + v.start() + } + + util.Logger.Info("applied the first config") + return +} + +func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { + util.Logger.Info("going to apply another config", zap.Int("number", s.numCfg), zap.Reflect("config", newCfg)) + if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) { + // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). + s.stopAllTasks() + // 2. Initialize clickhouse connections. + chCfg := &newCfg.Clickhouse + if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, + chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.DialTimeout); err != nil { + return + } + + // 3. Restart goroutine pools. + maxWorkers := len(newCfg.Clickhouse.Hosts) * newCfg.Clickhouse.MaxOpenConns + util.GlobalWritingPool.Resize(maxWorkers) + util.GlobalWritingPool.Restart() + util.Logger.Info("resized writing pool", zap.Int("maxWorkers", maxWorkers)) + go s.commitFn() + + // 4. Generate, initialize and run task + var tasksToStart []string + for _, taskCfg := range newCfg.Tasks { + if s.cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(s.httpAddr, taskCfg.Name) { + continue + } + var c *Consumer + var ok bool + if c, ok = s.consumers[taskCfg.ConsumerGroup]; ok { + if taskCfg.Earliest != c.cfgs[0].Earliest { + util.Logger.Fatal("Tasks are sharing same consumer group, but with different Earliest property specified!", + zap.String("task", taskCfg.Name), zap.String("task", c.cfgs[0].Name)) + } + } else { + c = newConsumer(s) + s.consumers[taskCfg.ConsumerGroup] = c + } + task := NewTaskService(newCfg, taskCfg, c) + if err = task.Init(); err != nil { + return + } + tasksToStart = append(tasksToStart, taskCfg.Name) + } + + // 5. Start fetching messages, only after all the taskCfg loaded to consumer.cfgs + s.curCfg = newCfg + for _, v := range s.consumers { + v.start() + } + + util.Logger.Info("started tasks", zap.Reflect("tasks", tasksToStart)) + } else if !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) || !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { + // 1. Find the config difference + newConsumers := make(map[string]map[string]*config.TaskConfig) + for _, taskCfg := range newCfg.Tasks { + if s.cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(s.httpAddr, taskCfg.Name) { + continue + } + var group map[string]*config.TaskConfig + var ok bool + if group, ok = newConsumers[taskCfg.ConsumerGroup]; !ok { + group = make(map[string]*config.TaskConfig) + newConsumers[taskCfg.ConsumerGroup] = group + } + group[taskCfg.Name] = taskCfg + } + + toCreate := make(map[string]map[string]*config.TaskConfig) + var deleteConsumers []string + for name, c := range s.consumers { + var group map[string]*config.TaskConfig + var ok bool + if group, ok = newConsumers[name]; !ok { + // consumer group no longer with this client, stop it + go c.stop(true) + deleteConsumers = append(deleteConsumers, name) + } else { + // find the one that need to be restart + var groupChanged bool + if len(group) != len(c.cfgs) { + groupChanged = true + } else { + for _, cfg := range c.cfgs { + var it *config.TaskConfig + var ok bool + if it, ok = group[cfg.Name]; !ok { + groupChanged = true + break + } else if !reflect.DeepEqual(it, cfg) { + groupChanged = true + break + } + } + } + // restart the group accordingly + if groupChanged { + go c.stop(true) + toCreate[name] = group + } + delete(newConsumers, name) + } + } + // create new consumers + for name, cfgs := range newConsumers { + toCreate[name] = cfgs + } + for _, name := range deleteConsumers { + delete(s.consumers, name) + } + for name, cfgs := range toCreate { + c := newConsumer(s) + s.consumers[name] = c + for _, cfg := range cfgs { + task := NewTaskService(newCfg, cfg, c) + if err = task.Init(); err != nil { + return + } + } + c.start() + } + } + // Record the new config + s.curCfg = newCfg + util.Logger.Info("applied another config", zap.Int("number", s.numCfg)) + s.numCfg++ + return +} + +func (s *Sinker) commitFn() { + for { + select { + case com := <-s.commits: + com.wg.Wait() + c := com.consumer + c.recMux.Lock() + err := json.Unmarshal(com.offsets, &c.recMap) + c.recMux.Unlock() + if err != nil { + util.Logger.Fatal("Failed to unmarshall offsets", zap.ByteString("offsets", com.offsets), zap.Error(err)) + } + for i, value := range c.recMap { + for k, v := range value { + if err := c.inputer.CommitMessages(&model.InputMessage{Topic: i, Partition: int(k), Offset: v}); err != nil { + // Note: kafka_go and sarama commit throws different error for context cancellation. + if errors.Is(err, context.Canceled) || errors.Is(err, io.ErrClosedPipe) { + util.Logger.Warn("Batch.Commit failed due to the context has been cancelled", zap.Error(err)) + } + util.Logger.Fatal("Batch.Commit failed with permanent error", zap.Error(err)) + } + } + } + c.mux.Lock() + c.numFlying-- + if c.numFlying == 0 { + c.commitDone.Broadcast() + } + c.mux.Unlock() + case <-s.stopCommit: + util.Logger.Info("stopped committing loop") + return + } + } +} diff --git a/task/task.go b/task/task.go index 9b8fcc26..f396805b 100644 --- a/task/task.go +++ b/task/task.go @@ -25,26 +25,20 @@ import ( "sync/atomic" "time" - "github.com/fagongzi/goetty" "github.com/housepower/clickhouse_sinker/config" - "github.com/housepower/clickhouse_sinker/input" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/output" "github.com/housepower/clickhouse_sinker/parser" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" - "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" "golang.org/x/time/rate" ) // TaskService holds the configuration for each task type Service struct { - sync.Mutex - inputer input.Inputer clickhouse *output.ClickHouse pp *parser.Pool - cfg *config.Config taskCfg *config.TaskConfig whiteList *regexp.Regexp blackList *regexp.Regexp @@ -59,32 +53,39 @@ type Service struct { newKeys sync.Map warnKeys sync.Map cntNewKeys int32 // size of newKeys - tid goetty.Timeout - rings []*Ring sharder *Sharder limiter1 *rate.Limiter limiter2 *rate.Limiter + consumer *Consumer +} + +// NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances +func cloneTask(s *Service) (service *Service) { + service = &Service{ + clickhouse: s.clickhouse, + pp: s.pp, + taskCfg: s.taskCfg, + consumer: s.consumer, + whiteList: s.whiteList, + blackList: s.blackList, + lblBlkList: s.lblBlkList, + } + service.Init() - wgRun sync.WaitGroup - state uint32 - numFlying int32 - taskDone *sync.Cond + return } // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances -func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) (service *Service) { +func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig, c *Consumer) (service *Service) { ck := output.NewClickHouse(cfg, taskCfg) pp, _ := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone, taskCfg.TimeUnit) - inputer := input.NewInputer(taskCfg.KafkaClient) service = &Service{ - inputer: inputer, clickhouse: ck, pp: pp, - cfg: cfg, taskCfg: taskCfg, + consumer: c, } - service.taskDone = sync.NewCond(service) if taskCfg.DynamicSchema.WhiteList != "" { service.whiteList = regexp.MustCompile(taskCfg.DynamicSchema.WhiteList) } @@ -101,8 +102,6 @@ func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig) (service *Se func (service *Service) Init() (err error) { taskCfg := service.taskCfg util.Logger.Info("task initializing", zap.String("task", taskCfg.Name)) - service.numFlying = 0 - atomic.StoreUint32(&service.state, util.StateRunning) if err = service.clickhouse.Init(); err != nil { return } @@ -114,17 +113,12 @@ func (service *Service) Init() (err error) { service.limiter1 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter2 = rate.NewLimiter(rate.Every(10*time.Second), 1) - service.rings = make([]*Ring, 0) if taskCfg.ShardingKey != "" { if service.sharder, err = NewSharder(service); err != nil { return } } - if err = service.inputer.Init(service.cfg, taskCfg, service.put, service.drain); err != nil { - return - } - if taskCfg.DynamicSchema.Enable { maxDims := math.MaxInt16 if taskCfg.DynamicSchema.MaxDims > 0 { @@ -145,285 +139,78 @@ func (service *Service) Init() (err error) { atomic.StoreInt32(&service.cntNewKeys, 0) } } - return -} - -// Run starts the task -func (service *Service) Run() { - var err error - service.wgRun.Add(1) - defer service.wgRun.Done() - taskCfg := service.taskCfg - if service.sharder != nil { - // schedule a delayed ForceFlush - if service.sharder.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.sharder.ForceFlush, nil); err != nil { - if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Info("Service.Run scheduling timer to a stopped timer wheel") - } else { - err = errors.Wrapf(err, "") - util.Logger.Fatal("scheduling timer filed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - } - } - service.inputer.Run() -} + service.consumer.tasks.Store(taskCfg.Name, service) + service.consumer.cfgs = append(service.consumer.cfgs, service.taskCfg) -func (service *Service) fnCommit(partition int, offset int64) (err error) { - msg := model.InputMessage{Topic: service.taskCfg.Topic, Partition: partition, Offset: offset} - if err = service.inputer.CommitMessages(&msg); err != nil { - return - } - util.Logger.Debug(fmt.Sprintf("committed topic %s, partition %d, offset %d", msg.Topic, msg.Partition, msg.Offset+1), zap.String("task", service.taskCfg.Name)) return } -func (service *Service) putToRing(msg *model.InputMessage) (ok bool) { +func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { taskCfg := service.taskCfg statistics.ConsumeMsgsTotal.WithLabelValues(taskCfg.Name).Inc() - // ensure ring for this message exist - service.Lock() - var ring *Ring - if msg.Partition < len(service.rings) { - ring = service.rings[msg.Partition] + var err error + var row *model.Row + var foundNewKeys bool + var metric model.Metric + + p := service.pp.Get() + if metric, err = p.Parse(msg.Value); err != nil { + // directly return, ignore the row with parsing errors + statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() + if service.limiter1.Allow() { + util.Logger.Error(fmt.Sprintf("failed to parse message(topic %v, partition %d, offset %v)", + msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", taskCfg.Name), zap.Error(err)) + } + return nil } else { - for i := len(service.rings); i < msg.Partition+1; i++ { - service.rings = append(service.rings, nil) + row = service.metric2Row(metric, msg) + if taskCfg.DynamicSchema.Enable { + foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys, &service.warnKeys, service.whiteList, service.blackList, msg.Partition, msg.Offset) } + // Dumping message and result + //util.Logger.Debug("parsed kafka message", zap.Int("partition", msg.Partition), zap.Int64("offset", msg.Offset), + // zap.String("message value", string(msg.Value)), zap.String("row(spew)", spew.Sdump(row))) } - - if ring == nil { - batchSizeShift := util.GetShift(taskCfg.BufferSize) - ringCap := int64(1 << (batchSizeShift + 1)) - ring := &Ring{ - ringBuf: nil, - ringCap: ringCap, - ringCapMask: ringCap - 1, - ringGroundOff: msg.Offset, - ringCeilingOff: msg.Offset, - ringFilledOffset: msg.Offset, - batchSizeShift: batchSizeShift, - idleCnt: 0, - isIdle: true, - partition: msg.Partition, - batchSys: model.NewBatchSys(taskCfg, service.fnCommit), - service: service, - } - ring.available = sync.NewCond(&ring.mux) - ring.PutMsgNolock(msg) - service.rings[msg.Partition] = ring - service.Unlock() - ok = true - } else { - service.Unlock() - ring.mux.Lock() - ring.QuitIdle() - if msg.Offset < ring.ringFilledOffset { - statistics.RingMsgsOffTooSmallErrorTotal.WithLabelValues(taskCfg.Name).Inc() - if service.limiter2.Allow() { - util.Logger.Warn(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) left to %v", - msg.Topic, msg.Partition, msg.Offset, ring.ringFilledOffset), zap.String("task", taskCfg.Name)) - } - ring.mux.Unlock() - } else if msg.Offset < ring.ringGroundOff+ring.ringCap { - ring.PutMsgNolock(msg) - ring.mux.Unlock() - ok = true - } else { - prevMsgOff := msg.Offset - 1 - for atomic.LoadUint32(&service.state) == util.StateRunning && !ring.isIdle && - msg.Offset == ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil { - // wait ring.PutElem/ring.ForceBatchOrShard to make room - util.Logger.Debug(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) while the ring is full, waiting...", - msg.Topic, msg.Partition, msg.Offset), zap.String("task", taskCfg.Name)) - ring.available.Wait() - util.Logger.Debug(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) while the ring is full, wake-up", - msg.Topic, msg.Partition, msg.Offset), zap.String("task", taskCfg.Name)) - } - if atomic.LoadUint32(&service.state) != util.StateRunning || ring.isIdle { - util.Logger.Debug(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) while the ring.isIdle %v, service.state %v", - msg.Topic, msg.Partition, msg.Offset, ring.isIdle, atomic.LoadUint32(&service.state)), zap.String("task", taskCfg.Name)) - ring.mux.Unlock() - } else if msg.Offset == ring.ringGroundOff || (msg.Offset < ring.ringGroundOff+ring.ringCap && ring.ringBuf[prevMsgOff&ring.ringCapMask].Msg != nil) { - ring.PutMsgNolock(msg) - ring.mux.Unlock() - ok = true - } else { - // discard messages to make room - ring.mux.Unlock() - statistics.RingMsgsOffTooLargeErrorTotal.WithLabelValues(taskCfg.Name).Inc() - util.Logger.Warn(fmt.Sprintf("got a message(topic %v, partition %d, offset %v) which's previous one is absent in ring offsets [%v, %v)", - msg.Topic, msg.Partition, msg.Offset, ring.ringGroundOff, ring.ringGroundOff+ring.ringCap), zap.String("task", taskCfg.Name)) - ring.MakeRoom(msg) - ring.PutMsgNolock(msg) - ok = true + // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. + service.pp.Put(p) + + if foundNewKeys { + cntNewKeys := atomic.AddInt32(&service.cntNewKeys, 1) + if cntNewKeys == 1 { + // the first message which contains new keys triggers the following: + // 1) restart the consumer group + // 1) stop the consumer to prevent blocking other consumers, stop will process until ChangeSchema completed + // 2) flush the shards + // 3) apply the schema change. + // 4) recreate the service + util.Logger.Warn("new key detected, consumer is going to restart", zap.String("consumer group", service.taskCfg.ConsumerGroup), zap.Error(err)) + service.consumer.state.Store(util.StateStopped) + go service.consumer.restart() + flushFn() + if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { + util.Logger.Fatal("clickhouse.ChangeSchema failed", zap.String("task", taskCfg.Name), zap.Error(err)) } + service.consumer.tasks.Store(service.taskCfg.Name, cloneTask(service)) + + return fmt.Errorf("consumer restart required due to new key") } } - return -} -func (service *Service) put(msg *model.InputMessage) { - if atomic.LoadUint32(&service.state) != util.StateRunning { - return - } - if !service.putToRing(msg) { - return - } - // submit message to the parsing pool - taskCfg := service.taskCfg - service.Lock() - service.numFlying++ - service.Unlock() - statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Inc() - _ = util.GlobalParsingPool.Submit(func() { - var err error - var row *model.Row - var foundNewKeys bool - var metric model.Metric - defer func() { - service.Lock() - service.numFlying-- - if service.numFlying == 0 { - service.taskDone.Broadcast() - } - service.Unlock() - statistics.ParsingPoolBacklog.WithLabelValues(taskCfg.Name).Dec() - }() - p := service.pp.Get() - metric, err = p.Parse(msg.Value) - // WARNNING: Always PutElem even if there's parsing error, so that this message can be acked to Kafka and skipped writing to ClickHouse. - if err != nil { - row = &model.FakedRow - statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() - if service.limiter1.Allow() { - util.Logger.Error(fmt.Sprintf("failed to parse message(topic %v, partition %d, offset %v)", - msg.Topic, msg.Partition, msg.Offset), zap.String("message value", string(msg.Value)), zap.String("task", taskCfg.Name), zap.Error(err)) + if atomic.LoadInt32(&service.cntNewKeys) == 0 && service.consumer.state.Load() == util.StateRunning { + msgRow := model.MsgRow{Msg: msg, Row: row} + if service.sharder != nil { + if msgRow.Shard, err = service.sharder.Calc(msgRow.Row); err != nil { + util.Logger.Fatal("shard number calculation failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - row = service.metric2Row(metric, msg) - if taskCfg.DynamicSchema.Enable { - foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys, &service.warnKeys, service.whiteList, service.blackList, msg.Partition, msg.Offset) - } - // Dumping message and result - //util.Logger.Debug("parsed kafka message", zap.Int("partition", msg.Partition), zap.Int64("offset", msg.Offset), - // zap.String("message value", string(msg.Value)), zap.String("row(spew)", spew.Sdump(row))) - } - // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. - service.pp.Put(p) - - if foundNewKeys { - cntNewKeys := atomic.AddInt32(&service.cntNewKeys, 1) - if cntNewKeys == 1 { - // The first message which contains new keys triggers flushing - // all messages and scheduling a delayed func to apply schema change. - for _, ring := range service.rings { - if ring != nil { - ring.ForceBatchOrShard(nil) - } - } - if service.sharder != nil { - service.sharder.ForceFlush(nil) - } - if service.tid, err = util.GlobalTimerWheel.Schedule(time.Duration(taskCfg.FlushInterval)*time.Second, service.changeSchema, nil); err != nil { - if errors.Is(err, goetty.ErrSystemStopped) { - util.Logger.Info("Service.put scheduling timer to a stopped timer wheel") - } else { - err = errors.Wrapf(err, "") - util.Logger.Fatal("scheduling timer failed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - } - } - } - if atomic.LoadInt32(&service.cntNewKeys) == 0 { - var ring *Ring - service.Lock() - ring = service.rings[msg.Partition] - service.Unlock() - ring.PutElem(model.MsgRow{Msg: msg, Row: row}) + util.Logger.Fatal("shardingKey must be specified!") } - }) -} - -// drain ensure we have completeted procession(discard or write&commit) for all received messages, and cleared service state. -func (service *Service) drain() { - savedState := atomic.SwapUint32(&service.state, util.StateStopped) - defer atomic.CompareAndSwapUint32(&service.state, util.StateStopped, savedState) - begin := time.Now() - service.Lock() - for service.numFlying != 0 { - service.taskDone.Wait() + service.sharder.PutElement(&msgRow) } - for _, ring := range service.rings { - if ring != nil { - ring.ForceBatchOrShard(nil) - } - } - service.rings = make([]*Ring, 0) - service.Unlock() - if service.sharder != nil { - service.sharder.ForceFlush(nil) - } - util.Logger.Debug("generated flying batches", - zap.String("task", service.taskCfg.Name), - zap.Duration("cost", time.Since(begin))) - service.clickhouse.Drain() - util.Logger.Debug("drained flying batches", - zap.String("task", service.taskCfg.Name), - zap.Duration("cost", time.Since(begin))) -} -func (service *Service) Flush(batch *model.Batch) (err error) { - if (len(*batch.Rows)) == 0 { - return batch.Commit() - } - service.clickhouse.Send(batch) return nil } -func (service *Service) changeSchema(arg interface{}) { - var err error - taskCfg := service.taskCfg - // change schema - if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { - util.Logger.Fatal("clickhouse.ChangeSchema failed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - // restart myself - service.Stop() - if err = service.Init(); err != nil { - util.Logger.Fatal("service.Init failed", zap.String("task", taskCfg.Name), zap.Error(err)) - } - go service.Run() -} - -// Stop stop kafka and clickhouse client. This is blocking. -func (service *Service) Stop() { - taskCfg := service.taskCfg - - util.Logger.Debug("stopping task service...", zap.String("task", taskCfg.Name)) - atomic.StoreUint32(&service.state, util.StateStopped) - for _, ring := range service.rings { - if ring != nil { - ring.mux.Lock() - ring.available.Broadcast() - ring.mux.Unlock() - } - } - - if service.sharder != nil { - service.sharder.tid.Stop() - } - service.tid.Stop() - util.Logger.Debug("stopped internal timers", zap.String("task", taskCfg.Name)) - - if err := service.inputer.Stop(); err != nil { - util.Logger.Fatal("service.inputer.Stop failed", zap.Error(err)) - } - util.Logger.Debug("stopped input", zap.String("task", taskCfg.Name)) - - service.wgRun.Wait() - util.Logger.Debug("stopped task", zap.String("task", taskCfg.Name)) -} - func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) (row *model.Row) { row = model.GetRow() if service.idxSerID >= 0 { diff --git a/util/common.go b/util/common.go index ca38a5b8..47df8b6d 100644 --- a/util/common.go +++ b/util/common.go @@ -25,52 +25,38 @@ import ( "os/exec" "path/filepath" "reflect" - "runtime" "strconv" "strings" - "time" "go.uber.org/zap" "go.uber.org/zap/zapcore" "gopkg.in/natefinch/lumberjack.v2" - "github.com/fagongzi/goetty" "github.com/thanos-io/thanos/pkg/errors" ) var ( - GlobalTimerWheel *goetty.TimeoutWheel //the global timer wheel - GlobalParsingPool *WorkerPool //for all tasks' parsing, cpu intensive - GlobalWritingPool *WorkerPool //the all tasks' writing ClickHouse, cpu-net balance + GlobalWritingPool *WorkerPool //the all tasks' writing ClickHouse, cpu-net balance Logger *zap.Logger logAtomLevel zap.AtomicLevel logPaths []string ) -// InitGlobalTimerWheel initialize the global timer wheel -func InitGlobalTimerWheel() { - if GlobalTimerWheel != nil { - return - } - GlobalTimerWheel = goetty.NewTimeoutWheel(goetty.WithTickInterval(time.Second)) -} - -// InitGlobalParsingPool initialize GlobalParsingPool -func InitGlobalParsingPool() { - if GlobalParsingPool != nil { - return - } - maxWorkers := 10 - if runtime.NumCPU() >= 2 { - if maxWorkers > runtime.NumCPU()/2 { - maxWorkers = runtime.NumCPU() / 2 - } - } else { - maxWorkers = 1 - } - queueSize := 1 << 16 - GlobalParsingPool = NewWorkerPool(maxWorkers, queueSize) - Logger.Info("initialized parsing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", queueSize)) +type CmdOptions struct { + ShowVer bool + LogLevel string // "debug", "info", "warn", "error", "dpanic", "panic", "fatal" + LogPaths string // comma-separated paths. "stdout" means the console stdout + HTTPPort int // 0 menas a randomly OS chosen port + PushGatewayAddrs string + PushInterval int + LocalCfgFile string + NacosAddr string + NacosNamespaceID string + NacosGroup string + NacosUsername string + NacosPassword string + NacosDataID string + NacosServiceName string // participate in assignment management if not empty } // InitGlobalWritingPool initialize GlobalWritingPool From d15caf7276820a15462736a257292e5945b49c28 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Sun, 25 Dec 2022 21:11:44 +0800 Subject: [PATCH 304/404] restart the consumergroup when facing offset commit error --- input/kafka_franz.go | 33 +++++----- task/consumer.go | 49 ++++++++++----- task/sinker.go | 143 ++++++++++++++++++++++++++++--------------- task/task.go | 14 +++-- 4 files changed, 152 insertions(+), 87 deletions(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 7a6769d8..e4c1ccea 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -47,11 +47,18 @@ const ( RetryBackoff = 5 * time.Second ) +type GroupConfig struct { + Name string + Topics []string + Earliest bool + FlushInterval int +} + // KafkaFranz implements input.Inputer // refers to examples/group_consuming/main.go type KafkaFranz struct { cfg *config.Config - taskCfgs []*config.TaskConfig + grpConfig *GroupConfig cl *kgo.Client ctx context.Context cancel context.CancelFunc @@ -66,9 +73,9 @@ func NewKafkaFranz() *KafkaFranz { } // Init Initialise the kafka instance with configuration -func (k *KafkaFranz) Init(cfg *config.Config, taskCfgs []*config.TaskConfig, f chan []*kgo.Record, cleanupFn func()) (err error) { +func (k *KafkaFranz) Init(cfg *config.Config, gCfg *GroupConfig, f chan []*kgo.Record, cleanupFn func()) (err error) { k.cfg = cfg - k.taskCfgs = taskCfgs + k.grpConfig = gCfg k.ctx, k.cancel = context.WithCancel(context.Background()) k.fetch = f k.cleanupFn = cleanupFn @@ -77,20 +84,16 @@ func (k *KafkaFranz) Init(cfg *config.Config, taskCfgs []*config.TaskConfig, f c if opts, err = GetFranzConfig(kfkCfg); err != nil { return } - var topics []string - for _, taskcfg := range k.taskCfgs { - topics = append(topics, taskcfg.Topic) - } opts = append(opts, - kgo.ConsumeTopics(topics...), - kgo.ConsumerGroup(taskCfgs[0].ConsumerGroup), + kgo.ConsumeTopics(k.grpConfig.Topics...), + kgo.ConsumerGroup(k.grpConfig.Name), kgo.DisableAutoCommit(), kgo.OnPartitionsRevoked(k.onPartitionRevoked), kgo.RebalanceTimeout(time.Minute*2), kgo.SessionTimeout(time.Minute*2), kgo.RequestTimeoutOverhead(time.Minute*1), ) - if !taskCfgs[0].Earliest { + if !k.grpConfig.Earliest { opts = append(opts, kgo.ConsumeResetOffset(kgo.NewOffset().AtEnd())) } @@ -186,7 +189,7 @@ func (k *KafkaFranz) Run() { k.fetch <- fetches.Records() } k.cl.Close() // will trigger k.onPartitionRevoked - util.Logger.Info("KafkaFranz.Run quit due to context has been canceled", zap.String("consumer group", k.taskCfgs[0].ConsumerGroup)) + util.Logger.Info("KafkaFranz.Run quit due to context has been canceled", zap.String("consumer group", k.grpConfig.Name)) } func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { @@ -229,17 +232,13 @@ func (k *KafkaFranz) Stop() error { // Description of this kafka consumer, which topic it reads from func (k *KafkaFranz) Description() string { - var topics []string - for _, taskcfg := range k.taskCfgs { - topics = append(topics, taskcfg.Topic) - } - return fmt.Sprint("kafka consumer of topic ", topics) + return fmt.Sprint("kafka consumer of topic ", k.grpConfig.Topics) } func (k *KafkaFranz) onPartitionRevoked(_ context.Context, _ *kgo.Client, _ map[string][]int32) { begin := time.Now() k.cleanupFn() util.Logger.Info("consumer group cleanup", - zap.String("consumer group", k.taskCfgs[0].ConsumerGroup), + zap.String("consumer group", k.grpConfig.Name), zap.Duration("cost", time.Since(begin))) } diff --git a/task/consumer.go b/task/consumer.go index 085247bd..fd9e5e85 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -22,7 +22,6 @@ import ( "sync/atomic" "time" - "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/input" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" @@ -43,11 +42,12 @@ type Consumer struct { sinker *Sinker inputer *input.KafkaFranz tasks sync.Map - cfgs []*config.TaskConfig + grpConfig *input.GroupConfig fetches chan []*kgo.Record processWg sync.WaitGroup stopProcess chan struct{} state atomic.Uint32 + errCommit bool recMap map[string]map[int32]int64 // committed RecMap recMux sync.Mutex @@ -66,6 +66,8 @@ func newConsumer(s *Sinker) *Consumer { c := &Consumer{ sinker: s, numFlying: 0, + errCommit: false, + grpConfig: &input.GroupConfig{}, stopProcess: make(chan struct{}), fetches: make(chan []*kgo.Record), recMap: make(map[string]map[int32]int64), @@ -75,18 +77,35 @@ func newConsumer(s *Sinker) *Consumer { return c } +func (c *Consumer) addTask(tsk *Service) { + cfg := tsk.taskCfg + if c.grpConfig.Name != "" && cfg.Earliest != c.grpConfig.Earliest { + util.Logger.Fatal("Tasks are sharing same consumer group, but with different Earliest property specified!", + zap.String("task", cfg.Name), zap.String("task", c.grpConfig.Name)) + } + c.grpConfig.Earliest = cfg.Earliest + c.grpConfig.Name = cfg.ConsumerGroup + c.grpConfig.FlushInterval = cfg.FlushInterval + c.grpConfig.Topics = append(c.grpConfig.Topics, cfg.Topic) + + c.tasks.Store(cfg.Name, tsk) +} + func (c *Consumer) start() { c.inputer = input.NewKafkaFranz() c.state.Store(util.StateRunning) - if err := c.inputer.Init(c.sinker.curCfg, c.cfgs, c.fetches, c.cleanupFn); err == nil { + if err := c.inputer.Init(c.sinker.curCfg, c.grpConfig, c.fetches, c.cleanupFn); err == nil { go c.inputer.Run() go c.processFetch() } else { - util.Logger.Fatal("failed to init consumer", zap.String("consumer", c.cfgs[0].ConsumerGroup), zap.Error(err)) + util.Logger.Fatal("failed to init consumer", zap.String("consumer", c.grpConfig.Name), zap.Error(err)) } } -func (c *Consumer) stop(force bool) (err error) { +func (c *Consumer) stop() (err error) { + if c.state.Load() == util.StateStopped { + return + } c.state.Store(util.StateStopped) // stop the processFetch routine, make sure no more input to the- commit chan & writing pool @@ -97,8 +116,8 @@ func (c *Consumer) stop(force bool) (err error) { } func (c *Consumer) restart() { - if err := c.stop(false); err != nil { - util.Logger.Fatal("failed to restart consumer group", zap.String("group", c.cfgs[0].ConsumerGroup)) + if err := c.stop(); err != nil { + util.Logger.Fatal("failed to restart consumer group", zap.String("group", c.grpConfig.Name)) } c.start() } @@ -120,7 +139,7 @@ func (c *Consumer) cleanupFn() { // ensure the completion of offset submission c.mux.Lock() for c.numFlying != 0 { - util.Logger.Debug("draining flying pending commits", zap.String("consumergroup", c.cfgs[0].ConsumerGroup), zap.Int32("pending", c.numFlying)) + util.Logger.Debug("draining flying pending commits", zap.String("consumergroup", c.grpConfig.Name), zap.Int32("pending", c.numFlying)) c.commitDone.Wait() } c.mux.Unlock() @@ -153,18 +172,20 @@ func (c *Consumer) processFetch() { c.mux.Lock() c.numFlying++ c.mux.Unlock() - c.sinker.commits <- &Commit{group: c.cfgs[0].ConsumerGroup, offsets: off, wg: &wg, consumer: c} + c.sinker.commits <- &Commit{group: c.grpConfig.Name, offsets: off, wg: &wg, consumer: c} } - for _, it := range c.cfgs { - bufThreshold += int64(it.BufferSize) - } + c.tasks.Range(func(key, value any) bool { + bufThreshold += int64(value.(*Service).taskCfg.BufferSize) + return true + }) + bufThreshold = bufThreshold * int64(len(c.sinker.curCfg.Clickhouse.Hosts)) * 4 / 5 if bufThreshold > MaxCountInBuf { bufThreshold = MaxCountInBuf } - ticker := time.NewTicker(time.Duration(c.cfgs[0].FlushInterval) * time.Second) + ticker := time.NewTicker(time.Duration(c.grpConfig.FlushInterval) * time.Second) defer ticker.Stop() for { select { @@ -244,7 +265,7 @@ func (c *Consumer) processFetch() { flushFn() case <-c.stopProcess: flushFn() - util.Logger.Info("stopped processing loop", zap.String("group", c.cfgs[0].ConsumerGroup)) + util.Logger.Info("stopped processing loop", zap.String("group", c.grpConfig.Name)) return } } diff --git a/task/sinker.go b/task/sinker.go index b6264368..1a4b1142 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -18,8 +18,6 @@ package task import ( "context" "encoding/json" - "errors" - "io" "os" "reflect" "strings" @@ -48,25 +46,27 @@ type Sinker struct { ctx context.Context cancel context.CancelFunc - consumers map[string]*Consumer - commits chan *Commit - exit chan struct{} - stopCommit chan struct{} + consumers map[string]*Consumer + commits chan *Commit + exit chan struct{} + stopCommit chan struct{} + consumerRestart chan *Consumer } // NewSinker get an instance of sinker with the task list func NewSinker(rcm cm.RemoteConfManager, http string, cmd *util.CmdOptions) *Sinker { ctx, cancel := context.WithCancel(context.Background()) s := &Sinker{ - rcm: rcm, - ctx: ctx, - cmdOps: cmd, - cancel: cancel, - commits: make(chan *Commit, 10), - exit: make(chan struct{}), - stopCommit: make(chan struct{}), - consumers: make(map[string]*Consumer), - httpAddr: http, + rcm: rcm, + ctx: ctx, + cmdOps: cmd, + cancel: cancel, + commits: make(chan *Commit, 10), + exit: make(chan struct{}), + stopCommit: make(chan struct{}), + consumerRestart: make(chan *Consumer), + consumers: make(map[string]*Consumer), + httpAddr: http, } return s } @@ -112,7 +112,29 @@ func (s *Sinker) Run() { util.Logger.Fatal("s.applyConfig failed", zap.Error(err)) return } - <-s.ctx.Done() + for { + select { + case <-s.ctx.Done(): + util.Logger.Info("Sinker.Run quit due to context has been canceled") + return + case c := <-s.consumerRestart: + // only restart the consumer which was not changed in applyAnotherConfig + if c == s.consumers[c.grpConfig.Name] { + newGroup := newConsumer(s) + s.consumers[c.grpConfig.Name] = newGroup + c.tasks.Range(func(key, value any) bool { + cloneTask(value.(*Service), newGroup) + return true + }) + newGroup.start() + util.Logger.Info("consumer restarted because of previous offset commit error", + zap.String("consumer", c.grpConfig.Name)) + } else { + util.Logger.Info("consumer restarted when applying another config", + zap.String("consumer", c.grpConfig.Name)) + } + } + } } else { if s.cmdOps.NacosServiceName != "" { go s.rcm.Run() @@ -138,6 +160,22 @@ func (s *Sinker) Run() { util.Logger.Error("s.applyConfig failed", zap.Error(err)) continue } + case c := <-s.consumerRestart: + // only restart the consumer which was not changed in applyAnotherConfig + if c == s.consumers[c.grpConfig.Name] { + newGroup := newConsumer(s) + s.consumers[c.grpConfig.Name] = newGroup + c.tasks.Range(func(key, value any) bool { + cloneTask(value.(*Service), newGroup) + return true + }) + newGroup.start() + util.Logger.Info("consumer restarted because of previous offset commit error", + zap.String("consumer", c.grpConfig.Name)) + } else { + util.Logger.Info("consumer restarted when applying another config", + zap.String("consumer", c.grpConfig.Name)) + } } } } @@ -169,15 +207,18 @@ func (s *Sinker) stopAllTasks() { if v.state.Load() == util.StateRunning { wg.Add(1) go func(c *Consumer) { - c.stop(false) + c.stop() wg.Done() }(v) } } wg.Wait() - util.Logger.Info("stopped all consumers") - s.stopCommit <- struct{}{} + + select { + case s.stopCommit <- struct{}{}: + default: + } util.Logger.Debug("stopped commit session") for name := range s.consumers { @@ -224,12 +265,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { } var c *Consumer var ok bool - if c, ok = s.consumers[taskCfg.ConsumerGroup]; ok { - if taskCfg.Earliest != c.cfgs[0].Earliest { - util.Logger.Fatal("Tasks are sharing same consumer group, but with different Earliest property specified!", - zap.String("task", taskCfg.Name), zap.String("task", c.cfgs[0].Name)) - } - } else { + if c, ok = s.consumers[taskCfg.ConsumerGroup]; !ok { c = newConsumer(s) s.consumers[taskCfg.ConsumerGroup] = c } @@ -276,12 +312,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { } var c *Consumer var ok bool - if c, ok = s.consumers[taskCfg.ConsumerGroup]; ok { - if taskCfg.Earliest != c.cfgs[0].Earliest { - util.Logger.Fatal("Tasks are sharing same consumer group, but with different Earliest property specified!", - zap.String("task", taskCfg.Name), zap.String("task", c.cfgs[0].Name)) - } - } else { + if c, ok = s.consumers[taskCfg.ConsumerGroup]; !ok { c = newConsumer(s) s.consumers[taskCfg.ConsumerGroup] = c } @@ -322,29 +353,31 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { var ok bool if group, ok = newConsumers[name]; !ok { // consumer group no longer with this client, stop it - go c.stop(true) + go c.stop() deleteConsumers = append(deleteConsumers, name) } else { // find the one that need to be restart var groupChanged bool - if len(group) != len(c.cfgs) { + if len(group) != len(c.grpConfig.Topics) { groupChanged = true } else { - for _, cfg := range c.cfgs { + c.tasks.Range(func(key, value any) bool { + cfg := value.(*Service).taskCfg var it *config.TaskConfig var ok bool if it, ok = group[cfg.Name]; !ok { groupChanged = true - break + return false } else if !reflect.DeepEqual(it, cfg) { groupChanged = true - break + return false } - } + return true + }) } // restart the group accordingly if groupChanged { - go c.stop(true) + go c.stop() toCreate[name] = group } delete(newConsumers, name) @@ -382,20 +415,28 @@ func (s *Sinker) commitFn() { case com := <-s.commits: com.wg.Wait() c := com.consumer - c.recMux.Lock() - err := json.Unmarshal(com.offsets, &c.recMap) - c.recMux.Unlock() - if err != nil { - util.Logger.Fatal("Failed to unmarshall offsets", zap.ByteString("offsets", com.offsets), zap.Error(err)) - } - for i, value := range c.recMap { - for k, v := range value { - if err := c.inputer.CommitMessages(&model.InputMessage{Topic: i, Partition: int(k), Offset: v}); err != nil { - // Note: kafka_go and sarama commit throws different error for context cancellation. - if errors.Is(err, context.Canceled) || errors.Is(err, io.ErrClosedPipe) { - util.Logger.Warn("Batch.Commit failed due to the context has been cancelled", zap.Error(err)) + + if !c.errCommit { + c.recMux.Lock() + err := json.Unmarshal(com.offsets, &c.recMap) + c.recMux.Unlock() + if err != nil { + util.Logger.Fatal("Failed to unmarshall offsets", zap.ByteString("offsets", com.offsets), zap.Error(err)) + } + LOOP: + for i, value := range c.recMap { + for k, v := range value { + if err := c.inputer.CommitMessages(&model.InputMessage{Topic: i, Partition: int(k), Offset: v}); err != nil { + c.errCommit = true + // restart the consumer when facing commit error, avoid change the s.consumers outside of s.Run + // error could be RebalanceInProgress, IllegalGeneration, UnknownMemberID + go func() { + c.stop() + s.consumerRestart <- c + }() + util.Logger.Warn("Batch.Commit failed, will restart later", zap.Error(err)) + break LOOP } - util.Logger.Fatal("Batch.Commit failed with permanent error", zap.Error(err)) } } } diff --git a/task/task.go b/task/task.go index f396805b..db38342b 100644 --- a/task/task.go +++ b/task/task.go @@ -61,7 +61,7 @@ type Service struct { } // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances -func cloneTask(s *Service) (service *Service) { +func cloneTask(s *Service, newGroup *Consumer) (service *Service) { service = &Service{ clickhouse: s.clickhouse, pp: s.pp, @@ -71,7 +71,12 @@ func cloneTask(s *Service) (service *Service) { blackList: s.blackList, lblBlkList: s.lblBlkList, } - service.Init() + if newGroup != nil { + service.consumer = newGroup + } + if err := service.Init(); err != nil { + util.Logger.Fatal("failed to clone task", zap.String("group", service.taskCfg.ConsumerGroup), zap.String("task", service.taskCfg.Name)) + } return } @@ -139,8 +144,7 @@ func (service *Service) Init() (err error) { atomic.StoreInt32(&service.cntNewKeys, 0) } } - service.consumer.tasks.Store(taskCfg.Name, service) - service.consumer.cfgs = append(service.consumer.cfgs, service.taskCfg) + service.consumer.addTask(service) return } @@ -190,7 +194,7 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { util.Logger.Fatal("clickhouse.ChangeSchema failed", zap.String("task", taskCfg.Name), zap.Error(err)) } - service.consumer.tasks.Store(service.taskCfg.Name, cloneTask(service)) + cloneTask(service, nil) return fmt.Errorf("consumer restart required due to new key") } From 379f6573aceeb2bbf18b688b1fda34f022d11165 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Wed, 28 Dec 2022 22:37:48 +0800 Subject: [PATCH 305/404] calculate the shard number based on message offset when shardingKey was not specified --- input/kafka_franz.go | 15 +++++----- task/consumer.go | 65 ++++++++++++++++++++++++++------------------ task/sharding.go | 6 ++-- task/sinker.go | 46 +++++++++++++++---------------- task/task.go | 17 ++++-------- 5 files changed, 79 insertions(+), 70 deletions(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index e4c1ccea..9bdb40b0 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -63,7 +63,7 @@ type KafkaFranz struct { ctx context.Context cancel context.CancelFunc wgRun sync.WaitGroup - fetch chan []*kgo.Record + fetch chan *kgo.Fetches cleanupFn func() } @@ -73,7 +73,7 @@ func NewKafkaFranz() *KafkaFranz { } // Init Initialise the kafka instance with configuration -func (k *KafkaFranz) Init(cfg *config.Config, gCfg *GroupConfig, f chan []*kgo.Record, cleanupFn func()) (err error) { +func (k *KafkaFranz) Init(cfg *config.Config, gCfg *GroupConfig, f chan *kgo.Fetches, cleanupFn func()) (err error) { k.cfg = cfg k.grpConfig = gCfg k.ctx, k.cancel = context.WithCancel(context.Background()) @@ -110,6 +110,7 @@ func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { // kgo.FetchMaxBytes(), // 50 MB -- take the default config // kgo.BrokerMaxReadBytes(), // 100 MB kgo.FetchMaxPartitionBytes(1 << 24), // 16MB + kgo.MaxConcurrentFetches(2), //kgo.MetadataMaxAge(...) corresponds to sarama.Config.Metadata.RefreshFrequency kgo.WithLogger(kzap.New(util.Logger)), } @@ -185,8 +186,8 @@ func (k *KafkaFranz) Run() { util.Logger.Info("kgo.Client.PollFetchs() got an error", zap.Error(err)) } - util.Logger.Debug("Records fetched", zap.String("records", strconv.Itoa(fetches.NumRecords()))) - k.fetch <- fetches.Records() + util.Logger.Debug("Records fetched", zap.String("records", strconv.Itoa(fetches.NumRecords())), zap.String("consumer group", k.grpConfig.Name)) + k.fetch <- &fetches } k.cl.Close() // will trigger k.onPartitionRevoked util.Logger.Info("KafkaFranz.Run quit due to context has been canceled", zap.String("consumer group", k.grpConfig.Name)) @@ -202,7 +203,7 @@ func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { } err = errors.Wrapf(err, "") if i < CommitRetries-1 && !errors.Is(err, context.Canceled) { - util.Logger.Error("cl.CommitRecords failed, will retry later", zap.String("topic", msg.Topic), zap.Int("try", i), zap.Error(err)) + util.Logger.Error("cl.CommitRecords failed, will retry later", zap.String("consumer group", k.grpConfig.Name), zap.Int("try", i), zap.Error(err)) time.Sleep(RetryBackoff) } } @@ -230,9 +231,9 @@ func (k *KafkaFranz) Stop() error { return nil } -// Description of this kafka consumer, which topic it reads from +// Description of this kafka consumer, consumer group name func (k *KafkaFranz) Description() string { - return fmt.Sprint("kafka consumer of topic ", k.grpConfig.Topics) + return fmt.Sprint("kafka consumer group ", k.grpConfig.Name) } func (k *KafkaFranz) onPartitionRevoked(_ context.Context, _ *kgo.Client, _ map[string][]int32) { diff --git a/task/consumer.go b/task/consumer.go index fd9e5e85..2b74c84e 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -39,15 +39,15 @@ type Commit struct { } type Consumer struct { - sinker *Sinker - inputer *input.KafkaFranz - tasks sync.Map - grpConfig *input.GroupConfig - fetches chan []*kgo.Record - processWg sync.WaitGroup - stopProcess chan struct{} - state atomic.Uint32 - errCommit bool + sinker *Sinker + inputer *input.KafkaFranz + tasks sync.Map + grpConfig *input.GroupConfig + fetchesCh chan *kgo.Fetches + processWg sync.WaitGroup + stopCh chan struct{} + state atomic.Uint32 + errCommit bool recMap map[string]map[int32]int64 // committed RecMap recMux sync.Mutex @@ -64,13 +64,13 @@ const ( func newConsumer(s *Sinker) *Consumer { c := &Consumer{ - sinker: s, - numFlying: 0, - errCommit: false, - grpConfig: &input.GroupConfig{}, - stopProcess: make(chan struct{}), - fetches: make(chan []*kgo.Record), - recMap: make(map[string]map[int32]int64), + sinker: s, + numFlying: 0, + errCommit: false, + grpConfig: &input.GroupConfig{}, + stopCh: make(chan struct{}), + fetchesCh: make(chan *kgo.Fetches), + recMap: make(map[string]map[int32]int64), } c.state.Store(util.StateStopped) c.commitDone = sync.NewCond(&c.mux) @@ -94,7 +94,7 @@ func (c *Consumer) addTask(tsk *Service) { func (c *Consumer) start() { c.inputer = input.NewKafkaFranz() c.state.Store(util.StateRunning) - if err := c.inputer.Init(c.sinker.curCfg, c.grpConfig, c.fetches, c.cleanupFn); err == nil { + if err := c.inputer.Init(c.sinker.curCfg, c.grpConfig, c.fetchesCh, c.cleanupFn); err == nil { go c.inputer.Run() go c.processFetch() } else { @@ -109,7 +109,7 @@ func (c *Consumer) stop() (err error) { c.state.Store(util.StateStopped) // stop the processFetch routine, make sure no more input to the- commit chan & writing pool - c.stopProcess <- struct{}{} + c.stopCh <- struct{}{} c.processWg.Wait() err = c.inputer.Stop() return err @@ -117,7 +117,7 @@ func (c *Consumer) stop() (err error) { func (c *Consumer) restart() { if err := c.stop(); err != nil { - util.Logger.Fatal("failed to restart consumer group", zap.String("group", c.grpConfig.Name)) + util.Logger.Fatal("failed to restart consumer group", zap.String("group", c.grpConfig.Name), zap.Error(err)) } c.start() } @@ -172,7 +172,7 @@ func (c *Consumer) processFetch() { c.mux.Lock() c.numFlying++ c.mux.Unlock() - c.sinker.commits <- &Commit{group: c.grpConfig.Name, offsets: off, wg: &wg, consumer: c} + c.sinker.commitsCh <- &Commit{group: c.grpConfig.Name, offsets: off, wg: &wg, consumer: c} } c.tasks.Range(func(key, value any) bool { @@ -189,11 +189,12 @@ func (c *Consumer) processFetch() { defer ticker.Stop() for { select { - case fetch := <-c.fetches: + case fetches := <-c.fetchesCh: if c.state.Load() == util.StateStopped { continue } + fetch := fetches.Records() items, done := int64(len(fetch)), int64(-1) var concurrency int if concurrency = int(items/1000) + 1; concurrency > MaxParallelism { @@ -249,12 +250,22 @@ func (c *Consumer) processFetch() { // record the latest offset in order // assume the c.state was reset to stopped when facing error, so that further fetch won't get processed if err == nil { - for i, it := range fetch { - if recMap[it.Topic] == nil { - recMap[it.Topic] = make(map[int32]int64) + for _, f := range *fetches { + for i := range f.Topics { + ft := &f.Topics[i] + if recMap[ft.Topic] == nil { + recMap[ft.Topic] = make(map[int32]int64) + } + for j := range ft.Partitions { + fpr := ft.Partitions[j].Records + lastOff := fpr[len(fpr)-1].Offset + + old, ok := recMap[ft.Topic][ft.Partitions[j].Partition] + if !ok || old < lastOff { + recMap[ft.Topic][ft.Partitions[j].Partition] = lastOff + } + } } - recMap[it.Topic][it.Partition] = it.Offset - fetch[i] = nil } } @@ -263,7 +274,7 @@ func (c *Consumer) processFetch() { } case <-ticker.C: flushFn() - case <-c.stopProcess: + case <-c.stopCh: flushFn() util.Logger.Info("stopped processing loop", zap.String("group", c.grpConfig.Name)) return diff --git a/task/sharding.go b/task/sharding.go index 5ede029e..c850ab16 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -119,8 +119,10 @@ func NewSharder(service *Service) (sh *Sharder, err error) { var policy *ShardingPolicy shards := pool.NumShard() taskCfg := service.taskCfg - if policy, err = NewShardingPolicy(taskCfg.ShardingKey, taskCfg.ShardingStripe, service.clickhouse.Dims, shards); err != nil { - return + if taskCfg.ShardingKey != "" { + if policy, err = NewShardingPolicy(taskCfg.ShardingKey, taskCfg.ShardingStripe, service.clickhouse.Dims, shards); err != nil { + return + } } sh = &Sharder{ service: service, diff --git a/task/sinker.go b/task/sinker.go index 1a4b1142..5b57d34f 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -46,27 +46,27 @@ type Sinker struct { ctx context.Context cancel context.CancelFunc - consumers map[string]*Consumer - commits chan *Commit - exit chan struct{} - stopCommit chan struct{} - consumerRestart chan *Consumer + consumers map[string]*Consumer + commitsCh chan *Commit + exitCh chan struct{} + stopCommitCh chan struct{} + consumerRestartCh chan *Consumer } // NewSinker get an instance of sinker with the task list func NewSinker(rcm cm.RemoteConfManager, http string, cmd *util.CmdOptions) *Sinker { ctx, cancel := context.WithCancel(context.Background()) s := &Sinker{ - rcm: rcm, - ctx: ctx, - cmdOps: cmd, - cancel: cancel, - commits: make(chan *Commit, 10), - exit: make(chan struct{}), - stopCommit: make(chan struct{}), - consumerRestart: make(chan *Consumer), - consumers: make(map[string]*Consumer), - httpAddr: http, + rcm: rcm, + ctx: ctx, + cmdOps: cmd, + cancel: cancel, + commitsCh: make(chan *Commit, 10), + exitCh: make(chan struct{}), + stopCommitCh: make(chan struct{}), + consumerRestartCh: make(chan *Consumer), + consumers: make(map[string]*Consumer), + httpAddr: http, } return s } @@ -84,7 +84,7 @@ func (s *Sinker) Run() { var err error var newCfg *config.Config defer func() { - s.exit <- struct{}{} + s.exitCh <- struct{}{} }() if s.cmdOps.PushGatewayAddrs != "" { addrs := strings.Split(s.cmdOps.PushGatewayAddrs, ",") @@ -117,7 +117,7 @@ func (s *Sinker) Run() { case <-s.ctx.Done(): util.Logger.Info("Sinker.Run quit due to context has been canceled") return - case c := <-s.consumerRestart: + case c := <-s.consumerRestartCh: // only restart the consumer which was not changed in applyAnotherConfig if c == s.consumers[c.grpConfig.Name] { newGroup := newConsumer(s) @@ -160,7 +160,7 @@ func (s *Sinker) Run() { util.Logger.Error("s.applyConfig failed", zap.Error(err)) continue } - case c := <-s.consumerRestart: + case c := <-s.consumerRestartCh: // only restart the consumer which was not changed in applyAnotherConfig if c == s.consumers[c.grpConfig.Name] { newGroup := newConsumer(s) @@ -190,7 +190,7 @@ func (s *Sinker) Close() { } // 2. Quit Run mainloop s.cancel() - <-s.exit + <-s.exitCh // 3. Stop tasks gracefully. s.stopAllTasks() // 4. Stop pusher @@ -216,7 +216,7 @@ func (s *Sinker) stopAllTasks() { util.Logger.Info("stopped all consumers") select { - case s.stopCommit <- struct{}{}: + case s.stopCommitCh <- struct{}{}: default: } util.Logger.Debug("stopped commit session") @@ -412,7 +412,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { func (s *Sinker) commitFn() { for { select { - case com := <-s.commits: + case com := <-s.commitsCh: com.wg.Wait() c := com.consumer @@ -432,7 +432,7 @@ func (s *Sinker) commitFn() { // error could be RebalanceInProgress, IllegalGeneration, UnknownMemberID go func() { c.stop() - s.consumerRestart <- c + s.consumerRestartCh <- c }() util.Logger.Warn("Batch.Commit failed, will restart later", zap.Error(err)) break LOOP @@ -446,7 +446,7 @@ func (s *Sinker) commitFn() { c.commitDone.Broadcast() } c.mux.Unlock() - case <-s.stopCommit: + case <-s.stopCommitCh: util.Logger.Info("stopped committing loop") return } diff --git a/task/task.go b/task/task.go index db38342b..938c751e 100644 --- a/task/task.go +++ b/task/task.go @@ -60,7 +60,7 @@ type Service struct { consumer *Consumer } -// NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances +// cloneTask create a new task by steal members from s instead of creating a new one func cloneTask(s *Service, newGroup *Consumer) (service *Service) { service = &Service{ clickhouse: s.clickhouse, @@ -75,7 +75,7 @@ func cloneTask(s *Service, newGroup *Consumer) (service *Service) { service.consumer = newGroup } if err := service.Init(); err != nil { - util.Logger.Fatal("failed to clone task", zap.String("group", service.taskCfg.ConsumerGroup), zap.String("task", service.taskCfg.Name)) + util.Logger.Fatal("failed to clone task", zap.String("group", service.taskCfg.ConsumerGroup), zap.String("task", service.taskCfg.Name), zap.Error(err)) } return @@ -118,10 +118,8 @@ func (service *Service) Init() (err error) { service.limiter1 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter2 = rate.NewLimiter(rate.Every(10*time.Second), 1) - if taskCfg.ShardingKey != "" { - if service.sharder, err = NewSharder(service); err != nil { - return - } + if service.sharder, err = NewSharder(service); err != nil { + return } if taskCfg.DynamicSchema.Enable { @@ -171,9 +169,6 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { if taskCfg.DynamicSchema.Enable { foundNewKeys = metric.GetNewKeys(&service.knownKeys, &service.newKeys, &service.warnKeys, service.whiteList, service.blackList, msg.Partition, msg.Offset) } - // Dumping message and result - //util.Logger.Debug("parsed kafka message", zap.Int("partition", msg.Partition), zap.Int64("offset", msg.Offset), - // zap.String("message value", string(msg.Value)), zap.String("row(spew)", spew.Sdump(row))) } // WARNNING: metric.GetXXX may depend on p. Don't call them after p been freed. service.pp.Put(p) @@ -202,12 +197,12 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { if atomic.LoadInt32(&service.cntNewKeys) == 0 && service.consumer.state.Load() == util.StateRunning { msgRow := model.MsgRow{Msg: msg, Row: row} - if service.sharder != nil { + if service.sharder.policy != nil { if msgRow.Shard, err = service.sharder.Calc(msgRow.Row); err != nil { util.Logger.Fatal("shard number calculation failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - util.Logger.Fatal("shardingKey must be specified!") + msgRow.Shard = int(msgRow.Msg.Offset>>17) % service.sharder.shards } service.sharder.PutElement(&msgRow) } From dfc271dfb425e74466cd59495aae312c30b77cf5 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 30 Dec 2022 15:41:33 +0800 Subject: [PATCH 306/404] Build up the Consumers map when normalizing the Config --- clickhouse_sinker-dashboard.json | 968 +++++++++++++++++-------------- cmd/nacos_publish_config/main.go | 2 +- config/config.go | 48 +- config_manager/nacos.go | 2 +- docker/test_auto_schema.hjson | 1 - docker/test_dynamic_schema.hjson | 1 - docker/test_fixed_schema.hjson | 1 - input/kafka_franz.go | 11 +- statistics/statistics.go | 88 +-- task/consumer.go | 65 +-- task/sinker.go | 196 +++---- 11 files changed, 703 insertions(+), 680 deletions(-) diff --git a/clickhouse_sinker-dashboard.json b/clickhouse_sinker-dashboard.json index dffe34b3..93a079fb 100644 --- a/clickhouse_sinker-dashboard.json +++ b/clickhouse_sinker-dashboard.json @@ -3,590 +3,689 @@ "list": [ { "builtIn": 1, - "datasource": "-- Grafana --", + "datasource": { + "type": "datasource", + "uid": "grafana" + }, "enable": true, "hide": true, "iconColor": "rgba(0, 211, 255, 1)", "name": "Annotations & Alerts", + "target": { + "limit": 100, + "matchAny": false, + "tags": [], + "type": "dashboard" + }, "type": "dashboard" } ] }, "editable": true, - "gnetId": null, + "fiscalYearStartMonth": 0, "graphTooltip": 0, - "id": 2, - "iteration": 1605515955085, + "id": 32, "links": [], + "liveNow": false, "panels": [ { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "$datasource", + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 18, + "panels": [], + "repeat": "datasource", + "repeatDirection": "h", + "title": "Global", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "$datasource" + }, "fieldConfig": { "defaults": { - "custom": {} + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" }, "overrides": [] }, - "fill": 1, - "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 0 + "y": 1 }, - "hiddenSeries": false, "id": 6, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "percentage": false, - "pluginVersion": "7.1.5", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "pluginVersion": "9.1.2", "targets": [ { - "expr": "sum by(task)(rate(clickhouse_sinker_consume_msgs_total[1m]))", + "datasource": { + "uid": "$datasource" + }, + "editorMode": "code", + "expr": "sum(sum by(task)(rate(clickhouse_sinker_consume_msgs_total[1m])))", "interval": "", "legendFormat": "", + "range": true, "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, "title": "clickhouse_sinker_consume_msgs_total rate", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "type": "timeseries" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "$datasource", + "datasource": { + "type": "prometheus", + "uid": "$datasource" + }, "fieldConfig": { "defaults": { - "custom": {} + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" }, "overrides": [] }, - "fill": 1, - "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 0 + "y": 1 }, - "hiddenSeries": false, "id": 16, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "percentage": false, - "pluginVersion": "7.1.5", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "pluginVersion": "9.1.2", "targets": [ { - "expr": "sum by(task) (rate(clickhouse_sinker_flush_msgs_total[1m]))", + "datasource": { + "uid": "$datasource" + }, + "editorMode": "code", + "expr": "sum(sum by(task) (rate(clickhouse_sinker_flush_msgs_total[1m])))", "interval": "", "legendFormat": "", + "range": true, "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, "title": "clickhouse_sinker_flush_msgs_total rate", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "type": "timeseries" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "$datasource", + "datasource": { + "type": "prometheus", + "uid": "$datasource" + }, "fieldConfig": { "defaults": { - "custom": {} + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" }, "overrides": [] }, - "fill": 1, - "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 8 + "y": 9 }, - "hiddenSeries": false, - "id": 8, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false + "id": 12, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "percentage": false, - "pluginVersion": "7.1.5", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "pluginVersion": "9.1.2", "targets": [ { - "expr": "sum by(task) (clickhouse_sinker_ring_msgs)", + "datasource": { + "uid": "$datasource" + }, + "editorMode": "code", + "expr": "sum(sum by(task) (clickhouse_sinker_writing_pool_backlog))", "interval": "", "legendFormat": "", + "range": true, "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "clickhouse_sinker_ring_msgs", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "title": "clickhouse_sinker_writing_pool_backlog", + "type": "timeseries" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "$datasource", + "datasource": { + "type": "prometheus", + "uid": "$datasource" + }, "fieldConfig": { "defaults": { - "custom": {} + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" }, "overrides": [] }, - "fill": 1, - "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 8 + "y": 9 }, - "hiddenSeries": false, "id": 10, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "percentage": false, - "pluginVersion": "7.1.5", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "pluginVersion": "9.1.2", "targets": [ { - "expr": "sum by(task) (clickhouse_sinker_shard_msgs)", + "datasource": { + "uid": "$datasource" + }, + "editorMode": "code", + "expr": "sum(sum by(task) (clickhouse_sinker_shard_msgs))", "interval": "", "legendFormat": "", + "range": true, "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, "title": "clickhouse_sinker_shard_msgs", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "type": "timeseries" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "$datasource", + "datasource": { + "type": "prometheus", + "uid": "$datasource" + }, "fieldConfig": { "defaults": { - "custom": {} + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" }, "overrides": [] }, - "fill": 1, - "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 16 + "y": 17 }, - "hiddenSeries": false, - "id": 14, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false + "id": 8, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "percentage": false, - "pluginVersion": "7.1.5", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "pluginVersion": "9.1.2", "targets": [ { - "expr": "sum by(task)(clickhouse_sinker_parsing_pool_backlog)", + "datasource": { + "uid": "$datasource" + }, + "editorMode": "code", + "expr": "sum by(task)(rate(clickhouse_sinker_parse_msgs_error_total[1m]))", "interval": "", "legendFormat": "", + "range": true, "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "clickhouse_sinker_parsing_pool_backlog", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "title": "clickhouse_sinker_parse_msgs_error_total", + "type": "timeseries" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "$datasource", + "datasource": { + "type": "prometheus", + "uid": "$datasource" + }, "fieldConfig": { "defaults": { - "custom": {} + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" }, "overrides": [] }, - "fill": 1, - "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 16 + "y": 17 }, - "hiddenSeries": false, - "id": 12, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false + "id": 14, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "percentage": false, - "pluginVersion": "7.1.5", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "pluginVersion": "9.1.2", "targets": [ { - "expr": "sum by(task) (clickhouse_sinker_writing_pool_backlog)", + "datasource": { + "uid": "$datasource" + }, + "editorMode": "code", + "expr": "sum(sum by(task)(rate(clickhouse_sinker_flush_msgs_error_total[1m])))", "interval": "", "legendFormat": "", + "range": true, "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "clickhouse_sinker_writing_pool_backlog", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" + "title": "clickhouse_sinker_flush_msgs_error_total", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "custom": { + "align": "auto", + "displayMode": "auto", + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] + "gridPos": { + "h": 9, + "w": 24, + "x": 0, + "y": 25 }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "id": 20, + "options": { + "footer": { + "fields": "", + "reducer": [ + "sum" + ], + "show": false }, + "frameIndex": 7, + "showHeader": true + }, + "pluginVersion": "9.1.2", + "targets": [ { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "clickhouse_sinker_consume_offsets", + "legendFormat": "__auto", + "range": true, + "refId": "A" } ], - "yaxis": { - "align": false, - "alignLevel": null - } + "title": "committed offset", + "type": "table" } ], - "refresh": "30s", - "schemaVersion": 26, + "refresh": "5s", + "schemaVersion": 37, "style": "dark", "tags": [], "templating": { "list": [ { "current": { - "selected": true, - "text": "prom106", - "value": "prom106" + "selected": false, + "text": "Prometheus-12", + "value": "Prometheus-12" }, "hide": 0, "includeAll": false, - "label": null, "multi": false, "name": "datasource", "options": [], @@ -600,7 +699,7 @@ ] }, "time": { - "from": "now-6h", + "from": "now-15m", "to": "now" }, "timepicker": { @@ -618,7 +717,8 @@ ] }, "timezone": "", - "title": "clickhouse_sinker", - "uid": "8f73nucGz", - "version": 32 + "title": "clickhouse_sinker_new", + "uid": "8f73nucGa", + "version": 3, + "weekStart": "" } \ No newline at end of file diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 78f9eead..8bbe8099 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -74,7 +74,7 @@ func PublishSinkerConfig() { return } - if err = cfg.Normallize(); err != nil { + if err = cfg.Normallize(false, ""); err != nil { util.Logger.Fatal("cfg.Normallize failed", zap.Error(err)) return } diff --git a/config/config.go b/config/config.go index 6d7f7fa0..559f6738 100644 --- a/config/config.go +++ b/config/config.go @@ -21,6 +21,7 @@ import ( "strings" "github.com/hjson/hjson-go/v4" + "go.uber.org/zap" "github.com/housepower/clickhouse_sinker/util" @@ -35,6 +36,7 @@ type Config struct { Tasks []*TaskConfig Assignment Assignment LogLevel string + Groups map[string]*GroupConfig `json:"-"` } // KafkaConfig configuration parameters @@ -146,7 +148,7 @@ type TaskConfig struct { // ShardingKey is the column name to which sharding against ShardingKey string `json:"shardingKey,omitempty"` - // ShardingStripe take effect iff the sharding key is numerical + // ShardingStripe take effect if the sharding key is numerical ShardingStripe uint64 `json:"shardingStripe,omitempty"` FlushInterval int `json:"flushInterval,omitempty"` @@ -155,6 +157,15 @@ type TaskConfig struct { TimeUnit float64 `json:"timeUnit"` } +type GroupConfig struct { + Name string + Topics []string + Earliest bool + FlushInterval int + BufferSize int + Configs map[string]*TaskConfig +} + type Assignment struct { Version int UpdatedAt int64 // timestamp when created @@ -175,7 +186,9 @@ const ( ) func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { - cfg = &Config{} + cfg = &Config{ + Groups: make(map[string]*GroupConfig), + } var b []byte b, err = os.ReadFile(cfgPath) if err != nil { @@ -190,7 +203,7 @@ func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { } // Normalize and validate configuration -func (cfg *Config) Normallize() (err error) { +func (cfg *Config) Normallize(constructGroup bool, httpAddr string) (err error) { if len(cfg.Clickhouse.Hosts) == 0 || cfg.Kafka.Brokers == "" { err = errors.Newf("invalid configuration") return @@ -244,6 +257,35 @@ func (cfg *Config) Normallize() (err error) { if err = cfg.normallizeTask(taskCfg); err != nil { return } + if constructGroup { + if httpAddr != "" && !cfg.IsAssigned(httpAddr, taskCfg.Name) { + continue + } + gCfg, ok := cfg.Groups[taskCfg.ConsumerGroup] + if !ok { + gCfg = &GroupConfig{ + Name: taskCfg.ConsumerGroup, + Earliest: taskCfg.Earliest, + Topics: []string{taskCfg.Topic}, + FlushInterval: taskCfg.FlushInterval, + BufferSize: taskCfg.BufferSize, + Configs: make(map[string]*TaskConfig), + } + gCfg.Configs[taskCfg.Name] = taskCfg + cfg.Groups[taskCfg.ConsumerGroup] = gCfg + } else { + if gCfg.Earliest != taskCfg.Earliest { + util.Logger.Fatal("Tasks are sharing same consumer group, but with different Earliest property specified!", + zap.String("task", gCfg.Name), zap.String("task", taskCfg.Name)) + } else if gCfg.FlushInterval != taskCfg.FlushInterval { + util.Logger.Fatal("Tasks are sharing same consumer group, but with different FlushInterval property specified!", + zap.String("task", gCfg.Name), zap.String("task", taskCfg.Name)) + } + gCfg.Topics = append(gCfg.Topics, taskCfg.Topic) + gCfg.BufferSize += taskCfg.BufferSize + gCfg.Configs[taskCfg.Name] = taskCfg + } + } } switch strings.ToLower(cfg.LogLevel) { case "debug", "info", "warn", "error", "dpanic", "panic", "fatal": diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 883b7443..766a3a14 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -136,7 +136,7 @@ func (ncm *NacosConfManager) GetConfig() (conf *config.Config, err error) { err = errors.Wrapf(err, "") return } - conf = &config.Config{} + conf = &config.Config{Groups: make(map[string]*config.GroupConfig)} if err = hjson.Unmarshal([]byte(content), conf); err != nil { err = errors.Wrapf(err, "") return diff --git a/docker/test_auto_schema.hjson b/docker/test_auto_schema.hjson index 59d5802d..efc3c7c8 100644 --- a/docker/test_auto_schema.hjson +++ b/docker/test_auto_schema.hjson @@ -22,7 +22,6 @@ parser: json autoSchema: true tableName: test_auto_schema - shardingKey: name excludeColumns: [] bufferSize: 50000 } diff --git a/docker/test_dynamic_schema.hjson b/docker/test_dynamic_schema.hjson index cec71991..7840d82d 100644 --- a/docker/test_dynamic_schema.hjson +++ b/docker/test_dynamic_schema.hjson @@ -30,7 +30,6 @@ dynamicSchema: { enable: true } - shardingKey: name bufferSize: 50000 } logLevel: info diff --git a/docker/test_fixed_schema.hjson b/docker/test_fixed_schema.hjson index 8b23fd24..aa5f2d6f 100644 --- a/docker/test_fixed_schema.hjson +++ b/docker/test_fixed_schema.hjson @@ -36,7 +36,6 @@ type: Float32 } ] - shardingKey: name bufferSize: 50000 } logLevel: info diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 9bdb40b0..6eb72209 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -47,18 +47,11 @@ const ( RetryBackoff = 5 * time.Second ) -type GroupConfig struct { - Name string - Topics []string - Earliest bool - FlushInterval int -} - // KafkaFranz implements input.Inputer // refers to examples/group_consuming/main.go type KafkaFranz struct { cfg *config.Config - grpConfig *GroupConfig + grpConfig *config.GroupConfig cl *kgo.Client ctx context.Context cancel context.CancelFunc @@ -73,7 +66,7 @@ func NewKafkaFranz() *KafkaFranz { } // Init Initialise the kafka instance with configuration -func (k *KafkaFranz) Init(cfg *config.Config, gCfg *GroupConfig, f chan *kgo.Fetches, cleanupFn func()) (err error) { +func (k *KafkaFranz) Init(cfg *config.Config, gCfg *config.GroupConfig, f chan *kgo.Fetches, cleanupFn func()) (err error) { k.cfg = cfg k.grpConfig = gCfg k.ctx, k.cancel = context.WithCancel(context.Background()) diff --git a/statistics/statistics.go b/statistics/statistics.go index 42a7be77..ffddbe04 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -40,13 +40,6 @@ var ( }, []string{"task"}, ) - ConsumeMsgsErrorTotal = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Name: prefix + "consumer_msgs_error_total", - Help: "total num of consume errors", - }, - []string{"group"}, - ) ParseMsgsErrorTotal = prometheus.NewCounterVec( prometheus.CounterOpts{ Name: prefix + "parse_msgs_error_total", @@ -54,41 +47,6 @@ var ( }, []string{"task"}, ) - RingMsgsOffTooSmallErrorTotal = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Name: prefix + "ring_msgs_offset_too_small_error_total", - Help: "total num of msgs with too small offset to put into ring", - }, - []string{"task"}, - ) - RingMsgsOffTooLargeErrorTotal = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Name: prefix + "ring_msgs_offset_too_large_error_total", - Help: "total num of msgs with too large offset to put into ring", - }, - []string{"task"}, - ) - RingNormalBatchsTotal = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Name: prefix + "ring_normal_batchs_total", - Help: "total num of normal batches generated", - }, - []string{"task"}, - ) - RingForceBatchsTotal = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Name: prefix + "ring_force_batchs_total", - Help: "total num of force batches generated", - }, - []string{"task"}, - ) - RingForceBatchAllTotal = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Name: prefix + "ring_force_batch_all_total", - Help: "total num of force batch_all generated", - }, - []string{"task"}, - ) FlushMsgsTotal = prometheus.NewCounterVec( prometheus.CounterOpts{ Name: prefix + "flush_msgs_total", @@ -108,21 +66,7 @@ var ( Name: prefix + "consume_offsets", Help: "last committed offset for each topic partition pair", }, - []string{"task", "topic", "partition"}, - ) - ParsedRingMsgs = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Name: prefix + "parsed_ring_msgs", - Help: "num of parsed msgs in ring", - }, - []string{"task"}, - ) - RingMsgs = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Name: prefix + "ring_msgs", - Help: "num of msgs in ring", - }, - []string{"task"}, + []string{"consumer", "topic", "partition"}, ) ShardMsgs = prometheus.NewGaugeVec( prometheus.GaugeOpts{ @@ -131,13 +75,6 @@ var ( }, []string{"task"}, ) - ParsingPoolBacklog = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Name: prefix + "parsing_pool_backlog", - Help: "GlobalParsingPool backlog", - }, - []string{"task"}, - ) WritingPoolBacklog = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Name: prefix + "writing_pool_backlog", @@ -193,20 +130,11 @@ var ( func init() { prometheus.MustRegister(ConsumeMsgsTotal) - prometheus.MustRegister(ConsumeMsgsErrorTotal) prometheus.MustRegister(ParseMsgsErrorTotal) - prometheus.MustRegister(RingMsgsOffTooSmallErrorTotal) - prometheus.MustRegister(RingMsgsOffTooLargeErrorTotal) - prometheus.MustRegister(RingNormalBatchsTotal) - prometheus.MustRegister(RingForceBatchsTotal) - prometheus.MustRegister(RingForceBatchAllTotal) prometheus.MustRegister(FlushMsgsTotal) prometheus.MustRegister(FlushMsgsErrorTotal) prometheus.MustRegister(ConsumeOffsets) - prometheus.MustRegister(ParsedRingMsgs) - prometheus.MustRegister(RingMsgs) prometheus.MustRegister(ShardMsgs) - prometheus.MustRegister(ParsingPoolBacklog) prometheus.MustRegister(WritingPoolBacklog) prometheus.MustRegister(WritingDurations) prometheus.MustRegister(WriteSeriesAllowNew) @@ -294,22 +222,18 @@ func (p *Pusher) reconnect() { } p.pusher = push.New(p.pgwAddrs[nextAddr], "clickhouse_sinker"). Collector(ConsumeMsgsTotal). - Collector(ConsumeMsgsErrorTotal). Collector(ParseMsgsErrorTotal). - Collector(RingMsgsOffTooSmallErrorTotal). - Collector(RingMsgsOffTooLargeErrorTotal). - Collector(RingNormalBatchsTotal). - Collector(RingForceBatchsTotal). - Collector(RingForceBatchAllTotal). Collector(FlushMsgsTotal). Collector(FlushMsgsErrorTotal). Collector(ConsumeOffsets). - Collector(RingMsgs). - Collector(ParsedRingMsgs). Collector(ShardMsgs). - Collector(ParsingPoolBacklog). Collector(WritingPoolBacklog). Collector(WritingDurations). + Collector(WriteSeriesAllowNew). + Collector(WriteSeriesAllowChanged). + Collector(WriteSeriesDropQuota). + Collector(WriteSeriesDropUnchanged). + Collector(WriteSeriesSucceed). Grouping("instance", p.instance).Format(expfmt.FmtText) p.inUseAddr = nextAddr } diff --git a/task/consumer.go b/task/consumer.go index 2b74c84e..569958c0 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -16,12 +16,11 @@ limitations under the License. package task import ( - "encoding/json" - "reflect" "sync" "sync/atomic" "time" + "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/input" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" @@ -33,7 +32,7 @@ import ( type Commit struct { group string - offsets []byte + offsets map[string]map[int32]int64 wg *sync.WaitGroup consumer *Consumer } @@ -42,16 +41,13 @@ type Consumer struct { sinker *Sinker inputer *input.KafkaFranz tasks sync.Map - grpConfig *input.GroupConfig + grpConfig *config.GroupConfig fetchesCh chan *kgo.Fetches processWg sync.WaitGroup stopCh chan struct{} state atomic.Uint32 errCommit bool - recMap map[string]map[int32]int64 // committed RecMap - recMux sync.Mutex - numFlying int32 mux sync.Mutex commitDone *sync.Cond @@ -62,15 +58,14 @@ const ( MaxParallelism = 10 ) -func newConsumer(s *Sinker) *Consumer { +func newConsumer(s *Sinker, gCfg *config.GroupConfig) *Consumer { c := &Consumer{ sinker: s, numFlying: 0, errCommit: false, - grpConfig: &input.GroupConfig{}, + grpConfig: gCfg, stopCh: make(chan struct{}), fetchesCh: make(chan *kgo.Fetches), - recMap: make(map[string]map[int32]int64), } c.state.Store(util.StateStopped) c.commitDone = sync.NewCond(&c.mux) @@ -78,17 +73,7 @@ func newConsumer(s *Sinker) *Consumer { } func (c *Consumer) addTask(tsk *Service) { - cfg := tsk.taskCfg - if c.grpConfig.Name != "" && cfg.Earliest != c.grpConfig.Earliest { - util.Logger.Fatal("Tasks are sharing same consumer group, but with different Earliest property specified!", - zap.String("task", cfg.Name), zap.String("task", c.grpConfig.Name)) - } - c.grpConfig.Earliest = cfg.Earliest - c.grpConfig.Name = cfg.ConsumerGroup - c.grpConfig.FlushInterval = cfg.FlushInterval - c.grpConfig.Topics = append(c.grpConfig.Topics, cfg.Topic) - - c.tasks.Store(cfg.Name, tsk) + c.tasks.Store(tsk.taskCfg.Name, tsk) } func (c *Consumer) start() { @@ -108,7 +93,7 @@ func (c *Consumer) stop() (err error) { } c.state.Store(util.StateStopped) - // stop the processFetch routine, make sure no more input to the- commit chan & writing pool + // stop the processFetch routine, make sure no more input to the commit chan & writing pool c.stopCh <- struct{}{} c.processWg.Wait() err = c.inputer.Stop() @@ -145,17 +130,25 @@ func (c *Consumer) cleanupFn() { c.mux.Unlock() } +func (c *Consumer) updateGroupConfig(g *config.GroupConfig) { + if c.state.Load() == util.StateStopped { + return + } + c.grpConfig = g + // stop the processFetch routine, make sure no more input to the commit chan & writing pool + c.stopCh <- struct{}{} + c.processWg.Wait() + go c.processFetch() +} + func (c *Consumer) processFetch() { c.processWg.Add(1) defer c.processWg.Done() recMap := make(map[string]map[int32]int64) - var bufLength, bufThreshold int64 + var bufLength int flushFn := func() { - c.recMux.Lock() - ok := reflect.DeepEqual(recMap, c.recMap) - c.recMux.Unlock() - if ok { + if len(recMap) == 0 { return } var wg sync.WaitGroup @@ -165,22 +158,15 @@ func (c *Consumer) processFetch() { return true }) bufLength = 0 - off, err := json.Marshal(recMap) - if err != nil { - return - } + c.mux.Lock() c.numFlying++ c.mux.Unlock() - c.sinker.commitsCh <- &Commit{group: c.grpConfig.Name, offsets: off, wg: &wg, consumer: c} + c.sinker.commitsCh <- &Commit{group: c.grpConfig.Name, offsets: recMap, wg: &wg, consumer: c} + recMap = make(map[string]map[int32]int64) } - c.tasks.Range(func(key, value any) bool { - bufThreshold += int64(value.(*Service).taskCfg.BufferSize) - return true - }) - - bufThreshold = bufThreshold * int64(len(c.sinker.curCfg.Clickhouse.Hosts)) * 4 / 5 + bufThreshold := c.grpConfig.BufferSize * len(c.sinker.curCfg.Clickhouse.Hosts) * 4 / 5 if bufThreshold > MaxCountInBuf { bufThreshold = MaxCountInBuf } @@ -258,6 +244,9 @@ func (c *Consumer) processFetch() { } for j := range ft.Partitions { fpr := ft.Partitions[j].Records + if len(fpr) == 0 { + continue + } lastOff := fpr[len(fpr)-1].Offset old, ok := recMap[ft.Topic][ft.Partitions[j].Partition] diff --git a/task/sinker.go b/task/sinker.go index 5b57d34f..23260d46 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -17,9 +17,10 @@ package task import ( "context" - "encoding/json" "os" "reflect" + "sort" + "strconv" "strings" "sync" "time" @@ -104,7 +105,11 @@ func (s *Sinker) Run() { util.Logger.Fatal("expect --local-cfg-file or --nacos-dataid") return } - if err = newCfg.Normallize(); err != nil { + ha := "" + if s.cmdOps.NacosServiceName != "" { + ha = s.httpAddr + } + if err = newCfg.Normallize(true, ha); err != nil { util.Logger.Fatal("newCfg.Normallize failed", zap.Error(err)) return } @@ -120,7 +125,7 @@ func (s *Sinker) Run() { case c := <-s.consumerRestartCh: // only restart the consumer which was not changed in applyAnotherConfig if c == s.consumers[c.grpConfig.Name] { - newGroup := newConsumer(s) + newGroup := newConsumer(s, c.grpConfig) s.consumers[c.grpConfig.Name] = newGroup c.tasks.Range(func(key, value any) bool { cloneTask(value.(*Service), newGroup) @@ -152,7 +157,11 @@ func (s *Sinker) Run() { util.Logger.Error("s.rcm.GetConfig failed", zap.Error(err)) continue } - if err = newCfg.Normallize(); err != nil { + ha := "" + if s.cmdOps.NacosServiceName != "" { + ha = s.httpAddr + } + if err = newCfg.Normallize(true, ha); err != nil { util.Logger.Error("newCfg.Normallize failed", zap.Error(err)) continue } @@ -163,7 +172,7 @@ func (s *Sinker) Run() { case c := <-s.consumerRestartCh: // only restart the consumer which was not changed in applyAnotherConfig if c == s.consumers[c.grpConfig.Name] { - newGroup := newConsumer(s) + newGroup := newConsumer(s, c.grpConfig) s.consumers[c.grpConfig.Name] = newGroup c.tasks.Range(func(key, value any) bool { cloneTask(value.(*Service), newGroup) @@ -259,26 +268,17 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { go s.commitFn() // 3. Generate, initialize and run task - for _, taskCfg := range newCfg.Tasks { - if s.cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(s.httpAddr, taskCfg.Name) { - continue - } - var c *Consumer - var ok bool - if c, ok = s.consumers[taskCfg.ConsumerGroup]; !ok { - c = newConsumer(s) - s.consumers[taskCfg.ConsumerGroup] = c - } - task := NewTaskService(newCfg, taskCfg, c) - if err = task.Init(); err != nil { - return - } - } - - // 4. Start fetching messages s.curCfg = newCfg - for _, v := range s.consumers { - v.start() + for group, grpCfg := range newCfg.Groups { + c := newConsumer(s, grpCfg) + for _, tsk := range grpCfg.Configs { + task := NewTaskService(newCfg, tsk, c) + if err = task.Init(); err != nil { + return + } + } + c.start() + s.consumers[group] = c } util.Logger.Info("applied the first config") @@ -306,104 +306,86 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 4. Generate, initialize and run task var tasksToStart []string - for _, taskCfg := range newCfg.Tasks { - if s.cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(s.httpAddr, taskCfg.Name) { - continue - } - var c *Consumer - var ok bool - if c, ok = s.consumers[taskCfg.ConsumerGroup]; !ok { - c = newConsumer(s) - s.consumers[taskCfg.ConsumerGroup] = c - } - task := NewTaskService(newCfg, taskCfg, c) - if err = task.Init(); err != nil { - return - } - tasksToStart = append(tasksToStart, taskCfg.Name) - } - - // 5. Start fetching messages, only after all the taskCfg loaded to consumer.cfgs s.curCfg = newCfg - for _, v := range s.consumers { - v.start() + for group, grpCfg := range newCfg.Groups { + c := newConsumer(s, grpCfg) + for _, tsk := range grpCfg.Configs { + task := NewTaskService(newCfg, tsk, c) + if err = task.Init(); err != nil { + return + } + tasksToStart = append(tasksToStart, tsk.Name) + } + c.start() + s.consumers[group] = c } util.Logger.Info("started tasks", zap.Reflect("tasks", tasksToStart)) } else if !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) || !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { - // 1. Find the config difference - newConsumers := make(map[string]map[string]*config.TaskConfig) - for _, taskCfg := range newCfg.Tasks { - if s.cmdOps.NacosServiceName != "" && !newCfg.IsAssigned(s.httpAddr, taskCfg.Name) { - continue - } - var group map[string]*config.TaskConfig - var ok bool - if group, ok = newConsumers[taskCfg.ConsumerGroup]; !ok { - group = make(map[string]*config.TaskConfig) - newConsumers[taskCfg.ConsumerGroup] = group - } - group[taskCfg.Name] = taskCfg - } - - toCreate := make(map[string]map[string]*config.TaskConfig) + // Find the consumer difference var deleteConsumers []string for name, c := range s.consumers { - var group map[string]*config.TaskConfig + var group *config.GroupConfig var ok bool - if group, ok = newConsumers[name]; !ok { - // consumer group no longer with this client, stop it - go c.stop() + if group, ok = newCfg.Groups[name]; !ok { deleteConsumers = append(deleteConsumers, name) } else { - // find the one that need to be restart - var groupChanged bool - if len(group) != len(c.grpConfig.Topics) { - groupChanged = true + if len(c.grpConfig.Topics) != len(group.Topics) { + deleteConsumers = append(deleteConsumers, name) } else { - c.tasks.Range(func(key, value any) bool { - cfg := value.(*Service).taskCfg - var it *config.TaskConfig - var ok bool - if it, ok = group[cfg.Name]; !ok { - groupChanged = true - return false - } else if !reflect.DeepEqual(it, cfg) { - groupChanged = true - return false - } - return true - }) - } - // restart the group accordingly - if groupChanged { - go c.stop() - toCreate[name] = group + sort.Strings(c.grpConfig.Topics) + sort.Strings(group.Topics) + if !reflect.DeepEqual(c.grpConfig.Topics, group.Topics) { + deleteConsumers = append(deleteConsumers, name) + } else { + c.updateGroupConfig(group) + } } - delete(newConsumers, name) } } - // create new consumers - for name, cfgs := range newConsumers { - toCreate[name] = cfgs + // 1) stop consumers no longer with newcfg + var wg sync.WaitGroup + for _, v := range deleteConsumers { + c := s.consumers[v] + if c.state.Load() == util.StateRunning { + wg.Add(1) + go func(c *Consumer) { + c.stop() + wg.Done() + }(c) + } + delete(s.consumers, v) } - for _, name := range deleteConsumers { - delete(s.consumers, name) + wg.Wait() + // 2) fire up new consumers + // Record the new config + s.curCfg = newCfg + for _, v := range newCfg.Groups { + if s.consumers[v.Name] == nil { + c := newConsumer(s, v) + for _, tCfg := range v.Configs { + task := NewTaskService(newCfg, tCfg, c) + if err = task.Init(); err != nil { + return + } + } + c.start() + s.consumers[v.Name] = c + } } - for name, cfgs := range toCreate { - c := newConsumer(s) - s.consumers[name] = c - for _, cfg := range cfgs { - task := NewTaskService(newCfg, cfg, c) - if err = task.Init(); err != nil { - return + // 3) apply TaskConfig Change + for name, c := range newCfg.Groups { + for _, tCfg := range c.Configs { + if !reflect.DeepEqual(tCfg, s.consumers[name].grpConfig.Configs[tCfg.Name]) { + task := NewTaskService(newCfg, tCfg, s.consumers[name]) + if err = task.Init(); err != nil { + return + } } } - c.start() + s.consumers[name].grpConfig = c } } - // Record the new config - s.curCfg = newCfg util.Logger.Info("applied another config", zap.Int("number", s.numCfg)) s.numCfg++ return @@ -417,14 +399,8 @@ func (s *Sinker) commitFn() { c := com.consumer if !c.errCommit { - c.recMux.Lock() - err := json.Unmarshal(com.offsets, &c.recMap) - c.recMux.Unlock() - if err != nil { - util.Logger.Fatal("Failed to unmarshall offsets", zap.ByteString("offsets", com.offsets), zap.Error(err)) - } LOOP: - for i, value := range c.recMap { + for i, value := range com.offsets { for k, v := range value { if err := c.inputer.CommitMessages(&model.InputMessage{Topic: i, Partition: int(k), Offset: v}); err != nil { c.errCommit = true @@ -436,6 +412,8 @@ func (s *Sinker) commitFn() { }() util.Logger.Warn("Batch.Commit failed, will restart later", zap.Error(err)) break LOOP + } else { + statistics.ConsumeOffsets.WithLabelValues(com.consumer.grpConfig.Name, i, strconv.Itoa(int(k))).Set(float64(v)) } } } From ac3ce76d33c4e71c2958311a167d4f33166e7153 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Mon, 2 Jan 2023 21:43:48 +0800 Subject: [PATCH 307/404] Update the doc and allow customization of DatabaseName in task level --- docs/configuration/config.md | 3 +- docs/dev/design.md | 26 +++++++---------- docs/dev/introduction.md | 28 +++--------------- go.test.sh | 1 + output/clickhouse.go | 56 +++++++++++++++++++++--------------- task/consumer.go | 2 +- task/task.go | 5 ++-- 7 files changed, 54 insertions(+), 67 deletions(-) diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 58ed7965..96a5342d 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -1,6 +1,6 @@ # Config Items -> Here we use json with comments for documentation +> Here we use json with comments for documentation, config file in hjson format is also supported ```json { @@ -105,6 +105,7 @@ "parser": "json", // clickhouse table name + // override the clickhouse.db with "db.tableName" format, eg "default.tbl1" "tableName": "daily", // columns of the table diff --git a/docs/dev/design.md b/docs/dev/design.md index 44a69d36..9aaf40cc 100644 --- a/docs/dev/design.md +++ b/docs/dev/design.md @@ -11,24 +11,20 @@ So if you setup ClickHouse properly(ReplacingMergeTree ORDER BY (__kafak_topic, It's hard for clickhouse_sinker to guarantee exactly-once semantic without ReplacingMergeTree. Kafka consumer group load-balance cause duplicated messages if one consumer crash suddenly. -### Sharding with kafka message offset stripe (default) +## Workflow -The flow is: +Internally, clickhouse_sinker groups tasks that with identical "consumerGroup" property set together for the purpose of reducing the number of Kafka client, so that Kafka server is able to handle more requests concurrently. And consequently, it's decided to commit Kafka offset only after messages in a whole fetch got written to clickhouse completely. -- Fetch message via Franz, Sarama, or kafka-go, which starts internally a goroutine for each partition. -- Parse messages in a global goroutine pool(pool size is customizable), fill the result into a ring according to the message's partition and offset. -- Generate a batch when messages in a ring reach a batchSize boundary or flush timer fire. For each message, the dest shard is determined by `(kafka_offset/roundup(buffer_size))%clickhouse_shards`. -- Write batch to ClickHouse in a global goroutine pool(pool size is fixed according to the number of tasks and Clickhouse shards). +The flow is like this: -### Sharding with custom key - -The flow is: - -- Fetch message via kafka-go or samara, which starts internally a goroutine for each partition. -- Parse messages in a global goroutine pool(pool size is customizable), fill the result into a ring according to the message's partition and offset. -- Shard messages in a ring when messages reach a batchSize boundary or flush timer fire. For each message, if the sharding key is numerical(integer, float, time, etc.), the dest shard is determined by `(shardingKey/shardingStripe)%clickhouse_shards`, otherwise it is determined by `xxHash64(shardingKey)%clickhouse_shards`. -- Generate batches for all shard slots if messages in one shard slot reach batchSize boundary or flush timer fire. Those batches form a `BatchGroup`. The `before` relationship could be impossible if messages of a partition are distributed to multiple batches. So those batches need to be committed after ALL of them have been written to Clickhouse. -- Write batch to ClickHouse in a global goroutine pool(pool size is fixed according to the number of tasks and Clickhouse shards). +- Group tasks with identical "consumerGroup" property together, fetch messages for the group of tasks with a single goroutine. +- Route fetched messages to the individual tasks for further parsing. By default, the mapping between messages and tasks is controlled by "topic" and "tableName" property. But for messages with Kafka header "__table_name" specified, the mapping between "__table_name" and "tableName" will override the default behavior. +- Parse messages and calculate the dest shard: +-- For tasks with "shardingkey" property specified, if the sharding key is numerical(integer, float, time, etc.), the dest shard is determined by `(shardingKey/shardingStripe)%clickhouse_shards`, if not, it is determined by `xxHash64(shardingKey)%clickhouse_shards`. +-- Otherwise, the dest shard for each message is determined by `(kafka_offset/roundup(buffer_size))%clickhouse_shards`. +- Generate batches for all shard slots that are in the same Group, when total cached message count in the Group reached `sum(batchSize)*80%` boundary or flush timer fire. +- Write batches to ClickHouse in a global goroutine pool(pool size is a fixed number based on the number of tasks and Clickhouse shards). +- Commit offset back to Kafka ## Task scheduling diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index c4526393..1e522322 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -12,9 +12,9 @@ Refers to [design](./design.md) for how it works. - Uses native ClickHouse client-server TCP protocol, with higher performance than HTTP. - Easy to use and deploy, you don't need write any hard code, just care about the configuration file - Support multiple parsers: fastjson(recommended), gjson, csv. -- Support multiple Kafka client: sarama(recommended), kafka-go. - Support multiple Kafka security mechanisms: SSL, SASL/PLAIN, SASL/SCRAM, SASL/GSSAPI and combinations of them. - Bulk insert (by config `bufferSize` and `flushInterval`). +- Powered by Franz-go, which is the fastest and most cpu and memory efficient Kafka client in Go. - Parse messages concurrently. - Write batches concurrently. - Every batch is routed to a determined clickhouse shard. Exit if loop write fail. @@ -35,6 +35,7 @@ Refers to [design](./design.md) for how it works. - [x] Enum - [x] Array(T), where T is one of above basic types - [x] Nullable(T), where T is one of above basic types +- [x] Map Note: @@ -164,7 +165,7 @@ Please follow [`Kafka SSL setup`](https://kafka.apache.org/documentation/#securi ### Kafka Authentication -clickhouse_sinker support following following authentication mechanisms: +clickhouse_sinker support the following authentication mechanisms: - No authentication @@ -339,13 +340,6 @@ type Parser interface { Parse(bs []byte) model.Metric } -type Inputer interface { - Init(cfg *config.Config, taskName string, putFn func(msg model.InputMessage)) error - Run(ctx context.Context) - Stop() error - CommitMessages(ctx context.Context, message *model.InputMessage) error -} - // RemoteConfManager can be implemented by many backends: Nacos, Consul, etcd, ZooKeeper... type RemoteConfManager interface { Init(properties map[string]interface{}) error @@ -366,18 +360,7 @@ type RemoteConfManager interface { Kafka release history is at [here](https://kafka.apache.org/downloads). Kafka broker [exposes versions of various APIs it supports since 0.10.0.0](https://kafka.apache.org/protocol#api_versions). -### Kafka-go - -- Kafka-go [negotiate it's protocol version](https://github.com/segmentio/kafka-go/blob/c66d8ca149e7f1a7905b47a60962745ceb08a6a9/conn.go#L209). -- Kafka-go [doesn't support Kerberos authentication](https://github.com/segmentio/kafka-go/issues/237). - -### Sarama - -- Sarama guarantees compatibility [with Kafka 2.6 through 2.8](https://github.com/Shopify/sarama/blob/master/README.md#compatibility-and-api-stability). -- Sarama [has tied it's protocol usage to the Version field in Config](https://github.com/Shopify/sarama/issues/1732). My experience is setting `Config.Version` to "0.11.0.0" or "2.5.0" cannot cowork with broker 2.2.0. -- Sarama consumer API provides generation cleanup callback. This ensures `exactly once` when consumer-group rebalance occur. - -### Franz +### Franz-go Kafka client - Franz negotiates it's protocol version. - Franz supports Kerberos authentication. @@ -385,6 +368,3 @@ Kafka release history is at [here](https://kafka.apache.org/downloads). Kafka br - Franz wins Sarama and Kafka-go at benchmark competition. - Franz project is young but very active. -### Conclusion - -Franz is the best Golang client library, though none is as mature as the officaial Kafka Java client. You need to try another if clickhouse_sinker fails to connect with Kafka. diff --git a/go.test.sh b/go.test.sh index 6fbef090..3df7bb08 100755 --- a/go.test.sh +++ b/go.test.sh @@ -45,6 +45,7 @@ echo "send messages to kafka" echo "cat /tmp/a.json | kafka-console-producer --topic topic1 --broker-list localhost:9092" > send.sh sudo docker cp a.json kafka:/tmp/ sudo docker cp send.sh kafka:/tmp/ +sudo docker exec kafka kafka-topics --bootstrap-server localhost:9093 --topic topic1 --delete sudo docker exec kafka sh /tmp/send.sh echo "start clickhouse_sinker to consume" diff --git a/output/clickhouse.go b/output/clickhouse.go index 6c328167..a89ee093 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -53,12 +53,15 @@ var ( // ClickHouse is an output service consumers from kafka messages type ClickHouse struct { - Dims []*model.ColumnWithType - NumDims int - IdxSerID int - NameKey string - cfg *config.Config - taskCfg *config.TaskConfig + Dims []*model.ColumnWithType + NumDims int + IdxSerID int + NameKey string + cfg *config.Config + taskCfg *config.TaskConfig + TableName string + DBName string + prepareSQL string promSerSQL string seriesTbl string @@ -211,7 +214,7 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( if numBad, err = writeRows(c.prepareSQL, *batch.Rows, 0, numDims, conn); err != nil { return } - statistics.WritingDurations.WithLabelValues(c.taskCfg.Name, c.taskCfg.TableName).Observe(time.Since(begin).Seconds()) + statistics.WritingDurations.WithLabelValues(c.taskCfg.Name, c.TableName).Observe(time.Since(begin).Seconds()) if numBad != 0 { statistics.ParseMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(numBad)) } @@ -262,7 +265,7 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { util.Logger.Info(fmt.Sprintf("skipped loading series from %v", tbl), zap.String("task", c.taskCfg.Name)) return } - query := fmt.Sprintf("SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s.%s ORDER BY sid", c.cfg.Clickhouse.DB, tbl) + query := fmt.Sprintf("SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s.%s ORDER BY sid", c.DBName, tbl) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs driver.Rows var seriesID, mgmtID int64 @@ -301,23 +304,23 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { } } if dimSerID == nil { - err = errors.Newf("Metric table %s shall have column `__series_id UInt64`.", c.taskCfg.TableName) + err = errors.Newf("Metric table %s.%s shall have column `__series_id UInt64`.", c.DBName, c.TableName) return } c.IdxSerID = len(c.Dims) c.Dims = append(c.Dims, dimSerID) // Add string columns from series table - c.seriesTbl = c.taskCfg.TableName + "_series" + c.seriesTbl = c.TableName + "_series" expSeriesDims := []*model.ColumnWithType{ {Name: "__series_id", Type: &model.TypeInfo{Type: model.Int64}}, {Name: "__mgmt_id", Type: &model.TypeInfo{Type: model.Int64}}, {Name: "labels", Type: &model.TypeInfo{Type: model.String}}, } var seriesDims []*model.ColumnWithType - if seriesDims, err = getDims(c.cfg.Clickhouse.DB, c.seriesTbl, nil, c.taskCfg.Parser, conn); err != nil { + if seriesDims, err = getDims(c.DBName, c.seriesTbl, nil, c.taskCfg.Parser, conn); err != nil { if errors.Is(err, ErrTblNotExist) { - err = errors.Wrapf(err, "Please create series table for %s.%s", c.cfg.Clickhouse.DB, c.taskCfg.TableName) + err = errors.Wrapf(err, "Please create series table for %s.%s", c.DBName, c.TableName) return } return @@ -354,7 +357,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) } c.promSerSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", - c.cfg.Clickhouse.DB, + c.DBName, c.seriesTbl, strings.Join(serDimsQuoted, ",")) @@ -377,13 +380,20 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { } func (c *ClickHouse) initSchema() (err error) { + if idx := strings.Index(c.taskCfg.TableName, "."); idx > 0 { + c.TableName = c.taskCfg.TableName[idx+1:] + c.DBName = c.taskCfg.TableName[0:idx] + } else { + c.TableName = c.taskCfg.TableName[idx+1:] + c.DBName = c.cfg.Clickhouse.DB + } sc := pool.GetShardConn(0) var conn clickhouse.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } if c.taskCfg.AutoSchema { - if c.Dims, err = getDims(c.cfg.Clickhouse.DB, c.taskCfg.TableName, c.taskCfg.ExcludeColumns, c.taskCfg.Parser, conn); err != nil { + if c.Dims, err = getDims(c.DBName, c.TableName, c.taskCfg.ExcludeColumns, c.taskCfg.Parser, conn); err != nil { return } } else { @@ -410,18 +420,18 @@ func (c *ClickHouse) initSchema() (err error) { quotedDms[i] = fmt.Sprintf("`%s`", c.Dims[i].Name) } c.prepareSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", - c.cfg.Clickhouse.DB, - c.taskCfg.TableName, + c.DBName, + c.TableName, strings.Join(quotedDms, ",")) util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.taskCfg.Name)) // Check distributed metric table if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { - if c.distMetricTbls, err = c.getDistTbls(c.taskCfg.TableName); err != nil { + if c.distMetricTbls, err = c.getDistTbls(c.TableName); err != nil { return } if c.distMetricTbls == nil { - err = errors.Newf("Please create distributed table for %s.", c.taskCfg.TableName) + err = errors.Newf("Please create distributed table for %s.", c.TableName) return } } @@ -475,12 +485,12 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } if c.taskCfg.PrometheusSchema { if intVal == model.String { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, c.seriesTbl, onCluster, strKey, strVal) + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.DBName, c.seriesTbl, onCluster, strKey, strVal) queries = append(queries, query) affectDistSeries = true } } else { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", chCfg.DB, taskCfg.TableName, onCluster, strKey, strVal) + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.DBName, c.TableName, onCluster, strKey, strVal) queries = append(queries, query) affectDistMetric = true } @@ -504,12 +514,12 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } if chCfg.Cluster != "" { if affectDistMetric { - if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, c.taskCfg.TableName, c.distMetricTbls, conn); err != nil { + if err = recreateDistTbls(chCfg.Cluster, c.DBName, c.TableName, c.distMetricTbls, conn); err != nil { return } } if affectDistSeries { - if err = recreateDistTbls(chCfg.Cluster, chCfg.DB, c.seriesTbl, c.distSeriesTbls, conn); err != nil { + if err = recreateDistTbls(chCfg.Cluster, c.DBName, c.seriesTbl, c.distSeriesTbls, conn); err != nil { return } } @@ -526,7 +536,7 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { return } query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed\(\'%s\', \'%s\', \'%s\'.*\)')`, - chCfg.DB, chCfg.Cluster, chCfg.DB, table) + c.DBName, chCfg.Cluster, c.DBName, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows driver.Rows diff --git a/task/consumer.go b/task/consumer.go index 569958c0..067f7822 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -218,7 +218,7 @@ func (c *Consumer) processFetch() { c.tasks.Range(func(key, value any) bool { tsk := value.(*Service) - if (tablename != "" && tsk.taskCfg.TableName == tablename) || tsk.taskCfg.Topic == rec.Topic { + if (tablename != "" && tsk.clickhouse.TableName == tablename) || tsk.taskCfg.Topic == rec.Topic { bufLength++ if e := tsk.Put(msg, flushFn); e != nil { atomic.StoreInt64(&done, items) diff --git a/task/task.go b/task/task.go index 938c751e..38d7c788 100644 --- a/task/task.go +++ b/task/task.go @@ -60,7 +60,7 @@ type Service struct { consumer *Consumer } -// cloneTask create a new task by steal members from s instead of creating a new one +// cloneTask create a new task by stealing members from s instead of creating a new one func cloneTask(s *Service, newGroup *Consumer) (service *Service) { service = &Service{ clickhouse: s.clickhouse, @@ -183,7 +183,6 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { // 3) apply the schema change. // 4) recreate the service util.Logger.Warn("new key detected, consumer is going to restart", zap.String("consumer group", service.taskCfg.ConsumerGroup), zap.Error(err)) - service.consumer.state.Store(util.StateStopped) go service.consumer.restart() flushFn() if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { @@ -202,7 +201,7 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { util.Logger.Fatal("shard number calculation failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - msgRow.Shard = int(msgRow.Msg.Offset>>17) % service.sharder.shards + msgRow.Shard = int(msgRow.Msg.Offset>>int64(util.GetShift(service.taskCfg.BufferSize))) % service.sharder.shards } service.sharder.PutElement(&msgRow) } From a8befbc226a13743be70e87a868ad00f54d2f88a Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Tue, 3 Jan 2023 10:25:50 +0800 Subject: [PATCH 308/404] Offset shift should only be calculated once per task --- task/task.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/task/task.go b/task/task.go index 38d7c788..ad2fb761 100644 --- a/task/task.go +++ b/task/task.go @@ -57,6 +57,7 @@ type Service struct { sharder *Sharder limiter1 *rate.Limiter limiter2 *rate.Limiter + offShift int64 consumer *Consumer } @@ -117,6 +118,7 @@ func (service *Service) Init() (err error) { service.nameKey = service.clickhouse.NameKey service.limiter1 = rate.NewLimiter(rate.Every(10*time.Second), 1) service.limiter2 = rate.NewLimiter(rate.Every(10*time.Second), 1) + service.offShift = int64(util.GetShift(taskCfg.BufferSize)) if service.sharder, err = NewSharder(service); err != nil { return @@ -201,7 +203,7 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { util.Logger.Fatal("shard number calculation failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - msgRow.Shard = int(msgRow.Msg.Offset>>int64(util.GetShift(service.taskCfg.BufferSize))) % service.sharder.shards + msgRow.Shard = int(msgRow.Msg.Offset>>service.offShift) % service.sharder.shards } service.sharder.PutElement(&msgRow) } From 74821f1cc5ce2854b7fc54ade1050e7be274f7ce Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Tue, 17 Jan 2023 16:11:08 +0800 Subject: [PATCH 309/404] Allow self-defined series table name --- config/config.go | 14 +---- config_manager/lags.go | 2 + docs/configuration/config.md | 10 ++-- output/clickhouse.go | 103 +++++++++++++++++++++-------------- statistics/statistics.go | 9 +++ 5 files changed, 81 insertions(+), 57 deletions(-) diff --git a/config/config.go b/config/config.go index 559f6738..011fa116 100644 --- a/config/config.go +++ b/config/config.go @@ -42,7 +42,6 @@ type Config struct { // KafkaConfig configuration parameters type KafkaConfig struct { Brokers string - Version string Security map[string]string TLS struct { Enable bool @@ -106,7 +105,6 @@ type ClickHouseConfig struct { type TaskConfig struct { Name string - KafkaClient string Topic string ConsumerGroup string @@ -117,7 +115,8 @@ type TaskConfig struct { CsvFormat []string Delimiter string - TableName string + TableName string + SeriesTableName string // AutoSchema will auto fetch the schema from clickhouse AutoSchema bool @@ -208,15 +207,6 @@ func (cfg *Config) Normallize(constructGroup bool, httpAddr string) (err error) err = errors.Newf("invalid configuration") return } - if cfg.Kafka.Version == "" { - // https://cwiki.apache.org/confluence/display/KAFKA/Compatibility+Matrix - // KIP-35 - Retrieving protocol version introduced a mechanism for dynamically determining the functionality of a Kafka broker. - // https://github.com/Shopify/sarama/issues/1732, - // Historically Sarama has tied it's protocol usage to the Version field in Config. - // https://kafka.apache.org/downloads - // 2.0.0 is released at July 30, 2018. - cfg.Kafka.Version = "2.0.0" - } cfg.convertKfkSecurity() if cfg.Kafka.TLS.CaCertFiles == "" && cfg.Kafka.TLS.TrustStoreLocation != "" { diff --git a/config_manager/lags.go b/config_manager/lags.go index b0205b8a..403c0df6 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -5,6 +5,7 @@ import ( "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/input" + "github.com/housepower/clickhouse_sinker/statistics" "github.com/thanos-io/thanos/pkg/errors" "github.com/twmb/franz-go/pkg/kadm" "github.com/twmb/franz-go/pkg/kgo" @@ -33,6 +34,7 @@ func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err return } stateLags[taskCfg.Name] = StateLag{State: state, Lag: totalLags} + statistics.ConsumeLags.WithLabelValues(taskCfg.ConsumerGroup, taskCfg.Topic, taskCfg.Name).Set(float64(totalLags)) } return } diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 96a5342d..dea9e23b 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -85,9 +85,6 @@ "disablepafxfast": false } }, - - // kafka version, if you use sarama, the version must be specified - "version": "2.5.0" }, "task": { @@ -106,7 +103,10 @@ // clickhouse table name // override the clickhouse.db with "db.tableName" format, eg "default.tbl1" - "tableName": "daily", + "tableName": "prom_metric", + + // name of the timeseries table, by default it is tableName with a "_series" suffix + "seriesTableName": "prom_metric_myseries", // columns of the table "dims": [ @@ -152,7 +152,7 @@ // shardingKey is the column name to which sharding against "shardingKey": "", - // shardingStripe take effect iff the sharding key is numerical + // shardingStripe take effect if the sharding key is numerical "shardingStripe": 0, // interval of flushing the batch. Default to 5, max to 600. diff --git a/output/clickhouse.go b/output/clickhouse.go index a89ee093..4f7519f3 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -49,8 +49,16 @@ var ( // zooKeeper Session expired issue: https://cwiki.apache.org/confluence/display/ZOOKEEPER/FAQ#:~:text=How%20should%20I%20handle%20SESSION_EXPIRED%3F replicaSpecificErrorCodes = []int32{225, 242, 252, 319, 999, 1000} //NO_ZOOKEEPER, TABLE_IS_READ_ONLY, TOO_MANY_PARTS, UNKNOWN_STATUS_OF_INSERT, KEEPER_EXCEPTION, POCO_EXCEPTION wrSeriesQuota = 16384 + seriesQuotas sync.Map ) +type seriesQuota struct { + sync.Mutex + nextResetQuota time.Time + bmSeries map[int64]int64 + wrSeries int +} + // ClickHouse is an output service consumers from kafka messages type ClickHouse struct { Dims []*model.ColumnWithType @@ -60,7 +68,7 @@ type ClickHouse struct { cfg *config.Config taskCfg *config.TaskConfig TableName string - DBName string + dbName string prepareSQL string promSerSQL string @@ -69,19 +77,17 @@ type ClickHouse struct { distMetricTbls []string distSeriesTbls []string - bmSeries map[int64]int64 - wrSeries int - numFlying int32 - mux sync.Mutex - taskDone *sync.Cond - nextResetQuota time.Time + seriesQuota *seriesQuota + + numFlying int32 + mux sync.Mutex + taskDone *sync.Cond } // NewClickHouse new a clickhouse instance func NewClickHouse(cfg *config.Config, taskCfg *config.TaskConfig) *ClickHouse { ck := &ClickHouse{cfg: cfg, taskCfg: taskCfg} ck.taskDone = sync.NewCond(&ck.mux) - ck.nextResetQuota = time.Now().Add(10 * time.Second) return ck } @@ -125,21 +131,21 @@ func (c *ClickHouse) Send(batch *model.Batch) { } func (c *ClickHouse) AllowWriteSeries(sid, mid int64) (allowed bool) { - c.mux.Lock() - defer c.mux.Unlock() - mid2, loaded := c.bmSeries[sid] + c.seriesQuota.Lock() + defer c.seriesQuota.Unlock() + mid2, loaded := c.seriesQuota.bmSeries[sid] if !loaded { allowed = true statistics.WriteSeriesAllowNew.WithLabelValues(c.taskCfg.Name).Inc() } else if mid != mid2 { - if c.wrSeries < wrSeriesQuota { - c.wrSeries++ + if c.seriesQuota.wrSeries < wrSeriesQuota { + c.seriesQuota.wrSeries++ allowed = true } else { now := time.Now() - if now.After(c.nextResetQuota) { - c.nextResetQuota = now.Add(10 * time.Second) - c.wrSeries = 1 + if now.After(c.seriesQuota.nextResetQuota) { + c.seriesQuota.nextResetQuota = now.Add(10 * time.Second) + c.seriesQuota.wrSeries = 1 allowed = true } } @@ -169,16 +175,16 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err err return } // update c.bmSeries **after** writing series - c.mux.Lock() + c.seriesQuota.Lock() for _, row := range seriesRows { sid := (*row)[c.IdxSerID].(int64) mid := (*row)[c.IdxSerID+1].(int64) - if _, loaded := c.bmSeries[sid]; loaded { - c.wrSeries-- + if _, loaded := c.seriesQuota.bmSeries[sid]; loaded { + c.seriesQuota.wrSeries-- } - c.bmSeries[sid] = mid + c.seriesQuota.bmSeries[sid] = mid } - c.mux.Unlock() + c.seriesQuota.Unlock() util.Logger.Info("ClickHouse.writeSeries succeeded", zap.Int("series", len(seriesRows)), zap.String("task", c.taskCfg.Name)) statistics.WriteSeriesSucceed.WithLabelValues(c.taskCfg.Name).Add(float64(len(seriesRows))) if numBad != 0 { @@ -260,12 +266,21 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { if c.cfg.Clickhouse.Cluster != "" { tbl = c.distSeriesTbls[0] } - c.bmSeries = make(map[int64]int64) + sq := &seriesQuota{} + if v, ok := seriesQuotas.LoadOrStore(tbl, sq); ok { + c.seriesQuota = v.(*seriesQuota) + } else { + sq.Lock() + sq.bmSeries = make(map[int64]int64) + sq.nextResetQuota = time.Now().Add(10 * time.Second) + sq.Unlock() + c.seriesQuota = sq + } if !c.taskCfg.LoadSeriesAtStartup { util.Logger.Info(fmt.Sprintf("skipped loading series from %v", tbl), zap.String("task", c.taskCfg.Name)) return } - query := fmt.Sprintf("SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s.%s ORDER BY sid", c.DBName, tbl) + query := fmt.Sprintf("SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s.%s ORDER BY sid", c.dbName, tbl) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs driver.Rows var seriesID, mgmtID int64 @@ -274,14 +289,17 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { return err } defer rs.Close() + + c.seriesQuota.Lock() + defer c.seriesQuota.Unlock() for rs.Next() { if err = rs.Scan(&seriesID, &mgmtID); err != nil { err = errors.Wrapf(err, "") return err } - c.bmSeries[seriesID] = mgmtID + c.seriesQuota.bmSeries[seriesID] = mgmtID } - util.Logger.Info(fmt.Sprintf("loaded %d series from %v", len(c.bmSeries), tbl), zap.String("task", c.taskCfg.Name)) + util.Logger.Info(fmt.Sprintf("loaded %d series from %v", len(c.seriesQuota.bmSeries), tbl), zap.String("task", c.taskCfg.Name)) return } @@ -299,28 +317,31 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) } else if dim.Type.Type == model.String { c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) + util.Logger.Warn("non-numeric type metric ignored", zap.String("metric name", dim.Name)) } else { i++ } } if dimSerID == nil { - err = errors.Newf("Metric table %s.%s shall have column `__series_id UInt64`.", c.DBName, c.TableName) + err = errors.Newf("Metric table %s.%s shall have column `__series_id UInt64`.", c.dbName, c.TableName) return } c.IdxSerID = len(c.Dims) c.Dims = append(c.Dims, dimSerID) // Add string columns from series table - c.seriesTbl = c.TableName + "_series" + if c.seriesTbl == "" { + c.seriesTbl = c.TableName + "_series" + } expSeriesDims := []*model.ColumnWithType{ {Name: "__series_id", Type: &model.TypeInfo{Type: model.Int64}}, {Name: "__mgmt_id", Type: &model.TypeInfo{Type: model.Int64}}, {Name: "labels", Type: &model.TypeInfo{Type: model.String}}, } var seriesDims []*model.ColumnWithType - if seriesDims, err = getDims(c.DBName, c.seriesTbl, nil, c.taskCfg.Parser, conn); err != nil { + if seriesDims, err = getDims(c.dbName, c.seriesTbl, nil, c.taskCfg.Parser, conn); err != nil { if errors.Is(err, ErrTblNotExist) { - err = errors.Wrapf(err, "Please create series table for %s.%s", c.DBName, c.TableName) + err = errors.Wrapf(err, "Please create series table for %s.%s", c.dbName, c.TableName) return } return @@ -331,7 +352,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { } else { for i := range expSeriesDims { if seriesDims[i].Name != expSeriesDims[i].Name || - seriesDims[i].Type != expSeriesDims[i].Type { + seriesDims[i].Type.Type != expSeriesDims[i].Type.Type { badFirst = true break } @@ -357,7 +378,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) } c.promSerSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", - c.DBName, + c.dbName, c.seriesTbl, strings.Join(serDimsQuoted, ",")) @@ -382,18 +403,20 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { func (c *ClickHouse) initSchema() (err error) { if idx := strings.Index(c.taskCfg.TableName, "."); idx > 0 { c.TableName = c.taskCfg.TableName[idx+1:] - c.DBName = c.taskCfg.TableName[0:idx] + c.dbName = c.taskCfg.TableName[0:idx] } else { c.TableName = c.taskCfg.TableName[idx+1:] - c.DBName = c.cfg.Clickhouse.DB + c.dbName = c.cfg.Clickhouse.DB } + c.seriesTbl = c.taskCfg.SeriesTableName + sc := pool.GetShardConn(0) var conn clickhouse.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } if c.taskCfg.AutoSchema { - if c.Dims, err = getDims(c.DBName, c.TableName, c.taskCfg.ExcludeColumns, c.taskCfg.Parser, conn); err != nil { + if c.Dims, err = getDims(c.dbName, c.TableName, c.taskCfg.ExcludeColumns, c.taskCfg.Parser, conn); err != nil { return } } else { @@ -420,7 +443,7 @@ func (c *ClickHouse) initSchema() (err error) { quotedDms[i] = fmt.Sprintf("`%s`", c.Dims[i].Name) } c.prepareSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", - c.DBName, + c.dbName, c.TableName, strings.Join(quotedDms, ",")) util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.taskCfg.Name)) @@ -485,12 +508,12 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } if c.taskCfg.PrometheusSchema { if intVal == model.String { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.DBName, c.seriesTbl, onCluster, strKey, strVal) + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.dbName, c.seriesTbl, onCluster, strKey, strVal) queries = append(queries, query) affectDistSeries = true } } else { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.DBName, c.TableName, onCluster, strKey, strVal) + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.dbName, c.TableName, onCluster, strKey, strVal) queries = append(queries, query) affectDistMetric = true } @@ -514,12 +537,12 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } if chCfg.Cluster != "" { if affectDistMetric { - if err = recreateDistTbls(chCfg.Cluster, c.DBName, c.TableName, c.distMetricTbls, conn); err != nil { + if err = recreateDistTbls(chCfg.Cluster, c.dbName, c.TableName, c.distMetricTbls, conn); err != nil { return } } if affectDistSeries { - if err = recreateDistTbls(chCfg.Cluster, c.DBName, c.seriesTbl, c.distSeriesTbls, conn); err != nil { + if err = recreateDistTbls(chCfg.Cluster, c.dbName, c.seriesTbl, c.distSeriesTbls, conn); err != nil { return } } @@ -536,7 +559,7 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { return } query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed\(\'%s\', \'%s\', \'%s\'.*\)')`, - c.DBName, chCfg.Cluster, c.DBName, table) + c.dbName, chCfg.Cluster, c.dbName, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows driver.Rows diff --git a/statistics/statistics.go b/statistics/statistics.go index ffddbe04..9fabf120 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -68,6 +68,13 @@ var ( }, []string{"consumer", "topic", "partition"}, ) + ConsumeLags = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "consume_lags", + Help: "message lags for each task, work with cluster of sinker", + }, + []string{"consumer", "topic", "task"}, + ) ShardMsgs = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Name: prefix + "shard_msgs", @@ -134,6 +141,7 @@ func init() { prometheus.MustRegister(FlushMsgsTotal) prometheus.MustRegister(FlushMsgsErrorTotal) prometheus.MustRegister(ConsumeOffsets) + prometheus.MustRegister(ConsumeLags) prometheus.MustRegister(ShardMsgs) prometheus.MustRegister(WritingPoolBacklog) prometheus.MustRegister(WritingDurations) @@ -226,6 +234,7 @@ func (p *Pusher) reconnect() { Collector(FlushMsgsTotal). Collector(FlushMsgsErrorTotal). Collector(ConsumeOffsets). + Collector(ConsumeLags). Collector(ShardMsgs). Collector(WritingPoolBacklog). Collector(WritingDurations). From faad0764590628e2f4b24fa35f6c2b46e304ef25 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 20 Jan 2023 14:39:33 +0800 Subject: [PATCH 310/404] Delete Quota when stopping corresponding tasks --- config/config.go | 2 -- output/clickhouse.go | 24 +++++++++++++++++++----- task/sinker.go | 16 ++++++++++++++++ 3 files changed, 35 insertions(+), 7 deletions(-) diff --git a/config/config.go b/config/config.go index 011fa116..5989c2c2 100644 --- a/config/config.go +++ b/config/config.go @@ -142,8 +142,6 @@ type TaskConfig struct { PrometheusSchema bool // fields match PromLabelsBlackList are not considered as labels. Requires PrometheusSchema be true. PromLabelsBlackList string // the regexp of black list - // whether load series at startup - LoadSeriesAtStartup bool // ShardingKey is the column name to which sharding against ShardingKey string `json:"shardingKey,omitempty"` diff --git a/output/clickhouse.go b/output/clickhouse.go index 4f7519f3..3ab39ee3 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -267,7 +267,7 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { tbl = c.distSeriesTbls[0] } sq := &seriesQuota{} - if v, ok := seriesQuotas.LoadOrStore(tbl, sq); ok { + if v, ok := seriesQuotas.LoadOrStore(c.GetSeriesQuotaKey(), sq); ok { c.seriesQuota = v.(*seriesQuota) } else { sq.Lock() @@ -276,10 +276,7 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { sq.Unlock() c.seriesQuota = sq } - if !c.taskCfg.LoadSeriesAtStartup { - util.Logger.Info(fmt.Sprintf("skipped loading series from %v", tbl), zap.String("task", c.taskCfg.Name)) - return - } + query := fmt.Sprintf("SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s.%s ORDER BY sid", c.dbName, tbl) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) var rs driver.Rows @@ -578,3 +575,20 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { } return } + +func UpdateSeriesQuotas(c map[string]struct{}) { + seriesQuotas.Range(func(key, value any) bool { + k := key.(string) + if _, ok := c[k]; !ok { + seriesQuotas.Delete(k) + } + return true + }) +} + +func (c *ClickHouse) GetSeriesQuotaKey() string { + if c.taskCfg.PrometheusSchema { + return c.dbName + "." + c.seriesTbl + } + return "" +} diff --git a/task/sinker.go b/task/sinker.go index 23260d46..ff76cfee 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -28,6 +28,7 @@ import ( "github.com/housepower/clickhouse_sinker/config" cm "github.com/housepower/clickhouse_sinker/config_manager" "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/output" "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" @@ -290,6 +291,8 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) { // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). s.stopAllTasks() + // 1.1 Cleanup the SeriesQuotas when stopping tasks + output.UpdateSeriesQuotas(make(map[string]struct{})) // 2. Initialize clickhouse connections. chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, @@ -357,6 +360,19 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { delete(s.consumers, v) } wg.Wait() + // 1.1) Update the SeriesQuotas when stopping tasks + tables := make(map[string]struct{}) + for _, c := range s.consumers { + c.tasks.Range(func(key, value any) bool { + k := value.(*Service).clickhouse.GetSeriesQuotaKey() + if k != "" { + tables[k] = struct{}{} + } + return true + }) + } + output.UpdateSeriesQuotas(tables) + // 2) fire up new consumers // Record the new config s.curCfg = newCfg From e34be1387c5972d3b6646f4a3c5c2f4ef4958f6c Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Tue, 31 Jan 2023 17:24:32 +0800 Subject: [PATCH 311/404] Add support of ingesting multi-value metrics --- cmd/kafka_gen_prom/main.go | 14 +++++++++++++- output/clickhouse.go | 13 +++++++------ 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index 2189055c..aa7fa48b 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -120,6 +120,9 @@ type Labels map[string]string type Datapoint struct { Timestamp time.Time Value float32 + Value1 float64 + Value2 int64 + Value3 bool Name string `json:"__name__"` Labels Labels LabelKeys []string @@ -141,7 +144,8 @@ func (dp Datapoint) MarshalJSON() ([]byte, error) { return nil, err } labels2 := labels[1 : len(labels)-1] - msg := fmt.Sprintf(`{"timestamp":"%s", "value":%f,"__name__":"%s","labels":%s,%s,"__series_id":%d,"__mgmt_id":%d}`, dp.Timestamp.Format(time.RFC3339), dp.Value, dp.Name, labels, labels2, seriesID, mgmtID) + msg := fmt.Sprintf(`{"timestamp":"%s", "value":%f, "value1":%g, "value2":%d, "value3":%t, "__name__":"%s", %s, "__series_id":%d, "__mgmt_id":%d}`, + dp.Timestamp.Format(time.RFC3339), dp.Value, dp.Value1, dp.Value2, dp.Value3, dp.Name, labels2, seriesID, mgmtID) return []byte(msg), nil } @@ -159,6 +163,11 @@ func randValue() (val string) { return } +func randBool() bool { + rand.Seed(time.Now().UnixNano()) + return rand.Intn(2) == 1 +} + func initMetrics() { metrics = make([]PromMetric, NumMetrics) for i := 0; i < NumMetrics; i++ { @@ -209,6 +218,9 @@ func generate() { dp := Datapoint{ Timestamp: timestamp, Value: rand.Float32(), + Value1: rand.Float64(), + Value2: rand.Int63(), + Value3: randBool(), Name: metrics[i].Name, Labels: make(Labels), LabelKeys: metrics[i].LabelKeys, diff --git a/output/clickhouse.go b/output/clickhouse.go index 3ab39ee3..0cd5ce60 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -503,13 +503,14 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { err = errors.Newf("%s: BUG: unsupported column type %s", taskCfg.Name, model.GetTypeName(intVal)) return false } - if c.taskCfg.PrometheusSchema { - if intVal == model.String { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.dbName, c.seriesTbl, onCluster, strKey, strVal) - queries = append(queries, query) - affectDistSeries = true - } + if c.taskCfg.PrometheusSchema && intVal == model.String { + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.dbName, c.seriesTbl, onCluster, strKey, strVal) + queries = append(queries, query) + affectDistSeries = true } else { + if c.taskCfg.PrometheusSchema && intVal > model.String { + util.Logger.Fatal("unsupported metric value type", zap.String("type", strVal), zap.String("name", strKey), zap.String("task", c.taskCfg.Name)) + } query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.dbName, c.TableName, onCluster, strKey, strVal) queries = append(queries, query) affectDistMetric = true From 2b7874d2b2c0c434a05fe3a847d42fc22433ea7e Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Mon, 6 Feb 2023 14:06:43 +0800 Subject: [PATCH 312/404] Ignore the SIGHUP signal --- cmd/kafka_gen_log/main.go | 6 +++--- cmd/kafka_gen_metric/main.go | 2 +- cmd/kafka_gen_prom/main.go | 2 +- util/app.go | 7 +++++-- util/signal.go | 6 +++--- 5 files changed, 13 insertions(+), 10 deletions(-) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 1aa06dd1..34c8677b 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -143,7 +143,7 @@ func (g *LogGenerator) Stat() (l, s int64) { return } -//reset logfiles +// reset logfiles func (g *LogGenerator) Init() error { g.logfiles = nil g.off = -1 @@ -186,7 +186,7 @@ func (g *LogGenerator) Init() error { return nil } -//switch to next log file +// switch to next log file func (g *LogGenerator) next() (err error) { g.scanner = nil if g.reader != nil { @@ -329,7 +329,7 @@ log_file_pattern: file name pattern, for example, '^secure.*$'`, os.Args[0], os. util.Logger.Fatal("got error", zap.Error(err)) } - ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) + ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM) g := &LogGenerator{} if err := g.Init(); err != nil { util.Logger.Fatal("got error", zap.Error(err)) diff --git a/cmd/kafka_gen_metric/main.go b/cmd/kafka_gen_metric/main.go index 69940790..6b70351b 100644 --- a/cmd/kafka_gen_metric/main.go +++ b/cmd/kafka_gen_metric/main.go @@ -210,7 +210,7 @@ topic: for example, sensor_dt_result_online`, os.Args[0], os.Args[0]) } var prevLines, prevSize int64 - ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) + ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM) go generate() ticker := time.NewTicker(10 * time.Second) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index aa7fa48b..c2b3acb2 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -279,7 +279,7 @@ topic: for example, prom_extend`, os.Args[0], os.Args[0]) } var prevLines, prevSize int64 - ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) + ctx, _ := signal.NotifyContext(context.Background(), os.Interrupt, syscall.SIGTERM) go generate() ticker := time.NewTicker(10 * time.Second) diff --git a/util/app.go b/util/app.go index f4501c1a..9528d454 100644 --- a/util/app.go +++ b/util/app.go @@ -16,6 +16,8 @@ limitations under the License. package util import ( + "fmt" + "go.uber.org/zap" ) @@ -31,8 +33,9 @@ func Run(appName string, initFunc, jobFunc, cleanupFunc func() error) { } }() - WaitForExitSign() - Logger.Info(appName + " got the exit signal, start to clean") + s := WaitForExitSign() + Logger.Info(fmt.Sprintf("%s got the exit signal %s, start to clean", appName, s)) + if err := cleanupFunc(); err != nil { Logger.Fatal(appName+" clean failed", zap.Error(err)) } diff --git a/util/signal.go b/util/signal.go index 58c433cb..7f146c93 100644 --- a/util/signal.go +++ b/util/signal.go @@ -21,8 +21,8 @@ import ( "syscall" ) -func WaitForExitSign() { +func WaitForExitSign() os.Signal { c := make(chan os.Signal, 1) - signal.Notify(c, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) - <-c + signal.Notify(c, os.Interrupt, syscall.SIGTERM) + return <-c } From f15f2cf4d335350851937a04a89c5e26d9060262 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Mon, 20 Feb 2023 23:37:05 +0800 Subject: [PATCH 313/404] Fix changing TimeUnit config property does not trigger config reload --- task/consumer.go | 3 ++- task/sinker.go | 21 +++++++++------------ 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/task/consumer.go b/task/consumer.go index 067f7822..56774666 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -135,7 +135,8 @@ func (c *Consumer) updateGroupConfig(g *config.GroupConfig) { return } c.grpConfig = g - // stop the processFetch routine, make sure no more input to the commit chan & writing pool + // restart the processFetch routine because of potential BufferSize or FlushInterval change + // make sure no more input to the commit chan & writing pool c.stopCh <- struct{}{} c.processWg.Wait() go c.processFetch() diff --git a/task/sinker.go b/task/sinker.go index ff76cfee..69c29c98 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -341,6 +341,15 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { if !reflect.DeepEqual(c.grpConfig.Topics, group.Topics) { deleteConsumers = append(deleteConsumers, name) } else { + // apply TaskConfig Change + for _, tCfg := range group.Configs { + if !reflect.DeepEqual(tCfg, s.consumers[name].grpConfig.Configs[tCfg.Name]) { + task := NewTaskService(newCfg, tCfg, s.consumers[name]) + if err = task.Init(); err != nil { + return + } + } + } c.updateGroupConfig(group) } } @@ -389,18 +398,6 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { s.consumers[v.Name] = c } } - // 3) apply TaskConfig Change - for name, c := range newCfg.Groups { - for _, tCfg := range c.Configs { - if !reflect.DeepEqual(tCfg, s.consumers[name].grpConfig.Configs[tCfg.Name]) { - task := NewTaskService(newCfg, tCfg, s.consumers[name]) - if err = task.Init(); err != nil { - return - } - } - } - s.consumers[name].grpConfig = c - } } util.Logger.Info("applied another config", zap.Int("number", s.numCfg)) s.numCfg++ From 6dc62be35e1e58b3f1075272559158d47d2180a9 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Mon, 20 Feb 2023 23:39:12 +0800 Subject: [PATCH 314/404] Fix sinker crash because of illegal TimeZone issue & Decimal parsing error --- CHANGELOG.md | 68 + Makefile | 1 + cmd/kafka_gen_prom/main.go | 17 +- config_manager/nacos.go | 8 +- docker-compose.yml | 2 + docker/metrika.xml | 45 + docker/test_prom_metric.data | 10000 ++++++++++++++++++++++++++++++++ docker/test_prom_metric.hjson | 34 + go.metrictest.sh | 59 + go.mod | 83 +- go.sum | 536 +- task/task.go | 5 +- 12 files changed, 10378 insertions(+), 480 deletions(-) create mode 100644 CHANGELOG.md create mode 100644 docker/metrika.xml create mode 100644 docker/test_prom_metric.data create mode 100644 docker/test_prom_metric.hjson create mode 100755 go.metrictest.sh diff --git a/CHANGELOG.md b/CHANGELOG.md new file mode 100644 index 00000000..54ec506a --- /dev/null +++ b/CHANGELOG.md @@ -0,0 +1,68 @@ +# Changelog + +#### Version 3.0.1 (not released yet) + +Bug Fixes: +- Fix nacos publish config error "BUG: got different config" +- Fix changing "TimeUnit" config property does not trigger config reload +- Fix illegal "TimeZone" value result in sinker crash +- Fix wrong parsing result of Decimal type [909](https://github.com/ClickHouse/clickhouse-go/pull/909) + + +#### Version 3.0.0 (2023-02-07) + +New Features: +- Add support of ingesting multi-value metrics, the metric table will be expanded accordingly +- Allow specifying the series table name +- Allow customization of DatabaseName in task level + +Improvements: +- Group the tasks by consumerGroup property to reduce number of kafka client, see design.md for details + +Deprecation: +- Kafka-go and Sarama are no longer internal options for sinker +- + + +#### Version 2.6.9 (2023-02-07) + +Improvements: +- Ignore SIGHUP signal, so that fire up sinker with nohup could work correctly +- Stop retrying when facing offsets commit error, leave it to the future commitment to sync the offsets +- Offsets commit error should not result in a process abort + + +#### Version 2.6.8 (2022-12-10) + +New Features: +- Add clickhouse Map type support +- Small updates to allow TLS connections for AWS MSK, etc. + ([169](https://github.com/housepower/clickhouse_sinker/pull/169)) + +Bug Fixes: +- Fix ClickHouse.Init goroutine leak + + +#### Version 2.6.7 (2022-12-07) + +Improvements: +- Add new sinker metrics to show the wrSeriesQuota status +- Always allow writing new series to avoid data mismatch between series and metrics table + + +#### Version 2.6.6 (2022-12-05) + +Bug Fixes: +- reset wrSeries timely to avoid failure of writing metric data to clickhouse + + +#### Version 2.6.5 (2022-11-30) + +Bug Fixes: +- Fix the 'segmentation violation' in ch-go package +- Fix the create table error 'table already exists' when trying to create a distribution table + + +#### Previous releases + +See https://github.com/housepower/clickhouse_sinker/releases \ No newline at end of file diff --git a/Makefile b/Makefile index acdcdacb..029413ee 100644 --- a/Makefile +++ b/Makefile @@ -18,6 +18,7 @@ benchtest: pre go test -bench=. ./... systest: build bash go.test.sh + bash go.metrictest.sh lint: golangci-lint run --timeout=3m run: pre diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index c2b3acb2..649748d3 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -150,8 +150,9 @@ func (dp Datapoint) MarshalJSON() ([]byte, error) { } type PromMetric struct { - Name string - LabelKeys []string + Name string + LabelKeys []string + LabelValues []string } func randValue() (val string) { @@ -172,12 +173,14 @@ func initMetrics() { metrics = make([]PromMetric, NumMetrics) for i := 0; i < NumMetrics; i++ { m := PromMetric{ - Name: fmt.Sprintf("metric_%08d", i), - LabelKeys: make([]string, NumKeys), + Name: fmt.Sprintf("metric_%08d", i), + LabelKeys: make([]string, NumKeys), + LabelValues: make([]string, NumKeys), } for j := 0; j < NumKeys; j++ { - key := fmt.Sprintf("key_%06d", rand.Intn(NumAllKeys+1)) + key := fmt.Sprintf("key_%d", rand.Intn(NumAllKeys)) m.LabelKeys[j] = key + m.LabelValues[j] = randValue() } sort.Strings(m.LabelKeys) metrics[i] = m @@ -225,8 +228,8 @@ func generate() { Labels: make(Labels), LabelKeys: metrics[i].LabelKeys, } - for _, key := range metrics[i].LabelKeys { - dp.Labels[key] = randValue() + for valueuIndex, key := range metrics[i].LabelKeys { + dp.Labels[key] = metrics[i].LabelValues[valueuIndex] } _ = wp.Submit(func() { diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 766a3a14..370eda54 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -80,7 +80,9 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) if pop, ok := properties["logDir"]; ok { logDir, _ = pop.(string) } - logDir, _ = filepath.Abs(logDir) + if logDir, err = filepath.Abs(logDir); err != nil { + return errors.Wrapf(err, "") + } cc := constant.ClientConfig{ NamespaceId: namespaceID, TimeoutMs: 5000, @@ -160,6 +162,10 @@ func (ncm *NacosConfManager) PublishConfig(conf *config.Config) (err error) { err = errors.Wrapf(err, "") return } + // update the conf so that properties with default values are no longer nil + if err = hjson.Unmarshal(bs, conf); err != nil { + err = errors.Wrapf(err, "") + } return } diff --git a/docker-compose.yml b/docker-compose.yml index 675909d7..98471402 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -10,6 +10,7 @@ services: - "2181:2181" environment: ALLOW_ANONYMOUS_LOGIN: 1 + ZOO_4LW_COMMANDS_WHITELIST: "*" security_opt: - label:disable kafka: @@ -44,6 +45,7 @@ services: hard: 262144 volumes: - "./docker-entrypoint-initdb.d:/docker-entrypoint-initdb.d" + - "./docker/metrika.xml:/etc/clickhouse-server/config.d/metrika.xml" security_opt: - label:disable nacos: diff --git a/docker/metrika.xml b/docker/metrika.xml new file mode 100644 index 00000000..07f3e30f --- /dev/null +++ b/docker/metrika.xml @@ -0,0 +1,45 @@ + + + 01 + clickhouse01 + + + 2181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 10000 + 30000 + trace + + + + + 1 + localhost + 9444 + + + + + + localhost + 2181 + + + :: + + + foo + + false + + localhost + 9000 + + + + + \ No newline at end of file diff --git a/docker/test_prom_metric.data b/docker/test_prom_metric.data new file mode 100644 index 00000000..9c7f3a1a --- /dev/null +++ b/docker/test_prom_metric.data @@ -0,0 +1,10000 @@ +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.860439, "value1":0.30809143156043617, "value2":8296859231789889769, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697105, "value1":0.07464704753781262, "value2":3626726022956376302, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499281, "value1":0.2598747988733079, "value2":1472024199460334836, "value3":false, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389559, "value1":0.9035230182410444, "value2":5735152112214129185, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259346, "value1":0.6088348511469825, "value2":2118449310673513905, "value3":false, "__name__":"metric_00000013", "key_6":"k","key_7":"e","key_3":"f", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265140, "value1":0.41315907782222017, "value2":6967606456828959585, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273243, "value1":0.3670231186776885, "value2":3094287292036347916, "value3":false, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697145, "value1":0.4712183861515544, "value2":233543654937300193, "value3":true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.570507, "value1":0.11967792858372435, "value2":1467009477255473639, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421572, "value1":0.25352243267763397, "value2":5433840270731074230, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423291, "value1":0.6724498851050211, "value2":652727500980374465, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072653, "value1":0.4078722430828867, "value2":6479204450678692857, "value3":false, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.576237, "value1":0.03506340983128008, "value2":2409142807432889102, "value3":false, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928873, "value1":0.9749671198417696, "value2":806127466997750314, "value3":false, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.121356, "value1":0.3710118368988569, "value2":2961209029330787894, "value3":true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.527082, "value1":0.427919527676362, "value2":4168209888386865559, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971852, "value1":0.4558638608790214, "value2":7271190379082523197, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758405, "value1":0.642984416191772, "value2":635944986021178390, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.191870, "value1":0.4028374291432409, "value2":4925274338231144549, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.177741, "value1":0.112412024356175, "value2":1799812085131373089, "value3":false, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.562889, "value1":0.9463319128677472, "value2":8604221624040619668, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.145145, "value1":0.6344301595600267, "value2":5934019334637844764, "value3":true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512765, "value1":0.12698238608047413, "value2":3179145500529665851, "value3":false, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.947023, "value1":0.9362933587076374, "value2":6552003615608175727, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479188, "value1":0.8368835012768996, "value2":1048667356432629681, "value3":true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.886895, "value1":0.147063588220359, "value2":2554258082886382046, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495557, "value1":0.4897837285320758, "value2":3386687451644953805, "value3":false, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267510, "value1":0.7137029646630675, "value2":4723657657521756601, "value3":true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662037, "value1":0.39308757655823606, "value2":5160142240464547787, "value3":true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.001360, "value1":0.2816582812804884, "value2":6076665375380746353, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.501969, "value1":0.4236689406205709, "value2":725918256085862567, "value3":true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.956392, "value1":0.4674704766085337, "value2":4478503621053198593, "value3":false, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442861, "value1":0.15012494378573155, "value2":6132679974455498265, "value3":true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394657, "value1":0.8655032404449484, "value2":2033181385746149676, "value3":true, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.574670, "value1":0.8578324023084722, "value2":9030512731073145161, "value3":false, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844866, "value1":0.401882448563829, "value2":6461587433040802039, "value3":false, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.677301, "value1":0.17574043431028194, "value2":13200916467489149, "value3":true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643407, "value1":0.09478762453322248, "value2":9040728601657069686, "value3":true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.461584, "value1":0.2313038745196538, "value2":8542291140491717734, "value3":false, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480373, "value1":0.33926139409918454, "value2":1470854458100424231, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.387467, "value1":0.9027179517679272, "value2":2152676801390448497, "value3":false, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.081729, "value1":0.5930629975633843, "value2":3441892662554339501, "value3":true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.523164, "value1":0.27807377593445504, "value2":8726571622906981746, "value3":true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643880, "value1":0.8856942439946653, "value2":6532657219501338712, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.233730, "value1":0.12117132337083063, "value2":5262200042073716921, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113242, "value1":0.15754750121773875, "value2":3512597140842356375, "value3":true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.592540, "value1":0.8501957441234611, "value2":1120617963379990831, "value3":false, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.754300, "value1":0.2516365689787487, "value2":1009529273537079157, "value3":true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.738781, "value1":0.9115797570978498, "value2":654040880693718238, "value3":false, "__name__":"metric_00000057", "key_5":"j","key_2":"a","key_3":"b", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730945, "value1":0.567968215036453, "value2":3597506204077270120, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.940173, "value1":0.9712119929757523, "value2":5983635236065602039, "value3":false, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326670, "value1":0.07295490390000714, "value2":6022184335907772623, "value3":true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.571799, "value1":0.5813154467308365, "value2":9191368645337346592, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.547202, "value1":0.12691857025568046, "value2":3820781159866636536, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097732, "value1":0.6481650315571341, "value2":6773475876286839908, "value3":true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.184127, "value1":0.5736971101036478, "value2":7284609149743260027, "value3":false, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.925677, "value1":0.3771049289873205, "value2":1181794630742224031, "value3":true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.157660, "value1":0.5233716379944181, "value2":4014079834827187874, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.413654, "value1":0.21255841055237895, "value2":4422822169208563976, "value3":true, "__name__":"metric_00000067", "key_9":"a","key_0":"c","key_4":"d", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282429, "value1":0.5134755380331291, "value2":2875914720894216239, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510799, "value1":0.4811695837424015, "value2":2131765002626290228, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855059, "value1":0.1803859944439004, "value2":4143870194342227353, "value3":true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841548, "value1":0.8921162909498762, "value2":2995639098842929032, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904367, "value1":0.6827455696983789, "value2":4486024070965990492, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903308, "value1":0.5275068815139481, "value2":4431489892412543662, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369269, "value1":0.3088848570530923, "value2":2711948390039375929, "value3":true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.246425, "value1":0.22469514297321577, "value2":3311794715258074199, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.627147, "value1":0.5085084020761419, "value2":7449844520407226749, "value3":true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757065, "value1":0.7340080407469496, "value2":1399142812283896331, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404001, "value1":0.6927872548099725, "value2":4611922647039691390, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.548980, "value1":0.9504303368975154, "value2":784663243740669037, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088930, "value1":0.5112126366657939, "value2":1061303860942908201, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808273, "value1":0.07227725639854525, "value2":705291637800185980, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.195134, "value1":0.9915791690400784, "value2":1380206536675804157, "value3":true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588271, "value1":0.9410252962292832, "value2":2976150472594021014, "value3":false, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830440, "value1":0.5990964794781901, "value2":6208539757039787288, "value3":true, "__name__":"metric_00000083", "key_9":"a","key_1":"g","key_6":"b", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.988764, "value1":0.12421784563285446, "value2":600116094525731026, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793643, "value1":0.013993638720735705, "value2":249013181149620349, "value3":false, "__name__":"metric_00000085", "key_5":"j","key_1":"d", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537335, "value1":0.9710360883032237, "value2":3990188937402642001, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.783911, "value1":0.5202417888338783, "value2":9207852713084863631, "value3":true, "__name__":"metric_00000086", "key_4":"b","key_6":"f","key_2":"k", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000369, "value1":0.4757654945131251, "value2":617662898409566699, "value3":true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720049, "value1":0.11616063522827627, "value2":1134111314294224704, "value3":true, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850537, "value1":0.856287241545387, "value2":3915015074142268096, "value3":false, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117449, "value1":0.23938708551093754, "value2":6182563875693526929, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720636, "value1":0.6929840624902686, "value2":7184719611552077354, "value3":false, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890463, "value1":0.39990668829147297, "value2":5236439635142539127, "value3":false, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598510, "value1":0.794840854439141, "value2":7790247245768394828, "value3":false, "__name__":"metric_00000094", "key_2":"e","key_7":"j","key_1":"c", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.103618, "value1":0.837788909134762, "value2":1746463551678400384, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890891, "value1":0.701693361054133, "value2":750546117784279688, "value3":true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243694, "value1":0.04252945224095913, "value2":7022047191936829657, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698720, "value1":0.8318935119459053, "value2":2403728383571794535, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.362462, "value1":0.5425805070018154, "value2":7722798226519636439, "value3":false, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447582, "value1":0.6901117678727204, "value2":1225566659535077888, "value3":false, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884009, "value1":0.9848633762091424, "value2":1508601246687431755, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.895176, "value1":0.1370326687436836, "value2":7673037864112111033, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.521633, "value1":0.14853540424497083, "value2":6875647668629772843, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388849, "value1":0.856681841687182, "value2":289008565390108021, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.747443, "value1":0.735786478790032, "value2":7309545501096360932, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528790, "value1":0.010059663849976508, "value2":3366632872144662566, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.586068, "value1":0.8316103940598538, "value2":6216090017784726937, "value3":true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154422, "value1":0.6449454723419513, "value2":5588370277273470937, "value3":true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.476987, "value1":0.688075774177057, "value2":3232344568636281729, "value3":false, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.176248, "value1":0.8630722552735859, "value2":7054623290814757387, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.553861, "value1":0.6275284095417549, "value2":8413581575795957099, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482699, "value1":0.30152766631318734, "value2":2728742455813076667, "value3":true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228157, "value1":0.09912287337858451, "value2":1368124455579266783, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.363726, "value1":0.2556344439654489, "value2":8083466247113334458, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598631, "value1":0.00654705967457182, "value2":678136325524679028, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188097, "value1":0.33693220746708563, "value2":5349169986765993063, "value3":false, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.591197, "value1":0.3158077067804786, "value2":8408038257495371649, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.723537, "value1":0.8935982855510853, "value2":6781320910854884909, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.714830, "value1":0.8483752317059017, "value2":4114947449469032042, "value3":true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510153, "value1":0.8569271663104114, "value2":2163617609666100779, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.556403, "value1":0.46846499185707535, "value2":4543935722642344774, "value3":false, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115027, "value1":0.13242237154578534, "value2":4001391755557856082, "value3":false, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700402, "value1":0.31919253163578853, "value2":9168039730828764814, "value3":true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473591, "value1":0.914966327309789, "value2":7574079944969934140, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657212, "value1":0.7123671399711553, "value2":5845893001941155972, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389666, "value1":0.46892477810031413, "value2":3773448585226589360, "value3":true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365529, "value1":0.10038955567825718, "value2":7677417085782135970, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.181780, "value1":0.3807188874415628, "value2":213730861941923164, "value3":false, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632990, "value1":0.6352610882615881, "value2":7695215886438259449, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.801703, "value1":0.5435516239407236, "value2":1197124892718412788, "value3":false, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943146, "value1":0.6459645738185148, "value2":257718117926443886, "value3":false, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047331, "value1":0.2599891672558978, "value2":1859555081827581052, "value3":false, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.329003, "value1":0.7536967883211322, "value2":8944365307464980699, "value3":true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750531, "value1":0.8871777984871712, "value2":4584326435451561707, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820378, "value1":0.6275908409813838, "value2":2808244151477882254, "value3":false, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.664852, "value1":0.971622999773127, "value2":3565786429109692783, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502157, "value1":0.19081539446623427, "value2":1479393056820182163, "value3":false, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396316, "value1":0.5123967312660013, "value2":8943627767884961054, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934631, "value1":0.19466544110858217, "value2":3956006751549658946, "value3":true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206549, "value1":0.017226775576345523, "value2":7020366329672792405, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118256, "value1":0.5188472954068934, "value2":8138153858856845378, "value3":true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584343, "value1":0.14448963894972008, "value2":4790553180218846981, "value3":false, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.877457, "value1":0.6880296047171478, "value2":1215785358113912003, "value3":true, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409285, "value1":0.05411761237341633, "value2":4709001633389005280, "value3":true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364854, "value1":0.8998190824029622, "value2":2511578657661549153, "value3":true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.973548, "value1":0.8069354470874932, "value2":2595652771451514899, "value3":true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.073905, "value1":0.2517811560702055, "value2":6140023292659546232, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.262163, "value1":0.5630653619265122, "value2":3018188000144695876, "value3":true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641275, "value1":0.6900145697913901, "value2":5216098013140123128, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392564, "value1":0.018717055152633055, "value2":8077083634713789634, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932220, "value1":0.8388125579510224, "value2":2685186121830778256, "value3":false, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206042, "value1":0.49376606624615665, "value2":5850726108082863170, "value3":false, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215184, "value1":0.356531340492791, "value2":4642776908341780996, "value3":true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.558758, "value1":0.575409651955595, "value2":2207683625879983306, "value3":true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953416, "value1":0.9073618953615663, "value2":5651749773058007750, "value3":true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.648335, "value1":0.46358450476088653, "value2":1290620516019440255, "value3":true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.487910, "value1":0.012547280333283235, "value2":4919259233823782968, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.378926, "value1":0.7085830860476057, "value2":2171649428344764858, "value3":true, "__name__":"metric_00000157", "key_8":"j","key_3":"b","key_4":"k", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802501, "value1":0.31728201505130527, "value2":5419079224591774860, "value3":false, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.656795, "value1":0.24875018847328167, "value2":485829881375178402, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938831, "value1":0.08769582144908596, "value2":6951540802412020838, "value3":false, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231956, "value1":0.7863053106916462, "value2":1030265256144215264, "value3":true, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.631246, "value1":0.46917511334330375, "value2":3774063736313175623, "value3":false, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305077, "value1":0.834556547518894, "value2":636203159633468096, "value3":true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808070, "value1":0.5245784354956413, "value2":5259003801435693215, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641827, "value1":0.7755759714857714, "value2":3467599768461222251, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.306769, "value1":0.47958408429359173, "value2":5365936635633468542, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294040, "value1":0.5158994633208911, "value2":7429034992856357163, "value3":true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.961943, "value1":0.7435043392340425, "value2":8500250963468273773, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935561, "value1":0.5801624578827164, "value2":8754050195411098007, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444846, "value1":0.030970547839451835, "value2":8387858736748608077, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358418, "value1":0.5401368375372775, "value2":7107146838213157067, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.546704, "value1":0.46330699877542014, "value2":8769213962266908815, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.270889, "value1":0.46793759900681503, "value2":6364231395353410490, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549392, "value1":0.3430929972372329, "value2":4966477192488000261, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428697, "value1":0.7243138275027047, "value2":2836857643122176656, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993399, "value1":0.40084653235121936, "value2":6794009333725376121, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.506725, "value1":0.07262914457616802, "value2":3459356114730929347, "value3":false, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202924, "value1":0.07381206265687722, "value2":2649194281139606297, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.411238, "value1":0.9684314554036981, "value2":3705584143921682379, "value3":false, "__name__":"metric_00000178", "key_5":"h","key_6":"k","key_4":"c", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358603, "value1":0.534711919539449, "value2":4138168909279457429, "value3":false, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989681, "value1":0.19207319251633845, "value2":1593591737448226138, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.052714, "value1":0.27756026777792103, "value2":6377557735209128817, "value3":true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614036, "value1":0.018601161874125947, "value2":151552447676967905, "value3":false, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305181, "value1":0.031288435638661224, "value2":3937949086874523036, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715232, "value1":0.6397334976781403, "value2":7614941574913315366, "value3":true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705549, "value1":0.6996034412212425, "value2":6200772172174019130, "value3":true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.302276, "value1":0.5239925847096263, "value2":1057710529698426724, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200111, "value1":0.2567985665426193, "value2":4529602646404884783, "value3":false, "__name__":"metric_00000002", "key_2":"g","key_8":"f","key_1":"a", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.727436, "value1":0.43685005741153693, "value2":3304580792456252079, "value3":false, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.351586, "value1":0.3164739225094333, "value2":3897998541684002875, "value3":false, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.665954, "value1":0.5906227199646251, "value2":6746215572356057637, "value3":true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361921, "value1":0.3364933451115498, "value2":761972198573021391, "value3":false, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.889779, "value1":0.36794570756599093, "value2":2090465055639313788, "value3":false, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.699184, "value1":0.5483254879513706, "value2":7720905658039555449, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388162, "value1":0.5287855210410441, "value2":7195732574645022274, "value3":false, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154496, "value1":0.6641476496489667, "value2":9137939751947784172, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327473, "value1":0.5518373975194654, "value2":3971910670147599886, "value3":true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063454, "value1":0.43820340286329656, "value2":5405166066848368254, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.781841, "value1":0.031101766659472223, "value2":4189381224837071716, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171755, "value1":0.46498479978119844, "value2":4931395791650935239, "value3":false, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.559617, "value1":0.8752912652134259, "value2":8586908304160851241, "value3":true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.323852, "value1":0.35821728605182146, "value2":5163275989952284916, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030550, "value1":0.8225486647874358, "value2":2481647473431927932, "value3":false, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.712533, "value1":0.368964069168292, "value2":8062115249497297834, "value3":true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298843, "value1":0.30262512534550123, "value2":5999722310995378983, "value3":false, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778244, "value1":0.3716152000047593, "value2":6978129792906951032, "value3":true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.144352, "value1":0.42812039567105226, "value2":6591666290600795514, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616112, "value1":0.5501694032291233, "value2":5953070646927834342, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843613, "value1":0.7957093053513518, "value2":779447045934204434, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.021516, "value1":0.3651263217338076, "value2":1101615937937937836, "value3":false, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.707183, "value1":0.9474415697823648, "value2":3163832075607398652, "value3":true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807066, "value1":0.328067579350897, "value2":1404186285892115543, "value3":true, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327059, "value1":0.4531450671817094, "value2":3186214527240993583, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.629874, "value1":0.555595498018525, "value2":7512625623217629070, "value3":false, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203064, "value1":0.24518269440439186, "value2":7974348531674940058, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865398, "value1":0.7125714321236861, "value2":1185618794516530658, "value3":false, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.098668, "value1":0.6050958429061423, "value2":7737776979675298527, "value3":false, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659514, "value1":0.8901996322311656, "value2":7752156196236457080, "value3":false, "__name__":"metric_00000220", "key_6":"h","key_9":"d","key_5":"g", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650309, "value1":0.7869508663362883, "value2":6010716207202506080, "value3":false, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.779127, "value1":0.6366846028665012, "value2":7797452629031711950, "value3":false, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.281652, "value1":0.9118975722819298, "value2":6103834325448620626, "value3":false, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868834, "value1":0.18117722257021981, "value2":6098726761586315616, "value3":true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865255, "value1":0.5168875940701739, "value2":7616170573520868343, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000667, "value1":0.8879098452429443, "value2":2716291319460579102, "value3":false, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938952, "value1":0.27931352835868933, "value2":2351410106131072829, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524371, "value1":0.5375950258397566, "value2":7234763394924546015, "value3":true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718980, "value1":0.7918266548241415, "value2":1156714261082582562, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532951, "value1":0.4063578415664877, "value2":2401980297367889095, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780599, "value1":0.15947463153475608, "value2":6812388220933080605, "value3":true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078621, "value1":0.6070073793267291, "value2":1012316510571499763, "value3":true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404621, "value1":0.9613908610430107, "value2":6888867545087821595, "value3":true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.130043, "value1":0.26321351304547996, "value2":4446851417687808950, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602508, "value1":0.38864606344069585, "value2":2797998328851090673, "value3":false, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244218, "value1":0.5263437247014857, "value2":884821963010111580, "value3":false, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415267, "value1":0.3515494200462383, "value2":3356463166369801537, "value3":false, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303765, "value1":0.5270577861362743, "value2":2138283533246685033, "value3":true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.254002, "value1":0.303619701206094, "value2":7629281847530948891, "value3":false, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588498, "value1":0.5481446854335168, "value2":7641330013258723790, "value3":false, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.620575, "value1":0.6420150180085478, "value2":7965173856733830205, "value3":false, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243174, "value1":0.400523011031962, "value2":4931302022232098402, "value3":false, "__name__":"metric_00000240", "key_9":"c","key_4":"d","key_8":"d", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.104291, "value1":0.4501470045022247, "value2":6419830717700946282, "value3":true, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605242, "value1":0.5489672412234975, "value2":25272414308366080, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705962, "value1":0.2696978321817019, "value2":7266661553560831399, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.204594, "value1":0.050667752810758936, "value2":4939383554296719335, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.694791, "value1":0.8451677349011486, "value2":1862202581082766068, "value3":true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178458, "value1":0.47634419936555994, "value2":3993368962808231698, "value3":true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879184, "value1":0.05969390345398635, "value2":5274917008847651276, "value3":false, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854470, "value1":0.8497662902937875, "value2":3035062077693412013, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.397422, "value1":0.08213093074253511, "value2":5967759139282299209, "value3":true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584032, "value1":0.08273000380656788, "value2":2735658568798834919, "value3":true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822339, "value1":0.581737952515166, "value2":2604615641240906232, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.308039, "value1":0.34242828701171635, "value2":3092061640651599674, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.913723, "value1":0.005597195694116873, "value2":2620704762797681315, "value3":false, "__name__":"metric_00000254", "key_3":"b","key_8":"c","key_1":"f", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618957, "value1":0.6413425640197833, "value2":5287503767189177120, "value3":false, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454987, "value1":0.32512559877918956, "value2":2506424876460932044, "value3":true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447066, "value1":0.20850990987663237, "value2":7977478851800531911, "value3":true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.683345, "value1":0.838649315370365, "value2":4999282712485449007, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935535, "value1":0.4089194818619663, "value2":722917113975878160, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439326, "value1":0.8812716905619982, "value2":6658838831662104981, "value3":false, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069099, "value1":0.15024751394280958, "value2":1328305014122533446, "value3":true, "__name__":"metric_00000258", "key_7":"f","key_3":"a","key_5":"e", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473312, "value1":0.9280459649583414, "value2":3438360323166932806, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215711, "value1":0.8566516621694267, "value2":1663749581830575806, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312134, "value1":0.566681231011358, "value2":6911167899525793851, "value3":true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862090, "value1":0.5364584771183928, "value2":7729755763890728353, "value3":true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456397, "value1":0.7096399907515605, "value2":5531147306124199678, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.587843, "value1":0.8985325214047336, "value2":4030305760543278105, "value3":false, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710467, "value1":0.5099257391896752, "value2":1179017586099298270, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188204, "value1":0.8723809422062396, "value2":5093258983701866401, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.681846, "value1":0.23032248565211208, "value2":3357194399728955391, "value3":true, "__name__":"metric_00000264", "key_2":"j","key_4":"c","key_1":"i", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614220, "value1":0.9249072784237294, "value2":4254302260697988757, "value3":true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.028000, "value1":0.2236758406407766, "value2":5499863050282572666, "value3":true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923027, "value1":0.4209047854675161, "value2":4430077989918325993, "value3":false, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203873, "value1":0.8404470170534011, "value2":532376833912389989, "value3":false, "__name__":"metric_00000269", "key_2":"f","key_6":"b","key_1":"d", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.541847, "value1":0.24084743744285772, "value2":2314457094095648600, "value3":true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932124, "value1":0.7743913731102018, "value2":7842905515694643998, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.986636, "value1":0.18488491173653657, "value2":1233277417792576747, "value3":false, "__name__":"metric_00000275", "key_6":"f","key_0":"j","key_2":"i", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.491189, "value1":0.5404267488261955, "value2":5193038540757554939, "value3":true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.449064, "value1":0.9915502184839048, "value2":3914891481887672833, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203773, "value1":0.11544040430435766, "value2":8720914144754355340, "value3":true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.637865, "value1":0.5397959742632489, "value2":5016935922539098494, "value3":true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.164476, "value1":0.40957356383804516, "value2":8101990321123394900, "value3":true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212208, "value1":0.31170589199654203, "value2":8172096864950686883, "value3":false, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427578, "value1":0.9500075990560398, "value2":4448206698192420471, "value3":false, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375888, "value1":0.08043137384857349, "value2":5428459057669693793, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.253419, "value1":0.30426981771564987, "value2":3701960397375147641, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.064111, "value1":0.6613366419136151, "value2":9214004723321204161, "value3":false, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618634, "value1":0.28919063506417325, "value2":2549339895850480456, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.581311, "value1":0.5011828194867549, "value2":2394702011956086199, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.619872, "value1":0.3579296101205142, "value2":3114339742928723546, "value3":true, "__name__":"metric_00000289", "key_6":"f","key_0":"c", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928952, "value1":0.7843473952520837, "value2":4007100053019753408, "value3":true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.678699460362967, "value2":1449401258062210262, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.791597, "value1":0.5225036343492986, "value2":7759663471842896645, "value3":true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794616, "value1":0.3395042657376233, "value2":1662502503940889595, "value3":true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.466754, "value1":0.9041538990976105, "value2":535273624649157699, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693734, "value1":0.009440601901378784, "value2":2148867264362048197, "value3":true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841713, "value1":0.059732250656449104, "value2":6677903169635737819, "value3":true, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.746032, "value1":0.16006431951166478, "value2":5022444125278904149, "value3":false, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.158292, "value1":0.8619758377742992, "value2":3793514584183151927, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364688, "value1":0.7050682785197309, "value2":8826236482901828470, "value3":true, "__name__":"metric_00000296", "key_9":"f","key_2":"e","key_7":"e", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273970, "value1":0.40265473643498934, "value2":8303926135399734280, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030530, "value1":0.7286415975665108, "value2":160039363895729267, "value3":true, "__name__":"metric_00000244", "key_5":"j","key_3":"g", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266449, "value1":0.033547786099323434, "value2":7620487585623102750, "value3":false, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025977, "value1":0.17793807237193235, "value2":8744525786084307091, "value3":false, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.806031, "value1":0.5052650450730324, "value2":4060064708931174932, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035732, "value1":0.9743453198182447, "value2":4099915557434576704, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.646092, "value1":0.21956311768718226, "value2":8589888788620337480, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078229, "value1":0.4079449893422866, "value2":6173402600986499343, "value3":false, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666881, "value1":0.9863097557928853, "value2":8279527785132450943, "value3":true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.741108, "value1":0.31651275509392585, "value2":1547895388613496809, "value3":true, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866620, "value1":0.6883437972608208, "value2":8802986125706842330, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718942, "value1":0.8847629636653906, "value2":800711854768452001, "value3":true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375188, "value1":0.9532398566953418, "value2":2394341312296044798, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624088, "value1":0.6934451210618476, "value2":8474880456434193538, "value3":true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680602, "value1":0.2209467711169388, "value2":8621094656072865370, "value3":false, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.522533, "value1":0.5439997834907557, "value2":8198708215843245045, "value3":false, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439975, "value1":0.8101860664171935, "value2":8929518033480490042, "value3":true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602144, "value1":0.17084337908018365, "value2":6000683736434217499, "value3":true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594415, "value1":0.3959186882607224, "value2":1454959879954934650, "value3":false, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917433, "value1":0.849867816493911, "value2":8615713829914363729, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439894, "value1":0.7578484758847723, "value2":3887712481108512291, "value3":true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.965697, "value1":0.8251003697840577, "value2":836422853680285494, "value3":true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259251, "value1":0.4596388195214864, "value2":1597474370288095053, "value3":true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512513, "value1":0.783528381108141, "value2":8045517537006666296, "value3":true, "__name__":"metric_00000314", "key_4":"i","key_6":"g","key_3":"c", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.452258, "value1":0.4477592871421156, "value2":1459300159451076858, "value3":true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735408, "value1":0.5067192602284456, "value2":7587131980242978873, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.108601, "value1":0.3076343047473968, "value2":5066182910326911353, "value3":true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419278, "value1":0.12962316194187043, "value2":8864033357034639760, "value3":true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336677, "value1":0.24448588819210298, "value2":4567827434979480913, "value3":true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131541, "value1":0.02538083388681256, "value2":6449819714583284688, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235312, "value1":0.7537710944451972, "value2":3840033782203811299, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713440, "value1":0.6660557019192793, "value2":6317879942694845725, "value3":true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250100, "value1":0.2226451425042414, "value2":6756354166360377412, "value3":false, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.545039, "value1":0.7203310314493744, "value2":6637075765032095898, "value3":false, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846538, "value1":0.5133638544522705, "value2":6146673076285104795, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.099134, "value1":0.26713136173061386, "value2":5047631075889612988, "value3":false, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.999599, "value1":0.19120338269199497, "value2":3009559225203387009, "value3":false, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456369, "value1":0.6496125050532283, "value2":8358687825156497010, "value3":false, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.882684, "value1":0.4767076839685737, "value2":2058131066392704343, "value3":true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326891, "value1":0.0072166654150041835, "value2":3877874561619455924, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.753206, "value1":0.9891987223468547, "value2":2500770621881849387, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.505150, "value1":0.33485751726586077, "value2":3075226382158740800, "value3":true, "__name__":"metric_00000334", "key_6":"k","key_3":"h", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.337847, "value1":0.7587725777413157, "value2":4030061892629350122, "value3":true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.061555, "value1":0.5715183894256888, "value2":1380477982282562120, "value3":false, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102225, "value1":0.18128048387136908, "value2":1107305884228635610, "value3":false, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.663571, "value1":0.2082433172088735, "value2":2331488443653185612, "value3":false, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400352, "value1":0.36344640764280667, "value2":4156784209596494480, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567282, "value1":0.6853752986060114, "value2":1738184775840508143, "value3":true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580625, "value1":0.21556187277712435, "value2":1716362597563830669, "value3":true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.068750, "value1":0.5617303119904301, "value2":215978150703712746, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948465, "value1":0.4015798445122638, "value2":4918723285934260509, "value3":true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818268, "value1":0.4923649301244579, "value2":2640505566674328479, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344093, "value1":0.24900402040054898, "value2":3729697701643357874, "value3":true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096571, "value1":0.84402994415599, "value2":5393515911153006286, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880117, "value1":0.7901487854013907, "value2":2074107282217049907, "value3":false, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923332, "value1":0.6838866899470619, "value2":1947909482655854334, "value3":false, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.432183, "value1":0.35953677139878504, "value2":3191972662401496477, "value3":false, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251030, "value1":0.9314169082560206, "value2":2792551904040732916, "value3":true, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.315849, "value1":0.9521659578315848, "value2":278261838735054052, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.790453, "value1":0.9152891069156645, "value2":1668356628128321724, "value3":true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.260477, "value1":0.2126605025888137, "value2":6877799862560969541, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.213669, "value1":0.023418304769384207, "value2":4056268380917249508, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.647354, "value1":0.1608657659472805, "value2":7997008030753632127, "value3":false, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364473, "value1":0.2924502770902306, "value2":2291771922128424424, "value3":false, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.399561, "value1":0.6211538305973102, "value2":6848308885300872697, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135252, "value1":0.6565821688014496, "value2":4331814359581194434, "value3":true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365542, "value1":0.9118204132722055, "value2":2417655238291803790, "value3":true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758482, "value1":0.17112508145844177, "value2":1991118257939966294, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.032543, "value1":0.08125554957598005, "value2":6803892048654362563, "value3":false, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.520072, "value1":0.4275632442269591, "value2":2250122493336577938, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.900940, "value1":0.14035051591842881, "value2":3790466116523762354, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794030, "value1":0.6000173402567552, "value2":1696544400981072969, "value3":true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.760746, "value1":0.10139199442682838, "value2":2726344565529290448, "value3":false, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687331, "value1":0.770768299751441, "value2":8765361349894188719, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035711, "value1":0.48562696854381604, "value2":95585559412343711, "value3":true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739784, "value1":0.5825982451536238, "value2":2923240563522303723, "value3":true, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454300, "value1":0.4006888000990195, "value2":2468975659398879205, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884626, "value1":0.8026957862944495, "value2":3605855909002945057, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395577, "value1":0.28352828399222396, "value2":7602348181385211161, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250646, "value1":0.2316228525268274, "value2":29961024893090406, "value3":true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421024, "value1":0.7990468794285002, "value2":2264750479744979673, "value3":false, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380413, "value1":0.677589393211687, "value2":1235396941989449609, "value3":true, "__name__":"metric_00000383", "key_4":"f","key_1":"j", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680912, "value1":0.25625503996995874, "value2":2396418552817135385, "value3":false, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624528, "value1":0.5836651399092099, "value2":8089438271359233374, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.578517, "value1":0.7858613429590018, "value2":3691936139324370664, "value3":false, "__name__":"metric_00000385", "key_7":"h","key_0":"b","key_2":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.824035, "value1":0.3471915666356168, "value2":7523600740099466883, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069777, "value1":0.6471590850539041, "value2":545376892289254611, "value3":false, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390523, "value1":0.8681181997676135, "value2":5603080382596898622, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.590994, "value1":0.6390330221088167, "value2":6898933669096943734, "value3":true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222958, "value1":0.7361676398339201, "value2":7798894502817396812, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.313614, "value1":0.9956617360294369, "value2":8114343594051219927, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.471497, "value1":0.2028990027253043, "value2":8672046519486478878, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299387, "value1":0.90821231206369, "value2":7139029616253188226, "value3":true, "__name__":"metric_00000373", "key_7":"c","key_9":"g","key_2":"j", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822834, "value1":0.7462012785016429, "value2":2497140784779218477, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.958689, "value1":0.6493610280581492, "value2":7375518635495424751, "value3":true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755680, "value1":0.3173769236575059, "value2":1038885732982663879, "value3":true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.906259, "value1":0.7803156579997583, "value2":1671762930600330124, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320586, "value1":0.047928946477745184, "value2":2364817790122909266, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.237873, "value1":0.5919698389786022, "value2":2456361237893496498, "value3":false, "__name__":"metric_00000391", "key_9":"c","key_2":"j","key_4":"d", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.183324, "value1":0.25157377599570596, "value2":3302857039571325572, "value3":true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427134, "value1":0.9570675751476113, "value2":241686111761765133, "value3":false, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.353599, "value1":0.19102611162010216, "value2":7635824178108933655, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933576, "value1":0.5497618436271264, "value2":178483151772454986, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.888923, "value1":0.6996148110998511, "value2":7261028738310365771, "value3":true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414681, "value1":0.45596913295291375, "value2":8486467978432271730, "value3":false, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354962, "value1":0.6882059355313528, "value2":4219371503140919650, "value3":true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149786, "value1":0.048507930319514725, "value2":3361043402215780354, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320396, "value1":0.48978604231367256, "value2":8391350623429868941, "value3":false, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.189903, "value1":0.49045511147932325, "value2":4568889448781542889, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.531173, "value1":0.015565420626107091, "value2":1532700275127700107, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.911622, "value1":0.9315116645649003, "value2":845964128004551720, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447558, "value1":0.6852687200103666, "value2":1997580526601582597, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389703, "value1":0.7679907754096043, "value2":8650394236218808766, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221750, "value1":0.8409665471906171, "value2":8068148192803213421, "value3":false, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.679200, "value1":0.5889377996015627, "value2":6603283664639278646, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.658212, "value1":0.20335938525562192, "value2":6814120282733950138, "value3":false, "__name__":"metric_00000411", "key_8":"c","key_4":"d","key_7":"g", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917850, "value1":0.07209516605018491, "value2":3465054040297242627, "value3":true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.463048, "value1":0.8121431937413348, "value2":7711657955494481167, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280683, "value1":0.9792224347770722, "value2":6637142200642308638, "value3":true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938341, "value1":0.6616785812007566, "value2":4430772053877820, "value3":true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778057, "value1":0.21301644528770988, "value2":5550954213897210258, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.435560, "value1":0.25143911919093503, "value2":8602791844656832235, "value3":false, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980296, "value1":0.4483217784998845, "value2":8061950324304064559, "value3":false, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.845189, "value1":0.23223688199088727, "value2":2019205569808444797, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030298, "value1":0.009150371898544056, "value2":2179401507551835981, "value3":true, "__name__":"metric_00000415", "key_9":"e","key_4":"j","key_7":"k", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439657, "value1":0.5569344336018628, "value2":3896413214679997959, "value3":false, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037278, "value1":0.0752646633945245, "value2":2986036073412577705, "value3":false, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005641, "value1":0.470809080359862, "value2":7672259019023724818, "value3":true, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169543, "value1":0.6908284999086921, "value2":6236799599986001574, "value3":false, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.168435, "value1":0.1656190822947285, "value2":2938449310867540367, "value3":true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901700, "value1":0.6243405512256929, "value2":2234990933462726180, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282060, "value1":0.14169460807404005, "value2":820280722029319872, "value3":true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868372, "value1":0.8809198418653182, "value2":7786878634034667094, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700989, "value1":0.028109982788470232, "value2":1201575202891520734, "value3":false, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.304734, "value1":0.8385104418360362, "value2":6703304420616815596, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901530, "value1":0.8011320750502264, "value2":7539936763030777880, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862233, "value1":0.47963788532278256, "value2":3960941336665870754, "value3":false, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.511307, "value1":0.1832580868434161, "value2":7188980001326450399, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700199, "value1":0.6958679254162045, "value2":2878046054907323632, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299563, "value1":0.31054488059495927, "value2":8075467755079023785, "value3":true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383092, "value1":0.290554447303189, "value2":6694033035935161393, "value3":false, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.696053, "value1":0.16066281765668294, "value2":3961201674767505870, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392128, "value1":0.5736001777474582, "value2":168331305812241403, "value3":false, "__name__":"metric_00000435", "key_9":"h","key_3":"j","key_5":"a", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752731, "value1":0.4825725339245043, "value2":7800665573914476062, "value3":false, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293547, "value1":0.12126767569308682, "value2":1612512565132344690, "value3":false, "__name__":"metric_00000437", "key_8":"a","key_0":"b","key_3":"d", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874204, "value1":0.7014202076829791, "value2":476647849984184674, "value3":false, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.132714, "value1":0.5122323816164429, "value2":646780496771729886, "value3":true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358264, "value1":0.8238092496263985, "value2":9061776775816645023, "value3":false, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358876, "value1":0.15679289393208987, "value2":8163846432367416312, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921374, "value1":0.8339787140810333, "value2":3649804286907707134, "value3":true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395687, "value1":0.875997280853855, "value2":9212367459114589124, "value3":false, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.459642, "value1":0.7528040980492385, "value2":4150547104368392230, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782674, "value1":0.7121212215364499, "value2":6531310437181949577, "value3":true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.229939, "value1":0.32814366772056375, "value2":6374161097327904245, "value3":true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594347, "value1":0.6163078141302316, "value2":1538685760517154018, "value3":true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.009874, "value1":0.36102842691855586, "value2":1131193318052688252, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.606094, "value1":0.7074086026345966, "value2":7018588286889757726, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.777708, "value1":0.9795566345359867, "value2":7358687237436509918, "value3":true, "__name__":"metric_00000445", "key_4":"a","key_2":"i","key_3":"k", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.963990, "value1":0.9872552891680104, "value2":6171290753066239119, "value3":true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772158, "value1":0.4830977012506835, "value2":6211566808050254036, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.952768, "value1":0.43798623227092814, "value2":3513514828808633618, "value3":true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566018, "value1":0.7713759201698291, "value2":5320413786417127410, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.563469, "value1":0.6599121888373632, "value2":1404388254364767252, "value3":true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786109, "value1":0.9075953036601083, "value2":8494075902382221765, "value3":true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812207, "value1":0.5042296550006388, "value2":2362200937900291190, "value3":true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093099, "value1":0.3386496883604208, "value2":7071312823587957868, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.381119, "value1":0.031714519630475106, "value2":2849328345417435639, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.815827, "value1":0.14737457839891155, "value2":199255383994378266, "value3":false, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050041, "value1":0.5503190199883978, "value2":7356958752349024534, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.472083, "value1":0.12465887530301682, "value2":6392360821928490144, "value3":false, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713056, "value1":0.12067504962982098, "value2":4673070422231359809, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.034034, "value1":0.25231429703394476, "value2":5489447259456982836, "value3":true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502818, "value1":0.4156967719918407, "value2":4612443309724781532, "value3":false, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427071, "value1":0.24921718574451088, "value2":7646229328406586652, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643652, "value1":0.07902787804812644, "value2":34188484750195348, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.488097, "value1":0.7204444807131233, "value2":4412925367441450672, "value3":true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.180195, "value1":0.3056776312063749, "value2":7687653658691030932, "value3":false, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.873382, "value1":0.6827451433784463, "value2":1751876103177277887, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529999, "value1":0.39651784939201806, "value2":1747188854631655971, "value3":true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971674, "value1":0.8695844176281972, "value2":2878591399245613156, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702173, "value1":0.9786793419693177, "value2":4388723377141224465, "value3":true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844886, "value1":0.690355517305138, "value2":6283981589400857877, "value3":false, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298362, "value1":0.6666814629745373, "value2":6187669811817639042, "value3":true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266804, "value1":0.7091063369563013, "value2":4664946045775147969, "value3":false, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400948, "value1":0.8630878707275965, "value2":7287257182404940937, "value3":true, "__name__":"metric_00000477", "key_3":"f","key_6":"k","key_0":"b", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135133, "value1":0.6122563921627874, "value2":2139209888725866722, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.370672, "value1":0.01650778323939298, "value2":3489547069770540399, "value3":false, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.279256, "value1":0.7619200195662909, "value2":5915401631721082002, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170496, "value1":0.01005373661086586, "value2":1482576688135431000, "value3":false, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499284, "value1":0.36435815211219275, "value2":1326048436273736829, "value3":true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185099, "value1":0.9391170841095738, "value2":8924546898546218998, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772826, "value1":0.7574432732083424, "value2":4488845007322620734, "value3":false, "__name__":"metric_00000484", "key_6":"h","key_5":"f", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169744, "value1":0.5739689347042248, "value2":741420931729893242, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.998044, "value1":0.8965817632976586, "value2":6239062200473429860, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594662, "value1":0.6301979977894112, "value2":2054875786295626784, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755433, "value1":0.06261106040596412, "value2":5455645386351916850, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.682584, "value1":0.03167274357401835, "value2":310883965859994658, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072455, "value1":0.9148162667555992, "value2":382836579698407350, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.725171, "value1":0.3647728943926306, "value2":2421892347307790025, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798022, "value1":0.22510760744817873, "value2":2442091377551743699, "value3":true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688933, "value1":0.07828818881955478, "value2":6735636563745227652, "value3":false, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.649962, "value1":0.9679958182791478, "value2":2607996809419409341, "value3":false, "__name__":"metric_00000493", "key_8":"d","key_9":"h","key_1":"d", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.382199, "value1":0.14064207963814382, "value2":1623159006724541520, "value3":true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293647, "value1":0.919909760697381, "value2":58019425907871235, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715683, "value1":0.40584027014114893, "value2":7884356555342436928, "value3":false, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.496280, "value1":0.6458580424511011, "value2":1385234809290785609, "value3":false, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037411, "value1":0.5356364996267606, "value2":4485181030850732806, "value3":false, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.092167, "value1":0.08471879259284998, "value2":1047090205307301338, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203791, "value1":0.9626197309889007, "value2":458130535579789756, "value3":false, "__name__":"metric_00000501", "key_5":"b","key_0":"d","key_2":"i", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395018, "value1":0.7684680824595476, "value2":787781374740198173, "value3":false, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408536, "value1":0.147741569501137, "value2":3967017244497727033, "value3":false, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693578, "value1":0.011896444619318294, "value2":4942230313540034093, "value3":false, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780418, "value1":0.8750655230631833, "value2":8170450018971591774, "value3":true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812004, "value1":0.011539724469496575, "value2":1314191368195668298, "value3":true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.090381, "value1":0.870578773268634, "value2":5701169500016143920, "value3":false, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512721, "value1":0.9238819426049193, "value2":5759090244464429103, "value3":false, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938235, "value1":0.5991043580028484, "value2":7404773139120927140, "value3":false, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.751490, "value1":0.1795090853269717, "value2":6360470713237738478, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267071, "value1":0.625707646354563, "value2":6040902311152523365, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.002110, "value1":0.08011219998212876, "value2":5850595301611238867, "value3":true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257610, "value1":0.8187562858163207, "value2":8061987864802598079, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.446622, "value1":0.7315172185892279, "value2":2428635533374338014, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730403, "value1":0.7209673234694193, "value2":8416924883569369777, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.814732, "value1":0.8815237283497432, "value2":1852270652578106849, "value3":true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.318594, "value1":0.610086630044871, "value2":4524314099691048466, "value3":true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.852591, "value1":0.9383715352044596, "value2":453270928675336465, "value3":false, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989585, "value1":0.8337913033606761, "value2":6440744488245743508, "value3":false, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666424, "value1":0.9182270518817748, "value2":6618322333494361668, "value3":true, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.010788, "value1":0.6687390259133706, "value2":1979105606239962267, "value3":false, "__name__":"metric_00000522", "key_9":"a","key_0":"c","key_6":"j", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752852, "value1":0.4586423067764851, "value2":3289626005749005579, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005307, "value1":0.4780784157090063, "value2":2142574953538675780, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752795, "value1":0.7908677214518265, "value2":5326267086837385392, "value3":false, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202377, "value1":0.001998326079086547, "value2":2359915256333417411, "value3":true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216786, "value1":0.03377151611828389, "value2":5536807322346137933, "value3":true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243069, "value1":0.48298272047333335, "value2":6984955197084383580, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199895, "value1":0.6668507451944505, "value2":1160751712622009388, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605559, "value1":0.014501073348709297, "value2":2976544013604671560, "value3":true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046312, "value1":0.45283346882557124, "value2":1106313893608419094, "value3":true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841418, "value1":0.9439193659449161, "value2":1585819181629654484, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.816329, "value1":0.8314432076191595, "value2":671388396305183300, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.672039, "value1":0.19428152515853786, "value2":4678503251263728412, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.652936, "value1":0.05999646842980615, "value2":6549432288618958292, "value3":true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212442, "value1":0.6061066332871502, "value2":6044859751282824016, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200351, "value1":0.9858548441265623, "value2":6095603519503403992, "value3":true, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047144, "value1":0.8823748802173365, "value2":2346527148275584633, "value3":false, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272669, "value1":0.49903571079222264, "value2":7606961266627055929, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244016, "value1":0.3505008339279857, "value2":9165454668662797150, "value3":false, "__name__":"metric_00000538", "key_6":"e","key_0":"a","key_4":"f", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595436, "value1":0.1769134432972796, "value2":4183298646463696655, "value3":true, "__name__":"metric_00000540", "key_7":"e","key_6":"f", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.832887, "value1":0.05781334979440461, "value2":2976605700126921242, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.417486, "value1":0.03855593830020744, "value2":6499287743834404350, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850592, "value1":0.8819483685824858, "value2":3365942495366951256, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904339, "value1":0.4096492831256871, "value2":7870603688751891889, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849740, "value1":0.5985549217817763, "value2":5972692186135936344, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080962, "value1":0.6508975263449264, "value2":620195756611450814, "value3":true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.172604, "value1":0.36758717062599394, "value2":3093546059237948200, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.611964, "value1":0.09212647541898711, "value2":600064894791793802, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.136731, "value1":0.3518781010346145, "value2":4172668838781409561, "value3":false, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221150, "value1":0.7696791785774878, "value2":4882972115027784491, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118097, "value1":0.3822037091545315, "value2":5547505099823892976, "value3":false, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.209184, "value1":0.3119528694472212, "value2":8126435007719933970, "value3":true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.708227, "value1":0.8414289260599137, "value2":1887259462865725736, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127263, "value1":0.417919506626244, "value2":3560935891426387222, "value3":false, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.500871, "value1":0.16110435350355187, "value2":2428171488019731436, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.848583, "value1":0.4961159478803282, "value2":5456304085389531044, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.468311, "value1":0.9289954682212651, "value2":4285742227173179225, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.160825, "value1":0.59740490878922, "value2":9140238758107732393, "value3":true, "__name__":"metric_00000559", "key_4":"d","key_1":"f","key_2":"j", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423818, "value1":0.2325925821959117, "value2":7321787707684172397, "value3":true, "__name__":"metric_00000558", "key_7":"k","key_5":"i","key_6":"g", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.689012, "value1":0.5753370946686959, "value2":5125074341961139213, "value3":true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941993, "value1":0.009508850201920292, "value2":2477284877979206282, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344874, "value1":0.012483207365877868, "value2":4898429158465049577, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625728, "value1":0.9281594076368007, "value2":771428325581269474, "value3":false, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701217, "value1":0.8504319614540372, "value2":3030161014659152421, "value3":true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.288648, "value1":0.7225335235603462, "value2":4184194222447924377, "value3":false, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735765, "value1":0.7338027307218647, "value2":8053319255149398119, "value3":false, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853312, "value1":0.31226366748735784, "value2":48160047989763460, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289371, "value1":0.5473984505709404, "value2":2530747852470429293, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825683, "value1":0.2907108545003152, "value2":3061253498575467483, "value3":false, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532231, "value1":0.5859058541023265, "value2":4975638816250812874, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512440, "value1":0.511793469522306, "value2":5419688896766823090, "value3":true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709116, "value1":0.0431590704274129, "value2":565903028384253279, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709410, "value1":0.45910825737421945, "value2":425558835807211279, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.970542, "value1":0.1219905172153899, "value2":24331626097709718, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.043601, "value1":0.49284843085319935, "value2":7672646678928068810, "value3":true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739151, "value1":0.4246916831482226, "value2":4312260845153215448, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489280, "value1":0.8582555742505251, "value2":3110601136234274420, "value3":false, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935791, "value1":0.4664631517475295, "value2":4539773222290649379, "value3":false, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.654384, "value1":0.10389342147634097, "value2":5731036292461383708, "value3":false, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193783, "value1":0.5459140576966053, "value2":6006070322533917783, "value3":false, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409382, "value1":0.472091611755319, "value2":4277330658526282078, "value3":true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423822, "value1":0.4751601960954781, "value2":2652715507916534737, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.660521, "value1":0.38225985421870495, "value2":9011573276516869930, "value3":true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975891, "value1":0.5238616298909866, "value2":472916590695431743, "value3":false, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843227, "value1":0.6679925899402425, "value2":906046342124593465, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829312, "value1":0.16298135421846682, "value2":1838361763539535864, "value3":false, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480674, "value1":0.259388407981778, "value2":6440770726430652573, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690879, "value1":0.0735354737983211, "value2":9181104589648553044, "value3":true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126782, "value1":0.4486397942700849, "value2":7462094819783188022, "value3":false, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948901, "value1":0.9959329498625173, "value2":5479458805805092746, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127088, "value1":0.9575385572151783, "value2":2275563983258531267, "value3":true, "__name__":"metric_00000592", "key_7":"b","key_0":"k","key_1":"e", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259685, "value1":0.3372857504409974, "value2":6098187454917492671, "value3":true, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336884, "value1":0.24553452230487985, "value2":8241989909296706238, "value3":false, "__name__":"metric_00000594", "key_3":"a","key_9":"e","key_1":"g", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169076, "value1":0.17392408086361133, "value2":7622066852267723117, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977238, "value1":0.5433155948072138, "value2":6898148363520327391, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.278455, "value1":0.6005033834895888, "value2":8001662694808432224, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993722, "value1":0.9597867721155696, "value2":1299805439881399559, "value3":false, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.530999, "value1":0.2209888826520892, "value2":4066778144109849949, "value3":true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782378, "value1":0.7523792160016672, "value2":5556510651542494721, "value3":false, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.219804, "value1":0.1717195689538366, "value2":7048403972015561437, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.946856, "value1":0.5985084131744225, "value2":4885828087902675075, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759489, "value1":0.27172565847552677, "value2":7664668067438271399, "value3":false, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687774, "value1":0.42250850460802286, "value2":7088707511295378411, "value3":true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560049, "value1":0.2616853019800064, "value2":2950184265013984605, "value3":false, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097585, "value1":0.2684028422029654, "value2":675122184817942746, "value3":true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205813, "value1":0.44731824636014533, "value2":1428863070750239410, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.543640, "value1":0.544977219392789, "value2":2157919960028175094, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566856, "value1":0.8008181036581771, "value2":7402387212023409312, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.765221, "value1":0.8596387394697663, "value2":1232636922510789742, "value3":false, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.386997, "value1":0.10171781350903462, "value2":2933191729226405023, "value3":true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.674282, "value1":0.05504930364585503, "value2":724010655267136473, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820827, "value1":0.1512145830665697, "value2":6612187124412654634, "value3":false, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537712, "value1":0.6818284933003707, "value2":833546983771380703, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.020134, "value1":0.7293182995525076, "value2":2126763229656176078, "value3":false, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.919742, "value1":0.47707314485124014, "value2":8035884626808128837, "value3":false, "__name__":"metric_00000616", "key_5":"d","key_7":"j","key_1":"k", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941486, "value1":0.13322259126479358, "value2":2182305210138540441, "value3":false, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.089091, "value1":0.6999475934755898, "value2":5942262186593070970, "value3":false, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.922220, "value1":0.01137462679977003, "value2":4337636877649564119, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846010, "value1":0.10326336153396976, "value2":8556623018979218867, "value3":true, "__name__":"metric_00000620", "key_9":"b","key_2":"j","key_7":"f", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361361, "value1":0.9603555432714893, "value2":7276258796427191775, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303375, "value1":0.9198009606494528, "value2":3654960055261327174, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354372, "value1":0.9154736221447883, "value2":6754725607301294498, "value3":false, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836769, "value1":0.17500751807732254, "value2":5225830062891206724, "value3":false, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.514481, "value1":0.712612132062798, "value2":5405795909104498480, "value3":false, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.857662, "value1":0.4771648896771537, "value2":9197955420056459031, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218504, "value1":0.43388279897052595, "value2":4876293374213376800, "value3":true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854318, "value1":0.29673697161398044, "value2":2557767286012732771, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.621120, "value1":0.07462842813102197, "value2":5246869953360521284, "value3":false, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482880, "value1":0.4261787117422624, "value2":7122235599178548659, "value3":false, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880280, "value1":0.38443671400954443, "value2":5167341510277945257, "value3":false, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849192, "value1":0.5161558417921461, "value2":4578699148249247151, "value3":false, "__name__":"metric_00000631", "key_8":"g","key_9":"d","key_1":"i", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826648, "value1":0.873997512950574, "value2":7985090519609883665, "value3":true, "__name__":"metric_00000632", "key_3":"i","key_7":"j","key_2":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199619, "value1":0.8346974218021976, "value2":8292656933621980920, "value3":true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.959402, "value1":0.6444072126847691, "value2":4320840087483118841, "value3":true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117065, "value1":0.9272062952128088, "value2":4910297155014939874, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.551716, "value1":0.13475804603313432, "value2":8222696695222433380, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643768, "value1":0.8311693472811903, "value2":7965400634034447655, "value3":false, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.484767, "value1":0.5064114690168263, "value2":5755735513825830154, "value3":true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377345, "value1":0.1504036301883094, "value2":2988203951245132390, "value3":true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991676, "value1":0.972772556086247, "value2":1470890803841584803, "value3":true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361353, "value1":0.6540839592057779, "value2":985314732060254627, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585584, "value1":0.7709261757498476, "value2":2565995410162492585, "value3":true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.704212, "value1":0.3118041755703027, "value2":6201623885191421475, "value3":true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.518173, "value1":0.9814365163985729, "value2":8388580742116222397, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.187366, "value1":0.6732011290892043, "value2":5408814647988876882, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.296035, "value1":0.5028422803953023, "value2":7236554080452628944, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.768145, "value1":0.8394583660519598, "value2":5325173567069752762, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549064, "value1":0.12911593374108088, "value2":8305911548302819241, "value3":true, "__name__":"metric_00000648", "key_9":"h","key_1":"e","key_2":"e", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758468, "value1":0.0009372001034959489, "value2":5525854549400493996, "value3":false, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.525822, "value1":0.9778487221852099, "value2":2741333418274183362, "value3":true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396148, "value1":0.9074357631670523, "value2":8809499083604789974, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414129, "value1":0.8132493023109502, "value2":4942425104780689529, "value3":false, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.398880, "value1":0.39814811004109957, "value2":6997600198610651439, "value3":false, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.667851, "value1":0.7812827159518755, "value2":111949400321343617, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.065047, "value1":0.5608722991855313, "value2":128957686943303349, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.494481, "value1":0.17816439204883128, "value2":8141342025149658982, "value3":true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.515335, "value1":0.9138017166447161, "value2":4317898835838327168, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713454, "value1":0.43899440111826654, "value2":143487884063527116, "value3":true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185777, "value1":0.5460630190657308, "value2":3180840112779032107, "value3":false, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450264, "value1":0.9313181115462055, "value2":8328870888335802008, "value3":true, "__name__":"metric_00000661", "key_5":"k","key_6":"e","key_1":"d", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567912, "value1":0.6046426149595721, "value2":3206194822374359841, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.896555, "value1":0.6026393505223315, "value2":4823045987798108744, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710017, "value1":0.5880756405721221, "value2":4714155846612246340, "value3":true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265002, "value1":0.824988737224255, "value2":8146626468221994382, "value3":true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029140, "value1":0.9928569418325714, "value2":1115616498765210900, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.994788, "value1":0.6191499887981317, "value2":1491651902318479820, "value3":true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310266, "value1":0.6540537390689438, "value2":8869916203173244502, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786610, "value1":0.5759332726822003, "value2":7232889220252179533, "value3":true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036697, "value1":0.31309276046690226, "value2":4197456548662367361, "value3":true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.684262, "value1":0.39030765212524027, "value2":8590977643860948235, "value3":true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.295632, "value1":0.4747190839423866, "value2":2819114746608115923, "value3":false, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.613481, "value1":0.02792670630263574, "value2":4669805487355644126, "value3":true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.462933, "value1":0.7475159252861773, "value2":2383279322222812533, "value3":true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179395, "value1":0.19102052131729996, "value2":1856517930791738382, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991666, "value1":0.3546374487300389, "value2":6354333338863606534, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.568174, "value1":0.4332276746738033, "value2":8645759887904070759, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.321899, "value1":0.2992200951220634, "value2":3116962887997170979, "value3":true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390308, "value1":0.02747272568305014, "value2":926243844801181597, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.805520, "value1":0.5096841522295752, "value2":7414378226567615667, "value3":true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230674, "value1":0.03960399937089453, "value2":1710851993634315848, "value3":true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.425150, "value1":0.4167707239792382, "value2":5244023674899540768, "value3":true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879335, "value1":0.4867774224489378, "value2":553896137255768087, "value3":false, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.021415180040758975, "value2":6576886716131511876, "value3":true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.875807, "value1":0.22236040526017317, "value2":8247944924957719281, "value3":true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.596786, "value1":0.5382707900600514, "value2":6504957777564642511, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853544, "value1":0.6591705273657046, "value2":6314593273328284879, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.792273, "value1":0.8726534877662263, "value2":8304088035724056780, "value3":true, "__name__":"metric_00000689", "key_8":"a","key_3":"i","key_5":"h", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050549, "value1":0.962790936441039, "value2":4045656520780865827, "value3":false, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493970, "value1":0.39340114596435655, "value2":5294670971014916935, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251929, "value1":0.5462996764305353, "value2":5897296033220091874, "value3":true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115726, "value1":0.8575474248088841, "value2":2146148594913460918, "value3":true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.787443, "value1":0.8185914401069716, "value2":6891346883169646879, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.982163, "value1":0.8404209142239473, "value2":2046890513913645569, "value3":false, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529940, "value1":0.15216580562912718, "value2":235763707139823565, "value3":true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036141, "value1":0.21797894391861414, "value2":2037043718914992589, "value3":false, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489425, "value1":0.578085671307559, "value2":1968373430108550733, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377536, "value1":0.5230650685409198, "value2":1948377520185967051, "value3":true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.644803, "value1":0.09118740593062584, "value2":3209698888069603256, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149640, "value1":0.2149828010685946, "value2":324803347557649497, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.788006, "value1":0.35151805877228876, "value2":5736730574020425869, "value3":true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528326, "value1":0.12892234054630594, "value2":40722644873633005, "value3":true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.517453, "value1":0.12096018419440699, "value2":8591818996293676301, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580879, "value1":0.18567989692889683, "value2":2424108579790648867, "value3":false, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.418474, "value1":0.10109548853357979, "value2":6805501278544557691, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192558, "value1":0.5389855440079544, "value2":2468356112053849635, "value3":true, "__name__":"metric_00000706", "key_6":"h","key_9":"d","key_5":"e", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700652, "value1":0.8426222872981881, "value2":5586434719045643994, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836161, "value1":0.6261463649204317, "value2":7998455739612119, "value3":false, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.410987, "value1":0.19322483253914569, "value2":7602154106800870019, "value3":true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920472, "value1":0.07895320792744571, "value2":3621399116338923589, "value3":false, "__name__":"metric_00000709", "key_8":"a","key_9":"g","key_1":"c", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058885, "value1":0.038506638786952846, "value2":8443519094492233492, "value3":false, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.357953, "value1":0.49873265104684444, "value2":6689514258543734175, "value3":true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444144, "value1":0.7257783571311162, "value2":2132381976912359864, "value3":true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.003034, "value1":0.21025100795379004, "value2":4437597220292182437, "value3":false, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794651, "value1":0.0987214108191527, "value2":4856891238525466286, "value3":true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.101167, "value1":0.5530900889416458, "value2":5869924593453820746, "value3":true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813524, "value1":0.7032391307910569, "value2":2962680547459287448, "value3":true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102641, "value1":0.05280577735508823, "value2":6197989994765662479, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.042807, "value1":0.46787236774210145, "value2":6987091087352317601, "value3":true, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023412, "value1":0.4117202092054757, "value2":5734736277249168119, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.582529, "value1":0.14733901739015626, "value2":2008347713878225513, "value3":false, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843555, "value1":0.45648522904759514, "value2":8748231721848476899, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.609920, "value1":0.5332106841490003, "value2":3789122902542055079, "value3":true, "__name__":"metric_00000722", "key_9":"d","key_0":"h","key_1":"b", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.004873, "value1":0.9250068088247592, "value2":4060601419917043509, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598410, "value1":0.20935422794575423, "value2":3458524447228408749, "value3":true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327673, "value1":0.6622534351078718, "value2":1975526610989314262, "value3":true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701834, "value1":0.018453031281240855, "value2":7208203361477507972, "value3":false, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.544888, "value1":0.022192838541852087, "value2":420807956823837942, "value3":true, "__name__":"metric_00000727", "key_8":"e","key_2":"i", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.333657, "value1":0.6116791297905804, "value2":6780782056382077493, "value3":true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.224370, "value1":0.58595743970275, "value2":2273745727787058306, "value3":false, "__name__":"metric_00000729", "key_5":"h","key_0":"f","key_1":"a", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874813, "value1":0.4160482798834957, "value2":5130896603060858656, "value3":false, "__name__":"metric_00000732", "key_8":"k","key_4":"i", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800557, "value1":0.6485257630686503, "value2":8645194213531066428, "value3":false, "__name__":"metric_00000730", "key_6":"b","key_2":"g","key_3":"e", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798578, "value1":0.3562881109226897, "value2":1936635714706401113, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222829, "value1":0.18800753190841593, "value2":3868467029118261102, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394596, "value1":0.9715849939407708, "value2":2892398559190644533, "value3":true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088141, "value1":0.19677712178571452, "value2":1290002108232651066, "value3":false, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046604, "value1":0.7961651460552367, "value2":6142675575156594148, "value3":false, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755275, "value1":0.26382736011465824, "value2":4790349544446362608, "value3":true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.300482, "value1":0.6458439650096045, "value2":5076464481756841749, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294901, "value1":0.3937755673758644, "value2":4837753175838197566, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717849, "value1":0.8465941273216573, "value2":4538488428913350745, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018027, "value1":0.7099513420503992, "value2":6589927070191288064, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686148, "value1":0.31341443389871104, "value2":4542163293900247497, "value3":true, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554562, "value1":0.6021518819649633, "value2":657935532838913237, "value3":false, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549050, "value1":0.9913051319754902, "value2":9133655827436281017, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.564021, "value1":0.3933144754852601, "value2":8691343872756358848, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.555563, "value1":0.12216755310514293, "value2":8181517965767093469, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991822, "value1":0.9046701031504584, "value2":7169764745600183537, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536610, "value1":0.6245723927615995, "value2":5410550698568344254, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.640481, "value1":0.7953791600529995, "value2":8896491255145975602, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.983650, "value1":0.39162313489970674, "value2":5865250699451794074, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.748358, "value1":0.3770029381067136, "value2":7671767291617145847, "value3":true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632308, "value1":0.9672641688276767, "value2":3043416827366033954, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829101, "value1":0.63693380465596, "value2":3688916592463553468, "value3":false, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.271187, "value1":0.7822523952945989, "value2":7538651950678502209, "value3":false, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757851, "value1":0.4993335289484615, "value2":8433879559771522559, "value3":false, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.022503, "value1":0.5685717542133112, "value2":5232283843486040657, "value3":false, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046586, "value1":0.6991953129630171, "value2":767301512539468704, "value3":true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170088, "value1":0.9099698273235788, "value2":6375281583140134087, "value3":true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.819909, "value1":0.9395198166293325, "value2":9208855013717569494, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078675, "value1":0.35105170816316494, "value2":7441300028994205, "value3":false, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977562, "value1":0.27747207540421476, "value2":2610474267165011454, "value3":false, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.577027, "value1":0.4084658901530552, "value2":7348713799224820014, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705040, "value1":0.13989635719875274, "value2":8770545555481284876, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.821460, "value1":0.5135499070770553, "value2":1707896495831766919, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369139, "value1":0.07309187404043328, "value2":5262455623289086241, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536485, "value1":0.3546193597483191, "value2":5532328840062432345, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632890, "value1":0.87913605477188, "value2":4206324763882989783, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.352893, "value1":0.22467668553540646, "value2":8020391351501404610, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058110, "value1":0.08476564402932071, "value2":2402316898103266567, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312920, "value1":0.14715981410041812, "value2":515215788552749041, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335941, "value1":0.5607746607254104, "value2":4029056354863603408, "value3":false, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493922, "value1":0.5228229815154212, "value2":5917395680452547526, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.721451, "value1":0.8180794492453682, "value2":1670905717947422538, "value3":false, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.347574, "value1":0.7935601761833552, "value2":6084751528698562987, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.722657, "value1":0.4075231573944958, "value2":7748408088453297294, "value3":false, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.451863, "value1":0.3657057400324225, "value2":7302526748126423806, "value3":false, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495468, "value1":0.10049524830706323, "value2":7515132691589300738, "value3":true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193570, "value1":0.5847911145927462, "value2":4656707981929867290, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866643, "value1":0.9003805975682342, "value2":3827225598428319501, "value3":false, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.535740, "value1":0.5365482035917958, "value2":3695801733453249426, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.197799, "value1":0.7228560209917014, "value2":5697296448849523746, "value3":false, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536756, "value1":0.2916665533141402, "value2":8616484406626615233, "value3":true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231668, "value1":0.5491980830875538, "value2":5401247188111922849, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836905, "value1":0.6668911346325062, "value2":5956397501314084543, "value3":true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.060078, "value1":0.41433534941101835, "value2":259739043603601632, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933921, "value1":0.15068665469322526, "value2":4615815953640952732, "value3":false, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.147080, "value1":0.17748824888193676, "value2":838395034976203435, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662077, "value1":0.7882764714348716, "value2":4457961488907810285, "value3":true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023154, "value1":0.24361462379316554, "value2":7839025824541823553, "value3":false, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826915, "value1":0.569504859021508, "value2":6251982817444532308, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585595, "value1":0.8340020457470065, "value2":2938024413765410558, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146190, "value1":0.9581727102338353, "value2":4413141265483838870, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.245440, "value1":0.6417183759653234, "value2":5364021531028647444, "value3":false, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493343, "value1":0.4938700840392069, "value2":3091997225543832896, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807379, "value1":0.4690524091058998, "value2":3061093742738985330, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.208752, "value1":0.8639618600039569, "value2":3187072739982271504, "value3":true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.011216, "value1":0.18367276511432976, "value2":2360627840986057827, "value3":true, "__name__":"metric_00000799", "key_8":"d","key_6":"i", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.440539, "value1":0.6977700817568234, "value2":3581694346818534680, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489990, "value1":0.309872070478555, "value2":7457465639523954616, "value3":false, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113130, "value1":0.6897957784827682, "value2":90019976280058795, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.604031, "value1":0.797243453747431, "value2":1234756804789352079, "value3":true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.881184, "value1":0.28155440991949554, "value2":4070028978929644169, "value3":true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.367426, "value1":0.016167903292426008, "value2":4863636678376461059, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.769132, "value1":0.967909876972054, "value2":9131005500727231455, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.905490, "value1":0.6194024275206055, "value2":210353074026913884, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.915138, "value1":0.6781979759382127, "value2":6372020977221093753, "value3":true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093709, "value1":0.6187727010105798, "value2":312866041452100946, "value3":true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758312, "value1":0.8273322264897901, "value2":7512565884920091651, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.284383, "value1":0.39739645167776044, "value2":8773985992873355471, "value3":true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216803, "value1":0.8653155039130678, "value2":8937229562905505386, "value3":true, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.770263, "value1":0.9293790766191896, "value2":4560937013728510181, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096191, "value1":0.22323546625261562, "value2":8010469458396680860, "value3":true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.984434, "value1":0.9454942818827864, "value2":660593493740706018, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222194, "value1":0.02420800916517378, "value2":5602553742039024765, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752900, "value1":0.3672193480709247, "value2":6716506478528347816, "value3":false, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.967703, "value1":0.2944363320939628, "value2":7783913193752729609, "value3":true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.600560, "value1":0.1440348243160143, "value2":9110640288768831559, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400353, "value1":0.2233756240085349, "value2":2151981935371948568, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.269597, "value1":0.9641297731318997, "value2":8169933506866910786, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.486730, "value1":0.1006988853266946, "value2":3116477845424070312, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953298, "value1":0.4756644907363518, "value2":2464865227545117596, "value3":false, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705992, "value1":0.8173093209310923, "value2":8219739418749111053, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.851844, "value1":0.28729444422491424, "value2":191979366247033806, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616037, "value1":0.08828307120467993, "value2":1006916952645293364, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.007265, "value1":0.43891779173017603, "value2":4642029911324023153, "value3":false, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408095, "value1":0.9633558663387003, "value2":2764041604557827140, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825923, "value1":0.12824733189500837, "value2":7952369571604255477, "value3":true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853643, "value1":0.8488628783599163, "value2":3110877858489702411, "value3":false, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686891, "value1":0.5538740126823626, "value2":7618257543247002890, "value3":true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408459, "value1":0.2427343835864084, "value2":1637171870696137413, "value3":false, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.635245, "value1":0.42768890444095087, "value2":4013428109133869754, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793205, "value1":0.23512439386454564, "value2":5799487535459689842, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920141, "value1":0.9415332414772868, "value2":1520706400052294853, "value3":true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.930133, "value1":0.47818008921632604, "value2":4567447002026438924, "value3":false, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.599550, "value1":0.9293362893760267, "value2":1195481394235809552, "value3":false, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903387, "value1":0.11210859774886604, "value2":3335652489364502188, "value3":true, "__name__":"metric_00000839", "key_4":"h","key_0":"f", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698033, "value1":0.3148016773642106, "value2":4939787017916634516, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813706, "value1":0.3325590668831616, "value2":3252287249574729793, "value3":false, "__name__":"metric_00000841", "key_2":"d","key_6":"b","key_1":"i", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.048837, "value1":0.053013772957975554, "value2":4819883401481045104, "value3":false, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310950, "value1":0.41839244392518726, "value2":8733225969530074129, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.062225, "value1":0.858129811136947, "value2":7961256500615532109, "value3":true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428880, "value1":0.2502872252911392, "value2":7064927828432353118, "value3":false, "__name__":"metric_00000842", "key_6":"c","key_0":"c","key_3":"b", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560002, "value1":0.11724155985130476, "value2":3115965127965032390, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657940, "value1":0.5473444163031204, "value2":8727302484888278411, "value3":true, "__name__":"metric_00000847", "key_9":"d","key_0":"k","key_5":"g", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554826, "value1":0.376684927630788, "value2":2585922874708871992, "value3":false, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079791, "value1":0.9982500603963644, "value2":7391513557121794219, "value3":true, "__name__":"metric_00000846", "key_8":"e","key_1":"h","key_4":"k", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.174380, "value1":0.13350980294424206, "value2":9115933928609206333, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690448, "value1":0.8022085172421284, "value2":5503579136981394270, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178959, "value1":0.5153762777175099, "value2":4750624720468516504, "value3":true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880368, "value1":0.3826318223193023, "value2":3994162284701586149, "value3":true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594211, "value1":0.0032667252772956594, "value2":211951125332337865, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080419, "value1":0.3371270895888467, "value2":970359359029436118, "value3":false, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354035, "value1":0.6830560620508377, "value2":351440888758257733, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686006, "value1":0.15832224781392848, "value2":7973904723234304705, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688086, "value1":0.14780349920952468, "value2":2027463870413046281, "value3":false, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.407959, "value1":0.04437078472668187, "value2":1566333717176214387, "value3":true, "__name__":"metric_00000856", "key_6":"d","key_3":"i","key_5":"a", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943264, "value1":0.6495072511389037, "value2":5853361075489349645, "value3":false, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400773, "value1":0.2676114612948212, "value2":6666522398737719346, "value3":true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228255, "value1":0.29124903807175784, "value2":128611338987653451, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702835, "value1":0.39306652609769904, "value2":4553551121428133233, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346580, "value1":0.9157209666515556, "value2":4177607716262676740, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000990, "value1":0.3155726084083431, "value2":7473596942971857359, "value3":true, "__name__":"metric_00000862", "key_8":"d","key_1":"b","key_4":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380807, "value1":0.5730835803736063, "value2":6788452783424900655, "value3":true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018359, "value1":0.11046900836774676, "value2":61241953976306769, "value3":true, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097897, "value1":0.6529098432421965, "value2":9123131154054179256, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118936, "value1":0.312336727346538, "value2":7294156215339952794, "value3":false, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.256520, "value1":0.817765585396917, "value2":4276314621602341409, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.239028, "value1":0.27988068073239764, "value2":8048493989378214976, "value3":false, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.359463, "value1":0.33204054027814056, "value2":4286372625461580954, "value3":false, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625120, "value1":0.8053374337323631, "value2":1700890355040268131, "value3":false, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.519453, "value1":0.2764225180730322, "value2":3111696426903897415, "value3":false, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.019685, "value1":0.5511793734194003, "value2":3991968620227079942, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.040695, "value1":0.6967938051735891, "value2":8919927137208366483, "value3":true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750443, "value1":0.9319592750217813, "value2":7402732360820495311, "value3":true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.497932, "value1":0.6545667018250457, "value2":7616613351817353333, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.773399, "value1":0.5580363664821956, "value2":314222158348617253, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.869323, "value1":0.7418088662526952, "value2":8657709877874943583, "value3":true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812016, "value1":0.4480192862830272, "value2":8470890763908250393, "value3":true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079326, "value1":0.3872756599905638, "value2":4349661970334556580, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.186686, "value1":0.8815811840060216, "value2":938888750792010987, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235117, "value1":0.8673030373216688, "value2":4951632884410168399, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335964, "value1":0.6073261439232968, "value2":8008517076279782152, "value3":false, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063573, "value1":0.12394184604832796, "value2":1362755555847887698, "value3":true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.685355, "value1":0.2361653248224142, "value2":2210259628223947558, "value3":true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029806, "value1":0.3517864573309168, "value2":6754043647766872898, "value3":true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.593026, "value1":0.38937941917051955, "value2":1945974862147158833, "value3":true, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595711, "value1":0.9836966964700219, "value2":4159340747167974849, "value3":true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.143026, "value1":0.5945202518873367, "value2":3646754986945085167, "value3":true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733493, "value1":0.18903944802082667, "value2":8837255893708962102, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.120646, "value1":0.4261710458329441, "value2":6946544380857137617, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.163665, "value1":0.24245216879890383, "value2":555667735507028882, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934782, "value1":0.31976487170355755, "value2":6301272802564976177, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975795, "value1":0.2007851651150272, "value2":866319078855783895, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630757, "value1":0.2788228743238552, "value2":2540783851122363232, "value3":false, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.661317, "value1":0.5218066875354199, "value2":8869741999431102494, "value3":true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980967, "value1":0.7624092409640837, "value2":628203903335846730, "value3":false, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380622, "value1":0.4837921884626896, "value2":3628530218991586661, "value3":false, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.248953, "value1":0.02097807162306582, "value2":4598542592315882026, "value3":true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.384743, "value1":0.7608954854738191, "value2":1886663813765371215, "value3":false, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.950140, "value1":0.2550158514673341, "value2":4065397223823067237, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.516999, "value1":0.6292445006161491, "value2":4125156478992786356, "value3":true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733841, "value1":0.37206344867268365, "value2":3082112298619357468, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086119, "value1":0.5738504787109479, "value2":5573707094695082413, "value3":true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800475, "value1":0.7767745931253813, "value2":6702226199646136395, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.175876, "value1":0.8346328055447434, "value2":1066680083349750545, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752359, "value1":0.7257723037237084, "value2":7568317247183343047, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.165816, "value1":0.19937439179339156, "value2":2068495750767911677, "value3":false, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.597980, "value1":0.6555890535602861, "value2":2883065227864053972, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.138569, "value1":0.22532513024599726, "value2":8297154458719010882, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.668715, "value1":0.11645984529741488, "value2":5555518408390232617, "value3":true, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063265, "value1":0.001988016117254865, "value2":8483308279719955566, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086644, "value1":0.4644931461203292, "value2":6366519140385299368, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.140571, "value1":0.5416650487590775, "value2":1665979086436164947, "value3":true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.045805, "value1":0.9888897436719353, "value2":7697021265197143502, "value3":true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272912, "value1":0.2088625700639475, "value2":8643238667192348189, "value3":true, "__name__":"metric_00000916", "key_9":"k","key_3":"e","key_7":"b", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289278, "value1":0.005139072821858179, "value2":1003021960883868455, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.242063, "value1":0.32725526486053264, "value2":2911166424297597473, "value3":false, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.393077, "value1":0.6314586962746932, "value2":2631691711252794732, "value3":true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.274319, "value1":0.9496413905130506, "value2":3462918152034782288, "value3":true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659954, "value1":0.9955163986622194, "value2":2213038727032863641, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657842, "value1":0.11877952023160425, "value2":2754577958762836530, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146197, "value1":0.5152391159350134, "value2":4396760573611697803, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759786, "value1":0.051119901574673096, "value2":965796598734594927, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.891935, "value1":0.6695409149263635, "value2":7204593893035944941, "value3":true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096828, "value1":0.7785299263381299, "value2":843269019498255824, "value3":true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822107, "value1":0.06887356745350987, "value2":3890683236062212546, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818726, "value1":0.3529788280342954, "value2":2023962428787621327, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.356164, "value1":0.8626419946204176, "value2":3137373073213134707, "value3":false, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855692, "value1":0.3727202998903748, "value2":1868074746512330224, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.899737, "value1":0.49932818019000214, "value2":5247723900283585273, "value3":false, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205758, "value1":0.752048623202744, "value2":405978535011434243, "value3":false, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.492517, "value1":0.7587591048194695, "value2":570855823822973980, "value3":true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025167, "value1":0.9352906688766658, "value2":409025014085718364, "value3":false, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524374, "value1":0.456131184246001, "value2":6492711389883342442, "value3":true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.173025, "value1":0.7216031002138218, "value2":1900883846963453532, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991241, "value1":0.9435537729789849, "value2":6744125432328938384, "value3":false, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126845, "value1":0.9483471232225058, "value2":2429204263257823531, "value3":true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079973, "value1":0.14998274238182085, "value2":1609358502486987778, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257883, "value1":0.09750371692825593, "value2":5050315729103715843, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.074081, "value1":0.21018141055409362, "value2":9169596995481234183, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218738, "value1":0.9047134659722221, "value2":9140209877004134552, "value3":false, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.745794, "value1":0.39056909591189853, "value2":4170147818303980952, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179295, "value1":0.9580414905588824, "value2":4615545745316982257, "value3":false, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442990, "value1":0.018901929544229163, "value2":2360761388936503714, "value3":true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.422188, "value1":0.5272513199907374, "value2":3200950480296318673, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192831, "value1":0.33577000839194515, "value2":8465993285574620045, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.227691, "value1":0.814891459750041, "value2":5020817184863163049, "value3":false, "__name__":"metric_00000947", "key_7":"f","key_1":"d","key_2":"h", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171624, "value1":0.32281185598307915, "value2":8148752929610663940, "value3":true, "__name__":"metric_00000949", "key_2":"c","key_7":"h","key_1":"b", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.490347, "value1":0.4777481785589013, "value2":2903978983753469512, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217872, "value1":0.6839140142887666, "value2":3130769003103218800, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.992440, "value1":0.1702388495618268, "value2":6704155277469548917, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.834388, "value1":0.855009067710764, "value2":8656769438285137265, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230809, "value1":0.6026285375258597, "value2":1622052925759175826, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344218, "value1":0.6418025631389815, "value2":8245500230948114788, "value3":true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086815, "value1":0.450124296857116, "value2":4866053118697224871, "value3":true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802304, "value1":0.9289196028774398, "value2":6637963067322995548, "value3":true, "__name__":"metric_00000955", "key_6":"j","key_7":"d","key_1":"e", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.840991, "value1":0.6695124623580792, "value2":5437276365728147032, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.833585, "value1":0.22730504686256575, "value2":1520963367111518952, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.744045, "value1":0.8193501328748979, "value2":279541774649644194, "value3":false, "__name__":"metric_00000959", "key_9":"g","key_0":"b","key_2":"h", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.736126, "value1":0.6177732267307673, "value2":738077807298530261, "value3":false, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131891, "value1":0.46856485152824584, "value2":1245296884531108950, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419961, "value1":0.9237029205265445, "value2":1223547652782219884, "value3":false, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512633, "value1":0.3170684617445516, "value2":3777467556351824325, "value3":false, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630355, "value1":0.021704073637768814, "value2":8674907845809369167, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.309670, "value1":0.16245539540033113, "value2":5206956291412580457, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.348146, "value1":0.7658591727792013, "value2":8113096525411729139, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346099, "value1":0.09833103805087655, "value2":3329067421410580481, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086337, "value1":0.8850305283927763, "value2":8626010840258936486, "value3":false, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479373, "value1":0.20404187483062786, "value2":2412557827504873850, "value3":false, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921982, "value1":0.7871473016496364, "value2":1874563226125050237, "value3":false, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415457, "value1":0.9305937676701617, "value2":4246181816757480088, "value3":false, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035701, "value1":0.11812212256088726, "value2":7168381109733048473, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.225156, "value1":0.9869870812720318, "value2":137813480312065053, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836380, "value1":0.860500054517146, "value2":6804060104865284962, "value3":false, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289062, "value1":0.385288303480441, "value2":3917001550422656910, "value3":false, "__name__":"metric_00000975", "key_7":"g","key_1":"k","key_2":"a", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885780, "value1":0.1315685249986314, "value2":2596575576892432777, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948363, "value1":0.36071231455630776, "value2":628986020889092104, "value3":true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369788, "value1":0.40617296244272144, "value2":1817522335798570734, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086188, "value1":0.6439285363835136, "value2":4867721675008841698, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.995659, "value1":0.2954211415738649, "value2":316663728175376955, "value3":true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.129393, "value1":0.050215829131005675, "value2":6231361900716566583, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400907, "value1":0.39112211803964236, "value2":5808809619414218326, "value3":true, "__name__":"metric_00000980", "key_7":"f","key_0":"d","key_4":"d", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717436, "value1":0.943850779238098, "value2":1447465700474277383, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217700, "value1":0.1707128166287673, "value2":4689212155367810810, "value3":true, "__name__":"metric_00000983", "key_9":"e","key_0":"b","key_7":"c", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.908215, "value1":0.13836337191309858, "value2":3286491733476954470, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383068, "value1":0.3171639131993519, "value2":5768120780013343252, "value3":true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758654, "value1":0.8248294031178783, "value2":1259633164417337834, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885742, "value1":0.7054910487633926, "value2":965825539457944695, "value3":false, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450755, "value1":0.5160791051802902, "value2":980291621965651603, "value3":true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280407, "value1":0.20023478153485316, "value2":216774102527727538, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953286, "value1":0.17708932991625895, "value2":7861520056013490110, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428154, "value1":0.3840775072053381, "value2":5128389599856507984, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650144, "value1":0.5475742595996844, "value2":37238921657180760, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880799, "value1":0.27347132520371564, "value2":7086278310918665364, "value3":true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179258, "value1":0.09483479166767275, "value2":3722096865477015794, "value3":false, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025175, "value1":0.24738190446733826, "value2":2326403401160366476, "value3":true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680444, "value1":0.6441894024482266, "value2":2704266700740984761, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830593, "value1":0.6314118537062262, "value2":8997005770669681872, "value3":false, "__name__":"metric_00000999", "key_9":"h","key_3":"c","key_5":"b", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177647, "value1":0.3203303212434556, "value2":8965923512665629393, "value3":false, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923489, "value1":0.29642566667944875, "value2":1203313530737226131, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.811446, "value1":0.5561455146353509, "value2":570880137152142782, "value3":false, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544802, "value1":0.9164364466036977, "value2":3723655635493793654, "value3":false, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.542566, "value1":0.7689437007202478, "value2":3938961297360550598, "value3":false, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819459, "value1":0.4225560622422357, "value2":2765920289473813577, "value3":true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971131, "value1":0.7588578513174423, "value2":554701584189491661, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638701, "value1":0.9383542993382983, "value2":2675775339679034911, "value3":false, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658985, "value1":0.8385957143790167, "value2":9118144992996965702, "value3":true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449591, "value1":0.6354046681746691, "value2":4796404340287643460, "value3":true, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.851526, "value1":0.7827632730352198, "value2":4383742830241177102, "value3":true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929626, "value1":0.7374713580232243, "value2":6076523832966313810, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.289625, "value1":0.6865021261714863, "value2":2978049869639889816, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394738, "value1":0.6344060546968167, "value2":293482016535092319, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.487799, "value1":0.01884442354627017, "value2":6314666407653758310, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.189751, "value1":0.5131352606856101, "value2":5370657937415537626, "value3":false, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.532641, "value1":0.7500243297537671, "value2":5814407549612083184, "value3":false, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413938, "value1":0.35879356972502013, "value2":7060341260629410136, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464627, "value1":0.5028910273480031, "value2":4767839332233680092, "value3":false, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.840321, "value1":0.13633703551235915, "value2":2307803453888302997, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177282, "value1":0.5212414108183161, "value2":7948818597964566740, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394656, "value1":0.22032364884959718, "value2":8010953658147063588, "value3":true, "__name__":"metric_00000023", "key_4":"b","key_2":"j", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.193561, "value1":0.6737972615452449, "value2":7205681986933877897, "value3":true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919748, "value1":0.5508022412545106, "value2":1935155841798114792, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.549841, "value1":0.6412542942797744, "value2":5725068566993480801, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755602, "value1":0.7570856701300407, "value2":2582449631680230244, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569314, "value1":0.9682571063393881, "value2":3213998604408735177, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220995, "value1":0.4682404519517274, "value2":1296712240822053674, "value3":false, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030188, "value1":0.13463362539315563, "value2":6167313691161388680, "value3":false, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955102, "value1":0.12893740940580098, "value2":937550317375879979, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912054, "value1":0.23584661351992364, "value2":8270059203922837294, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248552, "value1":0.873739706342713, "value2":5483897643292875706, "value3":false, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.668856, "value1":0.06480463060086489, "value2":3429839268398826587, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628990, "value1":0.06856495811246685, "value2":2353082280387987376, "value3":true, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.364427, "value1":0.8257731797802594, "value2":8162649044337833619, "value3":false, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488610, "value1":0.47476344691483213, "value2":7563742944450884141, "value3":true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.659600, "value1":0.22983785061318257, "value2":5566259052586123346, "value3":false, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715473, "value1":0.5778415124343756, "value2":379000163266106555, "value3":true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265567, "value1":0.856852436111815, "value2":96718295321822248, "value3":true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647902, "value1":0.9996700403145788, "value2":6837057660421261848, "value3":false, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.866334, "value1":0.8219801167193398, "value2":8280931000915145951, "value3":false, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144409, "value1":0.604589772741639, "value2":4205152374038619548, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192185, "value1":0.27115827101755235, "value2":5770338217899591097, "value3":true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300761, "value1":0.009332804989256857, "value2":4018403627826051427, "value3":false, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.312951, "value1":0.45431720098201106, "value2":2937253947933810175, "value3":false, "__name__":"metric_00000044", "key_4":"d","key_0":"j","key_1":"c", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647625, "value1":0.3268497558683786, "value2":5914220923444612314, "value3":true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878656, "value1":0.9409966965511731, "value2":1289273008938800365, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057134, "value1":0.3301792558402197, "value2":3318812578583185116, "value3":true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.439955, "value1":0.2652202727470005, "value2":6508684150448276421, "value3":false, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827995, "value1":0.5808204202626653, "value2":2858312188918612316, "value3":false, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863057, "value1":0.013094412054074, "value2":8217074167029879656, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936866, "value1":0.17076998229838808, "value2":1405785613574281058, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.188817, "value1":0.4102266284907626, "value2":9200292054736167191, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919291, "value1":0.849823159364745, "value2":8783211669019586620, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.684825, "value1":0.5612249810486886, "value2":2459193643509253683, "value3":false, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.132931, "value1":0.06165133682856612, "value2":3724912889225979746, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939721, "value1":0.7022523416408233, "value2":3787551523413319921, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423514, "value1":0.16397574471331047, "value2":2217544936229011402, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556341, "value1":0.3363631636072514, "value2":8296724122152500212, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.797045, "value1":0.9775102685569385, "value2":8717038963718857217, "value3":true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.037101, "value1":0.5188119627862039, "value2":8253271773680540946, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.472369, "value1":0.723269226446299, "value2":1830427992517853225, "value3":true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350765, "value1":0.45427831592086265, "value2":2459543486390463113, "value3":true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.172879, "value1":0.9405309891223543, "value2":2118657389939971315, "value3":true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902535, "value1":0.42952097429562314, "value2":3142662533255542276, "value3":true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863286, "value1":0.06876792355358291, "value2":1443459731022840202, "value3":false, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.137921, "value1":0.6964987593861792, "value2":7256675183245953924, "value3":true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.477178, "value1":0.3581558179516239, "value2":6351662172251250145, "value3":false, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286204, "value1":0.148299090562339, "value2":1443868893374130520, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286734, "value1":0.45842257280884463, "value2":1988366284391476937, "value3":false, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787180, "value1":0.5028191040745897, "value2":4148707399092925318, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.586935, "value1":0.0518139414168187, "value2":8586911735374624592, "value3":true, "__name__":"metric_00000072", "key_9":"k","key_0":"g","key_3":"g", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021854, "value1":0.8978228143933898, "value2":1413164249310483673, "value3":true, "__name__":"metric_00000073", "key_6":"b","key_4":"k", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318004, "value1":0.5329901050180244, "value2":700502281930980696, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381834, "value1":0.37533803419078043, "value2":1751644850942508358, "value3":false, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781575, "value1":0.7741607009730129, "value2":7455242385880444942, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.901091, "value1":0.9760509636456571, "value2":1834882752759920223, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805005, "value1":0.18640631542045302, "value2":3539140922926641426, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734757, "value1":0.14536274157280238, "value2":6893258320350162281, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.340380, "value1":0.2042782161421506, "value2":2006565586318459391, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.271132, "value1":0.008492488378786304, "value2":6129670920743570493, "value3":false, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947658, "value1":0.35165969319536833, "value2":4951697844001184553, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.735049, "value1":0.4770090728759761, "value2":2452955784370225782, "value3":true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337551, "value1":0.7050663230338188, "value2":4249578695875539573, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.633134, "value1":0.6874967232601639, "value2":8449525705234658507, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412613, "value1":0.8000775398308927, "value2":5399334563984433319, "value3":true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.520666, "value1":0.3288020539774752, "value2":1684836256027900746, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063862, "value1":0.4985615503389622, "value2":5250749533704010894, "value3":false, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.776154, "value1":0.039036691522280384, "value2":8622652029902283303, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.115396, "value1":0.2649092470624877, "value2":678667215803198769, "value3":false, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.728254, "value1":0.3531001293726284, "value2":2004607802946409884, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242732, "value1":0.9820165334533849, "value2":7506111952193927451, "value3":false, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969092, "value1":0.45874224514029954, "value2":6402112089715656350, "value3":false, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.528740, "value1":0.8320354986171156, "value2":8412309334445466976, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.907053, "value1":0.18136550029947487, "value2":6212227004856794281, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630067, "value1":0.8986694902601496, "value2":9213857752862359137, "value3":false, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133736, "value1":0.8699930415468984, "value2":4081508478635244211, "value3":false, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.198697, "value1":0.017191657622434766, "value2":1486415534842367537, "value3":true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.690493, "value1":0.3223491896226482, "value2":7372352323440542564, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.404617, "value1":0.6240663182478166, "value2":4691037631120171189, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120685, "value1":0.9919062682339983, "value2":8460618157376864427, "value3":true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.688005, "value1":0.3953177302348419, "value2":5961626802588462897, "value3":true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093767, "value1":0.23061987147968935, "value2":2892986034831352810, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.543984, "value1":0.16501271900248737, "value2":317259514400114949, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842969, "value1":0.25446808569640833, "value2":6002379582827585880, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269131, "value1":0.8842302265963506, "value2":2140205221312415384, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972033, "value1":0.9857791825041854, "value2":8003941482296671818, "value3":false, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508672, "value1":0.5492264125933453, "value2":4184449934578666153, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.812228, "value1":0.3691868332062656, "value2":940690450246364953, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033340, "value1":0.4347207480857335, "value2":7925423134984867831, "value3":true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921934, "value1":0.7008140011057534, "value2":3950204529429726081, "value3":false, "__name__":"metric_00000109", "key_8":"e","key_3":"i", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.159043, "value1":0.4015464920656329, "value2":6141367601228839763, "value3":false, "__name__":"metric_00000112", "key_8":"h","key_4":"e","key_5":"i", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207886, "value1":0.923323917554864, "value2":2538142530573084913, "value3":false, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560000, "value1":0.988479108310322, "value2":3590995919192964810, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324806, "value1":0.14623600559543842, "value2":7983994507018983485, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.278078, "value1":0.47875025775481017, "value2":9135606324353342316, "value3":false, "__name__":"metric_00000115", "key_9":"e","key_5":"i", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978376, "value1":0.646562029258319, "value2":6969654958747462453, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.038811, "value1":0.27278148584909157, "value2":3766949452289754200, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174153, "value1":0.456204859878279, "value2":5412969439184865040, "value3":false, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152394, "value1":0.6304591834176713, "value2":7508490770671251540, "value3":false, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079217, "value1":0.7093512830025359, "value2":8041107853739821078, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736508, "value1":0.9456361684627089, "value2":2695850503301130596, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.689579, "value1":0.8717416155878271, "value2":914033909063513507, "value3":true, "__name__":"metric_00000120", "key_7":"d","key_2":"d","key_3":"e", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716602, "value1":0.6181082501584888, "value2":7525886338688800823, "value3":false, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485128, "value1":0.805498923411989, "value2":3515190992320449261, "value3":false, "__name__":"metric_00000126", "key_2":"c","key_0":"k","key_1":"a", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761534, "value1":0.3987574394917293, "value2":565378419404884224, "value3":false, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234797, "value1":0.32687033897000584, "value2":4678721697918141366, "value3":false, "__name__":"metric_00000122", "key_9":"k","key_1":"g","key_8":"i", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099258, "value1":0.5165730450377122, "value2":7897877163898991964, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.855254, "value1":0.6113796894495002, "value2":5439835292581168290, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098218, "value1":0.6331475060559588, "value2":7978662530169854255, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.392907, "value1":0.7806228764363937, "value2":5349308434124377933, "value3":false, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174842, "value1":0.7994951238912167, "value2":213268455628772714, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.654871, "value1":0.23130216238112586, "value2":6353589500271587044, "value3":false, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255121, "value1":0.029192800621726373, "value2":2049991280590108935, "value3":false, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.454836, "value1":0.6345686250008671, "value2":8305250329173845884, "value3":false, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.196946, "value1":0.9679432223428265, "value2":4005113690213723694, "value3":false, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.514102, "value1":0.9460847081604973, "value2":2275533825263312731, "value3":true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.267365, "value1":0.5628401308203138, "value2":2718632724301463408, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.702558, "value1":0.3615743138551236, "value2":5265258619209069561, "value3":true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.157371, "value1":0.9914192491349261, "value2":7107333633697743515, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367362, "value1":0.30046228876641146, "value2":8731213027442541680, "value3":true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502940, "value1":0.33029188840622603, "value2":7282340644676767522, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773440, "value1":0.7398342089000881, "value2":6064862964524030137, "value3":false, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370133, "value1":0.5464083248908671, "value2":8595565634194089280, "value3":false, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819624, "value1":0.22563478104006193, "value2":6291072404703346422, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.822444, "value1":0.6431602385348965, "value2":3571751115605471571, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154889, "value1":0.661415650101091, "value2":2897716721956096259, "value3":false, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.706177, "value1":0.9541112843149568, "value2":4382405639585131280, "value3":false, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.892582, "value1":0.273637580698836, "value2":1569537455975017743, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912404, "value1":0.02958676932622315, "value2":864163789414463895, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164999, "value1":0.8749507184007825, "value2":9170076213645968437, "value3":false, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.609248, "value1":0.6230590575325244, "value2":1182671592625390639, "value3":true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402824, "value1":0.8555668782169765, "value2":3430464294959504849, "value3":false, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.905017, "value1":0.6152490194289306, "value2":3258848281050130827, "value3":true, "__name__":"metric_00000153", "key_8":"j","key_9":"d","key_5":"i", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.669648, "value1":0.6859907392101928, "value2":2219730699877135652, "value3":false, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435602, "value1":0.12441938240097333, "value2":3663118450934184743, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736951, "value1":0.9806346124707411, "value2":6495060662787801507, "value3":false, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502755, "value1":0.6962847488848957, "value2":7479724851955507611, "value3":true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.618421, "value1":0.31789340504585667, "value2":8738918799232514086, "value3":true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827476, "value1":0.7869780281846211, "value2":5474883946012414730, "value3":false, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490095, "value1":0.6962736110006601, "value2":6409713241665137545, "value3":false, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985050, "value1":0.2615773286934893, "value2":6009116516814474343, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502430, "value1":0.19305751383330208, "value2":1654152222563323901, "value3":true, "__name__":"metric_00000161", "key_3":"e","key_8":"b","key_2":"g", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349955, "value1":0.7772270603571774, "value2":73068209272270733, "value3":true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781804, "value1":0.6130002187106758, "value2":5094562490162409190, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.158107, "value1":0.12199479281505482, "value2":887695971080523742, "value3":false, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.575029, "value1":0.9090539881797902, "value2":3726306097696429975, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.197382, "value1":0.010351058305108105, "value2":3390177787518625882, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959617, "value1":0.10545857069830916, "value2":8523151888175341811, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789446, "value1":0.7606780493785521, "value2":2782849609854347711, "value3":true, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384471, "value1":0.3297433712153455, "value2":8732496713018613730, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.440642, "value1":0.05991406624275138, "value2":8221843418889420393, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485145, "value1":0.4324948321911064, "value2":1223798989569651011, "value3":false, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930239, "value1":0.797530519094042, "value2":3933281842105360464, "value3":false, "__name__":"metric_00000174", "key_7":"j","key_0":"h","key_4":"c", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921694, "value1":0.3827550255479697, "value2":1163470615431943983, "value3":false, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780691, "value1":0.8316468698353685, "value2":3969005124884722997, "value3":true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563279, "value1":0.5313651121371132, "value2":1275907147042029939, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099445, "value1":0.8066245132140291, "value2":1404833370385176869, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.897823, "value1":0.395354453907472, "value2":4125787827950753058, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562742, "value1":0.0008610845550682808, "value2":2683131454862577523, "value3":true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981050, "value1":0.6811899501971241, "value2":2963479936998599518, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.090076, "value1":0.7872489837661779, "value2":5920042185451264661, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904987, "value1":0.582385162743904, "value2":3052394546288639081, "value3":false, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.807620, "value1":0.8276864738323398, "value2":1770112128995343558, "value3":true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.313153, "value1":0.7000557306246208, "value2":8804754539464742313, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648178, "value1":0.6035738426070452, "value2":274869451073352526, "value3":false, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382503, "value1":0.7570596198853545, "value2":8494223821423922555, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981825, "value1":0.4954131435773338, "value2":5546139769442373925, "value3":false, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.896817, "value1":0.3057668662697838, "value2":2146747890118623175, "value3":false, "__name__":"metric_00000187", "key_5":"k","key_1":"b", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857258, "value1":0.7535991261099572, "value2":7241562330594412223, "value3":true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.954553, "value1":0.6272550393596944, "value2":7059824369010485707, "value3":true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611570, "value1":0.896770550906892, "value2":4157172203390286832, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.200060, "value1":0.6395440920752251, "value2":2313959820318356273, "value3":true, "__name__":"metric_00000189", "key_9":"k","key_2":"g","key_3":"j", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.305817, "value1":0.7944717065357103, "value2":9079049510197634026, "value3":false, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771177, "value1":0.5686457722223018, "value2":5926649416008895009, "value3":true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.194460, "value1":0.2030568091950128, "value2":5211606615226569619, "value3":false, "__name__":"metric_00000194", "key_2":"c","key_6":"b","key_1":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541777, "value1":0.38229879472765593, "value2":8330651996424197311, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566620, "value1":0.24307591832233774, "value2":2839998819511281148, "value3":false, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376323, "value1":0.5739727137071835, "value2":5399793491142182010, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480111, "value1":0.7895749118902282, "value2":7450043826116159674, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453645, "value1":0.4875344798695633, "value2":6926040231429456296, "value3":false, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734559, "value1":0.3865838466523982, "value2":9001676782517223243, "value3":true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.171659, "value1":0.33364493236506704, "value2":5739131266488499620, "value3":false, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.966196, "value1":0.1807801172943221, "value2":7703455497589255747, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087000, "value1":0.14745151659831435, "value2":3780273793092000978, "value3":false, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537557, "value1":0.6679609579098379, "value2":7524235382942600940, "value3":true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624433, "value1":0.43524838478159544, "value2":8707151351953730491, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.764878, "value1":0.9857854697655962, "value2":2628899540411422018, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087516, "value1":0.9235134986127471, "value2":1244667248757759893, "value3":false, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.092707, "value1":0.8123903328089516, "value2":1510741329638496293, "value3":true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212617, "value1":0.7803051371802152, "value2":1782941121051923216, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110139, "value1":0.9520250869731262, "value2":8515109734542519287, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217411, "value1":0.569745410096317, "value2":7158729472403289306, "value3":true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578681, "value1":0.4421036889589163, "value2":5172660091174561754, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.917747, "value1":0.9646587109518411, "value2":4932202474624997807, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490742, "value1":0.18375289991112742, "value2":5134730937919546380, "value3":true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698691, "value1":0.8462874020691947, "value2":6789099466073417633, "value3":false, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987429, "value1":0.47456618667904216, "value2":7333730409059985654, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.552031, "value1":0.8731675557704086, "value2":2383267067188016407, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.024510, "value1":0.7398931410894473, "value2":4688375563960023614, "value3":true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.885940, "value1":0.5267822661948509, "value2":3106942003445818164, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.128634, "value1":0.17304999217293912, "value2":4202058405826758012, "value3":true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919245, "value1":0.5246293149604686, "value2":7962001396490479168, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.784582, "value1":0.9051007718281217, "value2":6345441367106704431, "value3":true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.631365, "value1":0.18200897376922306, "value2":2743187358723124435, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632586, "value1":0.49203338164778787, "value2":310459975834593918, "value3":false, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.986910, "value1":0.21502125960121787, "value2":9213029499233048926, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.903530, "value1":0.20307359325640195, "value2":3425839773997981556, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495175, "value1":0.42906594953086846, "value2":1910613263092063181, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560265, "value1":0.27498766656590357, "value2":8893579102721900369, "value3":false, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.298348, "value1":0.05982261910728163, "value2":8036644582013512610, "value3":false, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624831, "value1":0.8394666399726175, "value2":1345154507982673038, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.123368, "value1":0.13834539586421774, "value2":6342191610731262762, "value3":false, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.086336, "value1":0.17439021756766837, "value2":6128400320706208802, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421248, "value1":0.39843282468447905, "value2":8680280397492385017, "value3":false, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139454, "value1":0.456434331043375, "value2":4836743646420773621, "value3":false, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351219, "value1":0.777135306661456, "value2":3903934605632263710, "value3":true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.414090, "value1":0.8143501039400677, "value2":4413351639615300728, "value3":false, "__name__":"metric_00000237", "key_7":"f","key_0":"d","key_5":"h", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231721, "value1":0.3138412091160347, "value2":9168517044165068794, "value3":true, "__name__":"metric_00000236", "key_1":"h","key_0":"d", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.230702, "value1":0.8804508000643604, "value2":512236854100511456, "value3":true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.708957, "value1":0.47311614120900003, "value2":7136526398161878544, "value3":false, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.475928, "value1":0.32108039730398563, "value2":7292530251105851041, "value3":true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.643383, "value1":0.9689860631255371, "value2":1734391035548707204, "value3":true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.792203, "value1":0.8844015947765959, "value2":6318884852252048224, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.323993, "value1":0.18316994233066625, "value2":3785806626345479252, "value3":true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.055153, "value1":0.9975090536646855, "value2":9101914663697333394, "value3":false, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762055, "value1":0.14538112505308043, "value2":6589674161435979608, "value3":false, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457017, "value1":0.9313588309515796, "value2":2386410717081263989, "value3":false, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118975, "value1":0.2538090940895061, "value2":2359952847484695171, "value3":true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887012, "value1":0.26127955373376677, "value2":7323797183712585078, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465777, "value1":0.39956363244564946, "value2":4290799816610955241, "value3":true, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919817, "value1":0.24654070770474018, "value2":6141631506827241212, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878721, "value1":0.7298060681766352, "value2":7628091662411871299, "value3":false, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308905, "value1":0.8062336262819177, "value2":3004783841504994012, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630086, "value1":0.9740479758667072, "value2":7021198218943014146, "value3":true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661157, "value1":0.0030467411917742306, "value2":8705979164600447258, "value3":true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865574, "value1":0.36919272421680055, "value2":4523649159933689100, "value3":true, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.185605, "value1":0.5443213859941823, "value2":3228781454351122262, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865038, "value1":0.09385639771871399, "value2":7227272424843868171, "value3":false, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798023, "value1":0.8684611657481685, "value2":2233953571760268466, "value3":true, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730804, "value1":0.08710486562835452, "value2":6236391362613657084, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.202824, "value1":0.4792114852099944, "value2":445497349923776863, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173553, "value1":0.039235946095809125, "value2":7869971288440786845, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908688, "value1":0.5052600498486294, "value2":4519674082380595002, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341267, "value1":0.007052586435709472, "value2":3150628489852334883, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881706, "value1":0.2885539517449333, "value2":1797762712648874995, "value3":false, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216914, "value1":0.4200170713130975, "value2":8711469515389655710, "value3":false, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.219622, "value1":0.8598302806140433, "value2":3863311400727592540, "value3":true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.656021, "value1":0.6463537908056233, "value2":1050074674903654190, "value3":true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641122, "value1":0.40560841010225995, "value2":742946648471490108, "value3":false, "__name__":"metric_00000268", "key_9":"d","key_6":"j","key_8":"c", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755478, "value1":0.42687719134929036, "value2":8589479839598188558, "value3":true, "__name__":"metric_00000270", "key_8":"d","key_0":"a","key_1":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.848432, "value1":0.9962729980353733, "value2":780000159187900550, "value3":true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.195275, "value1":0.6773988604333073, "value2":5522000829662160867, "value3":true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787345, "value1":0.6486468864516003, "value2":4138825123686123358, "value3":false, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.553619, "value1":0.7566207604613129, "value2":5341523136140526945, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982202, "value1":0.8174248836774913, "value2":4875324336295746206, "value3":true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317829, "value1":0.25574317763537074, "value2":3850513628767917465, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366612, "value1":0.634692324496315, "value2":8508176811169559100, "value3":false, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.931082, "value1":0.9070818575294899, "value2":2941126198839449331, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419466, "value1":0.6035953996525765, "value2":5226492465996637298, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207446, "value1":0.7148273386681443, "value2":6830161919863899962, "value3":false, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339115, "value1":0.9680080839353958, "value2":3346579129519946511, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329518, "value1":0.5052033589082856, "value2":8326166807060477796, "value3":false, "__name__":"metric_00000282", "key_2":"c","key_0":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314104, "value1":0.23744990369573346, "value2":6679180757317175227, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177954, "value1":0.6916907655144761, "value2":6598624424043999988, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220478, "value1":0.9419151786449748, "value2":6868454728302182507, "value3":true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.962386, "value1":0.29200153388880246, "value2":3531740125944089227, "value3":false, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.522912, "value1":0.4384617611060238, "value2":7012028395173793023, "value3":true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.701530, "value1":0.008494038169472137, "value2":4096870035163582532, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019574, "value1":0.540551382306373, "value2":7575443997806932703, "value3":true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558742, "value1":0.30242558718747614, "value2":1312291174476606343, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.597198, "value1":0.9990604033963557, "value2":5377437181323103863, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321750, "value1":0.8571357709325697, "value2":3223326577646383979, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976585, "value1":0.794736424631542, "value2":7297990135456560387, "value3":false, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053373, "value1":0.9010563829072253, "value2":5702408670366972568, "value3":false, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.500145, "value1":0.16016689363963474, "value2":7655127244946192653, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.347755, "value1":0.19254449629837572, "value2":1438411533080776332, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.047080, "value1":0.9596252631833848, "value2":4943940353426671568, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.737278, "value1":0.6304724450380977, "value2":5075506433971212474, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842472, "value1":0.9682019821931608, "value2":7537248780629599850, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210066, "value1":0.016073835132125416, "value2":1148941512571603438, "value3":false, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019003, "value1":0.15376503679496498, "value2":6744619422647456093, "value3":true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972351, "value1":0.6522824510722263, "value2":3635978187235320184, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464404, "value1":0.1987173098042302, "value2":2325620189736714816, "value3":false, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505822, "value1":0.8969967220271704, "value2":2772428523245878847, "value3":true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.858017, "value1":0.42532104055952885, "value2":7057664582771584406, "value3":false, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827767, "value1":0.11843519151662998, "value2":4642257980991996610, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457807, "value1":0.6680540521232995, "value2":1413093760337990232, "value3":true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307237, "value1":0.2483558233491924, "value2":1657535478313961463, "value3":false, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187495, "value1":0.8787274497214357, "value2":1012440122187350073, "value3":false, "__name__":"metric_00000309", "key_5":"h","key_6":"k","key_2":"b", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.935106, "value1":0.6622345557285527, "value2":873018521033719372, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212483, "value1":0.03664769373990424, "value2":8662944034974337497, "value3":false, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.279577, "value1":0.06940570148672788, "value2":3397490112016095138, "value3":false, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.679928, "value1":0.94467398885254, "value2":8140288354763339321, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348611, "value1":0.09177297933527545, "value2":2243053697955925595, "value3":true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033795, "value1":0.004266699548950342, "value2":8869596053230724066, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716957, "value1":0.952311087977132, "value2":3488323066281405120, "value3":true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275244, "value1":0.9153132837467353, "value2":6017024334903831134, "value3":false, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523199, "value1":0.8894820330529598, "value2":2495751532034294665, "value3":false, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.548536, "value1":0.36452752239967207, "value2":4087473006792877554, "value3":false, "__name__":"metric_00000319", "key_9":"g","key_2":"b", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452175, "value1":0.03794808741352215, "value2":8378297363369384496, "value3":false, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156999, "value1":0.3422567916396813, "value2":3950482866588863620, "value3":false, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508543, "value1":0.5441836551476523, "value2":5324891351187729379, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397072, "value1":0.8991563074648984, "value2":484650303993845367, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074791, "value1":0.08635154390096551, "value2":7636185572736407362, "value3":false, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214033, "value1":0.529318647161814, "value2":3428935700531030330, "value3":false, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023178, "value1":0.9863231971076605, "value2":952801547411635885, "value3":false, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.655562, "value1":0.1342499141610202, "value2":6341506872735720793, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834066, "value1":0.5801423696428913, "value2":2989261336292506730, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.107225, "value1":0.08595942695174727, "value2":7162816956772874074, "value3":true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.303930, "value1":0.5732092807009482, "value2":6606617148160680700, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.915970, "value1":0.10263255284529503, "value2":8480457956761770794, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356355, "value1":0.6141228451134201, "value2":7156945200312274543, "value3":false, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645757, "value1":0.7597072830851564, "value2":2605675246955821391, "value3":false, "__name__":"metric_00000333", "key_9":"e","key_1":"f","key_4":"b", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760044, "value1":0.048059905194606296, "value2":3102725962018871003, "value3":true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949450, "value1":0.803383211503761, "value2":5157179277981661443, "value3":true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124187, "value1":0.16836795463840862, "value2":5627024937490827799, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621202, "value1":0.11873453496853606, "value2":971627245321019634, "value3":true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762026, "value1":0.5957950719970895, "value2":1706809321238862597, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391493, "value1":0.7101377073031195, "value2":6966030075201405586, "value3":false, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.168903, "value1":0.22863352792292077, "value2":1427332489767053365, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698944, "value1":0.5031422927319381, "value2":2536670790706398066, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985917, "value1":0.7010791398723792, "value2":2424373004486759234, "value3":true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611987, "value1":0.023052053167488012, "value2":1572028785346213886, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.084732, "value1":0.8225285464369793, "value2":7247308476502074358, "value3":true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476584, "value1":0.6320619568566006, "value2":3013845267574452926, "value3":false, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.358865, "value1":0.9597368730404013, "value2":1895755548879639735, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559280, "value1":0.7939551495955407, "value2":9185811549617457738, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338153, "value1":0.6433609372521104, "value2":5456961662990895069, "value3":false, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120489, "value1":0.02598857402797232, "value2":2690067127374441413, "value3":false, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.026856, "value1":0.3509430162667547, "value2":5912250260618117609, "value3":true, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.911457, "value1":0.5658091707632997, "value2":3970272443516058585, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563432, "value1":0.24659033075994388, "value2":728046743750740729, "value3":true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.596212, "value1":0.5615056517168183, "value2":7033619460114819908, "value3":false, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051460, "value1":0.5771358250664421, "value2":8203484790496543671, "value3":false, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.824484, "value1":0.05697931696566816, "value2":6635808805223685263, "value3":true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627182, "value1":0.9130566878289693, "value2":8110244852760276756, "value3":false, "__name__":"metric_00000355", "key_3":"e","key_9":"e","key_2":"d", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207192, "value1":0.7274854390257628, "value2":5772335372245409001, "value3":true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799146, "value1":0.5204858317687999, "value2":128757778781441708, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014826, "value1":0.7097519460607616, "value2":9006806179184131719, "value3":true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648719, "value1":0.01042589454449313, "value2":1630317652722526040, "value3":false, "__name__":"metric_00000359", "key_6":"j","key_9":"k","key_3":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.665998, "value1":0.6184281613297867, "value2":3187261205020385384, "value3":false, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.501321, "value1":0.23807711629991066, "value2":1285207877035639838, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184165, "value1":0.7331865941823509, "value2":8439731025981805138, "value3":true, "__name__":"metric_00000360", "key_9":"f","key_0":"a","key_5":"j", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801569, "value1":0.9860558447882487, "value2":6774600722253198808, "value3":false, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384223, "value1":0.22369926816026842, "value2":5826656674460169842, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.980495, "value1":0.6229185444454209, "value2":4872634527752777210, "value3":false, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.380826, "value1":0.7806130098804899, "value2":384791734560367497, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163193, "value1":0.5398716860823138, "value2":4387950486971806786, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.823934, "value1":0.5810656874809516, "value2":450490831163611562, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.608002, "value1":0.7930931659235, "value2":7050949938710758790, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363629, "value1":0.5705690869328316, "value2":4003836970877260330, "value3":true, "__name__":"metric_00000369", "key_6":"e","key_7":"a","key_5":"i", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.997480, "value1":0.2946419071937674, "value2":5707538095602523121, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.835470, "value1":0.023853067484955352, "value2":5038338303194879356, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.720798, "value1":0.48474572095528634, "value2":682322783035637435, "value3":false, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.573983, "value1":0.3514813766672496, "value2":2903766540993385647, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730487, "value1":0.8217554814463921, "value2":2948047805979067123, "value3":true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016747, "value1":0.12213673109770266, "value2":3888549493731213502, "value3":false, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.081414, "value1":0.18665321280658956, "value2":6792269259144278363, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300754, "value1":0.9731729557371976, "value2":7090692680090167231, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337262, "value1":0.4023504772804693, "value2":4873228147801762854, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990413, "value1":0.6497908067795414, "value2":3504328384430870621, "value3":false, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.276664, "value1":0.785362017819063, "value2":1481672280882279109, "value3":true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.357896, "value1":0.03050056454630596, "value2":6649840382760115378, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.095552, "value1":0.31901448906359003, "value2":7956338378631849376, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.518227, "value1":0.922032137621878, "value2":785067472984268171, "value3":false, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094243, "value1":0.6170957228718421, "value2":6792110304537802959, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782822, "value1":0.20221744914264872, "value2":3178187263071942184, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.539464, "value1":0.9703158465709303, "value2":1060145839109092280, "value3":true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773211, "value1":0.9517836987483416, "value2":7463838442968425219, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.695794, "value1":0.34514076166527163, "value2":1111746250317888633, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074230, "value1":0.19136080585905593, "value2":4573587278465589025, "value3":false, "__name__":"metric_00000389", "key_6":"c","key_7":"b","key_0":"j", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982043, "value1":0.2232136951539441, "value2":7688263110504645334, "value3":false, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939724, "value1":0.3301646825680086, "value2":5317348954333303134, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275048, "value1":0.1567976387714283, "value2":8628385701502049400, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.251237, "value1":0.601867452525591, "value2":4258975777323734306, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051509, "value1":0.87447596502846, "value2":390569829027012116, "value3":true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453014, "value1":0.7517900109611292, "value2":5319112069730300764, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.859148, "value1":0.03968223769223721, "value2":6092044766196787249, "value3":false, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564895, "value1":0.8282334480284984, "value2":8177460795118738605, "value3":false, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.213064, "value1":0.37245338326241034, "value2":5313011500830153125, "value3":false, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234759, "value1":0.6391009807318464, "value2":3598159575817066989, "value3":true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922540, "value1":0.8881740732484977, "value2":4573970560902118758, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247842, "value1":0.6867454879676517, "value2":2359279288069431907, "value3":true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350225, "value1":0.6867879732162172, "value2":37306184729818872, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881388, "value1":0.4406197726220826, "value2":724124821973247241, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.950564, "value1":0.1322036312488787, "value2":4198819767892784951, "value3":true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.666395, "value1":0.2251980228042216, "value2":606006971675717615, "value3":false, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.344497, "value1":0.029652052432054064, "value2":5846707393638380751, "value3":false, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.703773, "value1":0.6376539079560296, "value2":6681127841717987695, "value3":false, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295458, "value1":0.5526352586149329, "value2":7043209016069060231, "value3":false, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435502, "value1":0.09713433968048614, "value2":1019638678991819627, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598132, "value1":0.963189125504533, "value2":6458016500274290636, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627640, "value1":0.4641061727729553, "value2":8689246494106761338, "value3":false, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254760, "value1":0.8326486356175677, "value2":4407797931839765907, "value3":false, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805732, "value1":0.6423132619728287, "value2":6833967417402080897, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.326823, "value1":0.9067956612338547, "value2":2841002873709302821, "value3":true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.088104, "value1":0.06965395874499884, "value2":1789284717431249823, "value3":true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.948712, "value1":0.9884448336440479, "value2":1290858986155893057, "value3":true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511709, "value1":0.8105556502461225, "value2":3470144572708895882, "value3":false, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163730, "value1":0.8658645130208062, "value2":6466367002189635673, "value3":false, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260153, "value1":0.5010858345134921, "value2":555883752624156654, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.274796, "value1":0.8267418905611275, "value2":7590347778036203754, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338150, "value1":0.7906313079639156, "value2":7837780129396014797, "value3":true, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938845, "value1":0.03371523590847688, "value2":2642591228031199330, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254822, "value1":0.42059549793106427, "value2":4614016824012787995, "value3":true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003989, "value1":0.05975709544316542, "value2":1382695454770978636, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.577151, "value1":0.8937534168973357, "value2":6521815205949311706, "value3":false, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.253675, "value1":0.40184282199800475, "value2":8286880530272716169, "value3":false, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205638, "value1":0.7633247932732301, "value2":6913337973948968095, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939093, "value1":0.03322965819693288, "value2":2797917596305577577, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248192, "value1":0.5307465178536894, "value2":6134584955455781673, "value3":true, "__name__":"metric_00000430", "key_9":"j","key_1":"j","key_4":"c", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.593487, "value1":0.05419262497736568, "value2":9190929989995856492, "value3":false, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.182515, "value1":0.035610068777040586, "value2":6469776877211677470, "value3":true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160124, "value1":0.662164487805106, "value2":8768604308735587138, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317998, "value1":0.9757873511779303, "value2":1199801679672428467, "value3":true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922339, "value1":0.9554985330074205, "value2":8286943158249961765, "value3":false, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023973, "value1":0.01669695080382125, "value2":4408497776354893523, "value3":true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.399088, "value1":0.37811424022789003, "value2":499643435878747492, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.974576, "value1":0.6392617923906977, "value2":1832499551397243524, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423426, "value1":0.1431493881970496, "value2":4016805275891833987, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220477, "value1":0.006541995474356055, "value2":5608461794122610089, "value3":false, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014455, "value1":0.20051860544075079, "value2":4298831902740539404, "value3":false, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771702, "value1":0.9711913949733121, "value2":4548136488165596312, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837484, "value1":0.3199900890395432, "value2":7619719494433917806, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632963, "value1":0.8179930116909135, "value2":42611344634328983, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739321, "value1":0.8571068088303897, "value2":1929086780074260377, "value3":true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169110, "value1":0.2241172770388991, "value2":3015343857994873164, "value3":true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558214, "value1":0.5245963976733786, "value2":4636974583954671385, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551826, "value1":0.8529460688457188, "value2":7676686801578586657, "value3":false, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317389, "value1":0.34597315939279244, "value2":8526544610592186971, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131556, "value1":0.9023875402465826, "value2":3147425851236763117, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330907, "value1":0.7187683220932978, "value2":6304935909987923644, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419527, "value1":0.3464275048663091, "value2":800017209533647436, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325070, "value1":0.32409707654758807, "value2":3093719017131899877, "value3":false, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094166, "value1":0.6576755239099957, "value2":4375841020494708801, "value3":false, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184361, "value1":0.454784333846635, "value2":8511384704032602153, "value3":false, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424067, "value1":0.7746556148202395, "value2":8409657959538616305, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.282726, "value1":0.19274194173178774, "value2":4484284290811930974, "value3":true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.044679, "value1":0.17881938606509734, "value2":2991918820431220404, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622917, "value1":0.5058188298556251, "value2":2680090310035550156, "value3":true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.686600, "value1":0.019991692028205214, "value2":8990563930001611676, "value3":true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.977493, "value1":0.8045660581528546, "value2":3965120458727174879, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.910280, "value1":0.15532301386230277, "value2":2751777764504449883, "value3":true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981678, "value1":0.6634934084628059, "value2":3338856320416894701, "value3":false, "__name__":"metric_00000463", "key_5":"a","key_0":"g","key_3":"e", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099077, "value1":0.4991967563162609, "value2":7696501412266603819, "value3":true, "__name__":"metric_00000464", "key_3":"c","key_0":"a","key_1":"j", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169242, "value1":0.7573800949559079, "value2":4682651688122452104, "value3":false, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.854077, "value1":0.38315712266426233, "value2":7277684317066282125, "value3":false, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017721, "value1":0.43030519598600153, "value2":8176728734487897933, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449823, "value1":0.0905672633924478, "value2":8038615141368276704, "value3":true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505199, "value1":0.29347749204963386, "value2":5939807103722962422, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.429412, "value1":0.3821527042310373, "value2":7616377560194630586, "value3":false, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645364, "value1":0.31800894756504583, "value2":5324276910842211792, "value3":false, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789951, "value1":0.12346800778619027, "value2":7282935021388766519, "value3":true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.005934, "value1":0.4652775738745803, "value2":5150985627139247681, "value3":true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770208, "value1":0.8318069314007921, "value2":8693296121103057557, "value3":false, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.036795, "value1":0.31504722243353955, "value2":5332439897432100886, "value3":false, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.557676, "value1":0.04547498827953966, "value2":1747115516814035342, "value3":true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295720, "value1":0.25765308010583876, "value2":7172964053161542723, "value3":true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.862280, "value1":0.6690305275979546, "value2":8530225967231964009, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464755, "value1":0.015096554191777058, "value2":2158727717387461667, "value3":true, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902724, "value1":0.3729818170388733, "value2":2937474665692773587, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.150211, "value1":0.9010548939518067, "value2":8689173712211659171, "value3":true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502794, "value1":0.637718327451074, "value2":1058416603563820737, "value3":true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756334, "value1":0.9093486580620052, "value2":3914139843013624351, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109763, "value1":0.3482544181233465, "value2":3984218999735374107, "value3":true, "__name__":"metric_00000482", "key_7":"g","key_2":"f","key_3":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.007357, "value1":0.530193826214922, "value2":7784257234281286192, "value3":false, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133042, "value1":0.14994561696634548, "value2":6913972686763727632, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.499117, "value1":0.9693770819840257, "value2":292923979595115495, "value3":true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.256986, "value1":0.8560085418753359, "value2":1633179891830212156, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348017, "value1":0.4525195870224779, "value2":1498113059165586202, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.389164, "value1":0.4747861877903318, "value2":5663253941505962378, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.712219, "value1":0.23770447169372633, "value2":7127281555547067699, "value3":false, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799547, "value1":0.4771589732318728, "value2":2601851094636806560, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761432, "value1":0.9902875537117704, "value2":2441735233721216793, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.594915, "value1":0.9327820854039164, "value2":7435962671657247487, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.018250, "value1":0.35410376862709275, "value2":6088069648662464618, "value3":true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.640810, "value1":0.6282310495408681, "value2":4206520774588356560, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929377, "value1":0.30194565379846455, "value2":371174649294622523, "value3":false, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.817275, "value1":0.4745483085572834, "value2":2810603774900787473, "value3":true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.046061, "value1":0.9540450542751275, "value2":6567731530813672613, "value3":false, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976385, "value1":0.2747180574854192, "value2":7052984344810070260, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833030, "value1":0.8111270330849011, "value2":2406611793520084033, "value3":true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146154, "value1":0.019440584474240373, "value2":1431804086688415352, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541319, "value1":0.42910271299796143, "value2":5008131544913964869, "value3":true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.478974, "value1":0.3992297014608565, "value2":6638251650936294767, "value3":false, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.591553, "value1":0.9631516171967569, "value2":6328755963396316225, "value3":true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.836675, "value1":0.8925642028256048, "value2":2169296529295363291, "value3":false, "__name__":"metric_00000506", "key_6":"g","key_3":"e","key_5":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240233, "value1":0.5601423681829663, "value2":2372802153601657174, "value3":false, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926558, "value1":0.8669054217402387, "value2":8750941328640682000, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.785340, "value1":0.9294643708923529, "value2":8967392703697598518, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321372, "value1":0.6020812110006013, "value2":1942102877471692500, "value3":false, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578919, "value1":0.17388396608421544, "value2":5054354296681821565, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830121, "value1":0.06504953435269434, "value2":7812868363877570134, "value3":true, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110821, "value1":0.12907416408116543, "value2":1398896999021006979, "value3":false, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318471, "value1":0.6665917244504806, "value2":5816507337427212221, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881196, "value1":0.35509697330525436, "value2":3799008703902351704, "value3":true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503152, "value1":0.686811705668727, "value2":1281918055642507847, "value3":false, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.393488, "value1":0.8980510009425112, "value2":3157487117727981952, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993113, "value1":0.8339917765115039, "value2":1528182230124691520, "value3":true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441452, "value1":0.5723022066521427, "value2":8611586563060872543, "value3":false, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902474, "value1":0.21553093452664723, "value2":1591141288904852990, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.937761, "value1":0.5975557556565823, "value2":1778079885483684894, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.509879, "value1":0.8107703383515009, "value2":7338854303608988296, "value3":true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321562, "value1":0.7839743525651934, "value2":407341572079351661, "value3":false, "__name__":"metric_00000523", "key_5":"k","key_0":"g","key_3":"g", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.083126, "value1":0.8340311472033722, "value2":414058498508156341, "value3":true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.266082, "value1":0.7848792912531005, "value2":1487975780946804498, "value3":false, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.001354, "value1":0.042658133657431034, "value2":2686477010080712855, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984348, "value1":0.12772415816911367, "value2":1744454398245616628, "value3":true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.545130, "value1":0.012749865379431171, "value2":3614758343709591135, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.211075, "value1":0.26931470974880833, "value2":5335597373851226907, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517993, "value1":0.8277823010534222, "value2":441076351209513172, "value3":true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.783905, "value1":0.6446774753163863, "value2":2444284131966329588, "value3":false, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140755, "value1":0.7307559658816861, "value2":1750453360454837174, "value3":false, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.243464, "value1":0.5611645445748624, "value2":5031122288522829688, "value3":false, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247597, "value1":0.26059857605419207, "value2":6822156384989813472, "value3":true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053905, "value1":0.6298269719196294, "value2":6661136217666523640, "value3":true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.058995, "value1":0.7306532383370615, "value2":5311315770055711299, "value3":true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030527, "value1":0.08351380687475202, "value2":7155821886727737087, "value3":true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.498371, "value1":0.2731396400770269, "value2":89197923125259866, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397662, "value1":0.0870595280214176, "value2":4020978501682465140, "value3":false, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173834, "value1":0.7945264755888277, "value2":3882559205765407193, "value3":false, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497055, "value1":0.09364593584875654, "value2":891682366412177732, "value3":false, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065765, "value1":0.835147994300049, "value2":2505979555724543511, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.765901, "value1":0.992206355507561, "value2":5788507149763623428, "value3":true, "__name__":"metric_00000543", "key_6":"a","key_2":"k","key_3":"h", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622912, "value1":0.44608293916585673, "value2":946502417712586156, "value3":true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366924, "value1":0.9938167708003688, "value2":7200121887993962206, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.072114, "value1":0.001073896039030629, "value2":8685581835197772410, "value3":false, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265774, "value1":0.30130760063683054, "value2":8411046196525683207, "value3":false, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.749589, "value1":0.9239863104020335, "value2":4219975890726145199, "value3":true, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190061, "value1":0.9734172533312344, "value2":8137079207203857103, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.519945, "value1":0.07187194342466732, "value2":7825744617647803155, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.649495, "value1":0.6408236940193898, "value2":3048798030528052662, "value3":false, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320456, "value1":0.5081060875649162, "value2":719352320345764599, "value3":true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476698, "value1":0.052307893890373255, "value2":2742427547083463898, "value3":false, "__name__":"metric_00000552", "key_8":"d","key_2":"i", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571531, "value1":0.4132003718642955, "value2":6484766831533191693, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682702, "value1":0.44751287991255284, "value2":4055042383084807325, "value3":true, "__name__":"metric_00000556", "key_4":"e","key_6":"b","key_3":"i", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.890718, "value1":0.33741266801359987, "value2":6934783766842267527, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452747, "value1":0.3663537726308516, "value2":3312655545498739922, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199996, "value1":0.15068464159094366, "value2":5665646131255744951, "value3":true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.983795, "value1":0.8775221552243728, "value2":9069927522091992178, "value3":false, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.583294, "value1":0.838152847228304, "value2":1035319401212201491, "value3":true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212587, "value1":0.8012210929096193, "value2":5585326063344558523, "value3":true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641779, "value1":0.9247211843697717, "value2":2117637859665911236, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.406293, "value1":0.6198872109290668, "value2":5911640262763562610, "value3":false, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.277897, "value1":0.027423430536495416, "value2":7628618445567261117, "value3":false, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.832579, "value1":0.5331161695907243, "value2":8305462638082561375, "value3":false, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205268, "value1":0.018199222166683245, "value2":475925606235710366, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.615075, "value1":0.9915416739226851, "value2":2204646069144435635, "value3":false, "__name__":"metric_00000565", "key_6":"d","key_7":"e","key_1":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424612, "value1":0.9757938651409276, "value2":6266511548869428739, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286078, "value1":0.0846398426026624, "value2":3081995387268873440, "value3":true, "__name__":"metric_00000569", "key_9":"g","key_4":"f", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787171, "value1":0.8365446036859473, "value2":5979460348923075463, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381115, "value1":0.9123835513620147, "value2":3398482109219075054, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.114157, "value1":0.804291931301114, "value2":6012424958858906101, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134911, "value1":0.00736667824808292, "value2":2400000632738793301, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164780, "value1":0.21287762980212552, "value2":8645521771535465924, "value3":true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710538, "value1":0.7411138160092244, "value2":8685307998868002992, "value3":false, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585644, "value1":0.08786909591629084, "value2":7343395510792995060, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125543, "value1":0.7332366768127239, "value2":8370868046930361421, "value3":false, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.932834, "value1":0.9015644945676353, "value2":5236314268804910193, "value3":false, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093732, "value1":0.497896904599941, "value2":6397325408101186539, "value3":false, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486123, "value1":0.5097658314843312, "value2":1337466807316653828, "value3":false, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904889, "value1":0.6556792189077274, "value2":4516335409033295305, "value3":true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.387498, "value1":0.9910872699779568, "value2":1557997601588004148, "value3":true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710931, "value1":0.5132845499506883, "value2":3319795795293053140, "value3":true, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.515215, "value1":0.049432502275940204, "value2":2300709145422274852, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821813, "value1":0.6825139883510115, "value2":8194491426131939536, "value3":true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865373, "value1":0.7044541486613806, "value2":3703555177588303304, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837580, "value1":0.8572176121238669, "value2":2344100903459752932, "value3":true, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314910, "value1":0.4347186053772807, "value2":7676829676819086499, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.946948, "value1":0.7950628854157453, "value2":2267010392465955325, "value3":false, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015107, "value1":0.8624602552899475, "value2":9148330689311945183, "value3":false, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488041, "value1":0.4618418780870866, "value2":8123711349904092581, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710726, "value1":0.4226915062812881, "value2":5491521860827839646, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841455, "value1":0.9825903973183027, "value2":3077499782633025057, "value3":true, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.592689, "value1":0.9193972918242918, "value2":929238328644383650, "value3":false, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993922, "value1":0.14695440679147787, "value2":8882303717901413025, "value3":true, "__name__":"metric_00000596", "key_4":"b","key_8":"b","key_0":"a", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731320, "value1":0.9934902241842611, "value2":1321453009633438522, "value3":true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604377, "value1":0.24174531205655925, "value2":5870091064000157680, "value3":true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628164, "value1":0.8064136026821553, "value2":8786888366338651141, "value3":false, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.484350, "value1":0.5079394885547678, "value2":6920245211342331930, "value3":true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978330, "value1":0.9354707910737625, "value2":7581764658078201917, "value3":true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632366, "value1":0.23595321295755245, "value2":1511878176205605753, "value3":true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142194, "value1":0.6306719216610874, "value2":2431345756658073693, "value3":false, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397195, "value1":0.7323027634538346, "value2":8521159495965572956, "value3":true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.996298, "value1":0.8484427529278864, "value2":7246857551445151219, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.328211, "value1":0.8531255802404514, "value2":4506921974707672140, "value3":false, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308884, "value1":0.5324739320542478, "value2":6505796324920546, "value3":false, "__name__":"metric_00000608", "key_4":"g","key_9":"i","key_1":"e", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.721813, "value1":0.2030510736245931, "value2":493933611483820748, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325138, "value1":0.6318807488600625, "value2":3427677815511198978, "value3":true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.717950, "value1":0.13850468444662895, "value2":3714125763364851426, "value3":false, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984959, "value1":0.2313158857459622, "value2":6179178857252455230, "value3":false, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767352, "value1":0.04737365094297706, "value2":7312736110130720225, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517326, "value1":0.38308887661557994, "value2":4397262624851606256, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537458, "value1":0.8630190277228629, "value2":1405251953788249605, "value3":true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.589024, "value1":0.6853812314292282, "value2":6568591835038822877, "value3":true, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013121, "value1":0.9022971605517441, "value2":5391410131187089227, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585652, "value1":0.027029007730938633, "value2":6270487841856618503, "value3":false, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.069148, "value1":0.5452016240009335, "value2":7230673973301658030, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870318, "value1":0.8893099847439015, "value2":2072311642337421510, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.525589, "value1":0.9489296395713415, "value2":1934395693866121179, "value3":true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.411127, "value1":0.5543622037481519, "value2":869973100557162310, "value3":true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870905, "value1":0.1429227422154167, "value2":1580405259981052384, "value3":true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214057, "value1":0.007127408341584659, "value2":5021161294907689526, "value3":false, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324636, "value1":0.2886203159980936, "value2":3536603792281429135, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566190, "value1":0.2435195285039059, "value2":7314231811000308948, "value3":true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.302446, "value1":0.3969373764014023, "value2":861549335043188440, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265270, "value1":0.8392596283575425, "value2":229012885919848214, "value3":false, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306041, "value1":0.6755326135952276, "value2":687305007623331029, "value3":true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.920842, "value1":0.14122582181388352, "value2":8912506495776009399, "value3":false, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970414, "value1":0.32375225588910955, "value2":3874667148173594242, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098047, "value1":0.5080753238940934, "value2":259459341225408666, "value3":true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814708, "value1":0.5851579994151463, "value2":4279032101744191170, "value3":false, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857725, "value1":0.8750883468056723, "value2":8160658677842295703, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288649, "value1":0.3318827172603761, "value2":8934632128032959148, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.651119, "value1":0.31525481325301247, "value2":8244510685212193822, "value3":true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065672, "value1":0.08637516713839134, "value2":707728667641140855, "value3":true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620357, "value1":0.8983386747310453, "value2":3756826533006983142, "value3":true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707292, "value1":0.4495030238963459, "value2":5996025393915497140, "value3":true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.443961, "value1":0.930457120039426, "value2":6519425791992110608, "value3":false, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109167, "value1":0.09168467762421384, "value2":2057706949013098262, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.481603, "value1":0.3529978146205129, "value2":6544227882618708766, "value3":false, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770746, "value1":0.6378677748113686, "value2":1182963631296444077, "value3":true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967875, "value1":0.8756904550723122, "value2":516292034493240162, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324624, "value1":0.03689035176673176, "value2":6723940798795470811, "value3":true, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556389, "value1":0.7611742354824225, "value2":6343475929372871725, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538576, "value1":0.1188977043392838, "value2":1679649620772949796, "value3":true, "__name__":"metric_00000646", "key_6":"k","key_7":"a","key_1":"j", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.531730, "value1":0.2793539457328025, "value2":8309265375086957584, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488778, "value1":0.8806255322454197, "value2":2637346612964090286, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174493, "value1":0.4058829790367517, "value2":4908420813809791761, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.875676, "value1":0.8084081237917962, "value2":1370185119560523418, "value3":true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214353, "value1":0.22148397653369978, "value2":3829075843579764833, "value3":true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402832, "value1":0.9983967808941802, "value2":373850521427271723, "value3":true, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.574399, "value1":0.1403913922108672, "value2":1025325851242540472, "value3":false, "__name__":"metric_00000651", "key_4":"j","key_2":"i", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538957, "value1":0.5637672549308663, "value2":180146514999582015, "value3":true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043198, "value1":0.6621042931314224, "value2":4672271958823003044, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.455391, "value1":0.37463255563131875, "value2":7050585404363795282, "value3":true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118353, "value1":0.7019235890065948, "value2":6561897677595087821, "value3":true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562842, "value1":0.4216296582302376, "value2":4671864964861835857, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.613979, "value1":0.9101669225413574, "value2":5476069179621211969, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079079, "value1":0.6285830126630634, "value2":1800090753316916399, "value3":true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990168, "value1":0.4228502254109105, "value2":4769618752656230984, "value3":true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630399, "value1":0.0593879368852638, "value2":8425926359679957872, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381514, "value1":0.3170447182462336, "value2":134775839032388004, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306533, "value1":0.09290124329204048, "value2":3878589204829724343, "value3":true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363988, "value1":0.7494592976903763, "value2":588119535129085693, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.726401, "value1":0.2843848594033847, "value2":797824336547840313, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.425192, "value1":0.6747059449207641, "value2":6618945540166900762, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486703, "value1":0.9234732601693532, "value2":7284867992059891857, "value3":false, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.162651, "value1":0.4998767239554866, "value2":3758664262196916666, "value3":true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638502, "value1":0.14222618431131662, "value2":7183266992692325102, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814797, "value1":0.04744514500578937, "value2":8807075724873574935, "value3":true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617563, "value1":0.7113712918981732, "value2":7369990382580999103, "value3":false, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391047, "value1":0.4667345592294561, "value2":2593217151341486604, "value3":false, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423210, "value1":0.2993388901152061, "value2":7736063295280636221, "value3":false, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.233156, "value1":0.2669621811468438, "value2":9163725732269873210, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780343, "value1":0.4836321272043857, "value2":6239122865745239252, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270794, "value1":0.8124904828437979, "value2":5664361933553425625, "value3":true, "__name__":"metric_00000676", "key_2":"k","key_5":"k","key_0":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.567390, "value1":0.1509148834766825, "value2":769811157679024909, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502511, "value1":0.3496329905819851, "value2":8064435486484275503, "value3":false, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.995318, "value1":0.07944294849213016, "value2":7290799161307742730, "value3":true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160027, "value1":0.9323372972664894, "value2":8694402707307208350, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265068, "value1":0.7804375599856387, "value2":2823069570137165166, "value3":true, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852790, "value1":0.04269382129150226, "value2":4247785862022220763, "value3":true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.813337, "value1":0.3041270110008107, "value2":1162341534698170174, "value3":true, "__name__":"metric_00000683", "key_6":"a","key_9":"c","key_1":"k", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821104, "value1":0.05863251524747595, "value2":5517633377182805397, "value3":false, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801029, "value1":0.7039209651517294, "value2":5777192245942090986, "value3":true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.292328, "value1":0.0789099713548181, "value2":4341141001609491414, "value3":true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.650140, "value1":0.6547556555322551, "value2":488100188330068974, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881952, "value1":0.6384170556890851, "value2":7393134503131526080, "value3":false, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019410, "value1":0.4521619097561246, "value2":2730473646049580719, "value3":false, "__name__":"metric_00000688", "key_7":"b","key_8":"c","key_2":"b", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413658, "value1":0.35836104075629194, "value2":5913006016396086189, "value3":false, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.388542, "value1":0.347328329631466, "value2":3415671885617145536, "value3":false, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908514, "value1":0.682932258250745, "value2":1535326911373174401, "value3":true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453424, "value1":0.011430118976046925, "value2":6615057602694952188, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661892, "value1":0.04490375141966574, "value2":3586691570128338490, "value3":true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598897, "value1":0.9840071764206562, "value2":6756637664735447989, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.614741, "value1":0.45336506256132186, "value2":7160462685010833845, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959437, "value1":0.33491802794663117, "value2":4263644093884006866, "value3":false, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394710, "value1":0.09508520954191016, "value2":8086669194950309707, "value3":true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.400389, "value1":0.748914928293926, "value2":7561318401530983897, "value3":true, "__name__":"metric_00000699", "key_2":"a","key_7":"c","key_0":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551520, "value1":0.29033239272123923, "value2":219973352444233301, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.433012, "value1":0.8026759332563937, "value2":1035833541375117050, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.097876, "value1":0.3123782080487283, "value2":97482352559342166, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971923, "value1":0.09350074090215915, "value2":6603209689656348295, "value3":true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385293, "value1":0.025071498526727982, "value2":7821611042801827653, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.582177, "value1":0.17895122462677535, "value2":4851272893829158392, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715577, "value1":0.12145793831580125, "value2":6773868244129033160, "value3":true, "__name__":"metric_00000706", "key_9":"d","key_5":"e","key_6":"h", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.245667, "value1":0.5083106874300092, "value2":5342790051223313569, "value3":true, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.191947, "value1":0.7213188287503117, "value2":8184500849109232134, "value3":false, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.527410, "value1":0.2848785581313827, "value2":8059297953437098619, "value3":true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023841, "value1":0.08963330011439892, "value2":8744048608508050406, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.861975, "value1":0.9569428255590289, "value2":5863440324678791040, "value3":true, "__name__":"metric_00000711", "key_4":"h","key_6":"i","key_0":"j", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.020139, "value1":0.5328792585389814, "value2":6851298079925452135, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383316, "value1":0.6736870667139099, "value2":1034982274545531879, "value3":true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490723, "value1":0.7069141316227696, "value2":1238860616186948989, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774619, "value1":0.15017350186839917, "value2":4145739713631541991, "value3":true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.724625, "value1":0.41136382513830944, "value2":2470375246542099625, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014004, "value1":0.45322459474776666, "value2":3424033124770728498, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370213, "value1":0.7936480778398273, "value2":7651653355869440262, "value3":true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930663, "value1":0.28375382304907193, "value2":4573094024293120951, "value3":true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.108477, "value1":0.2514498975622512, "value2":6565807048098733104, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318641, "value1":0.09067602148053568, "value2":2118726105929309608, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341849, "value1":0.5146408700742566, "value2":5544935195040202056, "value3":true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016279, "value1":0.2887329883061575, "value2":4016937139552222061, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696532, "value1":0.35345697419681527, "value2":7198819764217760607, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356409, "value1":0.6101596793669094, "value2":2338966817998610022, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882128, "value1":0.8757983585066079, "value2":1264113150578508254, "value3":true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214730, "value1":0.04033669277556926, "value2":2311999714610864651, "value3":true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125040, "value1":0.8510776359004306, "value2":9203972476999185975, "value3":false, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617466, "value1":0.4880211576700948, "value2":7876488006268555933, "value3":false, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.580824, "value1":0.738663404243217, "value2":6159559973328325759, "value3":false, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.894051, "value1":0.0394639970488642, "value2":477392450960926806, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112539, "value1":0.11952574252776728, "value2":2511563277039044978, "value3":true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.918404, "value1":0.30833608301592924, "value2":8563798643952041377, "value3":false, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385588, "value1":0.17478857683593713, "value2":6552271330519870932, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139243, "value1":0.42492669927426513, "value2":7691897138631144930, "value3":false, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715207, "value1":0.8647651137464607, "value2":8497862595581854195, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760625, "value1":0.24627316716260067, "value2":1409311771773591601, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.880305, "value1":0.7511958117841795, "value2":8663780143091433962, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.121986, "value1":0.503001087851081, "value2":86404765376501910, "value3":true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.378220, "value1":0.7154879418930685, "value2":4699613906263091859, "value3":false, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367776, "value1":0.3357733171654874, "value2":2109430818036060968, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051853, "value1":0.46132527877007656, "value2":1864815246368296251, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503984, "value1":0.5450705451757059, "value2":2199582755716092199, "value3":false, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270031, "value1":0.15857947720543178, "value2":5111440735129620852, "value3":false, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376563, "value1":0.08077432861716789, "value2":7964637453463198864, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017382, "value1":0.4697079883099492, "value2":6862028548685305626, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938781, "value1":0.3698227408392578, "value2":1642534448554432249, "value3":false, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.869838, "value1":0.4366942978360045, "value2":4007380928114996020, "value3":true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.309900, "value1":0.05509289000926218, "value2":8881026026944444737, "value3":true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321699, "value1":0.5508642725465765, "value2":1065357963175430930, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242134, "value1":0.05194281440569991, "value2":2955861771024227742, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394641, "value1":0.6802778418350491, "value2":7829903676952590571, "value3":false, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.011752, "value1":0.2679251298930048, "value2":46390372055260426, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156821, "value1":0.8759623901682069, "value2":6663991521677358276, "value3":true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621515, "value1":0.8461818248462066, "value2":4793482631115954367, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.906690, "value1":0.4454231810933155, "value2":1968123317376605778, "value3":false, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833508, "value1":0.5580322939662696, "value2":542437654648170086, "value3":false, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.864441, "value1":0.13709821675852493, "value2":3775094265545831122, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349796, "value1":0.5463888647509524, "value2":3843833923457334958, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210241, "value1":0.13053751777334896, "value2":2547919115148938351, "value3":false, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.534237, "value1":0.13104275846768723, "value2":2757438080188257397, "value3":false, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.479346, "value1":0.6936882347927468, "value2":3593937018182399578, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013374, "value1":0.647079383488511, "value2":6027310083597491507, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353768, "value1":0.5713686637635252, "value2":3396180021694291926, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704388, "value1":0.028813882790100154, "value2":6631675215659864441, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350375, "value1":0.09828506398906811, "value2":262432636815220332, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130984, "value1":0.05874581283090899, "value2":5455068607213223389, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269711, "value1":0.16897643510231944, "value2":2188619802248362391, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798909, "value1":0.49628972013165096, "value2":8871333460189078193, "value3":false, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314472, "value1":0.003776179814725882, "value2":5465069878443166467, "value3":true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841977, "value1":0.016019153138836187, "value2":5090055543864076033, "value3":true, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.431741, "value1":0.4503254906693805, "value2":3437452976536184232, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021144, "value1":0.7157770425720729, "value2":1627465028436054242, "value3":true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.492677, "value1":0.170722769795088, "value2":5333195898301269961, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.572483, "value1":0.07518223050679318, "value2":749837753058622914, "value3":false, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.070547, "value1":0.7819934575507779, "value2":1257841836293236196, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834097, "value1":0.7836504674507244, "value2":7822820024554629611, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.725415, "value1":0.6205675290840149, "value2":7546425291277331498, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756846, "value1":0.4577877981753711, "value2":521428696388054794, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551591, "value1":0.4656287234062602, "value2":3666151433771478256, "value3":true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367967, "value1":0.3303667849726779, "value2":1336037347437447384, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124888, "value1":0.9362913257587775, "value2":8565731764409689287, "value3":false, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356964, "value1":0.763103278688178, "value2":7384734667573077593, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.336557, "value1":0.18917586865284508, "value2":3164250330171533065, "value3":false, "__name__":"metric_00000786", "key_4":"e","key_1":"c", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947920, "value1":0.7279862820897369, "value2":5585150661666129475, "value3":false, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320214, "value1":0.02069659588289354, "value2":4096509130796828102, "value3":true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339884, "value1":0.3585015534256289, "value2":1745908303614221947, "value3":false, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736733, "value1":0.6300320540961489, "value2":4976479107859231308, "value3":true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843896, "value1":0.8618710521424175, "value2":409303658417169135, "value3":false, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351779, "value1":0.31536023315738654, "value2":7469097912657923197, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146297, "value1":0.47961745828082863, "value2":120290133475149079, "value3":false, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.299654, "value1":0.953430875530432, "value2":1804061300635059106, "value3":false, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.653880, "value1":0.6130300298494126, "value2":3600820249700194803, "value3":false, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.241886, "value1":0.4261506936601424, "value2":6410123065530378279, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.719862, "value1":0.5557094068088962, "value2":6619604443134241152, "value3":false, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192985, "value1":0.4372446351220488, "value2":5419318851099904514, "value3":false, "__name__":"metric_00000797", "key_5":"i","key_8":"b","key_0":"j", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.062254, "value1":0.35078106654250335, "value2":6337545142680530695, "value3":false, "__name__":"metric_00000795", "key_8":"i","key_9":"d","key_1":"f", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.826832, "value1":0.03911586835937395, "value2":8510739605077299393, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.466577, "value1":0.8386295972027331, "value2":3712956826274262169, "value3":false, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774816, "value1":0.2874652343932132, "value2":1661160295946815788, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497396, "value1":0.37671291600810797, "value2":7809177916397546184, "value3":false, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.061112, "value1":0.6756138122485794, "value2":3821150629117787840, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.711530, "value1":0.9726549946550717, "value2":4361793701491206585, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419723, "value1":0.8422780123510716, "value2":8387473970870580835, "value3":false, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740430, "value1":0.3687691277865257, "value2":49347396348935758, "value3":false, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544648, "value1":0.34026181496926355, "value2":3168616117094841931, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.031693, "value1":0.35157464321590376, "value2":6956034409157435335, "value3":true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682310, "value1":0.4968537230062955, "value2":1715909753935447521, "value3":false, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571570, "value1":0.7444971868018181, "value2":773947913317871760, "value3":true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412817, "value1":0.17790194335965473, "value2":453971816672635527, "value3":true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154029, "value1":0.695415820029077, "value2":4412491024371312379, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187573, "value1":0.025946190955721947, "value2":8268616070900349780, "value3":false, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297634, "value1":0.39036160577230816, "value2":9117936615987168462, "value3":true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882305, "value1":0.6498141826508745, "value2":1021392582780518091, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288985, "value1":0.6094023317572287, "value2":5522861781820654768, "value3":false, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.516532, "value1":0.4913308899910666, "value2":2877526282810678571, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.663856, "value1":0.6052287079408516, "value2":5244892044841001586, "value3":true, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177812, "value1":0.8518899507803316, "value2":4611087097036165557, "value3":false, "__name__":"metric_00000817", "key_5":"f","key_7":"d","key_3":"c", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789804, "value1":0.9253854154062459, "value2":4746855558380871130, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163177, "value1":0.054868844952025875, "value2":4671327502125519320, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753554, "value1":0.0480631824565462, "value2":7078677034582099346, "value3":false, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537888, "value1":0.3970428718849328, "value2":8317481519599523582, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887867, "value1":0.9625491057630078, "value2":851265403943296561, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739174, "value1":0.3876147534715181, "value2":5378120884849587893, "value3":true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263149, "value1":0.6578316627430515, "value2":199930804061246062, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926006, "value1":0.9018811245039401, "value2":8736925807617681839, "value3":false, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559532, "value1":0.017637548071686292, "value2":2677656350646305112, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.940793, "value1":0.36532273109204866, "value2":7505680103024683972, "value3":true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416010, "value1":0.7581313246147069, "value2":7561931289424116286, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130651, "value1":0.31761722951334315, "value2":7760092958155535983, "value3":false, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330390, "value1":0.3514672157667692, "value2":1963737467345840913, "value3":true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731094, "value1":0.5668209441148129, "value2":6834927574629705768, "value3":true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003464, "value1":0.44994944703983886, "value2":1268238147301227399, "value3":true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488880, "value1":0.15568001025824393, "value2":3802105015027414434, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099752, "value1":0.49578700566450296, "value2":6660233803101117286, "value3":true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131762, "value1":0.8279103415624635, "value2":8189053738835694283, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.818080, "value1":0.7683869146409037, "value2":4901733131698699140, "value3":false, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502240, "value1":0.2102196895388282, "value2":8419378719359168551, "value3":false, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130382, "value1":0.00017527290277822845, "value2":7209832064279367432, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697720, "value1":0.977742512542472, "value2":5886772442159349223, "value3":true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152164, "value1":0.5452125862262094, "value2":2301031936204454118, "value3":true, "__name__":"metric_00000843", "key_9":"i","key_0":"f", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307306, "value1":0.27827706723478657, "value2":8114605959354081308, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.687815, "value1":0.407696242982513, "value2":8251434258695514324, "value3":true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.957403, "value1":0.9248822561074296, "value2":5435740408050442827, "value3":false, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297405, "value1":0.735268728902142, "value2":6544542317901773513, "value3":false, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505182, "value1":0.6434041992526011, "value2":6723613820733288827, "value3":false, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953853, "value1":0.08239495901880306, "value2":7797622216144272838, "value3":true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.816223, "value1":0.026034555587018983, "value2":8247070392223609361, "value3":false, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781228, "value1":0.009350455430296063, "value2":6344110503212444401, "value3":false, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993318, "value1":0.7246275363827985, "value2":1117977296074531052, "value3":true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.346481, "value1":0.04745532854484892, "value2":8841440982349072666, "value3":true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240566, "value1":0.6244142217945766, "value2":4416869640928141403, "value3":false, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627039, "value1":0.8121528941309705, "value2":2364077225142023978, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.605014, "value1":0.041614775390040444, "value2":5587679092002855017, "value3":false, "__name__":"metric_00000854", "key_2":"f","key_4":"g","key_1":"d", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.675250, "value1":0.22265983684713628, "value2":852855496341453985, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.626117, "value1":0.13978101794521924, "value2":4898929345737475246, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.272053, "value1":0.8981952631418685, "value2":4155867998384044766, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216853, "value1":0.7930982218994412, "value2":1990843980684870249, "value3":true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231757, "value1":0.11381414833270587, "value2":5629821232198202573, "value3":true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967616, "value1":0.9325867021628451, "value2":6458012454076604601, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.105465, "value1":0.20026280910311015, "value2":1519145667363005058, "value3":true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959715, "value1":0.8692424311015335, "value2":3441946116265084781, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.916752, "value1":0.09650654250745302, "value2":7435925512873628832, "value3":true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314127, "value1":0.5339606052113064, "value2":8918601923652673265, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955659, "value1":0.21237559939985842, "value2":6700000271323408810, "value3":false, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.462571, "value1":0.8031758318689627, "value2":4200089724641662061, "value3":false, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704628, "value1":0.42388353536260726, "value2":1676230088334959718, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781871, "value1":0.7853951443498608, "value2":4677884824803639265, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093020, "value1":0.44592927039430036, "value2":5106772041668223293, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.066421, "value1":0.7167334468150509, "value2":916960599462889530, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753333, "value1":0.8855172765987326, "value2":3498076904045993942, "value3":true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263558, "value1":0.039242126444255614, "value2":1994687158267196250, "value3":false, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.529319, "value1":0.11390721956642542, "value2":6770255083967052135, "value3":true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.642302, "value1":0.25917237536080345, "value2":3561054032858646761, "value3":true, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.800222, "value1":0.5648139239977721, "value2":7859467413386922358, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.714908, "value1":0.3172565417584279, "value2":4018189432904936827, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495891, "value1":0.1699695672081134, "value2":2951696383925672800, "value3":true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.927247, "value1":0.5193998219352067, "value2":7461947996252867047, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465786, "value1":0.04561481590524083, "value2":8852624735734959732, "value3":true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970044, "value1":0.9502884615051707, "value2":5544298343674474830, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.224365, "value1":0.7835642667167426, "value2":7989661444310811565, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970171, "value1":0.044357973684653176, "value2":6240172232063272107, "value3":false, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.022705, "value1":0.2909464675793184, "value2":6791113566169749215, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.791002, "value1":0.30758766636926194, "value2":4087253744537420622, "value3":true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.526835, "value1":0.26218100233396685, "value2":761773481307699191, "value3":true, "__name__":"metric_00000882", "key_9":"c","key_3":"d","key_6":"a", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057757, "value1":0.5941262990661297, "value2":5203993881422592468, "value3":false, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621587, "value1":0.9726557103153585, "value2":4065263619832011048, "value3":true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936496, "value1":0.4220278511438201, "value2":537814119030374569, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476922, "value1":0.7453981222005723, "value2":3086149467770264289, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544208, "value1":0.1648752329298873, "value2":1528219915951406823, "value3":false, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480134, "value1":0.5864292944345278, "value2":1915038657623169872, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.634200, "value1":0.06085546420380519, "value2":1586772996290355012, "value3":true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.290157, "value1":0.4479966729250899, "value2":603667588545059551, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523397, "value1":0.8019975659364824, "value2":2263229113306312715, "value3":false, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260677, "value1":0.49446211956319075, "value2":5384294748496824064, "value3":true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.040570, "value1":0.2634688640512974, "value2":7081899407358309984, "value3":true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.670159, "value1":0.4242108328120063, "value2":1003697559007887859, "value3":true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.181699, "value1":0.7357308047023233, "value2":7674299813279760629, "value3":false, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.437960, "value1":0.34081697732260335, "value2":1366760825001014571, "value3":true, "__name__":"metric_00000899", "key_5":"f","key_0":"b","key_3":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585340, "value1":0.9880202469237767, "value2":5993013748473116382, "value3":true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758920, "value1":0.528974137063132, "value2":5387464413435497334, "value3":true, "__name__":"metric_00000900", "key_4":"c","key_7":"j","key_0":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190405, "value1":0.24309504115260316, "value2":8699961182370479292, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.045104, "value1":0.42560682798506067, "value2":3749517767528292571, "value3":false, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.239270, "value1":0.11837048427737668, "value2":4852849985587057450, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647950, "value1":0.24581023177518593, "value2":7946632180521405772, "value3":false, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969355, "value1":0.36754556888022905, "value2":7897267252653597746, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416692, "value1":0.9421287451192997, "value2":7881806374967586751, "value3":true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.052040, "value1":0.27660174901670026, "value2":5294738319793885165, "value3":false, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112174, "value1":0.8996109424352536, "value2":5125031699148488781, "value3":true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696416, "value1":0.42330986040349045, "value2":7061433961041368821, "value3":false, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127933, "value1":0.6743423459526591, "value2":5703788929519053213, "value3":false, "__name__":"metric_00000911", "key_9":"a","key_3":"k","key_4":"h", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421658, "value1":0.6878542321395752, "value2":3553548366690488868, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697493, "value1":0.169783750302659, "value2":7055001682108205827, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.550811, "value1":0.35231569219578235, "value2":1455572534223159241, "value3":false, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.493018, "value1":0.08084254027780423, "value2":2900266808751124367, "value3":true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949248, "value1":0.8847972628692381, "value2":4051840894997158475, "value3":true, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142802, "value1":0.3270345064861823, "value2":2685380632911712697, "value3":true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.004387, "value1":0.822755239334881, "value2":7165522026622587078, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620895, "value1":0.40285181555826477, "value2":7279952302942542040, "value3":true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.988553, "value1":0.42488091295486163, "value2":7195769969726296289, "value3":true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314916, "value1":0.24528560160904253, "value2":3412797507763244235, "value3":false, "__name__":"metric_00000921", "key_7":"b","key_3":"e","key_4":"e", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740841, "value1":0.7678978195926474, "value2":7353627394146413053, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767489, "value1":0.9849856324971001, "value2":4180092134684532856, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.681082, "value1":0.08407929345646925, "value2":2247545351887675892, "value3":false, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987608, "value1":0.3158564571415852, "value2":3372134575580371247, "value3":false, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953355, "value1":0.6808029749520165, "value2":6416401290847025754, "value3":false, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.934118, "value1":0.3815229371385279, "value2":5398231187188543220, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349522, "value1":0.008036202059570214, "value2":6632834525194225583, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852636, "value1":0.6082938917169154, "value2":2209452475312580755, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.491711, "value1":0.7070391677298287, "value2":5867779644024249420, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329284, "value1":0.23697436114335285, "value2":2747492895656100802, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217160, "value1":0.013558152161070156, "value2":543215008088311315, "value3":true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376202, "value1":0.3967940240541657, "value2":4348396634707940652, "value3":true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134419, "value1":0.2570940106942101, "value2":5495331795379375428, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.693581, "value1":0.6383501980446401, "value2":7332225459965791621, "value3":true, "__name__":"metric_00000934", "key_8":"i","key_7":"h", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843591, "value1":0.374550163160159, "value2":1957979143426115517, "value3":false, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127314, "value1":0.16562410193780275, "value2":5514782476287105847, "value3":false, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564210, "value1":0.47834402715548674, "value2":7864140566183633471, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.673777, "value1":0.656108656648855, "value2":6919590747679310147, "value3":false, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.071957, "value1":0.3923477172589352, "value2":7954657800672945455, "value3":true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497858, "value1":0.21899200976683347, "value2":5922902740218283725, "value3":false, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063776, "value1":0.819166087262725, "value2":497846112853863498, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.039234, "value1":0.06262728093114268, "value2":976242972769343091, "value3":true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658955, "value1":0.48196452599993433, "value2":2417102300700963835, "value3":true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255288, "value1":0.13639741045424703, "value2":7715812700116132347, "value3":true, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.496112, "value1":0.1915457705996175, "value2":7008066197109595718, "value3":false, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144334, "value1":0.011357361010319224, "value2":7600237475494790795, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.129843, "value1":0.45829887966275845, "value2":1540110795133843915, "value3":false, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.828068, "value1":0.8516996075016723, "value2":8405984298577097957, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870240, "value1":0.16467566190225996, "value2":1556056753506941522, "value3":false, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199050, "value1":0.7512917767548035, "value2":3201410658564101147, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830765, "value1":0.6115828253610872, "value2":1648965358200873623, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385351, "value1":0.44894635523974136, "value2":2127367263922899732, "value3":true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.555613, "value1":0.566989554108829, "value2":4275387093192470829, "value3":true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758445, "value1":0.45045350568682, "value2":6540058212119502734, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051410, "value1":0.07872483972360628, "value2":7661444664176600116, "value3":false, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.027756, "value1":0.1866583235803357, "value2":4402492796625360355, "value3":false, "__name__":"metric_00000957", "key_1":"i","key_5":"g","key_0":"c", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043704, "value1":0.3399629891033194, "value2":2843620218251838442, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.218394, "value1":0.6792705695737844, "value2":3717810518285734277, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.489979, "value1":0.2797515798899019, "value2":5603667040320572095, "value3":false, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394844, "value1":0.47076120967369356, "value2":2551470543073926967, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.176040, "value1":0.28649026862589944, "value2":8089352485313652968, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015062, "value1":0.8503661846365486, "value2":9115866206649167217, "value3":true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.867138, "value1":0.18222497145716, "value2":748036164126661849, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560015, "value1":0.16282129139411827, "value2":7758713833273415824, "value3":false, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782776, "value1":0.3488004874020465, "value2":715341283187680350, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.547186, "value1":0.07214699982761937, "value2":8071660169425611932, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383243, "value1":0.8304505195235525, "value2":4079430116382080627, "value3":true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551280, "value1":0.6676351538773236, "value2":4287835647581016602, "value3":true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.853493, "value1":0.3478301116617329, "value2":7815908195604525371, "value3":false, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511951, "value1":0.7049888722888338, "value2":1948135132689353800, "value3":false, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173192, "value1":0.5458720430431239, "value2":5871330428575335057, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413126, "value1":0.46553694703913845, "value2":7586060029018039815, "value3":false, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781439, "value1":0.705616760521343, "value2":1127274508185282406, "value3":false, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.732371, "value1":0.37205742400318453, "value2":5213749986182819618, "value3":false, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569115, "value1":0.03375202087271923, "value2":118632103331380976, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.102614, "value1":0.2822023902913289, "value2":1391807523431966730, "value3":true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140423, "value1":0.44611401204912965, "value2":828680952801675744, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.085991, "value1":0.7893667946565419, "value2":8411361994319110857, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.975471, "value1":0.7162325854468116, "value2":79462509030833666, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.244787, "value1":0.40407729683075916, "value2":2829176965864438571, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.304099, "value1":0.2959757161094293, "value2":5764137406549726855, "value3":false, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837106, "value1":0.4258452743236353, "value2":5341156407592196309, "value3":true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707606, "value1":0.031525280019894485, "value2":868621351492758022, "value3":false, "__name__":"metric_00000984", "key_9":"g","key_0":"k","key_3":"f", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441177, "value1":0.3225860911008584, "value2":3070966375799941978, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.077150, "value1":0.044429243112689185, "value2":8848387799149411370, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604969, "value1":0.8353491972869558, "value2":5277008629054064552, "value3":false, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537157, "value1":0.4103644112452139, "value2":9148697464094805661, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353178, "value1":0.8020014291715372, "value2":611786907029670311, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505264, "value1":0.10898025782239415, "value2":5937127177824419205, "value3":true, "__name__":"metric_00000989", "key_2":"h","key_7":"f","key_0":"c", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.228410, "value1":0.43346447533111176, "value2":7102784791187505051, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321463, "value1":0.9635846235850585, "value2":7091075060854093514, "value3":false, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.999019, "value1":0.15889354502430938, "value2":7074771352556045066, "value3":false, "__name__":"metric_00000992", "key_5":"d","key_6":"a","key_4":"k", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.766619, "value1":0.4943151198279696, "value2":6293058242069710093, "value3":true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382183, "value1":0.47579561094609324, "value2":2482575383850980092, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.054382, "value1":0.7191240626399533, "value2":7914822293313169877, "value3":true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255873, "value1":0.055171339528865125, "value2":1928364223437432733, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.535958, "value1":0.40417404005201557, "value2":8512533852699686666, "value3":true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394085, "value1":0.8317795195864898, "value2":6061937875766891692, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.752931, "value1":0.1811338646540346, "value2":3103290849364510613, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.049507, "value1":0.11124247410560267, "value2":8253232345223333946, "value3":false, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.067068, "value1":0.02792863936331565, "value2":1942738411033963266, "value3":true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746589, "value1":0.43603502537000616, "value2":5153565802559198697, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903064, "value1":0.3556713649253779, "value2":3489353708315351817, "value3":false, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.856458, "value1":0.28933348763855665, "value2":5247367677027973150, "value3":false, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397179, "value1":0.7130148119357395, "value2":5728198037062323847, "value3":true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.927222, "value1":0.7639936733901646, "value2":101990787192301191, "value3":false, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005808, "value1":0.03609120161396754, "value2":9216428818715898202, "value3":true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812165, "value1":0.4866254594918793, "value2":3026112033227979237, "value3":false, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420851, "value1":0.7402262167603453, "value2":6177484091405440598, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836705, "value1":0.0336701243488239, "value2":4778937227074309153, "value3":true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097260, "value1":0.6626429057561898, "value2":7257306969130399155, "value3":true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496088, "value1":0.36910009322506554, "value2":3657880390271983286, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870639, "value1":0.7067162570632016, "value2":1102103625009499222, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.020747, "value1":0.46587247219742656, "value2":5967339025562916268, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.724139, "value1":0.8163768770313428, "value2":2067500216739340556, "value3":false, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.369554, "value1":0.7671983408631192, "value2":4937837819229078201, "value3":false, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382589, "value1":0.94426076786917, "value2":575895696952869704, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115327, "value1":0.47575759482061636, "value2":2541949700573197783, "value3":false, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260734, "value1":0.8252666210219215, "value2":5316874961818228049, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.872358, "value1":0.8794099168533623, "value2":5800876034859982246, "value3":false, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.054141, "value1":0.05686166977091221, "value2":5301631175885078161, "value3":true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759774, "value1":0.8076817854451119, "value2":2675452361460215935, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921855, "value1":0.0467410376485931, "value2":7851926061831999904, "value3":false, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578515, "value1":0.9029496688391901, "value2":3737429676030030627, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812804, "value1":0.47887479306222563, "value2":1687173944192158201, "value3":false, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472625, "value1":0.8441160499425835, "value2":1515286134892734672, "value3":false, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760814, "value1":0.33274037254007843, "value2":1188458814270546919, "value3":true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788501, "value1":0.15942957594654944, "value2":2916736695606434573, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098854, "value1":0.8030098284257553, "value2":2336601872331852302, "value3":true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894386, "value1":0.4139981184160349, "value2":7897864995270774175, "value3":true, "__name__":"metric_00000032", "key_9":"k","key_5":"b","key_8":"g", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117046, "value1":0.5674622805765219, "value2":4791162537839256650, "value3":false, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.542843, "value1":0.19807908776500033, "value2":6807107659814067311, "value3":true, "__name__":"metric_00000033", "key_1":"h","key_9":"f","key_0":"i", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703834, "value1":0.5788042510612239, "value2":5007564675868943302, "value3":true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945291, "value1":0.3701740719620502, "value2":2459541142030887319, "value3":false, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070410, "value1":0.2627776276203505, "value2":9110732671786561088, "value3":false, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.305776, "value1":0.24403108521016115, "value2":3139654072672113070, "value3":true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.417910, "value1":0.549110839930364, "value2":2406040915101429610, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410216, "value1":0.8784609507591513, "value2":3467112924963980779, "value3":false, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186300, "value1":0.13500098796508653, "value2":4869858265003017538, "value3":false, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934357, "value1":0.7572145499314314, "value2":3619701318200032332, "value3":true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871110, "value1":0.4402673304510715, "value2":5018886154821992035, "value3":false, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.493200, "value1":0.1348835909121323, "value2":3845890133642201057, "value3":false, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875968, "value1":0.1364875892800979, "value2":570804136752641105, "value3":true, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.469938, "value1":0.5776925075890766, "value2":8313002338216576665, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.667608, "value1":0.9950302066492306, "value2":1913698187097493343, "value3":false, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497144, "value1":0.7591363060077458, "value2":4304322015184203986, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731147, "value1":0.743025674257032, "value2":3480752580915944185, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878760, "value1":0.5947269209561762, "value2":7998247484475194678, "value3":true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.606309, "value1":0.663213778377581, "value2":3067341192985682212, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.207494, "value1":0.8601914108563131, "value2":256790036338793553, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.144417, "value1":0.30286956006124033, "value2":5377965606661311252, "value3":false, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.134377, "value1":0.609971735912583, "value2":3774215546443285103, "value3":false, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.434483, "value1":0.986170039019965, "value2":5064754870308124270, "value3":true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.509040, "value1":0.5522321797777098, "value2":5012812814115566068, "value3":false, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903746, "value1":0.20330422291674133, "value2":6986910292987554191, "value3":false, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282547, "value1":0.04694512830804489, "value2":6329857494201458927, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354275, "value1":0.16833606088021735, "value2":3136536256813456866, "value3":true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774256, "value1":0.5410085627082251, "value2":3587475793865363544, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.762751, "value1":0.806841924275979, "value2":7358998298622408616, "value3":true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584082, "value1":0.6563492986452016, "value2":387593409104388354, "value3":true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.080055, "value1":0.5509982288474451, "value2":6251069533288673484, "value3":true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695506, "value1":0.8632479367125148, "value2":2061552818267451584, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489099, "value1":0.6265315870043634, "value2":4304737020249244240, "value3":true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098523, "value1":0.5339406052485941, "value2":4381041775949231103, "value3":false, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945882, "value1":0.23785010902188283, "value2":6729562900414289120, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.167154, "value1":0.3701120650676118, "value2":622381534969203518, "value3":true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775199, "value1":0.6184507954864329, "value2":5210419683928766855, "value3":true, "__name__":"metric_00000068", "key_9":"j","key_3":"f","key_4":"g", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.730074, "value1":0.38121631002699524, "value2":419350096079979395, "value3":false, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.013737, "value1":0.8747949109587233, "value2":1018960453244065142, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.091852, "value1":0.629803885859078, "value2":859062102274314528, "value3":true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166974, "value1":0.47395530253799945, "value2":6411055007256508887, "value3":true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.524472, "value1":0.9186585978945081, "value2":3492305084628232737, "value3":true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241236, "value1":0.3787867675534582, "value2":4760531059679736084, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918712, "value1":0.05290704362542312, "value2":1966436853098488271, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636045, "value1":0.9510956232200477, "value2":8704302710827007347, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187946, "value1":0.6635616634234764, "value2":1433605612315108919, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180096, "value1":0.173261531213523, "value2":3306945339384428282, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.453459, "value1":0.11094008182846338, "value2":5724966973695631624, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562127, "value1":0.3576118834730829, "value2":3463667757785807622, "value3":true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137341, "value1":0.6255329096036213, "value2":1381955744525597553, "value3":false, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913181, "value1":0.29162415237583095, "value2":2202030046648119531, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.443734, "value1":0.3209051100037936, "value2":3274712051745788108, "value3":false, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.302887, "value1":0.38589486316701666, "value2":1489273904023316120, "value3":true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205658, "value1":0.07820373463872678, "value2":7665901880569799873, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.211623, "value1":0.32477633025580727, "value2":6808357456829544888, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.697819, "value1":0.26244647649977987, "value2":6232754250974380613, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853360, "value1":0.3310597786987246, "value2":7565924568483154342, "value3":true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.135295, "value1":0.049613381396899345, "value2":6155724101399152110, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701749, "value1":0.9716574010135762, "value2":3704214151049295904, "value3":false, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527554, "value1":0.6532141955951636, "value2":2786469958733724605, "value3":false, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866043, "value1":0.03422399382505321, "value2":5496039574170303988, "value3":true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.415307, "value1":0.682109162147481, "value2":4924263240012975638, "value3":false, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631078, "value1":0.7274897566111301, "value2":8520095366826456731, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.549725, "value1":0.7077865773466269, "value2":2977543758339035237, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.154150, "value1":0.14439730391589514, "value2":7810498955756317569, "value3":false, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007570, "value1":0.8087681893470107, "value2":4017899853890170979, "value3":false, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.055629, "value1":0.44977291321386215, "value2":3453485470431150033, "value3":false, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189056, "value1":0.012064619774392913, "value2":3564594888238224552, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.545459, "value1":0.412297537176583, "value2":7021178975362987850, "value3":true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.116215, "value1":0.3276649448479836, "value2":3143364922694929733, "value3":true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529456, "value1":0.4708787441244093, "value2":2999813825732785635, "value3":true, "__name__":"metric_00000102", "key_8":"d","key_1":"i", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.966668, "value1":0.04918146120370962, "value2":870818154766064140, "value3":false, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601524, "value1":0.4723442759712491, "value2":5773596887056998694, "value3":false, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712431, "value1":0.04260252713403969, "value2":4497318416983710614, "value3":true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452968, "value1":0.03699415334331326, "value2":3190104546355404213, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.394341, "value1":0.05586282401938057, "value2":2019354346274899062, "value3":true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.084367, "value1":0.35383668982583555, "value2":14041413767842483, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227017, "value1":0.7278732120270098, "value2":8901905120302064125, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494493, "value1":0.0910027308098097, "value2":3614458258424175747, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059547, "value1":0.2958045744232146, "value2":5813283620430694858, "value3":true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.423389, "value1":0.9221509815116762, "value2":6742495498273189642, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501855, "value1":0.44198471706869047, "value2":7662744388645110665, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293650, "value1":0.512834114916101, "value2":5984393328938374467, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614743, "value1":0.8714098760733937, "value2":937734757736172502, "value3":false, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406459, "value1":0.17711840072487878, "value2":749048739837121472, "value3":false, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711112, "value1":0.5465711310503617, "value2":2943773951314932011, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335697, "value1":0.8515828942170133, "value2":7615557480020888685, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548405, "value1":0.13026413723454033, "value2":4513263208882824306, "value3":true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.351448, "value1":0.28264554746804893, "value2":1701229548455066314, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501558, "value1":0.7474271542447126, "value2":4300562906938627060, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107125, "value1":0.5220893826927367, "value2":3433346642469038441, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968552, "value1":0.32921550916944375, "value2":4204134061287881453, "value3":false, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.438315, "value1":0.5786006279022659, "value2":7016921001360584415, "value3":false, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.834753, "value1":0.15159677478724748, "value2":3404421917920731295, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454269, "value1":0.5292362536342466, "value2":6241633082661695873, "value3":false, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137463, "value1":0.4976726181585776, "value2":7634359338923847776, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744484, "value1":0.7709542443985684, "value2":2984932547540133289, "value3":false, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.653649, "value1":0.40168424113567774, "value2":9208485552143870835, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420303, "value1":0.8834331384252292, "value2":6602388659890449882, "value3":false, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.995276, "value1":0.35489802779155194, "value2":3627299399432728520, "value3":false, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.609717, "value1":0.32106453173709887, "value2":6997589541390821763, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.764384, "value1":0.448976213692369, "value2":2970007228856945436, "value3":true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548393, "value1":0.15204462152678552, "value2":7646037909450468928, "value3":false, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379101, "value1":0.7807002939639928, "value2":783999596388900221, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.278309, "value1":0.014487780492841778, "value2":1057643982300790379, "value3":false, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.016491, "value1":0.40903973612790867, "value2":453932377401006766, "value3":true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347001, "value1":0.5622168932310977, "value2":5705011066230310463, "value3":true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.086911, "value1":0.6494793319729653, "value2":861844073039964568, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.997904, "value1":0.8338228010696771, "value2":1349114457177401043, "value3":false, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266213, "value1":0.43023507437229846, "value2":2397780341037792730, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068462, "value1":0.6058806411980994, "value2":78381759723174420, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358282, "value1":0.2854157607268251, "value2":9047800747480831869, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845468, "value1":0.009988708005244632, "value2":2440593977353687165, "value3":true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780415, "value1":0.6187822246822943, "value2":4482602300599359335, "value3":true, "__name__":"metric_00000145", "key_1":"g","key_4":"k","key_0":"c", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700223, "value1":0.018684194191095932, "value2":6529685586040726217, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.792345, "value1":0.9111940168010332, "value2":1575039147844860051, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275696, "value1":0.01410734034670126, "value2":8535583193313743852, "value3":false, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458977, "value1":0.23636701604224677, "value2":314843807550918715, "value3":true, "__name__":"metric_00000148", "key_9":"d","key_3":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711597, "value1":0.1315269488618045, "value2":914748291669479577, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003734, "value1":0.48566417416368673, "value2":5809881749485776988, "value3":false, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.587116, "value1":0.411572893699433, "value2":582776826400065747, "value3":true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000207, "value1":0.25509895950578154, "value2":5802214523620869062, "value3":false, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082969, "value1":0.48435989736978513, "value2":7352735247711683766, "value3":false, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.042350, "value1":0.16157141460949973, "value2":1607874054697622469, "value3":true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.765374, "value1":0.15664293959132114, "value2":3675493379037113802, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.359224, "value1":0.887036256695556, "value2":7786059543942584229, "value3":true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293533, "value1":0.267785564160632, "value2":2723125994900979360, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964294, "value1":0.716239726237387, "value2":8651041612345523274, "value3":true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076213, "value1":0.9749455558859894, "value2":5297461003834502762, "value3":false, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.896241, "value1":0.7102104646771853, "value2":2795778846362770396, "value3":true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903120, "value1":0.14414851964547873, "value2":7334142904098197, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.843431, "value1":0.8361555205773405, "value2":7976205336110881786, "value3":true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731615, "value1":0.4179919061811569, "value2":1554283199870929425, "value3":true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.953374, "value1":0.7537780463182495, "value2":2591672750977463865, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400999, "value1":0.7471726171159019, "value2":7300350647929857982, "value3":false, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232296, "value1":0.7720730243870363, "value2":1060384392061020703, "value3":true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219798, "value1":0.16805543570986609, "value2":1462620146278820176, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364330, "value1":0.7255148218772242, "value2":713753994079127610, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152830, "value1":0.13410318940561705, "value2":9140011796352402602, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585566, "value1":0.4130103851752984, "value2":2970454230901035575, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.718094, "value1":0.5500539331076559, "value2":8305617631847453693, "value3":false, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.624058, "value1":0.9500416769324029, "value2":2935568409193402069, "value3":false, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539303, "value1":0.05177136725388866, "value2":7442175860869173746, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.172986, "value1":0.4456449462659514, "value2":137655062794989901, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631420, "value1":0.9356892224700029, "value2":8946945917645913041, "value3":true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948627, "value1":0.9988907819529511, "value2":7978995836658846886, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946323, "value1":0.5613474012283862, "value2":648588536937617868, "value3":true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361054, "value1":0.4585620050578102, "value2":5573215420419545492, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.817729, "value1":0.3241298418342054, "value2":2657202509964663855, "value3":false, "__name__":"metric_00000179", "key_6":"i","key_7":"h","key_2":"b", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713670, "value1":0.7338301625220077, "value2":7217765288314019300, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.521076, "value1":0.5030076209647331, "value2":3777966789579493364, "value3":true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.855317, "value1":0.7172152098421418, "value2":8060266815999977878, "value3":true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849239, "value1":0.9561486509896251, "value2":266727525021228946, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008199, "value1":0.11740640397171373, "value2":3689690733597943394, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936773, "value1":0.795334335887194, "value2":4733404788345141419, "value3":true, "__name__":"metric_00000185", "key_5":"d","key_7":"k","key_1":"g", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888974, "value1":0.9298030625919304, "value2":6088282595701162644, "value3":true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247054, "value1":0.1558941325474189, "value2":460928608228918469, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853182, "value1":0.8511278083029166, "value2":5598797292292719359, "value3":false, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.252753, "value1":0.16156992511009938, "value2":4368635811765896864, "value3":true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309449, "value1":0.5849973411963314, "value2":683197153873995001, "value3":true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.222187, "value1":0.6637363404506509, "value2":201431956551451886, "value3":false, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805299, "value1":0.03082292423828661, "value2":3135350466088347105, "value3":false, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489193, "value1":0.47023015827549575, "value2":301183049195776618, "value3":true, "__name__":"metric_00000194", "key_6":"b","key_1":"b","key_2":"c", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.058774, "value1":0.7620086190415307, "value2":2178333587571299612, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098933, "value1":0.696442402358351, "value2":427216344556980093, "value3":true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497026, "value1":0.18415993654905655, "value2":8786493859199944811, "value3":true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257640, "value1":0.6674105512961352, "value2":395753979079709412, "value3":true, "__name__":"metric_00000197", "key_9":"g","key_1":"d","key_4":"d", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490944, "value1":0.5114071451614219, "value2":2042645288775439509, "value3":true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455445, "value1":0.7431063214848364, "value2":8165863124378358117, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.796800, "value1":0.6414858212054014, "value2":2442066548061613486, "value3":true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616766, "value1":0.17517079750483502, "value2":2134452812266479738, "value3":true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060207, "value1":0.5103041206672632, "value2":6515531846051600666, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677324, "value1":0.2542456767596465, "value2":3968114223247313347, "value3":false, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.487523, "value1":0.6903207412936282, "value2":1424416428597884416, "value3":false, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601220, "value1":0.7953800597000228, "value2":3247496941215159385, "value3":false, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.642959, "value1":0.2541441152051556, "value2":7121652419723619875, "value3":true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268832, "value1":0.025435318199453064, "value2":8277309445690585191, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591576, "value1":0.98502525758626, "value2":3261712292707439368, "value3":false, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696131, "value1":0.602844278741062, "value2":9150721424292798196, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.799050, "value1":0.6397730903214256, "value2":322841440821332477, "value3":false, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703698, "value1":0.7179811740811757, "value2":6911091771320522730, "value3":false, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.276968, "value1":0.16623158682521033, "value2":5763688015485978321, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.932237, "value1":0.9953773456479392, "value2":1648531648235955285, "value3":true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562810, "value1":0.4786357292651346, "value2":3349388012721867031, "value3":true, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496221, "value1":0.10819101220823944, "value2":5211051877655840528, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720431, "value1":0.552698162040604, "value2":7021843927329324470, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.087213, "value1":0.8472571863880398, "value2":7745061885728922306, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.039167, "value1":0.21533794157440472, "value2":3121229544727240655, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309356, "value1":0.1590321315931928, "value2":710479249184112134, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.191661, "value1":0.040375310871212516, "value2":1055365248220450266, "value3":false, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189905, "value1":0.8222460270476257, "value2":9109111690041797580, "value3":true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517425, "value1":0.7168387126170568, "value2":6945048447223853682, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916881, "value1":0.37193142727878464, "value2":2747126651524473231, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.395094, "value1":0.6414572510838905, "value2":2905014786439561176, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.163189, "value1":0.7311236407483467, "value2":5070208982215207577, "value3":false, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827333, "value1":0.19235777061627696, "value2":1042609530408390018, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107479, "value1":0.11539837467269841, "value2":7629188841800355160, "value3":false, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186100, "value1":0.23799692868617986, "value2":3114291156883869525, "value3":false, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790442, "value1":0.4974549500784167, "value2":3142531162087829163, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.797183, "value1":0.252808246395566, "value2":7867371410034903023, "value3":true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269019, "value1":0.4459897405815389, "value2":5769972800266784623, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299688, "value1":0.7900084672305991, "value2":4400086899085085411, "value3":true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.295251, "value1":0.4992500267841033, "value2":8698415822608825668, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.992903, "value1":0.9703688948654993, "value2":699484569689785049, "value3":false, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694010, "value1":0.24590491974037657, "value2":6378096972799959498, "value3":true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916779, "value1":0.03667685970441408, "value2":8236016681410946158, "value3":false, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.018930, "value1":0.7762719211488223, "value2":5653992369569699642, "value3":true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673243, "value1":0.9005498092358467, "value2":2541259280078576557, "value3":false, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.403564, "value1":0.7066442393226117, "value2":7468478788293853391, "value3":true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912162, "value1":0.4479784585493673, "value2":2451129253364350697, "value3":false, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622846, "value1":0.6985884362308626, "value2":1822728881940552123, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707469, "value1":0.052057120254889885, "value2":640515677830640949, "value3":true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433576, "value1":0.4097754058261397, "value2":2475392911974322330, "value3":false, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374459, "value1":0.1353067477342801, "value2":7368967313707141232, "value3":false, "__name__":"metric_00000245", "key_7":"k","key_3":"g","key_5":"g", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993651, "value1":0.06711618906287087, "value2":1750993237587336816, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219084, "value1":0.5316119999916927, "value2":5790597415731545344, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527236, "value1":0.6470232539834205, "value2":5706809590317379695, "value3":false, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566174, "value1":0.9745838789323467, "value2":774234656544468494, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803752, "value1":0.8708197206158679, "value2":3775873873449515760, "value3":true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.388879, "value1":0.4862649086023325, "value2":8962983929778901050, "value3":true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471944, "value1":0.25578979603582636, "value2":7732635688969517720, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017485, "value1":0.4968624551232297, "value2":202940911271524303, "value3":true, "__name__":"metric_00000254", "key_8":"c","key_1":"f","key_3":"b", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.333780, "value1":0.8299780273551013, "value2":895214629529935810, "value3":true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.164082, "value1":0.9321061863901389, "value2":7407307236544459993, "value3":true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225068, "value1":0.9163062543062077, "value2":8739271132887488724, "value3":true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304547, "value1":0.45077586944816295, "value2":1933302507685789986, "value3":false, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.538110, "value1":0.4128481092985977, "value2":3178182543294139274, "value3":true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541173, "value1":0.7483924955860858, "value2":212944257013368531, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255386, "value1":0.14438260405328707, "value2":7645966424158688784, "value3":true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.073000, "value1":0.7628193459385832, "value2":4973597494190231390, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408853, "value1":0.5564603800067697, "value2":4401411283807625564, "value3":true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447512, "value1":0.8939878640344981, "value2":2693396599053914337, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048017, "value1":0.34230394362828026, "value2":3182689568942606164, "value3":true, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057151, "value1":0.8042880864594656, "value2":2051969816473375198, "value3":false, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746476, "value1":0.7332341667442952, "value2":461488002348760082, "value3":false, "__name__":"metric_00000267", "key_6":"d","key_1":"i","key_4":"h", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180890, "value1":0.31049348146155, "value2":1020023460971269956, "value3":true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550108, "value1":0.4362790210713837, "value2":7901529415931806303, "value3":false, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345598, "value1":0.11823802615751057, "value2":8356166230167153323, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804625, "value1":0.5643553567149021, "value2":2154734444228835687, "value3":true, "__name__":"metric_00000268", "key_8":"c","key_9":"d","key_6":"j", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247313, "value1":0.6975496564473892, "value2":3820052334665018388, "value3":true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805735, "value1":0.31084346436090265, "value2":4462974662487547374, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364439, "value1":0.1744566217814337, "value2":607910964577919864, "value3":true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589313, "value1":0.37010824428777245, "value2":5796079422541877022, "value3":true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931293, "value1":0.5772458152930102, "value2":1027336122468786461, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527679, "value1":0.2977670251954975, "value2":3769527680254605539, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.404475, "value1":0.38496439646172187, "value2":2284351654690866371, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772590, "value1":0.07775425303960136, "value2":8477700234367515830, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722168, "value1":0.17927054370632736, "value2":7192360571641392544, "value3":true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712775, "value1":0.576069645932529, "value2":4594581975868935835, "value3":false, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.126652, "value1":0.5061094013712955, "value2":3728550795113093627, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321145, "value1":0.13357161718693558, "value2":588806792143157793, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.101550, "value1":0.6394371793311785, "value2":3045802217092318405, "value3":true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.547834, "value1":0.5303880277608923, "value2":244812322497139670, "value3":true, "__name__":"metric_00000287", "key_5":"f","key_6":"f","key_2":"k", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.022447, "value1":0.813105541632356, "value2":4932627202785941294, "value3":false, "__name__":"metric_00000285", "key_8":"b","key_9":"k","key_5":"b", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382289, "value1":0.3508103161551219, "value2":2376325264202595095, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.611464, "value1":0.9618639190404236, "value2":2197184281244482985, "value3":false, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.911985, "value1":0.3625352195059055, "value2":211893969967676184, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268188, "value1":0.6899041268799436, "value2":3263903700628732234, "value3":true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229932, "value1":0.7830409765107325, "value2":3581295224187821255, "value3":true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921675, "value1":0.15132831757259205, "value2":8941349375775958121, "value3":true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.738768, "value1":0.28818505448700765, "value2":5311365998633458584, "value3":true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.670459, "value1":0.7891122563117983, "value2":5569764792017894880, "value3":false, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.893664, "value1":0.863874518524225, "value2":4812640494718764785, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484101, "value1":0.6065100518640419, "value2":3350972220626617309, "value3":false, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048572, "value1":0.15248106917852358, "value2":6593923637797377418, "value3":false, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.043823, "value1":0.6456350127530551, "value2":3888964250592362865, "value3":false, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109495, "value1":0.6845812758644145, "value2":167613517010834758, "value3":true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.328450, "value1":0.42643027428907515, "value2":8129659398848977292, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.429304, "value1":0.14015794848590404, "value2":4681962038728020905, "value3":true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.776267, "value1":0.3289665984746857, "value2":3318083720052608693, "value3":true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496512, "value1":0.3624675397742157, "value2":6289410424653796398, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441324, "value1":0.9094233421689666, "value2":2912289985138568601, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.650459, "value1":0.3752516427539381, "value2":4398293649161961566, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.350383, "value1":0.1741603700667637, "value2":3124160867383954120, "value3":true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.051974, "value1":0.5242492694985263, "value2":6760802540060503552, "value3":false, "__name__":"metric_00000306", "key_9":"i","key_5":"d","key_8":"f", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.291840, "value1":0.5576830705012547, "value2":4155187805029620559, "value3":true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853505, "value1":0.13344813988799406, "value2":7220245991757698014, "value3":false, "__name__":"metric_00000308", "key_4":"b","key_1":"f","key_2":"f", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.480822, "value1":0.34390971406763976, "value2":4047856794594370906, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192205, "value1":0.46496172552445103, "value2":2904944897204510913, "value3":false, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668616, "value1":0.9800566015891223, "value2":3768818727218430516, "value3":true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.940469, "value1":0.8502278446477077, "value2":5643864232780485221, "value3":false, "__name__":"metric_00000312", "key_4":"c","key_9":"i","key_0":"a", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775796, "value1":0.26560945264856717, "value2":4386556505603475589, "value3":false, "__name__":"metric_00000313", "key_4":"c","key_9":"g","key_2":"f", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484224, "value1":0.10363743205437767, "value2":7594526116903338826, "value3":true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.889136, "value1":0.4627114619785784, "value2":4680730415934145352, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739229, "value1":0.1719548250883568, "value2":6803717844294234894, "value3":true, "__name__":"metric_00000316", "key_8":"i","key_4":"f","key_6":"k", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057765, "value1":0.3081555912049451, "value2":8499651141401941047, "value3":true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.450693, "value1":0.8168675860307004, "value2":7949382588313472478, "value3":true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770167, "value1":0.272960033553786, "value2":1543554906153054869, "value3":true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411881, "value1":0.5252975573521564, "value2":7223448923269287286, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.892105, "value1":0.15152104472471403, "value2":3531358618763240055, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.882618, "value1":0.45516649656010977, "value2":2288125701245215412, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913392, "value1":0.6620512228658078, "value2":4363674990969315932, "value3":false, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.835393, "value1":0.7513525395048825, "value2":5883431192279251653, "value3":true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.563057, "value1":0.9008981984383352, "value2":6465724162584986904, "value3":false, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007001, "value1":0.9502622268086611, "value2":6989543064471041687, "value3":false, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688255, "value1":0.07451084769674933, "value2":2957263551734111550, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751629, "value1":0.2617947338456188, "value2":684618542420066244, "value3":false, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930787, "value1":0.6308597904360509, "value2":6601900249320434792, "value3":true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.363243, "value1":0.8139409689083822, "value2":1497160325123262680, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676085, "value1":0.9711732869605988, "value2":1463546357374490230, "value3":true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.161858, "value1":0.6374277110955211, "value2":6435065381978315913, "value3":false, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757941, "value1":0.910443361797238, "value2":2854035357843167779, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.331899, "value1":0.6895825352637633, "value2":621471764809456095, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473746, "value1":0.2844187612168192, "value2":5704986401969616545, "value3":true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048060, "value1":0.9984260267119236, "value2":4771344600707329825, "value3":false, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.030112, "value1":0.9637634957448155, "value2":5568130719531472451, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257160, "value1":0.5940271387669745, "value2":142865511754737819, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760326, "value1":0.4334581523321684, "value2":6413870941968139402, "value3":true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827687, "value1":0.8136794802681319, "value2":746965665674196204, "value3":true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447680, "value1":0.3168283761550826, "value2":7144161377877398366, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690596, "value1":0.3996428785716089, "value2":3598801694226963836, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.009204, "value1":0.21753660582669873, "value2":1790271338088923331, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.432520, "value1":0.8760470338201594, "value2":2329654518443264597, "value3":false, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558450, "value1":0.5241764449812006, "value2":8062760419700144076, "value3":true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.999165, "value1":0.4211253053940594, "value2":6241537222130424287, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.492382, "value1":0.19419908429465416, "value2":8132562796409382183, "value3":true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968488, "value1":0.8740668460370395, "value2":5462263580744941212, "value3":false, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566020, "value1":0.3116629548456918, "value2":1660846180499855539, "value3":true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899292, "value1":0.4455952655758791, "value2":5641720897227183391, "value3":true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213999, "value1":0.7107285930376039, "value2":320006539559036783, "value3":false, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177859, "value1":0.6189254938745993, "value2":3916861895859871884, "value3":false, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.942059, "value1":0.3464916414346185, "value2":1739217875853687994, "value3":false, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637964, "value1":0.4027137396066144, "value2":498622351647583844, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994610, "value1":0.7530828834500288, "value2":5890892402021233384, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.727487, "value1":0.28182955033112955, "value2":4630833361364112333, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.915590, "value1":0.5470194104324091, "value2":1666728635470642191, "value3":true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757932, "value1":0.5371051329548212, "value2":1493513093655584392, "value3":true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.254302, "value1":0.06977656180754654, "value2":5979425073849108121, "value3":false, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.475877, "value1":0.8550799756217883, "value2":624522672976326814, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247089, "value1":0.39743079305559637, "value2":4830779617580090488, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015595, "value1":0.2577347123153436, "value2":2406835300488043720, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945719, "value1":0.3129335496013211, "value2":820286347352483926, "value3":false, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081969, "value1":0.6051672450326654, "value2":8261558360059795606, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948207, "value1":0.7815897187663656, "value2":3429909037501105433, "value3":false, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.935696, "value1":0.909809980891043, "value2":6385383912360129741, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.248852, "value1":0.9247408795385116, "value2":9175721939467544545, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934738, "value1":0.6334884200328613, "value2":327826698570505025, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102837, "value1":0.44571438142580455, "value2":4747306485524936018, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.885921, "value1":0.35994074928816305, "value2":1395907621788196782, "value3":false, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.949368, "value1":0.7895021421389461, "value2":7949753331449968910, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.633040, "value1":0.5458972679689471, "value2":4245189254820710416, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.150358, "value1":0.7548359500337994, "value2":8481176564399105584, "value3":true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.557167, "value1":0.37427466994480085, "value2":1133639012241894316, "value3":true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.093134, "value1":0.29043113295756745, "value2":6947797053182077820, "value3":true, "__name__":"metric_00000375", "key_7":"d","key_1":"j","key_6":"a", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.136231, "value1":0.7872857259243544, "value2":2466956534940008420, "value3":true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.329501, "value1":0.01079935258997052, "value2":4968522926518293278, "value3":false, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.825131, "value1":0.6767719760068325, "value2":8529334985450219291, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807971, "value1":0.7351117134307579, "value2":3019346910002173461, "value3":true, "__name__":"metric_00000379", "key_9":"k","key_3":"f","key_7":"a", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017959, "value1":0.6253725541571399, "value2":8835344453005145545, "value3":false, "__name__":"metric_00000380", "key_1":"f","key_4":"j","key_0":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.280599, "value1":0.16481336771423033, "value2":1417072867431315310, "value3":false, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673036, "value1":0.8178932673930933, "value2":6530240293167023011, "value3":false, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192297, "value1":0.7505738676572953, "value2":6861957449422419505, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517236, "value1":0.639000099787322, "value2":5815570450456127187, "value3":true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739883, "value1":0.17493563192408712, "value2":7179940437785343272, "value3":true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.531912, "value1":0.08360001327419847, "value2":2755543678440921421, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452307, "value1":0.8726532834145122, "value2":187164282595967676, "value3":false, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517272, "value1":0.9444444024596198, "value2":5607680802025895328, "value3":true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289453, "value1":0.15619031798053584, "value2":810705789880173134, "value3":false, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397508, "value1":0.9544477853731341, "value2":7641027031682219689, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812107, "value1":0.9448725135844366, "value2":5145031592575451663, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.063548, "value1":0.922662738511914, "value2":2569888172146766439, "value3":true, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.959005, "value1":0.7947728545231653, "value2":4794325670292386969, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.570203, "value1":0.9426031560649749, "value2":8759901536059040166, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.514218, "value1":0.47519036709767065, "value2":1247765910819397634, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541370, "value1":0.3774216285225394, "value2":7667033726860743169, "value3":false, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.938282, "value1":0.020047520852319107, "value2":7849085976338159942, "value3":false, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874020, "value1":0.47048043771282444, "value2":2196635070664843443, "value3":false, "__name__":"metric_00000407", "key_8":"c","key_3":"a","key_6":"e", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513588, "value1":0.012232986779526358, "value2":8478348320905895865, "value3":true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109700, "value1":0.7338145643706604, "value2":6797416603289211588, "value3":false, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.488779, "value1":0.6031093288566471, "value2":5300319596464382751, "value3":true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931594, "value1":0.42339447548233033, "value2":969762807855801088, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.553860, "value1":0.676076748018672, "value2":1192139536623982501, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.132006, "value1":0.6728558661354047, "value2":1380355336108501777, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.669226, "value1":0.1577792131508693, "value2":4512376080159929307, "value3":true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921683, "value1":0.24615483388690507, "value2":5797393083888487885, "value3":false, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.800085, "value1":0.768303750055719, "value2":6971653838071612447, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468961, "value1":0.45803974634578515, "value2":1199483949895079721, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356489, "value1":0.49471640885346424, "value2":4138046086256944727, "value3":true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225992, "value1":0.6741558407603926, "value2":5216648174057731100, "value3":true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.779458, "value1":0.11692394331277675, "value2":9011198363894466670, "value3":true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.683290, "value1":0.48772002686762633, "value2":7212512979389072906, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053472, "value1":0.5939009672642634, "value2":1146669122452133702, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790089, "value1":0.8261492714018603, "value2":6290707817110359790, "value3":false, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145153, "value1":0.6394833078139336, "value2":7849898590496421466, "value3":false, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.367040, "value1":0.6490663681468899, "value2":3967326627567255091, "value3":true, "__name__":"metric_00000415", "key_7":"k","key_9":"e","key_4":"j", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930388, "value1":0.6376629127038135, "value2":3197193413798224716, "value3":true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471623, "value1":0.06021884177520976, "value2":4773928340410962251, "value3":false, "__name__":"metric_00000418", "key_7":"e","key_1":"c","key_6":"i", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201531, "value1":0.7136591237382468, "value2":6260011441045155926, "value3":true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064553, "value1":0.2818642070197672, "value2":2495939851413003901, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282911, "value1":0.1452427125541155, "value2":8365230104234658997, "value3":true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279591, "value1":0.8354266947002554, "value2":1254307733558365753, "value3":false, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849657, "value1":0.6912085697845711, "value2":1824933885382646361, "value3":false, "__name__":"metric_00000424", "key_2":"d","key_8":"b","key_1":"i", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.092000, "value1":0.053974617818740134, "value2":8465803905922172931, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408792, "value1":0.3821176162488874, "value2":4135301501032886839, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.728999, "value1":0.9460512775720927, "value2":5232062325949200093, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.706765, "value1":0.05808795299595995, "value2":3187473873599955497, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689611, "value1":0.07108495627051868, "value2":8720638688812490559, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.686555, "value1":0.11535487696002504, "value2":3943663806047625686, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815986, "value1":0.6951101939977824, "value2":2103945278295322543, "value3":false, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991212, "value1":0.9405521412873721, "value2":3787480236551923641, "value3":false, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665833, "value1":0.21165663807431181, "value2":3904059491287927557, "value3":true, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183315, "value1":0.5795855388985459, "value2":1854253684581147206, "value3":false, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344965, "value1":0.8721578775869344, "value2":2246100236955108961, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671953, "value1":0.31034394563958956, "value2":193813594216774336, "value3":true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.127315, "value1":0.4080665667532778, "value2":360930115537783747, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.947191, "value1":0.901998732832304, "value2":1708333379014319670, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894585, "value1":0.9195635726221545, "value2":8084121448593915190, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004792, "value1":0.9591461919610487, "value2":8675745838539187803, "value3":true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900181, "value1":0.4274569433819623, "value2":6055452196459435249, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948861, "value1":0.954396067309185, "value2":2700832246849384276, "value3":true, "__name__":"metric_00000450", "key_8":"k","key_9":"g","key_3":"h", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656208, "value1":0.4983661330975837, "value2":3291693517576289438, "value3":false, "__name__":"metric_00000451", "key_4":"h","key_5":"c","key_1":"a", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.937253, "value1":0.2620189984734357, "value2":8181737649910995938, "value3":true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867100, "value1":0.06224158128949376, "value2":4580692356241748765, "value3":false, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827749, "value1":0.03871136162211054, "value2":7089705182458313228, "value3":true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241402, "value1":0.4482242754297532, "value2":9114765255689214165, "value3":true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.602059, "value1":0.29735202117340814, "value2":3162172555777612351, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.249371, "value1":0.23918886296603434, "value2":9067747281684729145, "value3":true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725148, "value1":0.8102090769702104, "value2":4626069739522513533, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.928403, "value1":0.3838571333237863, "value2":8647736244601572769, "value3":true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358211, "value1":0.32922133279873583, "value2":8174411590575614648, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347825, "value1":0.7167150468238062, "value2":8311537141574084420, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875897, "value1":0.26919747077534717, "value2":9053609159881163713, "value3":true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392772, "value1":0.2800052196883224, "value2":7203447031464069407, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849662, "value1":0.5009846634201038, "value2":939832331440858045, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.024201, "value1":0.9391704477843268, "value2":2885798586864811264, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952269, "value1":0.2664477356829927, "value2":847075590517072325, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.307777, "value1":0.6456685214669929, "value2":3218983200036055999, "value3":true, "__name__":"metric_00000435", "key_5":"a","key_9":"h","key_3":"j", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.832715, "value1":0.2591746701265793, "value2":3426357136923132270, "value3":true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.510375, "value1":0.5688430382533638, "value2":7025749889591390600, "value3":false, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.850762, "value1":0.6438552947023466, "value2":1197489157433338286, "value3":false, "__name__":"metric_00000438", "key_6":"f","key_1":"j", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.905930, "value1":0.54095928495375, "value2":67603635153932551, "value3":true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636437, "value1":0.5854506016456503, "value2":5128251159570110333, "value3":false, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836009, "value1":0.4067978187295729, "value2":6225162557978433985, "value3":true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.781439, "value1":0.8809907639802731, "value2":1945481468549859562, "value3":false, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722225, "value1":0.6928638550983661, "value2":7984089816823103368, "value3":false, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631076, "value1":0.0305606186093776, "value2":3704333676945193488, "value3":true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.515877, "value1":0.03574457632343526, "value2":8116748669267665996, "value3":true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.062029, "value1":0.8880344844442181, "value2":7205869492721817899, "value3":true, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.753934, "value1":0.5309770343574366, "value2":3276732533413239990, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598973, "value1":0.9458250274999828, "value2":4588112807086674988, "value3":false, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284154, "value1":0.6864088859089572, "value2":8516198907569397198, "value3":false, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.337595, "value1":0.45387405891882765, "value2":2878690183576728505, "value3":false, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743636, "value1":0.5411320007572493, "value2":7926291410401458702, "value3":true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751569, "value1":0.29015226479219164, "value2":550259652101408680, "value3":false, "__name__":"metric_00000476", "key_6":"b","key_9":"f","key_4":"g", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696737, "value1":0.19318283293628083, "value2":8059270019889405703, "value3":false, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558974, "value1":0.5527141281446059, "value2":4981805365285955545, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.499157, "value1":0.3593055667554787, "value2":8763899128009608550, "value3":true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725437, "value1":0.38132949820890993, "value2":2593622500148824140, "value3":true, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575337, "value1":0.27925006017882537, "value2":4225105729105614711, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.168187, "value1":0.8455269167046058, "value2":4706079022430056900, "value3":false, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.223652, "value1":0.14768410638621007, "value2":2848574409182395374, "value3":false, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.655719, "value1":0.26992469602574687, "value2":4754721503564673004, "value3":false, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972901, "value1":0.22308996900209793, "value2":4121883961065445917, "value3":true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513268, "value1":0.763713042109811, "value2":1389619580667003266, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335708, "value1":0.9988121809461258, "value2":8201212630370607829, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874837, "value1":0.37662051730821194, "value2":1239286216512679885, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.761409, "value1":0.08352300660036191, "value2":788197175065166874, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993753, "value1":0.08212627471454952, "value2":2431495613386540436, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513306, "value1":0.09225555422185021, "value2":6434312790658522957, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899387, "value1":0.2747071960330611, "value2":3838090571407685488, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498947, "value1":0.19881833202736185, "value2":645491056031774982, "value3":false, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851971, "value1":0.9432399401550566, "value2":3792284814302154775, "value3":true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912337, "value1":0.6898029391029513, "value2":8695381256661600831, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060802, "value1":0.4672690888050405, "value2":509110642459148760, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.988275, "value1":0.6949519126514042, "value2":2125445866842945317, "value3":false, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.537780, "value1":0.9644299177479598, "value2":2844641921809856363, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.129171, "value1":0.2870831250353682, "value2":5356758362083554180, "value3":true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.699225, "value1":0.7476204022496519, "value2":4681919372415529338, "value3":true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.798856, "value1":0.5178345021294356, "value2":8950909328927419020, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356529, "value1":0.9689016091606751, "value2":778967583500360424, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.740297, "value1":0.48229504021118547, "value2":319778397891873168, "value3":false, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082023, "value1":0.8672498335471771, "value2":2761738695533665878, "value3":false, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.737582, "value1":0.21539065762206674, "value2":1173702480391317018, "value3":true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.203204, "value1":0.04428967569336166, "value2":6796467582468653168, "value3":false, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197343, "value1":0.5038157716364858, "value2":5419554006840711744, "value3":false, "__name__":"metric_00000506", "key_5":"g","key_6":"g","key_3":"e", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166454, "value1":0.4520211285816449, "value2":4140960458683685597, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.810425, "value1":0.9556399667218088, "value2":2345821074968960700, "value3":false, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708611, "value1":0.36562484484170277, "value2":7482183983302020104, "value3":true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614909, "value1":0.010690625631515951, "value2":4802163912353727023, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529200, "value1":0.23500509922901588, "value2":1518336865591090789, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957588, "value1":0.8999494296908893, "value2":5272580661109552970, "value3":false, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900198, "value1":0.3844554073160095, "value2":4170365040112200144, "value3":true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.579116, "value1":0.16747277518786968, "value2":6804977291625507700, "value3":false, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250708, "value1":0.8577877748125339, "value2":590691746141274872, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721810, "value1":0.7147800394287138, "value2":4035652902850373019, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.922402, "value1":0.0627394141655773, "value2":4185528067707010977, "value3":true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.198128, "value1":0.08114002676829289, "value2":2239252166892691326, "value3":false, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803033, "value1":0.0019520065249596544, "value2":1312973954738137976, "value3":false, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.174637, "value1":0.09909477348410087, "value2":93134409084713078, "value3":true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250984, "value1":0.49581172361364056, "value2":9059996107352612477, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381289, "value1":0.5781636662002811, "value2":3051612121774242926, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.046509, "value1":0.758530400177533, "value2":67989907532933127, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.933345, "value1":0.9339175035199608, "value2":377586629316226209, "value3":false, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963510, "value1":0.7004527478348754, "value2":1390034556068597934, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918093, "value1":0.17000680115237268, "value2":4215018843369459603, "value3":true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.569630, "value1":0.20175634343868196, "value2":8381338885007503558, "value3":true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.883668, "value1":0.06695323081792075, "value2":1589024156310802370, "value3":false, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.178981, "value1":0.37314887727591756, "value2":518499450705948775, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187208, "value1":0.4599180344115974, "value2":4887550148987301900, "value3":true, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952934, "value1":0.8319538072999415, "value2":3671167315474130299, "value3":false, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.588275, "value1":0.5426790448893685, "value2":6523903627951338788, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.128001, "value1":0.37530546288075156, "value2":5616976404881722053, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004862, "value1":0.8346907316991129, "value2":4019033006145719569, "value3":false, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826250, "value1":0.5254457071340125, "value2":972903594117286740, "value3":true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344623, "value1":0.6999472357226462, "value2":1701492823670778543, "value3":false, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639041, "value1":0.2248828857383105, "value2":4810585903388633809, "value3":false, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.571165, "value1":0.22946219998225406, "value2":8055176566662404630, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117315, "value1":0.7792786347828345, "value2":8503346841194050818, "value3":false, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.646178, "value1":0.29610828190559946, "value2":6707117461343778078, "value3":true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804384, "value1":0.8994527553958773, "value2":3055603759085197982, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.503237, "value1":0.038643663095115535, "value2":97929435615191250, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.917565, "value1":0.06570063912325382, "value2":7659557506835508250, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.194133, "value1":0.7280519586189449, "value2":607993872873285299, "value3":true, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197005, "value1":0.34702978428971004, "value2":2119181822487481024, "value3":false, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578715, "value1":0.4531709992663944, "value2":1719249508626109183, "value3":true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.221581, "value1":0.5476991652235295, "value2":8006956843834072386, "value3":true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177885, "value1":0.5137634978637854, "value2":5009892345362966157, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596214, "value1":0.2368591303155911, "value2":8792401284340375040, "value3":true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182004, "value1":0.5444756866358181, "value2":1210706677095755681, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972249, "value1":0.9909726775988351, "value2":1108183309913410572, "value3":false, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225911, "value1":0.47194616841751835, "value2":3490783413049237610, "value3":true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321843, "value1":0.5905642407899484, "value2":4893835600551421123, "value3":true, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.552992, "value1":0.8654702264058397, "value2":5106927081449112108, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782381, "value1":0.8285431175840194, "value2":9168936255854371872, "value3":true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410849, "value1":0.31656794437122665, "value2":2071986113321022167, "value3":false, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780581, "value1":0.051832044232056354, "value2":6484981440389875558, "value3":true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.072700, "value1":0.698857333675772, "value2":3443549406704488268, "value3":true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689951, "value1":0.5808073564555776, "value2":5048983612386176894, "value3":true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.733768, "value1":0.8029317643963917, "value2":7359595780765172048, "value3":true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364358, "value1":0.34011958562739014, "value2":999331274737013831, "value3":false, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.512816, "value1":0.34358105240557685, "value2":6714721903573983722, "value3":false, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711860, "value1":0.9821515464895118, "value2":5477696623940889135, "value3":true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.857930, "value1":0.697732523770638, "value2":6661494860267552531, "value3":false, "__name__":"metric_00000573", "key_8":"k","key_4":"d","key_5":"d", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114607, "value1":0.8236051117026364, "value2":4748182480100358673, "value3":true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.628228, "value1":0.168082038026373, "value2":7102000252406916356, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.687012, "value1":0.3638576608952103, "value2":8540460395700338240, "value3":true, "__name__":"metric_00000576", "key_9":"c","key_2":"k","key_5":"b", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169601, "value1":0.3111748596945155, "value2":8571165720141489367, "value3":false, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498080, "value1":0.6187426468242411, "value2":2451438877433776073, "value3":false, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866907, "value1":0.5131483959448067, "value2":8869138925582259535, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.396171, "value1":0.28292656018705137, "value2":5391262893377663378, "value3":true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114785, "value1":0.0649996585433473, "value2":3978180949228199287, "value3":true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671381, "value1":0.13833508369091968, "value2":5430930553797657293, "value3":false, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.923816, "value1":0.8519197270200854, "value2":9127083562371495095, "value3":false, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786218, "value1":0.02969493094352536, "value2":7671441472993176714, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774561, "value1":0.2102387127527105, "value2":4039185760762390304, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192344, "value1":0.144819186927003, "value2":378491549245728191, "value3":false, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076506, "value1":0.3187033255519536, "value2":8885020385104236250, "value3":false, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213717, "value1":0.8128476179407741, "value2":2181064442818506560, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528442, "value1":0.5936064804578587, "value2":5648031374498714859, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.767476, "value1":0.1348520099403021, "value2":4074505568425329493, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.449677, "value1":0.09144593933086395, "value2":8719212717808997934, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411794, "value1":0.8990540517914553, "value2":8624156888058647365, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.457637, "value1":0.3197628975782331, "value2":16763285997559821, "value3":false, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.330356, "value1":0.8095806602517515, "value2":4384365921475656332, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584148, "value1":0.16829289997155406, "value2":380241693738836834, "value3":false, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455567, "value1":0.7529538853574959, "value2":1681989614930760275, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.844936, "value1":0.6074576588778431, "value2":7722066461223215119, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926267, "value1":0.5797545736876416, "value2":7883616781252732331, "value3":true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238789, "value1":0.014736618507623488, "value2":4669225171297242340, "value3":true, "__name__":"metric_00000590", "key_9":"g","key_0":"k", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361896, "value1":0.17876601404118195, "value2":5115665659478830691, "value3":false, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756049, "value1":0.04403510631259223, "value2":7660735889331080164, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.031957, "value1":0.33256068212920453, "value2":4453878565561794031, "value3":false, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.828157, "value1":0.39237914319665085, "value2":7249969180646160990, "value3":false, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952636, "value1":0.372690613847562, "value2":6192025452112268327, "value3":true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991675, "value1":0.5494769946982794, "value2":90791903671198766, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240830, "value1":0.9786829569747533, "value2":5978162907079513697, "value3":false, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279525, "value1":0.1494973578442592, "value2":6869066553359800738, "value3":false, "__name__":"metric_00000607", "key_8":"h","key_5":"d", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.939830, "value1":0.37432428722372013, "value2":4226472512491590319, "value3":false, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.564559, "value1":0.7159468168647539, "value2":1512961341899043326, "value3":true, "__name__":"metric_00000609", "key_7":"e","key_6":"k", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048658, "value1":0.5401236641884074, "value2":8347872809477134111, "value3":false, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.340095, "value1":0.4533223235157631, "value2":1670286383442367211, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293295, "value1":0.4258630060523724, "value2":5674260405699571928, "value3":true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.239916, "value1":0.9150796129317534, "value2":8474870548791768473, "value3":true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744309, "value1":0.6470478918019383, "value2":7704476841600519211, "value3":false, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458393, "value1":0.05603920506738755, "value2":1942285693271412968, "value3":true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720119, "value1":0.6222144605927992, "value2":5009900948732974588, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435194, "value1":0.6528463342414516, "value2":2372394551953155351, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.847658, "value1":0.12821161771950726, "value2":1481519210091358924, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.083324, "value1":0.3648356544882518, "value2":15076769986125901, "value3":false, "__name__":"metric_00000619", "key_5":"h","key_0":"h","key_1":"i", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696866, "value1":0.5460078372418026, "value2":8482317071463194706, "value3":false, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878357, "value1":0.9158893232321995, "value2":4112830082593015136, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815751, "value1":0.7832798664333082, "value2":6621546193633794291, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.036509, "value1":0.7258315650178256, "value2":2697123926599878172, "value3":false, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786542, "value1":0.050969235189444476, "value2":960522431282362262, "value3":false, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293909, "value1":0.1637483242537312, "value2":4063809715912746117, "value3":true, "__name__":"metric_00000599", "key_6":"d","key_1":"g","key_2":"e", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960513, "value1":0.15412859037505927, "value2":3609332046279378552, "value3":true, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845902, "value1":0.9308703210994502, "value2":6609460372849170625, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750261, "value1":0.12234055832157559, "value2":8912704681070762337, "value3":false, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458834, "value1":0.9318510467298435, "value2":8026158416345347023, "value3":false, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859979, "value1":0.9301579137721765, "value2":7925354941740095610, "value3":true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.783146, "value1":0.4471773356574157, "value2":8481603802647763313, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.505375, "value1":0.5948200403445978, "value2":52874652951483269, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005495, "value1":0.829951436170396, "value2":3721373420722024974, "value3":false, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867411, "value1":0.6049905245498157, "value2":8221468316724652544, "value3":false, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435857, "value1":0.8205378674820365, "value2":4321340481613778781, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.618588, "value1":0.6552813364407445, "value2":3804139748589484208, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.027076, "value1":0.23841518035800105, "value2":7013010516589186218, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285111, "value1":0.5613630899205873, "value2":1517169132984045486, "value3":false, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117978, "value1":0.3226349150452925, "value2":5930250973192218816, "value3":false, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238673, "value1":0.398077529845209, "value2":8495752825324479706, "value3":true, "__name__":"metric_00000640", "key_7":"d","key_8":"j","key_3":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258260, "value1":0.3301633034128501, "value2":3240643823549476632, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.373493, "value1":0.3529131970207001, "value2":3035971070657682432, "value3":true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665486, "value1":0.1772430533425256, "value2":5999482327853244380, "value3":false, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358796, "value1":0.18816869038874143, "value2":5134838279453664617, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140177, "value1":0.8387362327402264, "value2":2310199526324609410, "value3":false, "__name__":"metric_00000645", "key_9":"j","key_0":"a","key_1":"c", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059616, "value1":0.8521328351977854, "value2":8719119695607623072, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121795, "value1":0.3605246283964201, "value2":1540950724302287517, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760661, "value1":0.21327358886251288, "value2":5483717677008529333, "value3":true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.019253, "value1":0.7234842919041711, "value2":3995891303198614565, "value3":true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468371, "value1":0.026572272284047523, "value2":2431068728438057213, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397658, "value1":0.22598294609730654, "value2":2698499054966120972, "value3":true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755700, "value1":0.09282245685151808, "value2":1822149750384470026, "value3":true, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304716, "value1":0.6358956357038871, "value2":1038515895167880998, "value3":false, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673410, "value1":0.01619219159654624, "value2":999540262235584439, "value3":false, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406798, "value1":0.2801610335950404, "value2":5737646822014765045, "value3":true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472994, "value1":0.14109660013219333, "value2":5384594772062199779, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601871, "value1":0.291815873607527, "value2":1847519318868439297, "value3":false, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994725, "value1":0.6730562186913949, "value2":5489090928652625049, "value3":false, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205639, "value1":0.42334867153762545, "value2":1823044221508369143, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.904945, "value1":0.598173556849254, "value2":1029832085092338611, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507464, "value1":0.8462370553826191, "value2":8938945324991684299, "value3":true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105783, "value1":0.4362851240411875, "value2":8230814679005715011, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.130819, "value1":0.5311370726009594, "value2":2799369947502242194, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.436621, "value1":0.5406911040823768, "value2":5927551572729154830, "value3":false, "__name__":"metric_00000655", "key_6":"e","key_0":"b", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081043, "value1":0.9897107711937903, "value2":1409810039307886257, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.360124, "value1":0.8898589942584547, "value2":1182666162202869439, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473406, "value1":0.9785032169444227, "value2":7933797459692204307, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711755, "value1":0.8440901040738314, "value2":1266347640739805258, "value3":true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836845, "value1":0.13202825619743236, "value2":6494324863003452768, "value3":true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.814171, "value1":0.22371325434459086, "value2":3679612694652027947, "value3":true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707264, "value1":0.43001516107021553, "value2":4022260893191171434, "value3":false, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585576, "value1":0.9028218878505339, "value2":3312996140868409539, "value3":false, "__name__":"metric_00000662", "key_8":"h","key_2":"j","key_3":"d", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.253947, "value1":0.5596795145791367, "value2":9053991858276197521, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.793355, "value1":0.655432892066219, "value2":2171965578246650022, "value3":false, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003349, "value1":0.32642930281521576, "value2":4724329284506985861, "value3":true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532693, "value1":0.47384645211929527, "value2":166053326506373649, "value3":true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258148, "value1":0.8755464478705562, "value2":1367687629182661630, "value3":true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284672, "value1":0.969983045672162, "value2":7709886208589610791, "value3":false, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656648, "value1":0.7178009692517222, "value2":378528392759859903, "value3":true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.138190, "value1":0.17961568288211016, "value2":6086554307721297156, "value3":false, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755588, "value1":0.7476161070292083, "value2":3339991721000583202, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.160231, "value1":0.4667023272863336, "value2":8520356596039544954, "value3":false, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888583, "value1":0.20049555103413333, "value2":2444783554472337568, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008015, "value1":0.08008213078983399, "value2":3073434633965527342, "value3":true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094964, "value1":0.7132542439294741, "value2":5312205610465131789, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.234374, "value1":0.3178507521158574, "value2":1341935169276159022, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772701, "value1":0.9799507521983674, "value2":4679043722393447880, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.742787, "value1":0.8436883610291797, "value2":8206108321797356735, "value3":false, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285496, "value1":0.02119049506813719, "value2":5643416354242611531, "value3":true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041734, "value1":0.8593746773495375, "value2":7177072845753511018, "value3":false, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102405, "value1":0.258959468834628, "value2":5506364318868451234, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.617097, "value1":0.5213134879577571, "value2":7820417886370593685, "value3":true, "__name__":"metric_00000683", "key_9":"c","key_1":"k","key_6":"a", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614285, "value1":0.8586887753126803, "value2":146426955523604107, "value3":false, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094475, "value1":0.3100760743003405, "value2":3216906708607385583, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400149, "value1":0.40305327199936813, "value2":4063666163388559666, "value3":true, "__name__":"metric_00000686", "key_4":"a","key_8":"g","key_1":"d", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.741707, "value1":0.07472258132971145, "value2":4583642984114208044, "value3":true, "__name__":"metric_00000688", "key_8":"c","key_2":"b","key_7":"b", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289818, "value1":0.21965034622340104, "value2":548484284878385021, "value3":false, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788770, "value1":0.8645491106416102, "value2":6428112142936498186, "value3":true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589911, "value1":0.8352505870700639, "value2":8807067943589232363, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980194, "value1":0.4222199616754522, "value2":1431823521353243628, "value3":true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347713, "value1":0.3792858568507692, "value2":5699682291213510899, "value3":false, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790574, "value1":0.48263080283098414, "value2":3899441712388650234, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.586360, "value1":0.39598813053011617, "value2":8951581838322949271, "value3":true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147821, "value1":0.696249186695191, "value2":5997811343683052027, "value3":true, "__name__":"metric_00000695", "key_9":"g","key_6":"a","key_8":"d", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375267, "value1":0.30594268061015806, "value2":7589869429394133120, "value3":false, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.662739, "value1":0.8061361815679017, "value2":5904903629197529499, "value3":true, "__name__":"metric_00000696", "key_4":"h","key_1":"d","key_2":"d", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447051, "value1":0.37750215516426633, "value2":8371518067848222420, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121391, "value1":0.6696130432092745, "value2":3528483712948148367, "value3":false, "__name__":"metric_00000701", "key_7":"a","key_2":"h","key_5":"i", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114395, "value1":0.9303974176092045, "value2":539479413974722614, "value3":true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.440024, "value1":0.33141470193064243, "value2":492711817812030768, "value3":false, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830103, "value1":0.7998316524208386, "value2":4274003622712511224, "value3":true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.343594, "value1":0.948502518427523, "value2":1092062054441297015, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.820370, "value1":0.6468806610248711, "value2":5700638394649707417, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169752, "value1":0.427387565910367, "value2":7968698722232869415, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.352398, "value1":0.008797452976395297, "value2":5389560588922529986, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.574184, "value1":0.6925729377012323, "value2":7921127567234590177, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708589, "value1":0.6874360024316648, "value2":7727336634389454587, "value3":false, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.647250, "value1":0.2053256936728798, "value2":8730436625808972720, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522110, "value1":0.7058378270168358, "value2":5838333049897839477, "value3":false, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894605, "value1":0.3141493596388785, "value2":2076790219296988613, "value3":true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008782, "value1":0.590566630449788, "value2":571655682516576680, "value3":true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.424761, "value1":0.6098530285996386, "value2":7846500926381318806, "value3":false, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591196, "value1":0.33824124551827933, "value2":646303391431941929, "value3":true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964204, "value1":0.7366285135213106, "value2":4137944794465811468, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643081, "value1":0.3218928504433246, "value2":9121706025453589663, "value3":false, "__name__":"metric_00000717", "key_9":"e","key_0":"b","key_4":"f", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804724, "value1":0.014940637092911, "value2":7451060690570027498, "value3":true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694428, "value1":0.21460306796333273, "value2":2982813118003123474, "value3":true, "__name__":"metric_00000718", "key_6":"g","key_9":"e","key_5":"b", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.122258, "value1":0.49140759630280006, "value2":8832276444519664682, "value3":true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.996796, "value1":0.2474526024511197, "value2":7613005431630356063, "value3":true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238269, "value1":0.8267663978098108, "value2":2724285849945025892, "value3":true, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.644529, "value1":0.4649514742650587, "value2":7589769671176102226, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.376035, "value1":0.7135202605627624, "value2":4131507979199286159, "value3":false, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958098, "value1":0.5742451996367589, "value2":7655662522626761261, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478186, "value1":0.30259713893200063, "value2":7366343481003162441, "value3":false, "__name__":"metric_00000725", "key_6":"g","key_9":"d","key_3":"a", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299728, "value1":0.5700777203840441, "value2":8077274544410191414, "value3":false, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.685643, "value1":0.9403232361639245, "value2":809458262071959965, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478225, "value1":0.7234775762369925, "value2":1256131637116314402, "value3":false, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.445423, "value1":0.17591193935592764, "value2":6211570042356723652, "value3":true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.362681, "value1":0.09007902579380467, "value2":5590217114587646147, "value3":false, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140908, "value1":0.3872422494802931, "value2":3824505998771737796, "value3":true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.312510, "value1":0.26492926145602147, "value2":2580605589364820036, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639094, "value1":0.8695708814560201, "value2":828997494663746747, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532778, "value1":0.8659642362585516, "value2":489767661854306328, "value3":false, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.332818, "value1":0.5085353492466338, "value2":7131627003266463948, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616148, "value1":0.6665482572291218, "value2":9183191141357087143, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807708, "value1":0.03703617074385954, "value2":1634911105603876624, "value3":true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.990782, "value1":0.8721197542810023, "value2":829062251335888311, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.678575, "value1":0.8518891534609417, "value2":5490128933307846252, "value3":false, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676580, "value1":0.032403043603468545, "value2":1860775688299500894, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.011776, "value1":0.3310374930985375, "value2":6454291476355278297, "value3":false, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.902226, "value1":0.03335506845841715, "value2":4281002708262490682, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.884426, "value1":0.26587448707952976, "value2":2888077545939138944, "value3":false, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721322, "value1":0.9352434968753679, "value2":7587503328880398082, "value3":false, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674491, "value1":0.13470131213667097, "value2":733064028083204302, "value3":false, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.422073, "value1":0.34800250717845166, "value2":5657610491232907815, "value3":false, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232687, "value1":0.8974445402000435, "value2":5670871752075567567, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.603175, "value1":0.8044440847017621, "value2":8279540241317103114, "value3":false, "__name__":"metric_00000757", "key_5":"b","key_6":"j","key_3":"e", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354995, "value1":0.9370809546532062, "value2":2794379758286031831, "value3":true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598521, "value1":0.03162604483850602, "value2":8898609192002715752, "value3":false, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489635, "value1":0.09343989179985779, "value2":6354680197733735029, "value3":true, "__name__":"metric_00000760", "key_5":"k","key_2":"d","key_4":"i", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.766631, "value1":0.6557065025532866, "value2":8867906113412686551, "value3":false, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.110067, "value1":0.37336348255028134, "value2":8359961637534300323, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.734771, "value1":0.6467506386274888, "value2":1558727129876488311, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751073, "value1":0.5011707406625697, "value2":165276254030404304, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903806, "value1":0.883542811993649, "value2":5713497784481830170, "value3":false, "__name__":"metric_00000765", "key_9":"b","key_3":"a","key_5":"h", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068830, "value1":0.41287620615552584, "value2":110550725395415731, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.481998, "value1":0.9176610757654294, "value2":3543653242038441064, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.206639, "value1":0.6881985031716263, "value2":3991865736711226608, "value3":true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682333, "value1":0.7354938188464508, "value2":2336942143484925177, "value3":true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.909363, "value1":0.6550859218093187, "value2":7848025848676081478, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.439129, "value1":0.4735897904277586, "value2":1730101214885033885, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788447, "value1":0.7029400400744759, "value2":7348647096857266019, "value3":false, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656519, "value1":0.8226813316731755, "value2":8565628989265102630, "value3":true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.286018, "value1":0.1715470955708772, "value2":281139728200192436, "value3":false, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.316461, "value1":0.970829440713552, "value2":8266115606666039974, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.131938, "value1":0.06029126490427053, "value2":6088969321338942970, "value3":false, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936137, "value1":0.16630484422294922, "value2":368627481682565524, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.632956, "value1":0.4597748954914025, "value2":614883467360042989, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441534, "value1":0.3169134259464135, "value2":4894244187289767341, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.536363, "value1":0.21988172452701402, "value2":3848642916965059480, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722069, "value1":0.0035036640993316324, "value2":1784910019753576066, "value3":true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189179, "value1":0.5999084267877618, "value2":589928855378135734, "value3":false, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070147, "value1":0.2816071736469606, "value2":1978576359562742327, "value3":true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.474328, "value1":0.12233813345274026, "value2":7757125810591739471, "value3":true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756802, "value1":0.8734910453077231, "value2":2858305999840238155, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830462, "value1":0.7809071944843656, "value2":6070440638222443699, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.158081, "value1":0.8007440801809812, "value2":3893225795949780633, "value3":false, "__name__":"metric_00000778", "key_7":"g","key_1":"b","key_3":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266044, "value1":0.5291100849459248, "value2":8570983485516004790, "value3":false, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375100, "value1":0.43060898088087474, "value2":5072556087103000232, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433713, "value1":0.4065332038272028, "value2":7498601821703546789, "value3":false, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293147, "value1":0.2008523459507916, "value2":3469755278652299519, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.464083, "value1":0.7193413705182999, "value2":8360590737420028572, "value3":true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182168, "value1":0.12869784027227613, "value2":2436732966596869973, "value3":true, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152849, "value1":0.915235247509805, "value2":6575468662860533653, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.106299, "value1":0.6574795299867987, "value2":5839146977664216835, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.842382, "value1":0.6684180756051546, "value2":6255233029731307668, "value3":true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.976893, "value1":0.9921834371308854, "value2":286112572755130788, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.747197, "value1":0.6687884895336262, "value2":5026368687057544137, "value3":true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227443, "value1":0.8444750407098351, "value2":3238045843783198862, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575432, "value1":0.04959245001196866, "value2":4763760968874648742, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484574, "value1":0.8794886478305205, "value2":2929187105321944273, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.613332, "value1":0.4649488124035846, "value2":8038984692264433406, "value3":false, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.156389, "value1":0.013056913554735015, "value2":1073629030912865258, "value3":false, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614657, "value1":0.21062410476220902, "value2":6415897327379032774, "value3":true, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238729, "value1":0.10820884210252545, "value2":9083864450943421396, "value3":false, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241678, "value1":0.22605677165183255, "value2":191747579342742576, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958794, "value1":0.08847610578305673, "value2":7897538472323716766, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335228, "value1":0.7181578715519717, "value2":7450823110233453413, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408629, "value1":0.007049664704797943, "value2":1505104843936344580, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558681, "value1":0.17466226992103584, "value2":4472498115626926601, "value3":true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682989, "value1":0.9645272985758268, "value2":1982644188431990793, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.179200, "value1":0.6439691808346755, "value2":8186363896835482044, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.823991, "value1":0.6478071089364331, "value2":8599197823923500800, "value3":false, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109730, "value1":0.43770117688691335, "value2":8534783631376673343, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.659077, "value1":0.2388079100822681, "value2":5323293096195049116, "value3":false, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.120405, "value1":0.7373368673794832, "value2":8559881035375574344, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.355361, "value1":0.6166673498020887, "value2":5456145421934358189, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.824086, "value1":0.4534358999465994, "value2":6384310233648456918, "value3":true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871013, "value1":0.5576949556795737, "value2":1428733241052481930, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507168, "value1":0.14733425604177453, "value2":4932780474472793744, "value3":true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.399272, "value1":0.7455596536595573, "value2":7959578558660693956, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015542, "value1":0.3661445195957949, "value2":5873286557095287645, "value3":false, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.103786, "value1":0.04914833432556202, "value2":3473465672536909160, "value3":true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201038, "value1":0.09243610446589845, "value2":317387159009776683, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637510, "value1":0.05764711949860248, "value2":683074398669967457, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000515, "value1":0.27555419865710506, "value2":1427932259617448392, "value3":true, "__name__":"metric_00000816", "key_6":"d","key_4":"e", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455871, "value1":0.4421289415542984, "value2":2179707427691179241, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.816179, "value1":0.0315328103269199, "value2":8955771787657622788, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.012467, "value1":0.2583652156574764, "value2":3152397034013397022, "value3":false, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.323088, "value1":0.6137213421267609, "value2":6194154265710187463, "value3":false, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361691, "value1":0.2728847707358428, "value2":1410878768807423021, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507208, "value1":0.40681095070477, "value2":2031127740762892841, "value3":true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.943788, "value1":0.4688930142953297, "value2":4135242725922960343, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528649, "value1":0.6786255340569864, "value2":5287486612492903502, "value3":false, "__name__":"metric_00000824", "key_3":"f","key_5":"a","key_0":"f", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225718, "value1":0.4601298705932887, "value2":9052049426787719860, "value3":true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.308880, "value1":0.4295083103049244, "value2":5651329885903965921, "value3":true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759618, "value1":0.16482488772851353, "value2":7857083585674436753, "value3":true, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416052, "value1":0.9111867810311565, "value2":6723934981662609251, "value3":true, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.962464, "value1":0.4935260151955853, "value2":3632302155175562257, "value3":true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.771117, "value1":0.5157133736913164, "value2":2578240794064018102, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690480, "value1":0.6806227775589547, "value2":478506980366332127, "value3":false, "__name__":"metric_00000832", "key_3":"h","key_0":"e", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.819739, "value1":0.8780417506767497, "value2":5796860147973283852, "value3":true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.393173, "value1":0.42102685254727695, "value2":655129410007603946, "value3":true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.648983, "value1":0.7792161980083644, "value2":3901696553081135725, "value3":true, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746096, "value1":0.4415965960336573, "value2":5128962497088709971, "value3":false, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550611, "value1":0.9164668411003939, "value2":8252426555782396131, "value3":false, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.026169, "value1":0.7150672668373745, "value2":4764749266652855333, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.684226, "value1":0.7136962442214909, "value2":5607090679190953591, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382560, "value1":0.4327657862928645, "value2":2199828856833521565, "value3":true, "__name__":"metric_00000840", "key_7":"c","key_2":"b","key_3":"i", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.467808, "value1":0.22363679307659168, "value2":2768232122997302049, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575206, "value1":0.7602213610975361, "value2":2779299954374668017, "value3":true, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591751, "value1":0.6270567668817301, "value2":8638942281069975034, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374899, "value1":0.3628294711980554, "value2":2256958770162190672, "value3":true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490274, "value1":0.7562635891941201, "value2":828105904736409259, "value3":true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.170118, "value1":0.5429488039839387, "value2":2772778506127755922, "value3":true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240769, "value1":0.2820088361870865, "value2":5639438631779439142, "value3":false, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.052386, "value1":0.43149632078191064, "value2":1748438881846630926, "value3":true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507233, "value1":0.961520455982136, "value2":538372936759435184, "value3":true, "__name__":"metric_00000849", "key_4":"e","key_3":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705937, "value1":0.5267821110187133, "value2":2615203611933488638, "value3":true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750925, "value1":0.12125960803758137, "value2":9007375284837027095, "value3":false, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631618, "value1":0.07157188871273036, "value2":8155465841105503917, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695843, "value1":0.7797612580730544, "value2":174508687346238960, "value3":true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269295, "value1":0.7420818036952436, "value2":4380281606468315484, "value3":true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566418, "value1":0.31419999154507045, "value2":1448626920753535043, "value3":true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.125439, "value1":0.23644172282231143, "value2":6454497651916669000, "value3":false, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260012, "value1":0.7675706994384546, "value2":5131372429329822614, "value3":false, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674484, "value1":0.024443192261290648, "value2":1060802392645575205, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870912, "value1":0.40232027987742836, "value2":7398514903536659789, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.193840, "value1":0.5573502520764032, "value2":2891807756570316806, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496711, "value1":0.0561679292808207, "value2":7275940738251409744, "value3":true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007546, "value1":0.4548616529539407, "value2":3391843142730199327, "value3":false, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532516, "value1":0.5229547783776293, "value2":306719517858726688, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.879419, "value1":0.738955824150393, "value2":4551796041817468625, "value3":false, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.037090, "value1":0.16862399179431758, "value2":4308060124699455866, "value3":false, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.748879, "value1":0.537219693168335, "value2":464744057316162498, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.610036, "value1":0.35423811263377813, "value2":8058264218672212470, "value3":false, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.212290, "value1":0.8726353918052155, "value2":7894596078524999317, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.876606, "value1":0.6546517022010405, "value2":6758959777685985912, "value3":true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140504, "value1":0.347953522010423, "value2":6223000072814932347, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.829533, "value1":0.7297577355885363, "value2":7098834979468843426, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770147, "value1":0.5158958137849098, "value2":7721786354129545110, "value3":true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960566, "value1":0.08984803039947419, "value2":7183642332674708296, "value3":false, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.002511, "value1":0.7991212300299283, "value2":1907144646133978540, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.969474, "value1":0.8490958586264525, "value2":2475634499457331111, "value3":true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757044, "value1":0.08525717165380897, "value2":6126821447855017519, "value3":false, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826913, "value1":0.3832744407117778, "value2":7549472912902563562, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713539, "value1":0.040929240199395034, "value2":8725477692408636856, "value3":true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.235038, "value1":0.21655074702892513, "value2":6340750264489982261, "value3":false, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713507, "value1":0.09698561541865447, "value2":6976532341223849395, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926365, "value1":0.6869464705551463, "value2":1770406309916093131, "value3":false, "__name__":"metric_00000890", "key_8":"h","key_0":"a", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.159778, "value1":0.37025745483297273, "value2":1667176166944781722, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964175, "value1":0.3460815492793407, "value2":8584884445165219279, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.789212, "value1":0.5752530695794612, "value2":7518543931942002634, "value3":true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946193, "value1":0.9389444654094391, "value2":4798866777286425255, "value3":true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183973, "value1":0.42689788996190486, "value2":2707564478214913458, "value3":false, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381159, "value1":0.19968138225970908, "value2":8498929039632010906, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.535645, "value1":0.9768502318817727, "value2":6273017261118239502, "value3":true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957340, "value1":0.1584492922790321, "value2":1071530234423723880, "value3":false, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622320, "value1":0.6014449518439046, "value2":5905866866815287771, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.839825, "value1":0.6412078194818629, "value2":8581664149275929445, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.925054, "value1":0.6499072539736884, "value2":3213784800648371267, "value3":true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.654382, "value1":0.9107398509050536, "value2":6290871463312648713, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.795697, "value1":0.8789558932554133, "value2":9118966352973366626, "value3":false, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299533, "value1":0.18449853142128472, "value2":6924501956061845944, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993498, "value1":0.10358257618685637, "value2":949727120096577179, "value3":true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192327, "value1":0.14944601875044164, "value2":3650903859653754777, "value3":false, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115925, "value1":0.7654222794387283, "value2":1807303188151583637, "value3":false, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290372, "value1":0.022817260035797225, "value2":3306384333491756709, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.402670, "value1":0.5073298240798392, "value2":4756233885952638662, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682572, "value1":0.44581853587828163, "value2":8977717914108281095, "value3":true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.176704, "value1":0.0686762673056191, "value2":418784156784186861, "value3":true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676421, "value1":0.9540024144812602, "value2":5040838132393700704, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851525, "value1":0.4501599756607321, "value2":7291682933774791859, "value3":false, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.593343, "value1":0.506206698395899, "value2":622013338659516904, "value3":true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454600, "value1":0.4498330838292854, "value2":2669650474822317581, "value3":false, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527805, "value1":0.10877922841749062, "value2":2131131607855096268, "value3":true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.196472, "value1":0.04264644660597854, "value2":2656288085872651609, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743799, "value1":0.2248320170707705, "value2":6330060840606217610, "value3":false, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137446, "value1":0.5799142292246671, "value2":4848799016740596142, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.405919, "value1":0.3378516880177889, "value2":3532844197361780130, "value3":false, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.085405, "value1":0.11768848717895854, "value2":3011043909687794254, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494421, "value1":0.45496539093555294, "value2":1968423041425508639, "value3":false, "__name__":"metric_00000901", "key_5":"f","key_0":"b","key_4":"h", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.846925, "value1":0.23911281903346318, "value2":1229857216789047456, "value3":false, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.320172, "value1":0.7911586278138405, "value2":145982991700455098, "value3":false, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688187, "value1":0.9272721413003945, "value2":5752958588041193324, "value3":false, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.069720, "value1":0.006222818414235851, "value2":7251719396221186260, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.974434, "value1":0.7327419748667513, "value2":8892941322690231954, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.891519, "value1":0.2559114110516683, "value2":8453661615833220464, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596204, "value1":0.8329616164402396, "value2":6548226786173702319, "value3":true, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539908, "value1":0.7401840588881007, "value2":2823865774842747097, "value3":true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865375, "value1":0.23950780446408748, "value2":6387790804623201730, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121268, "value1":0.4818898421020435, "value2":5213834205313537165, "value3":true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865802, "value1":0.3275415536098801, "value2":2802490407634150279, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105641, "value1":0.6210077776605651, "value2":7224328772436945002, "value3":true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592291, "value1":0.821033330430236, "value2":3130486219263774432, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497809, "value1":0.5925199380020667, "value2":341686057277963037, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097734, "value1":0.7071847698151175, "value2":3681912566029517435, "value3":true, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.763574, "value1":0.5357729458141934, "value2":4576393986863014129, "value3":false, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626276, "value1":0.4467400550839313, "value2":8832239215373133572, "value3":false, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626952, "value1":0.8566744061055088, "value2":2681684884383601627, "value3":false, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980666, "value1":0.620719669726131, "value2":3866315867973394400, "value3":false, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.970686, "value1":0.492660470887635, "value2":8584579737939983838, "value3":true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205981, "value1":0.41980273151462594, "value2":1272721076115788334, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643071, "value1":0.9850402079127909, "value2":6866475251892359276, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.520241, "value1":0.021531538514302357, "value2":3680419636166133733, "value3":false, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689781, "value1":0.7246711061095096, "value2":9156928465977729149, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936886, "value1":0.30593890582238353, "value2":8650088989944818585, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064882, "value1":0.9419400037260646, "value2":5139525288533697167, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255204, "value1":0.6562362511494719, "value2":5377088753944330401, "value3":true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147096, "value1":0.12576537131594923, "value2":696189011986700613, "value3":false, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.199366, "value1":0.8703136109731814, "value2":8758678284656449473, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782148, "value1":0.21588917591543605, "value2":7157001910263594492, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053985, "value1":0.9864976225072658, "value2":3999466034196061068, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.409718, "value1":0.4052988292496616, "value2":7239010836693528474, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918878, "value1":0.9460279119002769, "value2":8629146616157881411, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744254, "value1":0.12263478198823631, "value2":8698623269262602240, "value3":false, "__name__":"metric_00000937", "key_9":"i","key_4":"i","key_8":"h", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701868, "value1":0.3178667417931335, "value2":8069435536591676411, "value3":true, "__name__":"metric_00000938", "key_6":"k","key_7":"h","key_5":"b", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.794445, "value1":0.7546856348043545, "value2":1056257035145843208, "value3":false, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.318803, "value1":0.8976234734615749, "value2":7848445975050376420, "value3":false, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.146350, "value1":0.31748244669278847, "value2":2926268572806049868, "value3":false, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392349, "value1":0.655725592207486, "value2":4190046781739044777, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.749690, "value1":0.6239130127907853, "value2":5206683397328774242, "value3":true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744300, "value1":0.6764999956977692, "value2":1477381647586507515, "value3":true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.758303, "value1":0.5258481908865111, "value2":5235026862755190183, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379762, "value1":0.890935177217856, "value2":6322796509408703364, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592912, "value1":0.6886535422027805, "value2":3968885283796366911, "value3":true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859203, "value1":0.3806206868492738, "value2":6588306706789304138, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.565175, "value1":0.823280011831958, "value2":1840222942885696142, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.095858, "value1":0.18658533157232132, "value2":3040729957448861027, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.567863, "value1":0.1544724870114228, "value2":5739061219420103119, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739840, "value1":0.5915071280708923, "value2":6986464734573649185, "value3":false, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.200755, "value1":0.4500618730121175, "value2":4400580498849630870, "value3":false, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358028, "value1":0.8986628799372285, "value2":3320447208954210899, "value3":false, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017181, "value1":0.4972731292168526, "value2":5687205385943047330, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229539, "value1":0.5257635254456374, "value2":1562089048942850119, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.171044, "value1":0.8736127789799626, "value2":7151271249368522054, "value3":true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.384626, "value1":0.2370246172687121, "value2":2826036906774872004, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522883, "value1":0.6013577976454938, "value2":890649018657102134, "value3":true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.877768, "value1":0.6077458920362099, "value2":8966895480129060070, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.224033, "value1":0.0076863111785604905, "value2":6569047840639595510, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.311682, "value1":0.07593318166504835, "value2":3253540135578122766, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490459, "value1":0.4609451528285016, "value2":2994036544656680091, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.378898, "value1":0.9545401129487334, "value2":3668542248171100045, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416785, "value1":0.35541532907488144, "value2":94039845626229270, "value3":false, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275554, "value1":0.421200916650073, "value2":5818521501894570082, "value3":false, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619269, "value1":0.30772799684273167, "value2":4876402407267868155, "value3":false, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.702725, "value1":0.15927682148224467, "value2":1903456341847904299, "value3":false, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232604, "value1":0.7711772747260455, "value2":3866638254479737513, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.217772, "value1":0.8540182257688077, "value2":7309608218029816756, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705505, "value1":0.48163362162995105, "value2":1550283929814058088, "value3":false, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900193, "value1":0.23760307179291948, "value2":8646186763820142769, "value3":true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.208626, "value1":0.38931880434870053, "value2":5483530341831301025, "value3":false, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.568347, "value1":0.2801336860166125, "value2":9164397534745927432, "value3":false, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117458, "value1":0.981405746517363, "value2":6802896340427379587, "value3":false, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963035, "value1":0.7214166769862517, "value2":8440456704015434339, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700137, "value1":0.5934611351717767, "value2":6333818613628602547, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145303, "value1":0.6922047329264945, "value2":3009832451157760657, "value3":true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345925, "value1":0.7232346453151443, "value2":4791014890475627712, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619327, "value1":0.9932237622196707, "value2":3624811523135333449, "value3":false, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269730, "value1":0.7866373764087783, "value2":3746435711914598124, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.265782, "value1":0.05991121042342668, "value2":1951849832097700576, "value3":false, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.175543, "value1":0.5444637318639066, "value2":5220283889625062347, "value3":false, "__name__":"metric_00000991", "key_8":"h","key_5":"i", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677825, "value1":0.904620506580413, "value2":4669605336529676512, "value3":false, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668600, "value1":0.17366696171797397, "value2":8249554071220606769, "value3":true, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041222, "value1":0.3254693615668756, "value2":5262353711290630554, "value3":true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.243762, "value1":0.8759302538831941, "value2":7311945236881805486, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958625, "value1":0.5492731854343449, "value2":5435276136514133338, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290213, "value1":0.7949670770789082, "value2":7111766404096808279, "value3":false, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025988, "value1":0.1026128579407924, "value2":1755616401770874869, "value3":true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595099, "value1":0.18775955592545962, "value2":1656134813479826833, "value3":true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.671098, "value1":0.8079739199189518, "value2":3665225572712988915, "value3":false, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.542084, "value1":0.5181312401815146, "value2":1481804189008155276, "value3":true, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.784956, "value1":0.40357178109169595, "value2":9035114082415618141, "value3":true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.276517, "value1":0.24301724893563487, "value2":4602891919019880467, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064974, "value1":0.37706581069603073, "value2":7053383221578853097, "value3":false, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698510, "value1":0.45184858839742487, "value2":2258520036203869092, "value3":true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.218133, "value1":0.3113727295782559, "value2":6841422843138500205, "value3":false, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668881, "value1":0.25909755219748914, "value2":1185172632185740020, "value3":false, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.049479, "value1":0.9359900704241361, "value2":8443878057371326705, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.674394, "value1":0.5330986303950987, "value2":4597086230952392696, "value3":true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195073, "value1":0.23095513043718424, "value2":886484372856556987, "value3":false, "__name__":"metric_00000010", "key_6":"a","key_1":"e","key_5":"c", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.380352, "value1":0.9222632147295795, "value2":2011984636052581606, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.302156, "value1":0.7967664965647748, "value2":1936857391665696956, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673174, "value1":0.8085420465867998, "value2":6974688906284239573, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.013224, "value1":0.01982064999844403, "value2":3955016096628456095, "value3":true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090855, "value1":0.01304461484505885, "value2":8778318340262765164, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.795784, "value1":0.12217080131703652, "value2":2014947645475742270, "value3":false, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284442, "value1":0.42738762978686595, "value2":2584606560267930457, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.741760, "value1":0.5561052082820417, "value2":7263332550390986826, "value3":false, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035524, "value1":0.908173032025188, "value2":5685807943763240264, "value3":false, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025764, "value1":0.20925710637592332, "value2":6562668387404626293, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.530188, "value1":0.6447955243050295, "value2":3888614173619999623, "value3":false, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424157, "value1":0.17487330381635957, "value2":5301061833157028268, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026934, "value1":0.0422857387218024, "value2":8303721108125129087, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841847, "value1":0.5206472086096076, "value2":74348635869177909, "value3":true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.414469, "value1":0.4278094238360654, "value2":7010327929309184782, "value3":false, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491788, "value1":0.747156270778759, "value2":4790193170750716406, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399141, "value1":0.47388249427808166, "value2":8836306836420904183, "value3":false, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.160250, "value1":0.29496035919361546, "value2":1937679940636334558, "value3":true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787511, "value1":0.40198955034703965, "value2":5313833463463440565, "value3":true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684988, "value1":0.6177512623201419, "value2":1449869359073755283, "value3":false, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803253, "value1":0.8970896940692594, "value2":6178067064755804966, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964550, "value1":0.8199555931693112, "value2":140777809986023618, "value3":true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456407, "value1":0.2401625723920763, "value2":4030355648738753679, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849585, "value1":0.2629587147468046, "value2":7076050191155576404, "value3":false, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.824331, "value1":0.6892589926802947, "value2":6399755277819070915, "value3":false, "__name__":"metric_00000046", "key_4":"e","key_6":"h","key_2":"k", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388915, "value1":0.8127879446503501, "value2":5588278675440700300, "value3":false, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782971, "value1":0.22771808116681516, "value2":6261899644042013732, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712419, "value1":0.2905286238822736, "value2":5614218841386062674, "value3":false, "__name__":"metric_00000049", "key_3":"h","key_9":"j","key_1":"d", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249696, "value1":0.9445558673553629, "value2":1404486883141712140, "value3":true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601814, "value1":0.6990549633969345, "value2":7198014296771075924, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543813, "value1":0.07857387425059083, "value2":1217712980435541695, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123893, "value1":0.07983470162468347, "value2":7171526612504871775, "value3":false, "__name__":"metric_00000053", "key_4":"a","key_7":"k","key_1":"f", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703648, "value1":0.36447696895349185, "value2":5146632909527815894, "value3":true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166353, "value1":0.7098874569397392, "value2":5324193865568739754, "value3":true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399594, "value1":0.7745072356158436, "value2":7831862313374173936, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.439724, "value1":0.2037239485962775, "value2":3755069786706852308, "value3":true, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839998, "value1":0.6732355725193216, "value2":2639271267504971364, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228435, "value1":0.897840654678523, "value2":5116662644663764475, "value3":false, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973287, "value1":0.9461886626837847, "value2":5279391211817478931, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.441360, "value1":0.9419535299333034, "value2":2098864241711696584, "value3":false, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.780978, "value1":0.7212145622817746, "value2":8825784157879516761, "value3":true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.342306, "value1":0.40763808526506856, "value2":615440271373894957, "value3":true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964992, "value1":0.8428995412660205, "value2":5458378648341607708, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386901, "value1":0.3242872455426699, "value2":1659876068547712992, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.960102, "value1":0.2464238639205716, "value2":699750419933475491, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585099, "value1":0.11986390379255613, "value2":3678288424535052375, "value3":false, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.154037, "value1":0.7785920099800233, "value2":8275166075671164173, "value3":false, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211892, "value1":0.7853941226491261, "value2":6369532531235861115, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.746677, "value1":0.6664817644749024, "value2":4658328841108884689, "value3":true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.941862, "value1":0.846994201539624, "value2":1274645788690532949, "value3":true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.733827, "value1":0.13169457879746155, "value2":3221287182313964764, "value3":true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870127, "value1":0.37466507316122794, "value2":8856106508914542017, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798298, "value1":0.22462880312469063, "value2":6798911070080579970, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993366, "value1":0.9961738964349245, "value2":1193549068611970903, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.157412, "value1":0.4452570786392375, "value2":3487651503828456956, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.529021, "value1":0.26905395113611574, "value2":2025155869621977272, "value3":true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.701922, "value1":0.8730175680632801, "value2":997347931310956157, "value3":true, "__name__":"metric_00000078", "key_1":"a","key_9":"h","key_0":"b", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234577, "value1":0.7588051800901922, "value2":5319874977525060911, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943591, "value1":0.47603601658101, "value2":5946410545801626105, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.292747, "value1":0.03173464098642328, "value2":7226691638812613138, "value3":false, "__name__":"metric_00000057", "key_3":"b","key_5":"j","key_2":"a", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608463, "value1":0.2957253043607161, "value2":122362680185330457, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.643245, "value1":0.7167827619023723, "value2":5186237060875400709, "value3":false, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456051, "value1":0.2188755629521717, "value2":1845588654842798990, "value3":false, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231176, "value1":0.6418253580281563, "value2":3690123298454825, "value3":false, "__name__":"metric_00000061", "key_4":"d","key_5":"d","key_3":"f", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158253, "value1":0.7945876103242618, "value2":9004322525035166688, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234079, "value1":0.5202371436378874, "value2":1591535835129382659, "value3":false, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.875339, "value1":0.37493709867362873, "value2":8328159676060260451, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907034, "value1":0.20583691616211497, "value2":3631192064418800289, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.518876, "value1":0.16335464977880865, "value2":4643059165781979124, "value3":false, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185787, "value1":0.018176867671764584, "value2":3270163687436586406, "value3":true, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308787, "value1":0.4773555846410255, "value2":3253306657765008963, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297154, "value1":0.246942999840671, "value2":6872861714776356785, "value3":true, "__name__":"metric_00000084", "key_3":"k","key_9":"j","key_1":"c", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.817582, "value1":0.8165235911801363, "value2":8310469403306376604, "value3":true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493041, "value1":0.38500411997916434, "value2":6747567659837060435, "value3":false, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224106, "value1":0.9994238326374754, "value2":7099587552820760691, "value3":false, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031476, "value1":0.09066884358892856, "value2":5945790442760569800, "value3":true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311356, "value1":0.5806991883130863, "value2":6895176067401972683, "value3":true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653725, "value1":0.15503072276739358, "value2":7449412216901454949, "value3":false, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.598231, "value1":0.20373471056711764, "value2":1437658094331870355, "value3":false, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798108, "value1":0.6097451467316525, "value2":7133130757829716058, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709054, "value1":0.7371923755130838, "value2":7543405219351917468, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104790, "value1":0.21083306139991156, "value2":7929604552171608387, "value3":false, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.187877, "value1":0.5608708053678212, "value2":3368623446076987813, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749130, "value1":0.657437311445075, "value2":8949805346712651308, "value3":false, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582481, "value1":0.42362389624895797, "value2":8717288037606444575, "value3":true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.558438, "value1":0.370755803762819, "value2":1107999166234163628, "value3":false, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802675, "value1":0.8677299277452752, "value2":5915000168574046056, "value3":true, "__name__":"metric_00000108", "key_3":"j","key_0":"j","key_1":"c", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.208039, "value1":0.38781051864576976, "value2":3375198001539598542, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680278, "value1":0.886048145499295, "value2":4081527324669320081, "value3":false, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162287, "value1":0.7118801419064047, "value2":2390093012206667849, "value3":true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.092015, "value1":0.9942758150465311, "value2":4638994738698984564, "value3":true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039345, "value1":0.641165676244722, "value2":2976540841974698815, "value3":true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.408130, "value1":0.5042712547968512, "value2":8947846924433036861, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259088, "value1":0.514500988249709, "value2":2770449998520564629, "value3":false, "__name__":"metric_00000092", "key_4":"b","key_0":"j","key_3":"i", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293773, "value1":0.36521689212647834, "value2":67527608081891502, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646136, "value1":0.04048325982754749, "value2":4743191660759249600, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.047497, "value1":0.5502497475964416, "value2":7169106217640232030, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876800, "value1":0.3799501707452096, "value2":6757458314368411993, "value3":false, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968057, "value1":0.7499783045753542, "value2":1200059614173097893, "value3":true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.871681, "value1":0.6987940576652765, "value2":3787158661733344974, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.998419, "value1":0.7974184353878208, "value2":3011281681090708365, "value3":true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663978, "value1":0.1875336848417326, "value2":7682956450262415743, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799777, "value1":0.13431952299825461, "value2":8472008336913527353, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224565, "value1":0.7432835827699946, "value2":5363247813194187485, "value3":true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896272, "value1":0.1238866666650729, "value2":1249324477450532329, "value3":true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630720, "value1":0.15283582580094462, "value2":1819923308050662735, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405798, "value1":0.9413697480070442, "value2":6170928651103063980, "value3":true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249174, "value1":0.3594131764729996, "value2":6196624819261733193, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628627, "value1":0.5009434579166352, "value2":4913727766417338788, "value3":true, "__name__":"metric_00000130", "key_6":"d","key_9":"g","key_2":"e", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949656, "value1":0.677579274449204, "value2":4321897610912596942, "value3":false, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.452302, "value1":0.8465391778403211, "value2":8767695810058898547, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224741, "value1":0.9419255467202373, "value2":3942608565088456699, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832311, "value1":0.1844332411242929, "value2":7897621420940762914, "value3":false, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364450, "value1":0.6062623207548122, "value2":3048338263269528616, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458762, "value1":0.8964243000562225, "value2":6980106557138925502, "value3":true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090342, "value1":0.9353992150482529, "value2":3024974544306240484, "value3":true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680282, "value1":0.32391766175077186, "value2":855225884463637074, "value3":false, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949004, "value1":0.8992840018801868, "value2":5343754317476027418, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.564072, "value1":0.04591172153502947, "value2":15119713343690272, "value3":false, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.015169, "value1":0.2775250663576938, "value2":8329492228577328961, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035102, "value1":0.8955268703368635, "value2":1413978015063203049, "value3":true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.922310, "value1":0.5335365949387284, "value2":7752415045685811238, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.862402, "value1":0.328052481246616, "value2":1049471938369225232, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.632372, "value1":0.702701110691373, "value2":736041481181376478, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886888, "value1":0.8185435772395079, "value2":6423113939133946645, "value3":true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087072, "value1":0.4700081672384728, "value2":4642202730229975792, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.936650, "value1":0.18762496890040795, "value2":3355401686890613388, "value3":false, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032044, "value1":0.12307064568471555, "value2":5991756874915773532, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093295, "value1":0.22401921911686248, "value2":6120641321792430058, "value3":false, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.777098, "value1":0.29744828015347885, "value2":3084065102603938152, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.437450, "value1":0.2884592585353877, "value2":4044254904136084327, "value3":false, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.017135, "value1":0.7608822818277434, "value2":1205572766308158879, "value3":false, "__name__":"metric_00000143", "key_7":"a","key_4":"g","key_6":"d", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907629, "value1":0.7081298253215954, "value2":7688268403734929948, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924934, "value1":0.7192898599410195, "value2":1421354730652485196, "value3":false, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.736050, "value1":0.7502596301905261, "value2":2582210329738662673, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515235, "value1":0.09310016531489594, "value2":5123514648664672005, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.117213, "value1":0.07556633746315061, "value2":3501827326756011617, "value3":false, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.485929, "value1":0.9272991096008794, "value2":5159915940333201308, "value3":false, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296263, "value1":0.11108812397005534, "value2":8753717861080819638, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990089, "value1":0.527982415726353, "value2":5638704340740792965, "value3":false, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835455, "value1":0.062168150269998286, "value2":1944114908616022523, "value3":true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297201, "value1":0.30562887567201646, "value2":2937626694886920620, "value3":false, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974272, "value1":0.7221303752638616, "value2":4841762343070531864, "value3":true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.477606, "value1":0.011147194457821816, "value2":7938710785429203576, "value3":true, "__name__":"metric_00000152", "key_9":"h","key_0":"d","key_2":"k", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129918, "value1":0.5408385148641213, "value2":9195358942783956586, "value3":true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364995, "value1":0.0773648729840119, "value2":2987992944358913043, "value3":false, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.776833, "value1":0.8273262149027085, "value2":3110126394788152042, "value3":true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.610517, "value1":0.6781728258681893, "value2":4793737503812454856, "value3":true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896976, "value1":0.37188254064396126, "value2":3144974162911084078, "value3":true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202562, "value1":0.5271477787382647, "value2":6411589275983706377, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906770, "value1":0.4476129119528191, "value2":424159247916997254, "value3":true, "__name__":"metric_00000159", "key_9":"j","key_0":"e","key_3":"c", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289456, "value1":0.541155755467242, "value2":2999265325007368522, "value3":false, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.860878, "value1":0.4758705080993404, "value2":6690006483968082589, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.365448, "value1":0.027420735129761006, "value2":825754437394215422, "value3":true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212551, "value1":0.16839070098231337, "value2":1942763578810483030, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870900, "value1":0.4994077584006896, "value2":5993483889801645858, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.455124, "value1":0.26138016691309135, "value2":4427691071851307843, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.569012, "value1":0.4778238812129043, "value2":8556858598621845387, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.362641, "value1":0.2203054974232932, "value2":3477406531290022646, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493850, "value1":0.4787384781915333, "value2":2198408185614770688, "value3":false, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.513028, "value1":0.6429388402116987, "value2":7593443154020639176, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386729, "value1":0.17475131034566962, "value2":3647348637356138676, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053464, "value1":0.6091636844532319, "value2":5763485176228727883, "value3":false, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628348, "value1":0.023668653461306262, "value2":6178270391876561387, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913114, "value1":0.637992847841559, "value2":8248068010165238913, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.413158, "value1":0.5132505144116425, "value2":3488568643829613122, "value3":true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517474, "value1":0.21449952384417484, "value2":4829637720497890669, "value3":false, "__name__":"metric_00000177", "key_8":"d","key_4":"i","key_7":"c", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.659125, "value1":0.35348852774479717, "value2":929980299829000703, "value3":false, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071642, "value1":0.5598789974197187, "value2":7463399784012985552, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993416, "value1":0.6475276350489354, "value2":3252439000616730752, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003658, "value1":0.897666564581801, "value2":508174253835823210, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.629566, "value1":0.49532352920736916, "value2":3384274756820723223, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.084906, "value1":0.40471552937588173, "value2":2358566109120320700, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.108543, "value1":0.8814759719911462, "value2":940297244819349653, "value3":false, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138889, "value1":0.48057446422385713, "value2":5651682080133821762, "value3":false, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.468024, "value1":0.6607643026338559, "value2":3567273041505939519, "value3":false, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.278010, "value1":0.05185404011612328, "value2":9174087231001182435, "value3":false, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.095328, "value1":0.4998002617147629, "value2":4028395944074542796, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.962889, "value1":0.7966578010574328, "value2":907724426763786137, "value3":false, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.434695, "value1":0.596944451662117, "value2":6599947217349296930, "value3":false, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.695515, "value1":0.23670726317490795, "value2":2923186045419354751, "value3":false, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830693, "value1":0.43683284863611044, "value2":9110976313628947704, "value3":true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311809, "value1":0.18924644468054935, "value2":2283072307874475430, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405030, "value1":0.5026514010742291, "value2":7499615277819756210, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.985393, "value1":0.2214787811403981, "value2":1167173892132804942, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123017, "value1":0.24050302556695696, "value2":3512829503479749438, "value3":true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.275825, "value1":0.8989518053152799, "value2":7332940306078599750, "value3":false, "__name__":"metric_00000207", "key_3":"g","key_0":"k","key_1":"e", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.935513, "value1":0.422470893258827, "value2":1965139573445622612, "value3":true, "__name__":"metric_00000208", "key_4":"e","key_2":"a", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697239, "value1":0.9715985007012725, "value2":2441260821113474270, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297829, "value1":0.4121340516781959, "value2":7327020552945137669, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.488215, "value1":0.22392461646214185, "value2":5911938621135820572, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.206243, "value1":0.020325317058324376, "value2":9207086370751028426, "value3":true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183572, "value1":0.23206347801714602, "value2":6792315880866236659, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524533, "value1":0.9124779133103079, "value2":3221463708308920869, "value3":false, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.893528, "value1":0.4848003839426891, "value2":3814492558819679029, "value3":true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.752371, "value1":0.8170507918067189, "value2":6527155317426998401, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916551, "value1":0.342137272729241, "value2":1602041662641372111, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.037592, "value1":0.6500639685761466, "value2":3567065955735150085, "value3":false, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902253, "value1":0.7531115895009749, "value2":2543364566980292046, "value3":false, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683579, "value1":0.30275430657154007, "value2":7640650810082868472, "value3":true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956614, "value1":0.5403322951696091, "value2":332198111972672347, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202908, "value1":0.3832878669702297, "value2":880756788127038920, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.820586, "value1":0.34286605495032285, "value2":6526500882733787563, "value3":true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930022, "value1":0.3539159969493719, "value2":370073818816792799, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023457, "value1":0.047546161290410185, "value2":5398553854250083599, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303770, "value1":0.7186586727923895, "value2":9169471131017267259, "value3":false, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.756889, "value1":0.31103630013784395, "value2":3594098461981248414, "value3":false, "__name__":"metric_00000218", "key_8":"a","key_2":"k","key_7":"j", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212371, "value1":0.3882294279634762, "value2":2428299028227660876, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.472113, "value1":0.19389271171622363, "value2":7737936344522555466, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051213, "value1":0.12920469147183916, "value2":7837057207029745863, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907093, "value1":0.017956122904157993, "value2":4878755396476119431, "value3":true, "__name__":"metric_00000232", "key_4":"e","key_7":"a","key_2":"g", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.525258, "value1":0.49782305829726053, "value2":2466466392623808778, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913492, "value1":0.4704351769152427, "value2":7908561685667216260, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290702, "value1":0.4198914182070395, "value2":6819583912532498504, "value3":false, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125217, "value1":0.04410603241691789, "value2":6832161683827996201, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004937, "value1":0.13145801093930243, "value2":5314457688060891725, "value3":true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.656896, "value1":0.41192120722520353, "value2":5457893604153821743, "value3":false, "__name__":"metric_00000238", "key_5":"k","key_7":"e","key_4":"a", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.700497, "value1":0.8047394500892703, "value2":8180560267702238361, "value3":true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553716, "value1":0.7484558334038893, "value2":1003966921986785454, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966183, "value1":0.8167158291471999, "value2":2250434045285686743, "value3":false, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663750, "value1":0.7127353578019723, "value2":3741760060639503306, "value3":true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686363, "value1":0.9712701187599743, "value2":3622116724705792870, "value3":true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.103723, "value1":0.4724493721726449, "value2":8181583911299132150, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.484649, "value1":0.18448759636607337, "value2":4053994543260017135, "value3":false, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.925440, "value1":0.06382834538328273, "value2":6916231632621539461, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939805, "value1":0.22424399111258897, "value2":4351472309231431136, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792665, "value1":0.33436392964606154, "value2":2696421529224538356, "value3":true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054274, "value1":0.9272819232934056, "value2":7264556561094806515, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.983823, "value1":0.7989127551970725, "value2":5239695063979297186, "value3":true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714766, "value1":0.6892792624795272, "value2":270252571976728726, "value3":false, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.263106, "value1":0.64122047400478, "value2":3421795383581628735, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720260, "value1":0.8087795736115097, "value2":8426217720617766298, "value3":true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839070, "value1":0.5917446619321536, "value2":390571547986932413, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966990, "value1":0.2432955344040808, "value2":8461111581798764026, "value3":false, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.808072, "value1":0.3216500857620197, "value2":4440941489040196963, "value3":false, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091958, "value1":0.3995806295500445, "value2":1180768478663237716, "value3":true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139366, "value1":0.11507703668400583, "value2":4516311843825616388, "value3":false, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.251512, "value1":0.7632852678548858, "value2":6418768969046377858, "value3":true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963533, "value1":0.8213013619231073, "value2":6101164365690068241, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.466433, "value1":0.28350424710105476, "value2":5563469458031181794, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831271, "value1":0.8821293582204699, "value2":3430349356503249989, "value3":true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003553, "value1":0.7323647142880576, "value2":5739925461894436779, "value3":true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.241144, "value1":0.3850622637508932, "value2":6137203197960360313, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.872100, "value1":0.3184658084196574, "value2":2061373587207508318, "value3":false, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.900532, "value1":0.2888524955130242, "value2":6600516250776964704, "value3":false, "__name__":"metric_00000256", "key_9":"c","key_3":"g","key_4":"b", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142058, "value1":0.7572218923237818, "value2":5437617110907280837, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.167933, "value1":0.20087813542991115, "value2":8497531403193171323, "value3":false, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977657, "value1":0.49027491644969856, "value2":6846007575513730281, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.337497, "value1":0.7535889949814326, "value2":4191915280375173080, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091305, "value1":0.41827531366800946, "value2":3553824659208645760, "value3":false, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.260475, "value1":0.39071909544867817, "value2":3828966522631037754, "value3":false, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421117, "value1":0.6070501451115596, "value2":637677502764251310, "value3":false, "__name__":"metric_00000263", "key_6":"d","key_3":"a","key_5":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949074, "value1":0.8719785824911194, "value2":739855289497042435, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005847, "value1":0.564214486086032, "value2":3879428464931256295, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608985, "value1":0.8120552257475274, "value2":6222621751128276018, "value3":true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.505971, "value1":0.44491083173920337, "value2":2966233843840872053, "value3":false, "__name__":"metric_00000269", "key_6":"b","key_1":"d","key_2":"f", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686524, "value1":0.910573536000882, "value2":7259657495340557823, "value3":false, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386181, "value1":0.5690887314353372, "value2":7696797346941049468, "value3":true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004437, "value1":0.2837421940276855, "value2":3630048754825451133, "value3":true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197835, "value1":0.3478392366591849, "value2":8924518323717745322, "value3":true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404754, "value1":0.14818758453959469, "value2":3841840031846521039, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.847397, "value1":0.10611042816524652, "value2":3749767974486199531, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773493, "value1":0.16710420925739453, "value2":2156267597938791257, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253265, "value1":0.39770730374803015, "value2":8078016986748379936, "value3":true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720298, "value1":0.3023956927195079, "value2":2183933305568131133, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.306860, "value1":0.13870609943567333, "value2":5783110454957795198, "value3":true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189190, "value1":0.3825441585131095, "value2":2493897460883724129, "value3":true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842956, "value1":0.712749117246484, "value2":1337306979661530320, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045437, "value1":0.9278250040101162, "value2":6116936767056337825, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.606083, "value1":0.4732072236705937, "value2":6967738612762397759, "value3":false, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094844, "value1":0.9269374913746184, "value2":2591132307871233919, "value3":false, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369784, "value1":0.09045009486516034, "value2":7834710052863465672, "value3":false, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032637, "value1":0.16606039490952637, "value2":2737666598235173614, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.394163, "value1":0.47507719343340277, "value2":313537922371752138, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.987576, "value1":0.5780445815848603, "value2":4108970207058212125, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906463, "value1":0.03465970899762972, "value2":6841259554497483309, "value3":false, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.266096, "value1":0.6742131298888875, "value2":7025803245311237704, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963062, "value1":0.5904971275955457, "value2":2254362569087327315, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.130304, "value1":0.868734941900672, "value2":1538504759997860509, "value3":true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.955104, "value1":0.5994289988135004, "value2":3881947199005484386, "value3":false, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.480748, "value1":0.49853130606809243, "value2":727938692981473193, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.682757, "value1":0.38003013064353247, "value2":8479678523613035623, "value3":false, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792753, "value1":0.16576560662385773, "value2":7797109252869584262, "value3":false, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.510440, "value1":0.6049726150607012, "value2":8669482832353600257, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.638136, "value1":0.15014964585075946, "value2":6225706505785608080, "value3":true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.522674, "value1":0.3512111717300567, "value2":3641814037368457348, "value3":false, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785473, "value1":0.4544503987969124, "value2":4635364210597620767, "value3":true, "__name__":"metric_00000308", "key_2":"f","key_4":"b","key_1":"f", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110745, "value1":0.5221211963164464, "value2":4903849187407455848, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195405, "value1":0.08594736781118724, "value2":2859597321511204836, "value3":true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051298, "value1":0.8256101721258909, "value2":290470632378621656, "value3":true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284220, "value1":0.27211501738111415, "value2":2134420797562982003, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792536, "value1":0.9372077558354763, "value2":7927244557057049645, "value3":false, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.681776, "value1":0.4477409740262762, "value2":3995977364357841936, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231810, "value1":0.6078487538143919, "value2":3261959160546744959, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.131613, "value1":0.7340176818759, "value2":3565658759712151137, "value3":true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.908393, "value1":0.28199967714832963, "value2":3350748733246898915, "value3":false, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698846, "value1":0.043919603046489966, "value2":4515802998753688491, "value3":false, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803882, "value1":0.9014217952934828, "value2":8345388974528376469, "value3":true, "__name__":"metric_00000291", "key_2":"a","key_3":"i","key_0":"d", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501652, "value1":0.45069419646062914, "value2":4400331011414194610, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.240902, "value1":0.34849466754791153, "value2":3738192721082184330, "value3":false, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.950263, "value1":0.5119937423488377, "value2":8919897235633479114, "value3":false, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.443897, "value1":0.08318284858972097, "value2":8878946431124018702, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653076, "value1":0.2711114129019499, "value2":4904832384185588956, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199562, "value1":0.5179430768249581, "value2":2338622690587545147, "value3":true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902254, "value1":0.8456128690164353, "value2":3512723913452353513, "value3":true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424568, "value1":0.358235963026908, "value2":1951463002559117594, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.891020, "value1":0.899730727943035, "value2":384925737643795292, "value3":false, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242327, "value1":0.8194521049409685, "value2":1651591731883214718, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064957, "value1":0.5094969888256025, "value2":1137790773894543912, "value3":false, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032406, "value1":0.3455331481317415, "value2":2958859262258749395, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553817, "value1":0.484360827667417, "value2":3796071195831724862, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197274, "value1":0.4874943149488065, "value2":1044578434864087492, "value3":true, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.384646, "value1":0.3107814255944672, "value2":1519235564474675279, "value3":true, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286271, "value1":0.42853641527210906, "value2":4722523490137371754, "value3":false, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175308, "value1":0.3946055220580569, "value2":6954032044064595773, "value3":true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.793904, "value1":0.06297728493572644, "value2":5259172994496601557, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.062223, "value1":0.013288825435356888, "value2":6140160587654305146, "value3":true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.768622, "value1":0.39922765836242, "value2":6629678017270316706, "value3":false, "__name__":"metric_00000339", "key_5":"b","key_9":"g","key_4":"k", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.203380, "value1":0.9727752066385791, "value2":960040697676306380, "value3":true, "__name__":"metric_00000340", "key_8":"i","key_0":"i","key_3":"d", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.845192, "value1":0.9218669222513459, "value2":8161692681657646744, "value3":true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625176, "value1":0.7818690860470178, "value2":6286486510404426651, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943535, "value1":0.004848635402133272, "value2":8498014405632225004, "value3":true, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.204456, "value1":0.5833134681081519, "value2":7410659829376107064, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.576114, "value1":0.9192372118815992, "value2":6657353407785586037, "value3":true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118284, "value1":0.8313517670516266, "value2":7764880758391828110, "value3":true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.535569, "value1":0.24827312684994104, "value2":2040106521001592824, "value3":true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.549323, "value1":0.8166897779780931, "value2":1564430022376434258, "value3":true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881818, "value1":0.09926733907609549, "value2":3471220368243877954, "value3":false, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374110, "value1":0.5149369120619428, "value2":2786214942305343351, "value3":true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770127, "value1":0.7580522569953325, "value2":4637323015723129783, "value3":true, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178499, "value1":0.146709310096475, "value2":7936042836162333987, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274162, "value1":0.4372447015799375, "value2":9083170531856859501, "value3":true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491205, "value1":0.022796016225098064, "value2":8123705023600354614, "value3":true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016451, "value1":0.9256503393490508, "value2":6970208660239633828, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087308, "value1":0.4462899441715343, "value2":1749521530182375024, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094183, "value1":0.3475409881711034, "value2":5390535090747181247, "value3":true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.735505, "value1":0.53372762504523, "value2":8372678000097135865, "value3":true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.114823, "value1":0.029693120979849812, "value2":6638472287910475525, "value3":true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242928, "value1":0.3050035672228757, "value2":1792917800424529821, "value3":true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703291, "value1":0.31694612370345837, "value2":2183523770285388049, "value3":true, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.331904, "value1":0.47208239473287594, "value2":6204971001548635721, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.345432, "value1":0.4951178136444847, "value2":3855507055566939240, "value3":true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136145, "value1":0.6933172444157865, "value2":1571591012651895136, "value3":false, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303587, "value1":0.05226903625691088, "value2":6994464805121874342, "value3":true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.604568, "value1":0.14657045529476723, "value2":2698838307937260948, "value3":true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376282, "value1":0.8133509413304958, "value2":8456674940436282992, "value3":false, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843511, "value1":0.6514108937273068, "value2":7171882174493021649, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888039, "value1":0.34434154548426427, "value2":5546478000801091154, "value3":false, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663092, "value1":0.2145932369532521, "value2":1924108819168508153, "value3":true, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178424, "value1":0.3811642005135661, "value2":5277232346465674780, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.982420, "value1":0.509903018966164, "value2":1366973490493465213, "value3":false, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.789914, "value1":0.5590817594064089, "value2":2545283978114147454, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.858245, "value1":0.2990011327668893, "value2":7402472987921684090, "value3":true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045358, "value1":0.4729812652892598, "value2":6922132388747645101, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212994, "value1":0.6311902604147308, "value2":4414936740257073237, "value3":true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357341, "value1":0.06207502931784455, "value2":4475293686563973956, "value3":false, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.975194, "value1":0.41868388111482774, "value2":1854059337783469990, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841014, "value1":0.21676180216590327, "value2":4591196137073503812, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.978140, "value1":0.3174552368999545, "value2":1901352595947591853, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.267125, "value1":0.18049579402909974, "value2":4625546630316797154, "value3":true, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916965, "value1":0.5612711107070505, "value2":6029229351915879062, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.927166, "value1":0.40285609194050886, "value2":5993132840270480501, "value3":true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791545, "value1":0.4639522865587751, "value2":7463548107279858536, "value3":true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770063, "value1":0.26886367149111556, "value2":3097459774572125541, "value3":true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077189, "value1":0.7572977836842385, "value2":8298753827870878088, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.287375, "value1":0.621000003066286, "value2":5005851683916784910, "value3":true, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517180, "value1":0.6660153705066499, "value2":8387087977559252312, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.675034, "value1":0.9813881401454424, "value2":3741676304938251291, "value3":true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.554896, "value1":0.7715073370679849, "value2":6589060977722716090, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319288, "value1":0.12279901479898898, "value2":3422770912327659239, "value3":true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703895, "value1":0.8358860148624778, "value2":8257794540400854550, "value3":false, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493064, "value1":0.048118109532117544, "value2":4134373887037968422, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799328, "value1":0.11593192650373021, "value2":1597084067805135642, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.848966, "value1":0.3986720724086452, "value2":2405658508790196207, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792447, "value1":0.5248994141540482, "value2":6715402324504357457, "value3":false, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376734, "value1":0.7992683845538152, "value2":636776368616526730, "value3":false, "__name__":"metric_00000366", "key_8":"b","key_2":"j","key_6":"i", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646082, "value1":0.7040359351145827, "value2":8792341972386350386, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.358717, "value1":0.15149972457126396, "value2":8525880284771508381, "value3":false, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038854, "value1":0.6775307407388963, "value2":4497884453622086771, "value3":false, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313173, "value1":0.6408664944912432, "value2":9170385529548968990, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.209148, "value1":0.8824832965439914, "value2":6994782182036294339, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.349829, "value1":0.6332729838723153, "value2":8895300460695986955, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877356, "value1":0.9629884067987685, "value2":5874587318078108417, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175639, "value1":0.2215504394755138, "value2":1645626442028267495, "value3":true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.867334, "value1":0.17245948139280137, "value2":7635160136876279780, "value3":true, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.304089, "value1":0.4417775050588339, "value2":4663940359268319500, "value3":true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.377260, "value1":0.3699273314519082, "value2":3098095013643812538, "value3":false, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.687645, "value1":0.49899496293822504, "value2":3483082050221607102, "value3":true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.764258, "value1":0.5854336662763441, "value2":9096847150126021331, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.395360, "value1":0.4621587456489886, "value2":4751829496417501448, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702903, "value1":0.41598100403233135, "value2":3929201336867975918, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107090, "value1":0.7914140452826858, "value2":3589076543012868088, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.487563, "value1":0.4474861481593704, "value2":2779456470910748695, "value3":true, "__name__":"metric_00000406", "key_6":"j","key_7":"f","key_2":"i", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512119, "value1":0.5021386207405886, "value2":5678715906223819621, "value3":false, "__name__":"metric_00000402", "key_4":"k","key_7":"j","key_2":"h", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346070, "value1":0.55706872148054, "value2":2039287696528568559, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.574280, "value1":0.3713448367655447, "value2":3929829805998447613, "value3":true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.892079, "value1":0.6942759872823322, "value2":7729227663415283404, "value3":false, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.344002, "value1":0.42092378887559295, "value2":4794883782773019081, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078411, "value1":0.6154878270099419, "value2":6707716797669517904, "value3":false, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.571429, "value1":0.1378751349573672, "value2":8074751840319910755, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.040055, "value1":0.6408943986817053, "value2":1079211030051415138, "value3":true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162869, "value1":0.5000621894976172, "value2":5126515351324361210, "value3":false, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.390768, "value1":0.5289053805740587, "value2":7495442762687396988, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259180, "value1":0.5649142133543891, "value2":4328071162032108601, "value3":true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842704, "value1":0.21495927580985347, "value2":2932451180465678632, "value3":true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650755, "value1":0.6984634581926075, "value2":5766024562422825714, "value3":false, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612080, "value1":0.22851599558447847, "value2":5344041318095855549, "value3":true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990865, "value1":0.7354893379622883, "value2":9025665650100041943, "value3":false, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787926, "value1":0.33568512834020503, "value2":6804173344109850974, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172355, "value1":0.9139245088414795, "value2":6155807217840657379, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.979541, "value1":0.24199196392953898, "value2":5042790445163279739, "value3":false, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.314666, "value1":0.3110154918771059, "value2":8906652806658780575, "value3":true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357408, "value1":0.8063999303874789, "value2":1651751776595868392, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.288676, "value1":0.9293974960471005, "value2":7670397636585827003, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293262, "value1":0.7818217756805212, "value2":8606918715162923765, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.083534, "value1":0.9670843857632683, "value2":5887729702260327460, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149981, "value1":0.6552447379991068, "value2":6725961755903338686, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.492842, "value1":0.7092364133108918, "value2":7796995215212529618, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944648, "value1":0.6256326288941183, "value2":4831063196252879183, "value3":true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.895337, "value1":0.4995752655149396, "value2":1408597026195247818, "value3":false, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944023, "value1":0.634711000263352, "value2":5584697240807054271, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809484, "value1":0.7426498850085607, "value2":8020687455958940501, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289743, "value1":0.008820277060811819, "value2":6337782472142563494, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.537111, "value1":0.14436676295527084, "value2":1783901321543975972, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749042, "value1":0.9467021656046419, "value2":279529980240550876, "value3":true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.261740, "value1":0.20334209964882055, "value2":3677408684840846866, "value3":false, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.272200, "value1":0.5140769192808836, "value2":3965788713393972569, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391452, "value1":0.5620808926889774, "value2":3432925103242153786, "value3":false, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.797201, "value1":0.24306630830309864, "value2":2212488327265931040, "value3":false, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296686, "value1":0.4958413853936345, "value2":2745010941408731624, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.508150, "value1":0.1764099610149601, "value2":6087888705783067229, "value3":true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.150030, "value1":0.6475061379725164, "value2":4152410772475568903, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406997, "value1":0.8552543065924282, "value2":6943121315248371505, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798595, "value1":0.4482032174944134, "value2":4591949889203793788, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427353, "value1":0.23427032816286456, "value2":8455901174434850105, "value3":true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.426947, "value1":0.2157937977395769, "value2":4812396978108020858, "value3":true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553964, "value1":0.6569888020417926, "value2":4304633456065882312, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830280, "value1":0.6753558122351391, "value2":5709360383833750840, "value3":true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335737, "value1":0.6229413264866215, "value2":774141312393774432, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934503, "value1":0.3184401780744072, "value2":992485213419150513, "value3":false, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286280, "value1":0.19701195292124982, "value2":5131579316119627914, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.027527, "value1":0.5595644624118316, "value2":7324008691917344768, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.445835, "value1":0.3878002605330551, "value2":5283397242149349986, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.317903, "value1":0.34865399024368093, "value2":919918660600814082, "value3":true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.988293, "value1":0.520877683344731, "value2":1362287203583282026, "value3":true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451918, "value1":0.17797518451009597, "value2":8069259017842130934, "value3":true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.416614, "value1":0.49438798293161773, "value2":8221530176741871409, "value3":true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930143, "value1":0.11046223267799947, "value2":1569220098038190448, "value3":false, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841002, "value1":0.2351074608491269, "value2":6987153620614536862, "value3":true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172031, "value1":0.30378326240838155, "value2":3438948688273621480, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427034, "value1":0.3726321157197697, "value2":3892410333661915590, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.996688, "value1":0.06148106153681236, "value2":7012776674540552021, "value3":true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178139, "value1":0.24652480255103007, "value2":7264782641186044646, "value3":true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.641890, "value1":0.18896088271258277, "value2":5281363384350384982, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020470, "value1":0.5427453076719968, "value2":4211018551963613162, "value3":true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548627, "value1":0.9467235670877983, "value2":1356346215173760766, "value3":true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923887, "value1":0.16868132666008845, "value2":2419587552380208808, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388273, "value1":0.34607395217688514, "value2":6360018437765039879, "value3":false, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199475, "value1":0.48700761251474795, "value2":4207130869528000977, "value3":false, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212632, "value1":0.7298887109862953, "value2":3855225371708173636, "value3":false, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869081, "value1":0.3040832001677732, "value2":1636252957386132843, "value3":false, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458358, "value1":0.25499479265263875, "value2":7951534915845928364, "value3":false, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673427, "value1":0.3560914688953808, "value2":4486505974707646981, "value3":false, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934304, "value1":0.9086314287420252, "value2":6454407382033994812, "value3":false, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077472, "value1":0.272412312785447, "value2":4055844210948982160, "value3":false, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391470, "value1":0.4094306269697527, "value2":8737172085844354362, "value3":false, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.507196, "value1":0.4293932426928758, "value2":7752579176339832618, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.354922, "value1":0.8782836749723913, "value2":3092074071663841238, "value3":false, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290283, "value1":0.6466038651665884, "value2":6445966108572410248, "value3":true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930993, "value1":0.2534798517547614, "value2":2212538268968967382, "value3":false, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078571, "value1":0.766186870171555, "value2":2276988139922811436, "value3":true, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512143, "value1":0.26492881415396713, "value2":7960469334919381452, "value3":false, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.664663, "value1":0.9293891578195701, "value2":8363525489715122179, "value3":false, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.859809, "value1":0.5317709711783597, "value2":5646146960428890136, "value3":false, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755810, "value1":0.5033479266486215, "value2":1442438201922477703, "value3":false, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.014944, "value1":0.2736949295034319, "value2":4543402002213224798, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.019842, "value1":0.06301734338783647, "value2":7848421926410748778, "value3":true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.006460, "value1":0.7283128915704968, "value2":8696821133231766119, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.495665, "value1":0.22357458621454834, "value2":5013935622275517771, "value3":true, "__name__":"metric_00000490", "key_5":"j","key_9":"c","key_0":"k", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.829438, "value1":0.08516153321035196, "value2":6346747105122069788, "value3":false, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120479, "value1":0.6330301971897442, "value2":3074605270601257967, "value3":false, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406377, "value1":0.9646902889346889, "value2":9116844958473667659, "value3":false, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258382, "value1":0.9290313183449712, "value2":7925410787408077827, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340705, "value1":0.7460138699765932, "value2":1834174806073680705, "value3":false, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.660263, "value1":0.9250659950303504, "value2":6999248688568055694, "value3":false, "__name__":"metric_00000496", "key_6":"g","key_0":"h","key_5":"f", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252290, "value1":0.718659365662016, "value2":414864145410727491, "value3":false, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404912, "value1":0.5143974391105166, "value2":6178244170296726344, "value3":false, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.355231, "value1":0.7285417682392774, "value2":4099314188219635809, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232613, "value1":0.9628251247534773, "value2":2753347640662156751, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042238, "value1":0.40260777202369225, "value2":6405687740970757268, "value3":false, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.059614, "value1":0.6706802975294311, "value2":155105754975301806, "value3":false, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335427, "value1":0.03875943420721546, "value2":2621496673126808846, "value3":true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791994, "value1":0.4069226188854093, "value2":8911123850930154192, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.152311, "value1":0.001001700535447483, "value2":8160805255392424399, "value3":false, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.806126, "value1":0.09279090865200551, "value2":5165528109888127485, "value3":false, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181230, "value1":0.34297422042642206, "value2":5952935302808125089, "value3":false, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308758, "value1":0.39399952685268963, "value2":2291587630268260397, "value3":true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438229, "value1":0.008760206538330227, "value2":4899262456651209905, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253853, "value1":0.7708129134752543, "value2":5591299897276344295, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388133, "value1":0.4749440177332357, "value2":1978415349188199488, "value3":true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612871, "value1":0.07573141562457633, "value2":5953047090784791116, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.481277, "value1":0.12674805322449534, "value2":8950535310944284362, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026417, "value1":0.13620073115890582, "value2":5838422267573225081, "value3":true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491782, "value1":0.6697421435610075, "value2":5470323240276494634, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599889, "value1":0.2218252683274155, "value2":3863404473966363047, "value3":false, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028452, "value1":0.8531657224160458, "value2":1800467248475944001, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234267, "value1":0.2783481987951185, "value2":6997165241525177240, "value3":false, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369705, "value1":0.5636516367093837, "value2":4958650393346745090, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201031, "value1":0.628281679414999, "value2":3983210515283198963, "value3":true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.519582, "value1":0.950842451530494, "value2":8333725115642196357, "value3":false, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.800667, "value1":0.47362573201727787, "value2":5531339313881866007, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.022715, "value1":0.7456986400451252, "value2":6375649752818307396, "value3":true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026963, "value1":0.013441045212029653, "value2":8879122156886303116, "value3":false, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849167, "value1":0.7209297073484169, "value2":5571977114960029707, "value3":true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916214, "value1":0.07478238462060836, "value2":4957974530052652715, "value3":false, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.801842, "value1":0.5497676040427942, "value2":7068572572815929980, "value3":true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.180884, "value1":0.16934771421243305, "value2":4129529446885753081, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742335, "value1":0.17996128783258833, "value2":1612889506435491518, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.124131, "value1":0.6211285761421723, "value2":6936437753872174981, "value3":false, "__name__":"metric_00000531", "key_4":"j","key_2":"c","key_3":"h", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.626664, "value1":0.1412142921060844, "value2":7340286601023012024, "value3":true, "__name__":"metric_00000532", "key_9":"k","key_7":"e", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534599, "value1":0.1824543867650024, "value2":2710596260731528329, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.198428, "value1":0.563515863993397, "value2":8658671925349980639, "value3":true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939981, "value1":0.5602221119588965, "value2":1332664463057731952, "value3":true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749870, "value1":0.27530155912076365, "value2":3294589175109845443, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501991, "value1":0.28276078133217086, "value2":6074437039967216201, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886807, "value1":0.3878328960046618, "value2":6751632027412168403, "value3":true, "__name__":"metric_00000536", "key_8":"e","key_3":"d","key_7":"f", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.911289, "value1":0.6411363324230984, "value2":8252067497790790379, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.696807, "value1":0.43753398310350267, "value2":3790908907624313334, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319319, "value1":0.8437649419939369, "value2":4536112071098596928, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.734427, "value1":0.7768336856428384, "value2":577814262800235815, "value3":true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232482, "value1":0.4639181971383516, "value2":915233793101361418, "value3":true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033045, "value1":0.6840737140076493, "value2":6077817095892263388, "value3":true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427451, "value1":0.7412294644798527, "value2":5067651533453787714, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.086307, "value1":0.884916959850894, "value2":6213157740517107791, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032588, "value1":0.7692661642362982, "value2":3255905758159811447, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129133, "value1":0.9868818324607139, "value2":2840159926392322636, "value3":true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005933, "value1":0.4856659261691267, "value2":5401836471710156191, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078678, "value1":0.43872499297363293, "value2":7610348477992750069, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.101709, "value1":0.6553052079447295, "value2":5590613299241898040, "value3":false, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.691765, "value1":0.5500719022328135, "value2":2311787078857040546, "value3":true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.609841, "value1":0.20437362444327167, "value2":5976709159513367946, "value3":false, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.262383, "value1":0.988890785552926, "value2":5073782211732867167, "value3":false, "__name__":"metric_00000554", "key_5":"g","key_9":"g","key_1":"e", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382099, "value1":0.7056400640121613, "value2":1362786206799762754, "value3":false, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.910924, "value1":0.740118438943497, "value2":5204241364776707180, "value3":false, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526567, "value1":0.33926045568377805, "value2":8282435047009117311, "value3":true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259549, "value1":0.6968073477407744, "value2":5508984811117632066, "value3":false, "__name__":"metric_00000567", "key_6":"a","key_2":"i","key_4":"b", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.575379, "value1":0.9204583134801098, "value2":7328623658034172463, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.021181, "value1":0.8037794671210114, "value2":8763012613490416638, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666802, "value1":0.735290222214112, "value2":5898346647756703163, "value3":true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836023, "value1":0.9774841401477888, "value2":262883488175479807, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085239, "value1":0.2006523822964799, "value2":6927806981819194391, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722778, "value1":0.29750991877070054, "value2":7269146382542606407, "value3":true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585823, "value1":0.7790122698056057, "value2":5454870737254023995, "value3":false, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966399, "value1":0.9342783027092735, "value2":1736810747318512793, "value3":true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.363378, "value1":0.5421736888212692, "value2":8680345426115136675, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548518, "value1":0.6180502927475818, "value2":576215651567739722, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169252, "value1":0.8808696517971845, "value2":3266464067570846970, "value3":true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032461, "value1":0.3854322648479189, "value2":5351736110848205547, "value3":false, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418352, "value1":0.9282836374954182, "value2":4950655664039540565, "value3":true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.567245, "value1":0.06221685577996024, "value2":6376294794655595743, "value3":false, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.607825, "value1":0.8284877360321535, "value2":8836357162970050736, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.002720, "value1":0.9429757239354525, "value2":3712886085226721402, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772087, "value1":0.8045038867848936, "value2":4759138460922621996, "value3":true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515031, "value1":0.2984015395169622, "value2":2069584478283962675, "value3":false, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.141150, "value1":0.6666900797494949, "value2":268184099180001198, "value3":false, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.102173, "value1":0.08736073078451814, "value2":3669960405293392951, "value3":false, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.825677, "value1":0.883585003117732, "value2":4900464079888733154, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.613326, "value1":0.2003226722632331, "value2":3796686759340146598, "value3":true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.965080, "value1":0.24041472099907274, "value2":7454089011589464567, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139395, "value1":0.8846967167208682, "value2":3451086131264105278, "value3":false, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709618, "value1":0.40080113749800483, "value2":7347906568423400039, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.490979, "value1":0.10232414632757307, "value2":6898384399737921493, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.419810, "value1":0.9521194603097142, "value2":4165710553126975748, "value3":true, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142251, "value1":0.3656369505236563, "value2":2656042717790241170, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578775, "value1":0.42161623774225904, "value2":7959202830983535548, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340081, "value1":0.47208224231080925, "value2":1119972655203312039, "value3":true, "__name__":"metric_00000597", "key_3":"e","key_6":"j","key_2":"k", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201301, "value1":0.686282703890728, "value2":1347056528311053195, "value3":true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.905450, "value1":0.1504198445233754, "value2":4153183079049561671, "value3":true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.774660, "value1":0.04197441149840889, "value2":2892741854923361469, "value3":false, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.223828, "value1":0.9913278205978917, "value2":2634921019433133187, "value3":true, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145135, "value1":0.9764589636119458, "value2":1863333686324511619, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051457, "value1":0.9664880803510422, "value2":5468675095205300977, "value3":false, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.471202, "value1":0.9438594012185837, "value2":8711551131933841419, "value3":false, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438432, "value1":0.7332734301616686, "value2":6736601196370014167, "value3":false, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.018444, "value1":0.6166410041335356, "value2":7430067770903432071, "value3":false, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042785, "value1":0.3083845760668554, "value2":7529797289952981250, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.500528, "value1":0.41360737530731173, "value2":527946418421906650, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.639029, "value1":0.28120003518234893, "value2":7333246850823447562, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118602, "value1":0.22780592472876143, "value2":5562449967641903101, "value3":true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773447, "value1":0.34556648930141964, "value2":3454919255414457289, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.751050, "value1":0.5691145183911931, "value2":3420985612149611112, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.967122, "value1":0.010774686710818307, "value2":1857363403020110299, "value3":false, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211203, "value1":0.5071144540997159, "value2":3201415543625346596, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966034, "value1":0.10396047428400347, "value2":9014157472738440520, "value3":true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253319, "value1":0.43609058248995825, "value2":1183645881443278471, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104187, "value1":0.4324653404306875, "value2":2865322183450789108, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185518, "value1":0.8897028996083935, "value2":7566651396044601727, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930391, "value1":0.8041371194633276, "value2":7871648174569158725, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163568, "value1":0.9301616644769536, "value2":7748526202149251971, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869278, "value1":0.88349271636991, "value2":554007324522749523, "value3":true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.213408, "value1":0.664832495384472, "value2":2758153864408048603, "value3":true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451136, "value1":0.3702265627491039, "value2":494580882184796857, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915009, "value1":0.6519575592422996, "value2":6479471856073874185, "value3":true, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.581364, "value1":0.0703141245253348, "value2":899491222964044388, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142057, "value1":0.9957531092014932, "value2":5996532724034214303, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618561, "value1":0.31275866348570913, "value2":4261560321529985020, "value3":true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526426, "value1":0.8632813331612903, "value2":2280841193196363442, "value3":false, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.640741, "value1":0.4671010753968709, "value2":2523475617435456826, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686774, "value1":0.37684937337218677, "value2":6568317226871778750, "value3":false, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.547284, "value1":0.9412532971618462, "value2":6396494980432088364, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630465, "value1":0.10697292615695254, "value2":6790210135628014818, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156319, "value1":0.10115374361141051, "value2":2774605952238778168, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742080, "value1":0.4345761934130475, "value2":6896687242300333410, "value3":false, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662119, "value1":0.3886014719849511, "value2":2171378209788759856, "value3":false, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158805, "value1":0.4473032074837117, "value2":586039993535786710, "value3":false, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025398, "value1":0.0651048756141861, "value2":3136658443010387115, "value3":true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189314, "value1":0.11574796728414168, "value2":5756732023607570312, "value3":false, "__name__":"metric_00000631", "key_9":"d","key_1":"i","key_8":"g", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469000, "value1":0.3187212262703272, "value2":4880377874531438837, "value3":true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698910, "value1":0.9021085577061128, "value2":8314934978791640839, "value3":true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228153, "value1":0.7939593629687762, "value2":5721748474729655997, "value3":false, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.174833, "value1":0.47180057752688764, "value2":74850886504396865, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.373682, "value1":0.9894401032981868, "value2":2443814427210362559, "value3":false, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.230070, "value1":0.840516361148932, "value2":5295744989016708862, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595645, "value1":0.2788762608865926, "value2":898774770804749270, "value3":false, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583440, "value1":0.38764558337295657, "value2":4879715356875855821, "value3":false, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.818979, "value1":0.08550208893092236, "value2":7063465379400979205, "value3":true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990629, "value1":0.9735905702213057, "value2":9121417987396463994, "value3":false, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.904697, "value1":0.06007475161136332, "value2":1399260524832392153, "value3":true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832693, "value1":0.9560818456694564, "value2":6313771830503199817, "value3":false, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.273330, "value1":0.5911156951363433, "value2":6957059227202260296, "value3":false, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.069691, "value1":0.43486640701864104, "value2":5573463606213032969, "value3":true, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107074, "value1":0.28194511495703845, "value2":5008401247473276713, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.133554, "value1":0.6572805795366599, "value2":1430282776047161882, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.994425, "value1":0.7317293344785567, "value2":2161864319170825977, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703844, "value1":0.05169084281485511, "value2":7389845001390437940, "value3":true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667112, "value1":0.912614805848696, "value2":7494408396730454879, "value3":false, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175192, "value1":0.5654088358678295, "value2":7414573375849683263, "value3":true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.205436, "value1":0.5764549187518897, "value2":4315302504250390302, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.816073, "value1":0.3021595298462732, "value2":7835612651483818643, "value3":true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465800, "value1":0.7224511737802353, "value2":1475815608612889931, "value3":true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836860, "value1":0.13043025990273924, "value2":8725370796816365579, "value3":true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110112, "value1":0.2182570483232002, "value2":4212625314776126266, "value3":false, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.401613, "value1":0.3660560702023913, "value2":7136594683517469774, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772305, "value1":0.6420871129325768, "value2":346199260296482441, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142556, "value1":0.18492252698973582, "value2":2410884666534888495, "value3":true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.710999, "value1":0.03498389655141747, "value2":3857815905441742992, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588844, "value1":0.6843616917435196, "value2":925508006135613377, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902010, "value1":0.7892717951683124, "value2":7453201615304707478, "value3":true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071475, "value1":0.8597541524726022, "value2":5247483360729978449, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886399, "value1":0.896477399685476, "value2":3245685435537096080, "value3":false, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683519, "value1":0.7125828488615462, "value2":9221328263547905112, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650600, "value1":0.8976371405804583, "value2":6562020796772331341, "value3":false, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.316077, "value1":0.8989654150875539, "value2":2315139769444859937, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.844010, "value1":0.027932169490326376, "value2":8266191397214577403, "value3":false, "__name__":"metric_00000666", "key_3":"b","key_1":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.732165, "value1":0.9768852287807686, "value2":7871357409427659276, "value3":true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.644252, "value1":0.3652027068824268, "value2":4777422302644847840, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016877, "value1":0.4774488788727276, "value2":1359637185519527638, "value3":true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.121368, "value1":0.07637107971325026, "value2":225307864650781063, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.661741, "value1":0.9526993459994305, "value2":1751890497684728484, "value3":false, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534867, "value1":0.5672837246804853, "value2":3948370290656301760, "value3":false, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.400506, "value1":0.312527577003677, "value2":3200609992091774001, "value3":true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023204, "value1":0.9498281171738407, "value2":1775257641053296989, "value3":false, "__name__":"metric_00000674", "key_6":"c","key_7":"f","key_1":"a", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650859, "value1":0.7200243703698949, "value2":6459489446120996262, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125434, "value1":0.7513756216852647, "value2":6502722962310909561, "value3":false, "__name__":"metric_00000676", "key_5":"k","key_0":"k","key_2":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.861634, "value1":0.22132024628333183, "value2":4213082281103637151, "value3":true, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.616718, "value1":0.7492815151768418, "value2":8785198245169443750, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618495, "value1":0.5367662090666573, "value2":311068731789452160, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090805, "value1":0.6461806005354355, "value2":1567126231539851190, "value3":false, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.704866, "value1":0.6742157058257261, "value2":8258394004417852586, "value3":false, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178045, "value1":0.9347807954648532, "value2":5391643132437615590, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.903981, "value1":0.04205578457439769, "value2":4267367541005494603, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038592, "value1":0.9632414356901877, "value2":7847330231249131972, "value3":true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753078, "value1":0.025682952887214512, "value2":4407969986032631485, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.551387, "value1":0.9127538521307936, "value2":5532887277300530226, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340315, "value1":0.357568695323674, "value2":5839322022227051810, "value3":false, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253088, "value1":0.37455084749823464, "value2":919408856650004667, "value3":false, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.061612, "value1":0.08807754228035809, "value2":7674978240888726818, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350886, "value1":0.9187800273401259, "value2":3985979173089262193, "value3":false, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.997832, "value1":0.3332675485121975, "value2":6884650671561667205, "value3":false, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376107, "value1":0.882485292013383, "value2":980580007402095853, "value3":true, "__name__":"metric_00000691", "key_5":"h","key_6":"h","key_0":"b", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666200, "value1":0.543727834482791, "value2":7683502472635473948, "value3":true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.690345, "value1":0.9227402770528923, "value2":5963421474121503099, "value3":false, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831896, "value1":0.8899731417411955, "value2":3119002755449635021, "value3":true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583901, "value1":0.46723052632915907, "value2":4738569017001620646, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254234, "value1":0.07205706866452416, "value2":1464209096722519553, "value3":true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.940932, "value1":0.4880590550999508, "value2":3249029228406137695, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.819723, "value1":0.9180382594935601, "value2":5531777988899534137, "value3":true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.899553, "value1":0.5017555686141989, "value2":3526732457056334830, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364242, "value1":0.726219019872585, "value2":7463421321863859910, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591997, "value1":0.18380773538675038, "value2":6489374408484526555, "value3":true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268627, "value1":0.43767329135389277, "value2":7591255891965320507, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668426, "value1":0.8905993514345967, "value2":8025287728753470033, "value3":false, "__name__":"metric_00000713", "key_4":"c","key_2":"a","key_3":"b", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980089, "value1":0.9629455057472474, "value2":8613343366791814003, "value3":true, "__name__":"metric_00000714", "key_6":"g","key_3":"i","key_4":"d", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.561494, "value1":0.11038616417110533, "value2":5718549668479051731, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031316, "value1":0.5572446441362652, "value2":9073355869031665981, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913579, "value1":0.40513173097972344, "value2":1392267543875933322, "value3":true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469802, "value1":0.7197223556739805, "value2":8312522501295372114, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216239, "value1":0.38424010939993836, "value2":3429512093250859609, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053128, "value1":0.12167032010014549, "value2":2500133776481013473, "value3":false, "__name__":"metric_00000719", "key_9":"g","key_1":"d","key_4":"b", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755093, "value1":0.1432079670350677, "value2":4460076765766388831, "value3":true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939117, "value1":0.022396858430100586, "value2":1677736517466813137, "value3":true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730771, "value1":0.5975163721595746, "value2":8564236690483063617, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.763913, "value1":0.11111651544548351, "value2":2128389252611471929, "value3":false, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225483, "value1":0.1857169909861624, "value2":1459664257900170929, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.557521, "value1":0.10580492023702627, "value2":9211924519949290338, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.398250, "value1":0.8985634717833737, "value2":2690743666091944740, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.523236, "value1":0.8737716900990798, "value2":3717036232567418933, "value3":true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465788, "value1":0.8763409298655005, "value2":1486778905892801100, "value3":true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185627, "value1":0.4834833602007871, "value2":8283437701946785636, "value3":true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129438, "value1":0.4506377660858689, "value2":8577339860951120030, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.937075, "value1":0.6005010987020736, "value2":4798896188528165681, "value3":true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.387681, "value1":0.8777928078176396, "value2":4677891530420984908, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.074680, "value1":0.015131030176877407, "value2":2652924707092709319, "value3":true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032439, "value1":0.7762590461709376, "value2":5555968285860120909, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107619, "value1":0.0013683674938652786, "value2":8724767287173452904, "value3":true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199777, "value1":0.2705573241568207, "value2":5731172518242608315, "value3":true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.883455, "value1":0.440980354957089, "value2":717450023283511313, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.065153, "value1":0.7805040031236675, "value2":8249819238828017763, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028771, "value1":0.4266508385350679, "value2":3525220539697375602, "value3":false, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417030, "value1":0.9344530676800743, "value2":2840008251192450241, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169783, "value1":0.4944137362673325, "value2":8143727104076341090, "value3":true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309365, "value1":0.8233798931001785, "value2":4143752746550875358, "value3":true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405563, "value1":0.36473772798628423, "value2":8368477347330201864, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.080493, "value1":0.45312759136632275, "value2":5757367910796479605, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591589, "value1":0.5574497820130245, "value2":4692942231335997930, "value3":false, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770417, "value1":0.20880923062348417, "value2":4939040557737590500, "value3":false, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923414, "value1":0.8006055497749137, "value2":6979326850811422524, "value3":false, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249012, "value1":0.022621459630026125, "value2":6570512292324514521, "value3":false, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787566, "value1":0.5692664113038692, "value2":3307063324843670478, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.236668, "value1":0.706981031305401, "value2":6240895241125781235, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590979, "value1":0.9325091299580149, "value2":5714058795401825599, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782982, "value1":0.8159590900452445, "value2":3981533983292904484, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449106, "value1":0.2421958898621009, "value2":8819094433559346153, "value3":true, "__name__":"metric_00000746", "key_3":"d","key_8":"c","key_1":"i", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.721732, "value1":0.7320938316381037, "value2":1840371677400026121, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417920, "value1":0.3777544081714696, "value2":2517494055558796825, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.467240, "value1":0.07736271112081412, "value2":3355581288729910614, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802307, "value1":0.06164218213969723, "value2":6962215396306940249, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599871, "value1":0.2948698648725146, "value2":1065059716477496547, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.109220, "value1":0.11737528756979723, "value2":4468678220083400352, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156222, "value1":0.6294465098457002, "value2":2835781199255992964, "value3":false, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697592, "value1":0.9317953948461911, "value2":2221981181398322475, "value3":false, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785752, "value1":0.4235402956347778, "value2":5956435747949559681, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943686, "value1":0.28355998213096456, "value2":3638916038813769869, "value3":false, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.010104, "value1":0.22773696044954786, "value2":2360395284842638669, "value3":false, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.239178, "value1":0.14419564768805598, "value2":5315118297850048019, "value3":true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136093, "value1":0.3244550022291072, "value2":164222067931855660, "value3":false, "__name__":"metric_00000758", "key_6":"i","key_7":"b","key_2":"j", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968733, "value1":0.7486844786821671, "value2":6287914493893452990, "value3":false, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252999, "value1":0.24537563867085085, "value2":1626574969302079091, "value3":false, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.885743, "value1":0.9307243682752692, "value2":2858030878483913913, "value3":true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.631011, "value1":0.7084136481624964, "value2":2044467985498488215, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.853026, "value1":0.04425979450750048, "value2":1653452661642691666, "value3":false, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.991236, "value1":0.40886253057738015, "value2":7871621921656029433, "value3":true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.207156, "value1":0.37687587966259445, "value2":1943347287070225574, "value3":true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.708313, "value1":0.5785225454675275, "value2":1380941757138302107, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770913, "value1":0.9188180749639426, "value2":3884885787383779114, "value3":true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418857, "value1":0.9998412287287592, "value2":1886115929568662728, "value3":true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.850257, "value1":0.9796804114853653, "value2":4397109710918249601, "value3":true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339971, "value1":0.5089698391144044, "value2":8237713822108701891, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913742, "value1":0.9646995309003717, "value2":1173067889283022862, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067798, "value1":0.6899692743360695, "value2":1520636933221389194, "value3":false, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533696, "value1":0.8606713340189989, "value2":5737213954118900238, "value3":false, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212415, "value1":0.39075235526474594, "value2":6592432468398327445, "value3":false, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054486, "value1":0.34784858262092483, "value2":7843113252021213719, "value3":true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479147, "value1":0.5614107726843738, "value2":8690368331747046125, "value3":false, "__name__":"metric_00000785", "key_6":"d","key_9":"k","key_5":"j", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.622976, "value1":0.06775340365098863, "value2":5564636394288500497, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.315983, "value1":0.33546786831791003, "value2":3930900696554115736, "value3":true, "__name__":"metric_00000787", "key_6":"j","key_7":"g","key_3":"h", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.370627, "value1":0.779233269421853, "value2":3716543858406653649, "value3":false, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.969610, "value1":0.006014494345107452, "value2":8710808689518483016, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.724048, "value1":0.07768942947746518, "value2":496409043717037404, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590602, "value1":0.8288610653786054, "value2":1102610595846648990, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297543, "value1":0.37578423768641916, "value2":5536977376920161050, "value3":true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.155394, "value1":0.7083524353328224, "value2":4002932567237644088, "value3":false, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.459881, "value1":0.4873646265203241, "value2":5916271319076231574, "value3":false, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791505, "value1":0.5808298895011346, "value2":7841892156688279555, "value3":true, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067541, "value1":0.09121699649729094, "value2":8699612203169033522, "value3":true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138436, "value1":0.8850890106644654, "value2":9069485496265005186, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.812668, "value1":0.7098616610527902, "value2":8132915524494658053, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.190668, "value1":0.2761598493860578, "value2":651770389362313830, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.220527, "value1":0.17936744276680264, "value2":310375881133353905, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428315, "value1":0.49075022964169346, "value2":6493345006460964913, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595999, "value1":0.2755442227044263, "value2":4862778030716507189, "value3":false, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.719052, "value1":0.21758648924177262, "value2":9168976451471290619, "value3":true, "__name__":"metric_00000777", "key_4":"c","key_8":"i","key_0":"j", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166119, "value1":0.5187037640605351, "value2":7936720027148899998, "value3":false, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.361344, "value1":0.7201073761748152, "value2":8725593566170852352, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254415, "value1":0.07641284618924576, "value2":7978515171853518726, "value3":true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.981898, "value1":0.027541030165662613, "value2":6121611839007131360, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625954, "value1":0.22759978693770738, "value2":2989185053134172992, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350547, "value1":0.566849398731457, "value2":4327099372915065254, "value3":false, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799208, "value1":0.11746266545752859, "value2":7451515689230242023, "value3":true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.322064, "value1":0.26907348347198895, "value2":262434464387240754, "value3":true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.919974, "value1":0.6405108138252255, "value2":7284316699831173435, "value3":true, "__name__":"metric_00000803", "key_9":"f","key_3":"k","key_7":"c", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534467, "value1":0.9814271230191226, "value2":9214896283898892364, "value3":true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.291823, "value1":0.7119821636486502, "value2":4137668561036791084, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773725, "value1":0.05006867968524374, "value2":4350082823034910080, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.642706, "value1":0.8326077028765216, "value2":6301127653055186823, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.624652, "value1":0.6013140948710498, "value2":7150151669380473161, "value3":false, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663653, "value1":0.6471420893487286, "value2":2842026748176794650, "value3":false, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533665, "value1":0.40332997931033865, "value2":8939087210798770840, "value3":false, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915959, "value1":0.20834521315367086, "value2":7895351758501362895, "value3":false, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.238383, "value1":0.5265667505785955, "value2":2676293106518263900, "value3":true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582878, "value1":0.3420886897063641, "value2":1233598437288047014, "value3":false, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.489464, "value1":0.6718160824398975, "value2":4468628860088387255, "value3":true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.649028, "value1":0.7293826474439736, "value2":3127060444172041547, "value3":true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254010, "value1":0.8457444172560414, "value2":6200016452741633767, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.536370, "value1":0.16874770298054162, "value2":7575698105239291032, "value3":false, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809584, "value1":0.343753090691686, "value2":4854239248106071203, "value3":true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.334375, "value1":0.5540567040963329, "value2":2646076573445947055, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826524, "value1":0.7273180695893363, "value2":6564419565708307637, "value3":false, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071563, "value1":0.26385268835278003, "value2":4070457585628644142, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225965, "value1":0.9321283899737617, "value2":7702159669474089190, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458779, "value1":0.10195409940683517, "value2":5987602857042259052, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501592, "value1":0.07688705594071264, "value2":5259725214710983339, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.385240, "value1":0.8254376912417642, "value2":6760238975249610189, "value3":true, "__name__":"metric_00000818", "key_6":"f","key_3":"c","key_4":"g", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.760329, "value1":0.7530171422146449, "value2":1097173296466796516, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.597269, "value1":0.2403481802375547, "value2":5050041855629937310, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149349, "value1":0.43132198586517284, "value2":3287400838900732872, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.499626, "value1":0.737892470781718, "value2":4461721794139931373, "value3":false, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449953, "value1":0.8639558735483506, "value2":2903703899322529602, "value3":true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.912253, "value1":0.01605090964788225, "value2":5781647102967996198, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.4054065908363409, "value2":6440329633133258472, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038844, "value1":0.3763702930516151, "value2":1302491802347098686, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.453739, "value1":0.6265318797324549, "value2":819075302260626614, "value3":false, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590829, "value1":0.5479451127212884, "value2":8557856578535483978, "value3":true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714164, "value1":0.8513319596110408, "value2":3981392010073089983, "value3":false, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.767546, "value1":0.9500048124431112, "value2":2193055476612398128, "value3":false, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.685203, "value1":0.9241278767545394, "value2":5146494836784446946, "value3":false, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171752, "value1":0.05193751579817584, "value2":2407542541423061968, "value3":false, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620585, "value1":0.9449477088124408, "value2":293003481714135625, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197927, "value1":0.9081143587822024, "value2":336470947794228646, "value3":false, "__name__":"metric_00000841", "key_6":"b","key_1":"i","key_2":"d", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730165, "value1":0.4106378302625472, "value2":2305406403284031522, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993455, "value1":0.5015628664485756, "value2":3942941750281172936, "value3":false, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376150, "value1":0.08846010494510266, "value2":451409939393211294, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578966, "value1":0.5184884584460681, "value2":5013311361860229047, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980632, "value1":0.6414794039381226, "value2":4661775189685128413, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543153, "value1":0.7779597819383781, "value2":1678453405944867509, "value3":false, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.728157, "value1":0.4026694195455883, "value2":1247937331490084335, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039033, "value1":0.6078130901686731, "value2":6572519819620597387, "value3":false, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020089, "value1":0.16501689332992148, "value2":2135299756770180375, "value3":true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159279, "value1":0.06812322597368, "value2":1853312335979487824, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225355, "value1":0.31761717143878, "value2":1547635249117783016, "value3":false, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.579591, "value1":0.45080215259792744, "value2":830280592135153306, "value3":true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.989155, "value1":0.3246650009893744, "value2":3832412454221483471, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.898794, "value1":0.9353227689941251, "value2":5019805485542488656, "value3":true, "__name__":"metric_00000854", "key_4":"g","key_1":"d","key_2":"f", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.325402, "value1":0.40452074217950085, "value2":3779976710102146862, "value3":false, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.846682, "value1":0.8078556003817017, "value2":4563926776477066326, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.538107, "value1":0.6916244463696708, "value2":2616499684497809047, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601707, "value1":0.0005961819411869942, "value2":8421789457526552579, "value3":false, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.474772, "value1":0.40247038234287125, "value2":691809847413019723, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835874, "value1":0.24276779175523852, "value2":6899854861109897252, "value3":false, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.366826, "value1":0.8778001737651254, "value2":4321676554804962434, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826448, "value1":0.8031889549998205, "value2":5554188972077808449, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.432681, "value1":0.15017093541000798, "value2":7513115457016757405, "value3":true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.353975, "value1":0.23702527674889587, "value2":1810989557365945672, "value3":true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303934, "value1":0.2677306312444, "value2":430799962318472061, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163809, "value1":0.8138323281483592, "value2":8246241283973870498, "value3":false, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662355, "value1":0.9144882781553422, "value2":4359470640610896382, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120568, "value1":0.8056479342778862, "value2":6170593949410616781, "value3":true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.541822, "value1":0.248047143451303, "value2":8344606419798048221, "value3":false, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.147354, "value1":0.35385149944297156, "value2":6006537614889971434, "value3":true, "__name__":"metric_00000868", "key_8":"j","key_3":"i","key_4":"i", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.116235, "value1":0.6774547184266492, "value2":5503266725598369859, "value3":false, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.563395, "value1":0.978792023344712, "value2":8139355376515169099, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202304, "value1":0.42453838538082794, "value2":8767087562530261434, "value3":false, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553168, "value1":0.813931872578598, "value2":838817774250793063, "value3":true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.476339, "value1":0.2705647850316454, "value2":6004383053661805919, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067348, "value1":0.23127791420697405, "value2":614040873255409897, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.063340, "value1":0.04494057741571453, "value2":7945808906854676018, "value3":true, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877369, "value1":0.188685408716376, "value2":5653852277013510880, "value3":false, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145318, "value1":0.6394711610065485, "value2":6391486639411558835, "value3":false, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171038, "value1":0.9902721723985418, "value2":3163235560331954355, "value3":true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773084, "value1":0.40123815528773027, "value2":6772197516975478771, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.976169, "value1":0.608513336182585, "value2":3898075135558136236, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.460540, "value1":0.027884469593113505, "value2":1806632468596114826, "value3":false, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.411909, "value1":0.6437101253853217, "value2":4628507915366383990, "value3":true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.055892, "value1":0.4209111843259915, "value2":8498738002897413709, "value3":true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684315, "value1":0.3608627471653678, "value2":8911984339628672502, "value3":true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181887, "value1":0.04106030144724243, "value2":912973087706429869, "value3":true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.127221, "value1":0.8723975492127802, "value2":4688994221703849298, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881735, "value1":0.8013556912079172, "value2":643258613153646569, "value3":true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093171, "value1":0.9983142632275949, "value2":2242609892773836818, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702516, "value1":0.9951260414368723, "value2":9110402286568364937, "value3":true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712584, "value1":0.10494403348655837, "value2":4745554958401980793, "value3":false, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.699467, "value1":0.9010413033988799, "value2":9007036540377523860, "value3":false, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.132646, "value1":0.06467587338211019, "value2":1060827532336676254, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.552157, "value1":0.4330362437559917, "value2":7496787358324320393, "value3":false, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888799, "value1":0.2576583993305694, "value2":4018360458617386281, "value3":true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.498269, "value1":0.2224059001256479, "value2":4065063745853139952, "value3":false, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588721, "value1":0.14516718870277823, "value2":3512331805580372297, "value3":true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583233, "value1":0.7489522543167118, "value2":5610699138607696953, "value3":true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290155, "value1":0.44273304014865483, "value2":427744180869204041, "value3":true, "__name__":"metric_00000910", "key_9":"e","key_5":"a","key_8":"c", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183829, "value1":0.6345267428968718, "value2":1846381248733979645, "value3":false, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.323098, "value1":0.012439339538142696, "value2":8869073391299009493, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634145, "value1":0.9267919956221178, "value2":2118172993893609219, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794582, "value1":0.8353549954749553, "value2":1351674467301929869, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.008550, "value1":0.6040059322178531, "value2":753170312236461842, "value3":true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268043, "value1":0.31922185771526074, "value2":2365194637514291305, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.857933, "value1":0.8196275302693926, "value2":4478227043901542114, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.645585, "value1":0.17157442528490047, "value2":3055405974571687053, "value3":true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.277791, "value1":0.8508212157393535, "value2":475170132890875463, "value3":false, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123450, "value1":0.35015180460716977, "value2":7931493690032322348, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.918435, "value1":0.30789613557477924, "value2":4634204529169845529, "value3":false, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.747396, "value1":0.7119174217020244, "value2":5113967535733338451, "value3":true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346547, "value1":0.4105703222986521, "value2":5634326652610778742, "value3":false, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.873833, "value1":0.5726619572582797, "value2":4834039219162125836, "value3":true, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839553, "value1":0.3474160702051541, "value2":5912402851303338785, "value3":true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924943, "value1":0.25193375971882537, "value2":3157228766646586798, "value3":false, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309520, "value1":0.4856065585160012, "value2":12512784025771269, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612385, "value1":0.06742962293903079, "value2":5561454087904294314, "value3":true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634744, "value1":0.8158802826640159, "value2":4407063105915178233, "value3":false, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524442, "value1":0.05056002141365119, "value2":1069310785755082516, "value3":false, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.122461, "value1":0.8648592352685478, "value2":2607760955410972754, "value3":false, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308553, "value1":0.8340608354185821, "value2":5091217112123016682, "value3":true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.073014, "value1":0.412293314616913, "value2":1754188674597018639, "value3":false, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382506, "value1":0.7914084871078703, "value2":3113230598922027169, "value3":true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.494019, "value1":0.22683994673921629, "value2":345509562584529791, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770833, "value1":0.36946692588064045, "value2":3634543347045698698, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374724, "value1":0.7674203492653544, "value2":409087874753033650, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.33760436450813625, "value2":6378216485000854161, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753898, "value1":0.08146706117124274, "value2":4619087622758440080, "value3":true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.058417, "value1":0.0895913841094616, "value2":7775148933653129917, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.889748, "value1":0.9025957275835974, "value2":9060698938664882878, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.429099, "value1":0.45935248830386766, "value2":5813431686543708617, "value3":false, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.415280, "value1":0.8845089283715811, "value2":6384109853170568567, "value3":false, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956220, "value1":0.6163283411653032, "value2":7939487848850431764, "value3":false, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.813125, "value1":0.3306736514104797, "value2":717112968732186535, "value3":false, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.464126, "value1":0.23519387446018714, "value2":2061991374504688132, "value3":false, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.336194, "value1":0.6334359034532542, "value2":5464410230276800258, "value3":false, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722970, "value1":0.6104432497384834, "value2":3298076355265173187, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.246842, "value1":0.8069861206050694, "value2":7793368021912415700, "value3":false, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465645, "value1":0.6879012241509205, "value2":4196934016416565115, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876004, "value1":0.06724546167889978, "value2":7203308436755461322, "value3":true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667298, "value1":0.6826712318664662, "value2":6402839614176163878, "value3":true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.329694, "value1":0.7040366253423608, "value2":1262392626609016984, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591566, "value1":0.9312106031712911, "value2":6293442498282971529, "value3":true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.559072, "value1":0.4448009704050545, "value2":2669159126008899565, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431852, "value1":0.8238849009316198, "value2":1021818236630467563, "value3":true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722762, "value1":0.5948459056041813, "value2":6253806025921861085, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339501, "value1":0.14816011629555237, "value2":2437202342857552183, "value3":false, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313849, "value1":0.14517234363613402, "value2":3032490535727906731, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974064, "value1":0.5282428591902778, "value2":7306937940345849875, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930949, "value1":0.6595519497499615, "value2":2818227237339798252, "value3":false, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.995836, "value1":0.41346641045143234, "value2":457793590293209131, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.527926, "value1":0.17089664849686367, "value2":5825724904068162921, "value3":false, "__name__":"metric_00000955", "key_7":"d","key_1":"e","key_6":"j", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258906, "value1":0.5587901020638738, "value2":2247971244413895345, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232248, "value1":0.9182710496214209, "value2":2114818465791862400, "value3":false, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.075755, "value1":0.36489084685089246, "value2":294204895726685309, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.726685, "value1":0.6513663450138257, "value2":741257611118540876, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794966, "value1":0.07728058455296861, "value2":6115920270410886698, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.954148, "value1":0.6576477115837028, "value2":8442517348621889353, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.305446, "value1":0.8417215600157095, "value2":3960819244598684350, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749210, "value1":0.905397326364665, "value2":7392448658664215427, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479499, "value1":0.8192850944227448, "value2":1109448037735485747, "value3":true, "__name__":"metric_00000963", "key_8":"b","key_4":"h","key_5":"a", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228850, "value1":0.7549298352802428, "value2":2475887390541907125, "value3":false, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.256344, "value1":0.6191375565525401, "value2":6629221972448932333, "value3":true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843715, "value1":0.4048974244394111, "value2":2052496354441233400, "value3":true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.257771, "value1":0.11131625744600283, "value2":5527432282921695963, "value3":false, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033616, "value1":0.18479886541727453, "value2":9028612437715086928, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832552, "value1":0.6959404377768793, "value2":2173084661524542898, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431933, "value1":0.7816557853866045, "value2":7424533234329790131, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.264080, "value1":0.7382347817860379, "value2":5213561271316161382, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085647, "value1":0.4523244522101, "value2":7889921400021958159, "value3":true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004893, "value1":0.6556889205138186, "value2":3025561804921198898, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339616, "value1":0.2999169933424259, "value2":8184676717410157627, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216679, "value1":0.42313229610434316, "value2":4008780382662754625, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418495, "value1":0.49555776117242517, "value2":1018253790078366875, "value3":false, "__name__":"metric_00000985", "key_7":"j","key_0":"a", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033786, "value1":0.9209475895817386, "value2":5569575246133894540, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620702, "value1":0.6494468637932249, "value2":4953667604093058532, "value3":true, "__name__":"metric_00000987", "key_5":"b","key_7":"e","key_2":"a", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.706504, "value1":0.12760690080118256, "value2":6290715519152270329, "value3":true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.483588, "value1":0.5095141564714446, "value2":6862972202733799902, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159429, "value1":0.90417660237305, "value2":8461823351624136781, "value3":false, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.868532, "value1":0.20009311827615117, "value2":2269303027016825167, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.030853, "value1":0.07427016371278365, "value2":6524107618104112365, "value3":false, "__name__":"metric_00000983", "key_7":"c","key_9":"e","key_0":"b", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625191, "value1":0.3705374970999065, "value2":8940487375802886850, "value3":false, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274579, "value1":0.43745692673872216, "value2":7664930411577220439, "value3":false, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428630, "value1":0.527310522539046, "value2":6084859156132058763, "value3":true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.594595, "value1":0.21322470607998698, "value2":3416879710307779950, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.920870, "value1":0.47127845699263055, "value2":4564621285620268941, "value3":false, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930512, "value1":0.41802427118606883, "value2":8549501771062307593, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977996, "value1":0.1610907563584878, "value2":6578022034712623821, "value3":true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973727, "value1":0.9224145581231933, "value2":8668945457313971984, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421891, "value1":0.40172827078543016, "value2":2817865116264178423, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683482, "value1":0.8428158825760677, "value2":1087743773064912864, "value3":false, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.001802, "value1":0.5075672608227982, "value2":3748937813625482336, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924538, "value1":0.6184538471340483, "value2":8049426804122873881, "value3":false, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427400, "value1":0.23484522887445405, "value2":360795956748828928, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.371453, "value1":0.14713938237360089, "value2":2782450449945692027, "value3":false, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105401, "value1":0.7787234859135739, "value2":7103390400724679707, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608837, "value1":0.9083492888699478, "value2":591947190434298439, "value3":true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839636, "value1":0.20678896707817307, "value2":1816652983048222981, "value3":true, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.798496, "value1":0.27959153848419394, "value2":6453301245071446957, "value3":true, "__name__":"metric_00000002", "key_8":"f","key_1":"a","key_2":"g", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079294, "value1":0.9921346825877959, "value2":2610009567825131673, "value3":true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.932174, "value1":0.616531393243407, "value2":5793240958146632836, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352378, "value1":0.5817027431697587, "value2":1001435181122015935, "value3":false, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.892678, "value1":0.09115452403074104, "value2":5100882205034466347, "value3":false, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.976748, "value1":0.223689480612581, "value2":4832346905388106952, "value3":true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348707, "value1":0.4785116775888147, "value2":1474405574460091730, "value3":true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285197, "value1":0.2545105724094511, "value2":2069757700743153534, "value3":true, "__name__":"metric_00000007", "key_8":"a","key_3":"e","key_4":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886496, "value1":0.6267242677514145, "value2":5390965713876988864, "value3":true, "__name__":"metric_00000009", "key_3":"k","key_8":"b","key_0":"h", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.106653, "value1":0.29720904047717794, "value2":3776030286792185916, "value3":true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.360157, "value1":0.5546347320924953, "value2":330359320098225031, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229710, "value1":0.36539460584181066, "value2":2852256551896569615, "value3":false, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.107272, "value1":0.7519447135299673, "value2":479042341091737730, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.653206, "value1":0.055664103121505173, "value2":4715105139689145584, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339724, "value1":0.65279778916842, "value2":5051290581444035933, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605595, "value1":0.7185823471699857, "value2":6840084614633305448, "value3":true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923752, "value1":0.4375825544173804, "value2":5512879974255226786, "value3":true, "__name__":"metric_00000017", "key_5":"d","key_7":"e","key_0":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259124, "value1":0.1976251344530473, "value2":3865204942280496136, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833243, "value1":0.1327195116975274, "value2":25036382112222312, "value3":true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181954, "value1":0.03893696644874014, "value2":8620048848181171214, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241452, "value1":0.08765547856551259, "value2":478115772614972254, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.346035, "value1":0.8594527006005106, "value2":7112082294884150846, "value3":false, "__name__":"metric_00000021", "key_3":"f","key_9":"c","key_2":"i", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171651, "value1":0.03206744023244803, "value2":1423544456047900511, "value3":false, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863325, "value1":0.8536761474388656, "value2":6060590565202496674, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.441838, "value1":0.172156773217081, "value2":5954759661339252215, "value3":true, "__name__":"metric_00000025", "key_4":"j","key_5":"b","key_0":"a", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960980, "value1":0.32980876505437323, "value2":8725964433492010874, "value3":false, "__name__":"metric_00000026", "key_7":"d","key_1":"f","key_2":"c", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000707, "value1":0.24709283071336913, "value2":6676514692655151346, "value3":false, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.618373, "value1":0.9009517094483285, "value2":3595072304978000605, "value3":true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.170508, "value1":0.9280407408957537, "value2":5488749324303924425, "value3":true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863252, "value1":0.12661256202486373, "value2":8664732043269436820, "value3":true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.849403, "value1":0.16431470929214623, "value2":7019138744327165116, "value3":false, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464027, "value1":0.09323138351556215, "value2":7725624855401413056, "value3":false, "__name__":"metric_00000034", "key_7":"e","key_9":"c","key_6":"b", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.236399, "value1":0.8723974444748736, "value2":4852776678069519279, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748433, "value1":0.955013940926474, "value2":1647976132077310759, "value3":false, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519277, "value1":0.9878333472759755, "value2":1234045371574302701, "value3":false, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098817, "value1":0.01452665827731407, "value2":1567075824080468956, "value3":true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.086812, "value1":0.507976874584688, "value2":7088565570837880582, "value3":true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.442740, "value1":0.05234017389082447, "value2":6744325328771713826, "value3":true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200886, "value1":0.26486555470244233, "value2":7063921451775904700, "value3":true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542647, "value1":0.44652572463811874, "value2":881040498652574470, "value3":false, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403848, "value1":0.15638403887386948, "value2":881480808583221270, "value3":false, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.456051, "value1":0.8059172501323808, "value2":7272729849107639168, "value3":false, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529307, "value1":0.34486654778342174, "value2":392684728493023069, "value3":true, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171436, "value1":0.3729515434430422, "value2":3944928681545148043, "value3":false, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308638, "value1":0.7632957285940241, "value2":92442464729948625, "value3":false, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515220, "value1":0.6925288324736131, "value2":8180048442965609063, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584191, "value1":0.021892941959738798, "value2":8483219855752350520, "value3":false, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.062243, "value1":0.4180089249967854, "value2":6354312606503764778, "value3":true, "__name__":"metric_00000049", "key_9":"j","key_1":"d","key_3":"h", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341779, "value1":0.7379607607767219, "value2":5150239222005969615, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843413, "value1":0.6687750316343205, "value2":2316529591090571268, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.402005, "value1":0.9141363896275393, "value2":6270246938486716288, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128379, "value1":0.5150067708119237, "value2":678394980878451314, "value3":true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108482, "value1":0.3129644434816646, "value2":8384633845422523721, "value3":false, "__name__":"metric_00000052", "key_4":"a","key_9":"g","key_3":"c", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.594874, "value1":0.2519824660631957, "value2":6886759477377939773, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445205, "value1":0.903159708784011, "value2":7034171128847343546, "value3":false, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824198, "value1":0.6204070552794421, "value2":1059263092990498226, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.159620, "value1":0.8686561618779289, "value2":5404253444446120581, "value3":true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.365387, "value1":0.4572890031222697, "value2":7934471101097198056, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521775, "value1":0.36902024174358433, "value2":2258863231460147617, "value3":false, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.553157, "value1":0.44513850096108715, "value2":6217475414573430294, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141433, "value1":0.21220874698804243, "value2":93944351046750038, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.811456, "value1":0.8670934692477834, "value2":7403345830074134000, "value3":false, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003892, "value1":0.4239005643048954, "value2":7762289626422779835, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.067882, "value1":0.18864986614817061, "value2":2907310668904743062, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.750570, "value1":0.6373780024843834, "value2":3686123513663951346, "value3":false, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.386866, "value1":0.2003273945539795, "value2":5992425398301380670, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515245, "value1":0.516857446811694, "value2":4876668434287590384, "value3":false, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389071, "value1":0.9340489387136409, "value2":7504250901331893981, "value3":true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098316, "value1":0.29683731723021556, "value2":7480572590262205533, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906295, "value1":0.7596148064915852, "value2":2005574389285214762, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.469199, "value1":0.3474919541045514, "value2":4513611972684272623, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735221, "value1":0.44321964186413737, "value2":284056865289241089, "value3":false, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382330, "value1":0.4698238258381697, "value2":6544780705530113911, "value3":false, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644228, "value1":0.34547957427274706, "value2":5643558758908257061, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025533, "value1":0.9831418878764304, "value2":34382825795091331, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564530, "value1":0.1974264877985391, "value2":7202346148984902707, "value3":true, "__name__":"metric_00000076", "key_8":"h","key_4":"e", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.582038, "value1":0.6658499962262118, "value2":4779411307587076137, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.048223, "value1":0.28118182352648263, "value2":2406664371776445226, "value3":false, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.277281, "value1":0.3293088133074837, "value2":3125287577587960916, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711593, "value1":0.6513318644160435, "value2":6304109109655854952, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.136044, "value1":0.15479605681668362, "value2":4561542839689997911, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.143594, "value1":0.03881037667597162, "value2":5145183720039449728, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766270, "value1":0.5913740900789849, "value2":2880750585927778789, "value3":true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.330272, "value1":0.03345193869225257, "value2":5177206809108614692, "value3":true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893588, "value1":0.9545617077887952, "value2":1994751413642673810, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200065, "value1":0.7615854624227617, "value2":2555245387882162412, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.637143, "value1":0.8697167932948927, "value2":3163242108044176287, "value3":true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.640389, "value1":0.5659009743641423, "value2":7954937488899706120, "value3":true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.558497, "value1":0.928513605246244, "value2":1207841403052993714, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.244339, "value1":0.9485629290996225, "value2":8066024682557174519, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.368292, "value1":0.1771040772082859, "value2":5158146682102067527, "value3":false, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883369, "value1":0.26796315495485606, "value2":500188854907829642, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.603831, "value1":0.695040513930677, "value2":7670486818087410343, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635402, "value1":0.17089841071375036, "value2":2188177332357604180, "value3":false, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246584, "value1":0.638326848995027, "value2":2467157573588247428, "value3":false, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297753, "value1":0.4028871413838985, "value2":549392368948968713, "value3":true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061298, "value1":0.28566196711825703, "value2":4345219419785285366, "value3":true, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468603, "value1":0.15137756224435941, "value2":5857705640176057374, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705772, "value1":0.9900652870935787, "value2":4160023456685330050, "value3":true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628560, "value1":0.47755802914007767, "value2":2886069569872931687, "value3":true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.201527, "value1":0.3765200270511815, "value2":8192740497964394547, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405600, "value1":0.16568713498682006, "value2":807681981221494025, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.070083, "value1":0.5411063354146068, "value2":6815859051389366015, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339848, "value1":0.09610559077312085, "value2":5872072911320364504, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571126, "value1":0.2497057720663749, "value2":924893218806837423, "value3":true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659128, "value1":0.026188484488162033, "value2":8931856805131420281, "value3":true, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.127455, "value1":0.3964141576554178, "value2":2662798392903843365, "value3":false, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780076, "value1":0.15040609462861912, "value2":7522359721454017998, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108734, "value1":0.6578186778027336, "value2":2079921522814480295, "value3":true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.476594, "value1":0.5388243902425934, "value2":1295082543190183614, "value3":true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377611, "value1":0.4899107617022405, "value2":1810844261495017790, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.980760, "value1":0.052036560099483146, "value2":4808591658339710229, "value3":true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.662657, "value1":0.739342249293886, "value2":4457016924188070830, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573154, "value1":0.043565004929414355, "value2":2934360743315418976, "value3":false, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.792186, "value1":0.7194337918198027, "value2":8482147182498656703, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687869, "value1":0.5159874448621534, "value2":5870856463348491189, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.522440, "value1":0.8824876555801262, "value2":2534178016362914654, "value3":false, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132515, "value1":0.8599678891940701, "value2":8395948226155967317, "value3":true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.096926, "value1":0.9245913183435254, "value2":2358549994381165596, "value3":true, "__name__":"metric_00000120", "key_3":"e","key_7":"d","key_2":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486752, "value1":0.2842029485021584, "value2":2801356184403378281, "value3":false, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.475238, "value1":0.22293224326621178, "value2":443811864922674372, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132657, "value1":0.9371037777036672, "value2":4385576391999515299, "value3":true, "__name__":"metric_00000121", "key_5":"a","key_7":"h","key_3":"e", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.549121, "value1":0.03204732941506213, "value2":8714811939867543319, "value3":true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796275, "value1":0.21070900388130828, "value2":8667819799137478248, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927343, "value1":0.9159887689007843, "value2":7689292400204431331, "value3":false, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685507, "value1":0.7348923960578333, "value2":330857683592078866, "value3":false, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.865459, "value1":0.24066397037006473, "value2":203169051837200909, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.827976, "value1":0.4488303067264579, "value2":3646968045887419113, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900145, "value1":0.31889183176683844, "value2":5999122195912397909, "value3":true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.251489, "value1":0.5593865051855272, "value2":5772018587784459447, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191087, "value1":0.2891985736646055, "value2":4399230413942175266, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087488, "value1":0.5425699812510858, "value2":3965285909604337765, "value3":true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680134, "value1":0.18266599397918218, "value2":5571070206339456089, "value3":true, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551156, "value1":0.8876027936303559, "value2":6010745863587972423, "value3":false, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.803161, "value1":0.9394184911877158, "value2":571658979107886488, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.054336, "value1":0.03957759543141472, "value2":3918244459790702522, "value3":true, "__name__":"metric_00000144", "key_3":"b","key_9":"e","key_0":"g", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685390, "value1":0.5264564714839759, "value2":2232108588919897592, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.161082, "value1":0.32998817001502934, "value2":8460874534662447333, "value3":false, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778535, "value1":0.06811024060980113, "value2":7835773491677016452, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832955, "value1":0.028669939437680664, "value2":4299578964430488535, "value3":true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004645, "value1":0.8188081000238006, "value2":9221293293613853860, "value3":false, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205474, "value1":0.40545870686171354, "value2":1575377844766226698, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639268, "value1":0.8908404617416188, "value2":1540158992274106467, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.139661, "value1":0.3668772992670227, "value2":6483649363648949947, "value3":true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924909, "value1":0.16190741337415684, "value2":5257368157508278570, "value3":true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297500, "value1":0.4330469601614637, "value2":4632520395480129342, "value3":false, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101765, "value1":0.4685291307083107, "value2":1614399458247460286, "value3":false, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291465, "value1":0.6213831794241761, "value2":7754123020970248013, "value3":true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369394, "value1":0.5153486584517465, "value2":2263160774029528717, "value3":false, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719680, "value1":0.5383036464533613, "value2":3589028105702605678, "value3":false, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262982, "value1":0.02336562179973987, "value2":2500396082328773649, "value3":false, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.790458, "value1":0.7125005920534319, "value2":8453423485925643300, "value3":true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.937011, "value1":0.9029577396659265, "value2":3493444148478497784, "value3":false, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815751, "value1":0.34331005780857, "value2":4896770691763395578, "value3":true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.732239, "value1":0.5781653462685661, "value2":2815554048397289034, "value3":true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073195, "value1":0.6318398759739543, "value2":1443128294631046938, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.335280, "value1":0.2900799960272814, "value2":319439076295666505, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375292, "value1":0.8416100389726322, "value2":1104228901216289966, "value3":true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543223, "value1":0.30539264418826334, "value2":2529051273510780065, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850347, "value1":0.9475422757893678, "value2":3815335775316587120, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010486, "value1":0.9233328326166781, "value2":1132448560381334143, "value3":true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509503, "value1":0.43078826324145314, "value2":2585538196535057958, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440516, "value1":0.36575868540627, "value2":764298189191108299, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.576334, "value1":0.9992971397552057, "value2":8607203945347812848, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.970781, "value1":0.43725394836722203, "value2":8039899278253467744, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746791, "value1":0.5653040341344706, "value2":6676447946324724349, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636245, "value1":0.8420862976065959, "value2":1562613302672270132, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285262, "value1":0.32142916834577895, "value2":5289001385309911194, "value3":false, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.417392, "value1":0.049995803596490404, "value2":6607894685909801752, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584296, "value1":0.2409216093884783, "value2":1637965971530230928, "value3":true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325720, "value1":0.028777043283946674, "value2":963675377332588860, "value3":false, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566004, "value1":0.9002123352889874, "value2":4604125644132750665, "value3":true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.882236, "value1":0.33330368402055666, "value2":156824449049228711, "value3":true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.767642, "value1":0.13763101488639068, "value2":8064121309553799557, "value3":true, "__name__":"metric_00000152", "key_2":"k","key_9":"h","key_0":"d", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719598, "value1":0.8986180062766446, "value2":1742955026017794842, "value3":false, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886873, "value1":0.010775899740746808, "value2":1351720915728762737, "value3":false, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484279, "value1":0.5158913513515657, "value2":6695543448765155807, "value3":true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.120818, "value1":0.38000311042969326, "value2":8387696033998291, "value3":false, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.471169, "value1":0.5449952216814127, "value2":7977583153767662789, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.692704, "value1":0.644651424950153, "value2":1344814886763463146, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434917, "value1":0.5336620566427158, "value2":7974250102837601477, "value3":true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.231472, "value1":0.18429722211173638, "value2":239717814769298968, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351874, "value1":0.3302574440246314, "value2":116725965202686758, "value3":false, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.652256, "value1":0.8832602328157859, "value2":2443207413955073666, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392058, "value1":0.3666546013150111, "value2":5949156227430002451, "value3":false, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940471, "value1":0.038777989285728266, "value2":2871780422161853348, "value3":true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412070, "value1":0.5197207843322247, "value2":406042026849178989, "value3":false, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571008, "value1":0.7027610078055025, "value2":1383746034138340292, "value3":false, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900415, "value1":0.17910816553895145, "value2":401749706711859248, "value3":false, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.718726, "value1":0.2276914412180115, "value2":9221447498523708978, "value3":false, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.381086, "value1":0.9677704240195594, "value2":1303001585301279474, "value3":true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893466, "value1":0.2579622668275943, "value2":3971563895105696581, "value3":false, "__name__":"metric_00000193", "key_9":"e","key_2":"h","key_4":"a", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.400237, "value1":0.23652480428212666, "value2":6009422694154041952, "value3":true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481908, "value1":0.625595272995919, "value2":2204722405376257638, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.936595, "value1":0.2680275935509345, "value2":324738255121972023, "value3":false, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.983646, "value1":0.3650479381459544, "value2":6655499432411034428, "value3":true, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.489964, "value1":0.6666717547984726, "value2":2225549269590513897, "value3":true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641209, "value1":0.578327662161616, "value2":6547750503066251015, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134486, "value1":0.67209109592655, "value2":2885152331577461710, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073911, "value1":0.08294753363850706, "value2":7590236594974272032, "value3":true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181170, "value1":0.5500056163737558, "value2":6771546940482209584, "value3":true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.029045, "value1":0.6184324246909119, "value2":8362437134419045707, "value3":true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162843, "value1":0.4243438443026443, "value2":8729664805668824103, "value3":true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.808018, "value1":0.9740191657641797, "value2":811057613633969138, "value3":true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509186, "value1":0.5860682567407373, "value2":4715370569695436166, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664283, "value1":0.9354682602818816, "value2":4771667149292350278, "value3":true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220707, "value1":0.9895245632099, "value2":7384968594835669289, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389342, "value1":0.5213334242916959, "value2":9064052527115170337, "value3":true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977547, "value1":0.9968413793490465, "value2":1018668683107683896, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122093, "value1":0.22725801593699438, "value2":471678573457243344, "value3":false, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474960, "value1":0.6106407332478643, "value2":6142821675445007328, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770612, "value1":0.38366813133853406, "value2":4771686788392964587, "value3":false, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166046, "value1":0.6731562334089229, "value2":6765933549203517189, "value3":true, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000435, "value1":0.7725469520964668, "value2":1526254098614108268, "value3":true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.596567, "value1":0.33143109459424, "value2":6988963074743099921, "value3":false, "__name__":"metric_00000215", "key_2":"f","key_0":"f","key_1":"h", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832348, "value1":0.5082718126919952, "value2":7366669828299681751, "value3":true, "__name__":"metric_00000216", "key_5":"j","key_7":"b","key_1":"j", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.933461, "value1":0.2177858524105436, "value2":1927680313692641282, "value3":true, "__name__":"metric_00000217", "key_4":"a","key_9":"h","key_1":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800386, "value1":0.37752422688739345, "value2":3623415676274096956, "value3":true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602632, "value1":0.9591290646171173, "value2":149144865605034806, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805853, "value1":0.46005104238615857, "value2":5399494644643471050, "value3":false, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023064, "value1":0.2589954384060476, "value2":2161234059148173478, "value3":true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.663052, "value1":0.4340276216577473, "value2":1651362675477804080, "value3":false, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425648, "value1":0.5284587146267693, "value2":2204338428823542625, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297837, "value1":0.9930278173662268, "value2":1153109222914733395, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.661230, "value1":0.3189372627835483, "value2":8950530915947476877, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997529, "value1":0.9931406569786665, "value2":3171376634461957806, "value3":true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785991, "value1":0.6115707472871859, "value2":4480386318218501447, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909727, "value1":0.29199552201380746, "value2":3686357880544622730, "value3":false, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155119, "value1":0.5470008889969509, "value2":7858820522318450111, "value3":false, "__name__":"metric_00000238", "key_7":"e","key_4":"a","key_5":"k", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.530629, "value1":0.9873625937302516, "value2":5471095001682678938, "value3":false, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512132, "value1":0.14465321479404997, "value2":8852396394387214520, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.643903, "value1":0.81607168234715, "value2":7750000008183770029, "value3":false, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088371, "value1":0.9834997161657205, "value2":8265123019714389041, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.840291, "value1":0.31720704974294733, "value2":8838522133270072113, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607394, "value1":0.6158114593013463, "value2":3920524769932432062, "value3":false, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.891225, "value1":0.7260707014711243, "value2":8545845285837175547, "value3":false, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155524, "value1":0.5324837252402642, "value2":6050562650984829349, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463368, "value1":0.3636429297340659, "value2":7366387480034568615, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.051267, "value1":0.21240586224005645, "value2":3520112151921939439, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.713935, "value1":0.6148054965944443, "value2":4435199374933360163, "value3":true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229994, "value1":0.877849746015665, "value2":4328772383519778289, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.394088, "value1":0.7419214909083843, "value2":1533570891823401045, "value3":false, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311200, "value1":0.6547222273874411, "value2":603076909255570468, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593460, "value1":0.33723617074445394, "value2":4429464301654712087, "value3":false, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016002, "value1":0.7850512802013139, "value2":9218293660844009579, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293893, "value1":0.9024192531760731, "value2":874814059564514510, "value3":false, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197015, "value1":0.6346018344137921, "value2":2275287342249871781, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444015, "value1":0.07476763645648904, "value2":6294420894284163263, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.137036, "value1":0.7149572150590823, "value2":5890716035999924313, "value3":false, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382203, "value1":0.2373007200794848, "value2":4237441552449229366, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927338, "value1":0.5067232894842505, "value2":7166709656823836078, "value3":false, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.856975, "value1":0.7551172442205076, "value2":3114426699624764241, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290618, "value1":0.461652630545808, "value2":6870191814019559455, "value3":true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194719, "value1":0.28972301156720964, "value2":2266210405139781750, "value3":false, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.930146, "value1":0.8041428167626514, "value2":3758081748375193976, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.876259, "value1":0.8601408243336739, "value2":8153013110856241749, "value3":true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544542, "value1":0.4588432483941638, "value2":1078220017088765885, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861984, "value1":0.33210767773987954, "value2":6423637803003548004, "value3":false, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725768, "value1":0.13821724766383348, "value2":2685436575194046233, "value3":true, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422624, "value1":0.6174306852475553, "value2":622178413482526248, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.749470, "value1":0.7804428540523304, "value2":6875918761098582661, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.903503, "value1":0.7328633216838554, "value2":3622697451467218798, "value3":true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.116528, "value1":0.9193757905595477, "value2":6857103271674313745, "value3":false, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318972, "value1":0.35822522818261543, "value2":8829601973990551809, "value3":true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433538, "value1":0.5317306440222773, "value2":2556858263334639046, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497594, "value1":0.1404686557588262, "value2":5182788796116063454, "value3":false, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.731708, "value1":0.08352722566425953, "value2":57349948699707298, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052659, "value1":0.5013772188087474, "value2":4075548779807119976, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680970, "value1":0.7475466041318248, "value2":804362071276609030, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644235, "value1":0.13991370513704515, "value2":8012476351028838411, "value3":false, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.430709, "value1":0.07096170614998987, "value2":810033186472659399, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.283029, "value1":0.49501003784947184, "value2":5335533898113400696, "value3":false, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.213964, "value1":0.06985861165056967, "value2":3236599269070483646, "value3":false, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899939, "value1":0.8432631992057813, "value2":8331656940556283499, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447604, "value1":0.4633606486187383, "value2":3282862092550905414, "value3":false, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986819, "value1":0.31082999792855004, "value2":5075360107904774840, "value3":true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375656, "value1":0.6617351656082016, "value2":8930874302935418560, "value3":false, "__name__":"metric_00000278", "key_5":"j","key_1":"c","key_4":"h", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787774, "value1":0.5242892599116226, "value2":4186364548758456678, "value3":false, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918288, "value1":0.3268848136936015, "value2":2614642728372801254, "value3":true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535191, "value1":0.176268908097137, "value2":653040557920632015, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529661, "value1":0.35212458808041835, "value2":1525600014751099598, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.361165, "value1":0.4714568379001177, "value2":4598533494631122989, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898032, "value1":0.22769956750018774, "value2":5631589457641828317, "value3":false, "__name__":"metric_00000283", "key_7":"c","key_1":"f","key_5":"f", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483853, "value1":0.4317121876757055, "value2":564210499810148536, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512418, "value1":0.14841146100443744, "value2":6375007633580482412, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172575, "value1":0.4457725144006962, "value2":940411418055250112, "value3":true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109679, "value1":0.17465206890579033, "value2":619954528132439983, "value3":false, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016784, "value1":0.6172432789077167, "value2":8694611599799563, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405055, "value1":0.9156938137116516, "value2":1197968795715040120, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.153068, "value1":0.6193599211433017, "value2":3101256038412073082, "value3":false, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041261, "value1":0.0029965595633104653, "value2":1337872063966920132, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.852796, "value1":0.021368920853982197, "value2":7987040863826958067, "value3":false, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921132, "value1":0.47595655072486737, "value2":1700855145658274194, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.383399, "value1":0.26371142871266584, "value2":6832011082222328508, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.332123, "value1":0.8824714441278855, "value2":4889159017232292596, "value3":true, "__name__":"metric_00000294", "key_5":"a","key_7":"j","key_0":"f", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847690, "value1":0.5825485094693434, "value2":8301349950043802905, "value3":false, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004022, "value1":0.963420472883698, "value2":4714062367056435117, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178319, "value1":0.21282157659328568, "value2":5741804038235426956, "value3":true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226035, "value1":0.614117409288991, "value2":7916317937107113091, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188200, "value1":0.6715893678504037, "value2":7136230473959043902, "value3":true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709476, "value1":0.7492754619759127, "value2":6914951663040962797, "value3":false, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879839, "value1":0.9373684799176515, "value2":3333434004719572493, "value3":true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174084, "value1":0.14484188554360505, "value2":2035242236177706450, "value3":false, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.505125, "value1":0.57329431354178, "value2":5119683575480371621, "value3":false, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742853, "value1":0.7629472821449486, "value2":5737967107040353117, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226226, "value1":0.17730887465740228, "value2":4673386690110010077, "value3":false, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560859, "value1":0.9766459300795906, "value2":1996912721458005085, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262060, "value1":0.05094335959612934, "value2":1245357682330480297, "value3":true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432269, "value1":0.8390455137158405, "value2":1878222408225434076, "value3":true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113677, "value1":0.7363256343069756, "value2":2885521712019062646, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444967, "value1":0.7955117384805845, "value2":264780152192144506, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162729, "value1":0.6712123871834628, "value2":1119778679298263778, "value3":false, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.911266, "value1":0.9891407670464004, "value2":9103652755878870157, "value3":true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379007, "value1":0.2889256375011702, "value2":5327006092087239478, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.092408, "value1":0.21632368500444782, "value2":3442219206298318473, "value3":true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032852, "value1":0.524214031146113, "value2":6530887848447144232, "value3":false, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382209, "value1":0.7113041138812657, "value2":933816269978698137, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.334638, "value1":0.7695287307455443, "value2":7489040093668157103, "value3":true, "__name__":"metric_00000318", "key_4":"d","key_7":"f","key_1":"e", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372671, "value1":0.7439130206835773, "value2":4959199765009131376, "value3":true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900273, "value1":0.9052840812209041, "value2":965461062104746729, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.654196, "value1":0.8121385984212723, "value2":5665296699467259244, "value3":true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178571, "value1":0.19948342559708582, "value2":5100240681564871548, "value3":false, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770922, "value1":0.3886846963277322, "value2":619589477307797843, "value3":false, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.623448, "value1":0.010709272715713756, "value2":2405417022197491275, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.501430, "value1":0.49322743442420075, "value2":4840878662359361238, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268501, "value1":0.9623480970104531, "value2":7490234459966550271, "value3":true, "__name__":"metric_00000326", "key_7":"b","key_9":"c","key_4":"g", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403202, "value1":0.7953121912476196, "value2":6585060526257826515, "value3":true, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078706, "value1":0.6781344722473363, "value2":3204877843782827661, "value3":true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.343846, "value1":0.7227864686817432, "value2":3271852738757870545, "value3":true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.026648, "value1":0.47137422416076813, "value2":3734918729615183416, "value3":true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.691254, "value1":0.7660887564670621, "value2":790090719852690753, "value3":true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262583, "value1":0.666841123381339, "value2":2785906959504169261, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.234934, "value1":0.28467055157973764, "value2":5751420159217569904, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110004, "value1":0.6775088488593821, "value2":3553287660015547262, "value3":false, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766422, "value1":0.23227174275211976, "value2":623571748203397381, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800936, "value1":0.31353934764817454, "value2":4270432397290565810, "value3":false, "__name__":"metric_00000336", "key_7":"e","key_0":"k","key_3":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709087, "value1":0.45637800363407555, "value2":6299983262198737196, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756471, "value1":0.4807492151985086, "value2":7279712801686741955, "value3":false, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188086, "value1":0.36885137660554584, "value2":7300782007111153833, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.948483, "value1":0.37900022039799186, "value2":659834049221243616, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.589486, "value1":0.790378840630986, "value2":5696488550645422430, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783617, "value1":0.7580291430060272, "value2":3320953914407054988, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321932, "value1":0.6227238552522466, "value2":8556762730182657496, "value3":true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585825, "value1":0.8340787864264869, "value2":2874883662014238947, "value3":true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425246, "value1":0.4329750194122766, "value2":6011155334107441536, "value3":true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023097, "value1":0.474467449495024, "value2":9096564500742879948, "value3":false, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625423, "value1":0.1682038891195739, "value2":4431847776626169211, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289832, "value1":0.9685809921059807, "value2":1593648503211538110, "value3":false, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613185, "value1":0.10083138549838662, "value2":6012379324790188749, "value3":true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720225, "value1":0.22688014499451828, "value2":533494308206781076, "value3":true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.925929, "value1":0.3145810371686551, "value2":2521880979655219825, "value3":false, "__name__":"metric_00000360", "key_5":"j","key_9":"f","key_0":"a", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073559, "value1":0.8510652378781342, "value2":2771699030834759801, "value3":true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540880, "value1":0.19703714704465763, "value2":2229904355406996428, "value3":true, "__name__":"metric_00000362", "key_1":"d","key_0":"c", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358372, "value1":0.47865214913286414, "value2":991745116280869642, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.985891, "value1":0.5712880806403865, "value2":7108232781064624938, "value3":false, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787182, "value1":0.10623777547287752, "value2":5757177518524775613, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725343, "value1":0.4881539523572101, "value2":9008775172737657154, "value3":true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.276129, "value1":0.7963976864976252, "value2":4233311333311619132, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.296009, "value1":0.9362536518865696, "value2":6472012225611436212, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572284, "value1":0.5256827892320936, "value2":6967364447645723612, "value3":true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490559, "value1":0.5321215426269078, "value2":6272951345279554009, "value3":false, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.897929, "value1":0.9148681755770618, "value2":8519086271296505773, "value3":true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438881, "value1":0.9657537469163081, "value2":1825676918798292475, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.878233, "value1":0.44284605438890395, "value2":948269923000784878, "value3":false, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.240505, "value1":0.2373413838092857, "value2":510374534141290052, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041417, "value1":0.17592672383423122, "value2":3473808065501665173, "value3":true, "__name__":"metric_00000345", "key_8":"k","key_9":"a","key_1":"j", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191879, "value1":0.3751219146996357, "value2":7106698191372720163, "value3":true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694969, "value1":0.8445486295845246, "value2":672504742133677359, "value3":true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974994, "value1":0.6654201856510823, "value2":4512443800515268537, "value3":false, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.265020, "value1":0.3977309051692021, "value2":987280456712575618, "value3":true, "__name__":"metric_00000350", "key_9":"d","key_0":"f","key_7":"k", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.837282, "value1":0.6667916772962499, "value2":2649317741597325862, "value3":false, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584752, "value1":0.8184998685164273, "value2":6853243233010504233, "value3":true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956207, "value1":0.20740750871245983, "value2":3701599145151889680, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785977, "value1":0.42059150301577825, "value2":4300206568693523992, "value3":false, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206794, "value1":0.4065971742289675, "value2":5055750687883280777, "value3":true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.466356, "value1":0.24692397834834437, "value2":6503690772355582431, "value3":false, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374485, "value1":0.27439926954774063, "value2":2257827689674118963, "value3":false, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859317, "value1":0.15734285025101918, "value2":2188339349886556422, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.888728, "value1":0.2275635074824085, "value2":3959472993254545104, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552286, "value1":0.8140316248314068, "value2":28562363718590634, "value3":false, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101732, "value1":0.9986972418263012, "value2":6625998065774179710, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.064398, "value1":0.4719546471748007, "value2":5016107261432878232, "value3":false, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540095, "value1":0.05564657854486056, "value2":2892021965316178106, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052034, "value1":0.6085312696710864, "value2":267981962928859635, "value3":true, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479153, "value1":0.1864971302120239, "value2":4274890765687650257, "value3":true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285637, "value1":0.8618179927379825, "value2":98273395449385741, "value3":true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160949, "value1":0.1665709809492651, "value2":3549197933610586412, "value3":true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479807, "value1":0.9792753435060556, "value2":544874846395596701, "value3":true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.715679, "value1":0.955575677088658, "value2":7351930305259561505, "value3":true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655220, "value1":0.7716326625888155, "value2":490280663710319116, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799754, "value1":0.26438748028137354, "value2":2889376751051793411, "value3":false, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625103, "value1":0.1721904108442133, "value2":7991543743244408523, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.690861, "value1":0.007633155868195746, "value2":3073219921658712367, "value3":false, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308852, "value1":0.12391196742809665, "value2":1115580635235450437, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.013227, "value1":0.7267200692886595, "value2":1429240573369570759, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268822, "value1":0.6684636760714637, "value2":4339881778709198457, "value3":false, "__name__":"metric_00000396", "key_5":"f","key_2":"i","key_3":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160477, "value1":0.5396173638949372, "value2":4906545989468275563, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599858, "value1":0.07893489799204395, "value2":410345370022207222, "value3":true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584892, "value1":0.8068626341311561, "value2":4979004915398024459, "value3":true, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.987465, "value1":0.8935603600443448, "value2":8149710572361609309, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227032, "value1":0.3687576237691273, "value2":1391051851919204878, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833201, "value1":0.32203470987301264, "value2":2081318068679055279, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291063, "value1":0.034071004249405484, "value2":5342977357585868177, "value3":false, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290178, "value1":0.3044083195032883, "value2":4841141697891251443, "value3":true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109841, "value1":0.9228991323104464, "value2":1691775942527270832, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587504, "value1":0.5332728718746681, "value2":1334529969932810461, "value3":true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.920462, "value1":0.18350284649923848, "value2":2590094361323084553, "value3":true, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.966666, "value1":0.25923537198192, "value2":6116285159120956507, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480100, "value1":0.8741323699909098, "value2":1633286453079731557, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.175764, "value1":0.46174408884851265, "value2":5588956365079776400, "value3":true, "__name__":"metric_00000410", "key_7":"j","key_1":"i","key_5":"h", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357949, "value1":0.9471421832392611, "value2":2020538223652392065, "value3":false, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087613, "value1":0.8391096943297279, "value2":1883071449916289062, "value3":true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883487, "value1":0.5961374967262479, "value2":6748555704132728387, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.514904, "value1":0.43082646153613147, "value2":3827154946167603504, "value3":true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899576, "value1":0.926252451999718, "value2":6912098010507428750, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.708901, "value1":0.5214095428253637, "value2":8239976493295716264, "value3":false, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.784267, "value1":0.7119074977387614, "value2":5875315435697946397, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318238, "value1":0.2857683698624022, "value2":2168440855638261641, "value3":false, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434779, "value1":0.728469975577827, "value2":4027120624532411802, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432104, "value1":0.9159655292186285, "value2":1266477315347168519, "value3":true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.453551, "value1":0.9820731220635184, "value2":472902679452044251, "value3":false, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.239100, "value1":0.05657570742681277, "value2":1082500030056759801, "value3":true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918697, "value1":0.5552870083342735, "value2":3402695849932536811, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689901, "value1":0.372073062318352, "value2":4891331242898882829, "value3":false, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924258, "value1":0.5283619937827609, "value2":559122472611611400, "value3":false, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172686, "value1":0.4344219735503553, "value2":1248792109975059381, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894193, "value1":0.43426860175202114, "value2":3497813026288966485, "value3":true, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.327909, "value1":0.3878605121485491, "value2":5047756483827083362, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220825, "value1":0.07655058465784402, "value2":5252985818354538541, "value3":false, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742213, "value1":0.38430981024473343, "value2":5620787295533246773, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.733232, "value1":0.5555223164703845, "value2":631270712021422829, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940662, "value1":0.3098977203238577, "value2":5954879855194614083, "value3":true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.649184, "value1":0.3354932964670884, "value2":6051779552546236650, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832762, "value1":0.9063274954717605, "value2":3724265762255192701, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.385652, "value1":0.5036946685314467, "value2":5109085475601380385, "value3":false, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.722579, "value1":0.772640832685559, "value2":3780594965713240464, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655329, "value1":0.29356054638145535, "value2":4596604760791157242, "value3":false, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.459104, "value1":0.8777447945394818, "value2":1186981345641271839, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.399975, "value1":0.025674080171056102, "value2":8964167090414725476, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.123918, "value1":0.9272811881045043, "value2":7218529778562540903, "value3":true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747920, "value1":0.9264629051459597, "value2":8464913590880556582, "value3":false, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914960, "value1":0.32245869851475545, "value2":9109063864698425563, "value3":false, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.328190, "value1":0.7652891376156531, "value2":3024374861601192187, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593397, "value1":0.17904346621073747, "value2":3899031855173978925, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229837, "value1":0.5384685110923013, "value2":5235158233416481121, "value3":true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.698328, "value1":0.5982638259890453, "value2":6740778632490166175, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812614, "value1":0.1666081159883058, "value2":2800734738787135789, "value3":true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352669, "value1":0.33792886940688555, "value2":7496164581714310216, "value3":true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128711, "value1":0.2991783007626336, "value2":7338677993190210677, "value3":false, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552539, "value1":0.06968528327878533, "value2":8300480680068139329, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.581708, "value1":0.35231544880285015, "value2":1384864101595986439, "value3":false, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778952, "value1":0.8415964882413857, "value2":1745115595251159346, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.695991, "value1":0.5753223257206643, "value2":1886601101964605359, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664273, "value1":0.6725491194351899, "value2":5574699884022596470, "value3":true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711367, "value1":0.2595939895449639, "value2":7814193120722379636, "value3":true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.600545, "value1":0.944128951867823, "value2":6711650903238481221, "value3":false, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452146, "value1":0.9554010846729131, "value2":3169390990279464268, "value3":true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.140642, "value1":0.16345235341026784, "value2":2148518750069655597, "value3":true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388132, "value1":0.5979152819074895, "value2":7002872128153528233, "value3":true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078323, "value1":0.5697531979956287, "value2":980455479278929153, "value3":true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499938, "value1":0.388179798040964, "value2":2192712143484318261, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761109, "value1":0.2600517751619912, "value2":647733851046806648, "value3":false, "__name__":"metric_00000462", "key_9":"d","key_6":"d","key_8":"g", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088032, "value1":0.2162359479396891, "value2":7960761017907576840, "value3":true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886480, "value1":0.21950291960892157, "value2":7881441150954657573, "value3":true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.538255, "value1":0.36627249562354336, "value2":2697185681710870928, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956144, "value1":0.09482810661100612, "value2":3142048329987868522, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.610158, "value1":0.7330550786147262, "value2":4897289423318138878, "value3":true, "__name__":"metric_00000466", "key_5":"g","key_7":"h","key_0":"b", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641699, "value1":0.27867439634792107, "value2":621712637914424018, "value3":true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.218950, "value1":0.1329184934440547, "value2":3565116595465876544, "value3":true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.247888, "value1":0.7166338804544903, "value2":5367509956565676215, "value3":false, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.214416, "value1":0.3332699601933243, "value2":4699030477990361670, "value3":true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023652, "value1":0.5060732821933588, "value2":1384885559250896583, "value3":true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907328, "value1":0.1694015036022409, "value2":3592921690010847117, "value3":true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.642558, "value1":0.737568796988017, "value2":2406767446522063712, "value3":false, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273826, "value1":0.8447613680154935, "value2":6080438111270024688, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.984506, "value1":0.28262866688772725, "value2":6480278416681893463, "value3":false, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.868136, "value1":0.8910201029103536, "value2":6867531047341490726, "value3":false, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687877, "value1":0.8829884505686191, "value2":2249445178190997362, "value3":false, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850677, "value1":0.6276341510695712, "value2":2252184121487700107, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737726, "value1":0.12597210262772834, "value2":1348511802277935140, "value3":false, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.810752, "value1":0.2102822839010783, "value2":4831043683200695850, "value3":true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317105, "value1":0.817061302570146, "value2":8324179065679406640, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.426069, "value1":0.09982797254913842, "value2":6519984536301213031, "value3":true, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.121073, "value1":0.5197148618479782, "value2":3010915938164137229, "value3":false, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.264976, "value1":0.4225812260629186, "value2":5303229136813956536, "value3":true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572439, "value1":0.07380111555778088, "value2":4428887607089365223, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526101, "value1":0.2368198468823957, "value2":5676247054773494672, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.794040, "value1":0.8488571887152928, "value2":7926265788784400661, "value3":true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682501, "value1":0.6419011526811798, "value2":6127958830288275598, "value3":false, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.508833, "value1":0.08505936393863323, "value2":6956791791775335476, "value3":true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.485450, "value1":0.24440134625086388, "value2":8675077803244715182, "value3":false, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341184, "value1":0.7204080689080778, "value2":7288860462608592695, "value3":false, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300497, "value1":0.587674524187528, "value2":2136914643668219995, "value3":false, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646074, "value1":0.5687487761791935, "value2":5117858885820847437, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405215, "value1":0.63703146651586, "value2":8601605955587466200, "value3":true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040361, "value1":0.9056350649746174, "value2":4395139702119619272, "value3":true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042190, "value1":0.25258468888663055, "value2":6275160558122438530, "value3":true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436823, "value1":0.3052140189316623, "value2":1176588437754725430, "value3":true, "__name__":"metric_00000502", "key_9":"c","key_2":"d","key_4":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.885585, "value1":0.996098364991821, "value2":7425087259003526051, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.667291, "value1":0.01567259898271256, "value2":688757342279745926, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560861, "value1":0.8815928724937075, "value2":5127701164424889645, "value3":false, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921488, "value1":0.5280525030917018, "value2":8702967906065448733, "value3":false, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906624, "value1":0.204252621066632, "value2":7869988567138398448, "value3":false, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.855702, "value1":0.956230561465019, "value2":157946491688041108, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367342, "value1":0.6684623495570281, "value2":329650868812150871, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445102, "value1":0.6894360816538613, "value2":4920916512388188612, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.609045, "value1":0.0027756305160863397, "value2":5637928110041308291, "value3":true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.632075, "value1":0.04166500110036427, "value2":6157318199778992604, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972254, "value1":0.31480609981503055, "value2":4364758259634809973, "value3":true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.776493, "value1":0.13104837145124149, "value2":4732079918184171170, "value3":true, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238222, "value1":0.6559225899889957, "value2":3576156993215909053, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434447, "value1":0.24817502461178453, "value2":7546679218765777183, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040709, "value1":0.1860292914746309, "value2":1580899927050287482, "value3":false, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273335, "value1":0.9526744406123745, "value2":5086267667987270481, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052913, "value1":0.33164061518472737, "value2":6494564179935068135, "value3":true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093729, "value1":0.8324601543372498, "value2":4840824205416424699, "value3":false, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.167756, "value1":0.8292588795543363, "value2":926691352131787790, "value3":true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602891, "value1":0.7468923005026717, "value2":1499084894678948825, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939687, "value1":0.9093420550399373, "value2":7624268916058145495, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859436, "value1":0.018264976223293757, "value2":8966202800040990455, "value3":false, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540853, "value1":0.15398968210956118, "value2":1169217513145417422, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927943, "value1":0.6497527028232101, "value2":8964902118693660858, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032864, "value1":0.5440150103669782, "value2":3398396102381979841, "value3":true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440858, "value1":0.5805592700724092, "value2":598543749015746918, "value3":false, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220522, "value1":0.7634462305829404, "value2":4744586200886838973, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350814, "value1":0.8753641286301881, "value2":3594959380973533940, "value3":true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689230, "value1":0.11698455183819528, "value2":3362979353229964321, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369880, "value1":0.9959022990686378, "value2":913069268809019275, "value3":true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499935, "value1":0.42435230326696954, "value2":5642082683494178997, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.126493, "value1":0.24855180288733794, "value2":2553926153696879833, "value3":false, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241812, "value1":0.19715345400514303, "value2":7966035951898297683, "value3":true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.282709, "value1":0.26568871987724985, "value2":5614603759761174315, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634974, "value1":0.5611667195638738, "value2":719923428562838874, "value3":false, "__name__":"metric_00000533", "key_9":"f","key_2":"i","key_3":"b", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.437429, "value1":0.044098595487335565, "value2":7598428545433157434, "value3":false, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986084, "value1":0.15643080062821366, "value2":876340157580973000, "value3":false, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350215, "value1":0.5350647081004255, "value2":2039458830523373741, "value3":false, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.263790, "value1":0.13663992546983536, "value2":4887595679013193650, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761252, "value1":0.31778054800300737, "value2":4963237153707981360, "value3":false, "__name__":"metric_00000547", "key_5":"k","key_0":"k","key_1":"c", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358509, "value1":0.4977744678036884, "value2":2417156657544845857, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.825502, "value1":0.7927543298826962, "value2":7803607945365925463, "value3":true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463820, "value1":0.25407613728207273, "value2":7298874514736284603, "value3":true, "__name__":"metric_00000550", "key_7":"d","key_5":"f", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815985, "value1":0.03419039527127868, "value2":2983524479323907938, "value3":false, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648209, "value1":0.5111390924658735, "value2":4469943396931617311, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560198, "value1":0.3027175063306067, "value2":8582749896805213520, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293707, "value1":0.7381247559612524, "value2":267296433337218156, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436607, "value1":0.1678795867934854, "value2":2882160820519571353, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516040, "value1":0.02439990616294445, "value2":3459503829097802670, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861717, "value1":0.8549050337904404, "value2":5330118776081944380, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181801, "value1":0.014687212605387332, "value2":1741497345355098408, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398203, "value1":0.3791649455492745, "value2":7827883443566154722, "value3":true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004152, "value1":0.4627140852343998, "value2":5127693472090561791, "value3":false, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289305, "value1":0.9473077760244797, "value2":1972308969161346102, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.407619, "value1":0.22478804543381714, "value2":4393947727507143193, "value3":true, "__name__":"metric_00000553", "key_8":"g","key_5":"c","key_7":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500355, "value1":0.9148638467317476, "value2":2930801640270658847, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390931, "value1":0.03756752782865913, "value2":6450238118865267877, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436518, "value1":0.7302171451231968, "value2":8892033149379515881, "value3":false, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352126, "value1":0.6345536180749114, "value2":8076417892766647115, "value3":false, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211476, "value1":0.7869548859510007, "value2":2137142575900025284, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.620465, "value1":0.04777740137317064, "value2":2493346722675519040, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166994, "value1":0.5908047667032886, "value2":2133480026426783424, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799110, "value1":0.7447928303219921, "value2":6367274514210109366, "value3":true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.258756, "value1":0.5720021685376423, "value2":7819250772844902436, "value3":true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178485, "value1":0.22728962622611987, "value2":5810329748220466185, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756847, "value1":0.09118290676810949, "value2":2438579961381457911, "value3":true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447596, "value1":0.2813020748043202, "value2":4446085339672311465, "value3":false, "__name__":"metric_00000572", "key_6":"b","key_7":"i","key_1":"d", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.507148, "value1":0.8635348228135215, "value2":7235169187543778395, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412107, "value1":0.6168652299086078, "value2":8460560637760416654, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.916980, "value1":0.20506020965429841, "value2":1044309486752695204, "value3":true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358540, "value1":0.5251096014647666, "value2":6638181287796109637, "value3":false, "__name__":"metric_00000576", "key_5":"b","key_9":"c","key_2":"k", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.199047, "value1":0.6942630163772678, "value2":3410541935645339151, "value3":true, "__name__":"metric_00000577", "key_5":"h","key_6":"j","key_1":"g", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.754827, "value1":0.9837922147041048, "value2":3690131697027385202, "value3":false, "__name__":"metric_00000578", "key_4":"b","key_5":"h","key_2":"a", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.329509, "value1":0.8271404181330777, "value2":4227974754451880800, "value3":false, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.513229, "value1":0.672862073275747, "value2":3810363550665232995, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910574, "value1":0.9390557529752215, "value2":5133678849432201178, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181846, "value1":0.49307852075676845, "value2":6137025771035845818, "value3":false, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572559, "value1":0.2408765387072259, "value2":8298872358883983458, "value3":true, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078241, "value1":0.08393417297668128, "value2":8837983950648920911, "value3":true, "__name__":"metric_00000561", "key_9":"e","key_4":"k", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.056551, "value1":0.9755466271988745, "value2":2207490383904730952, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.814175, "value1":0.5043585864165596, "value2":5350947527006645760, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.066736, "value1":0.5177229679064868, "value2":2703347438865819514, "value3":false, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560402, "value1":0.4478041012053205, "value2":117552149576626326, "value3":false, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079515, "value1":0.133747701259907, "value2":5643720599026556636, "value3":false, "__name__":"metric_00000581", "key_9":"h","key_2":"f","key_3":"e", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246265, "value1":0.8787847052412423, "value2":8596628070583162060, "value3":true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.011510, "value1":0.3733460731999948, "value2":6601446014598669145, "value3":true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735379, "value1":0.9116679871732262, "value2":5605502556155572766, "value3":true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.574257, "value1":0.3506079241295861, "value2":1800090074854288106, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900580, "value1":0.8930823791186644, "value2":1843480248185265031, "value3":false, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.744935, "value1":0.754485340555253, "value2":1527411985430067052, "value3":false, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369195, "value1":0.7284643223167715, "value2":2551804296404453319, "value3":false, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500109, "value1":0.1715238288854149, "value2":3468590424262559119, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196638, "value1":0.4131391791311713, "value2":2036494112995161182, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660034, "value1":0.5162081878070082, "value2":683174465375000181, "value3":false, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.324311, "value1":0.6392052032119463, "value2":6854067358310903979, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477405, "value1":0.9223995048777075, "value2":8678575474954356403, "value3":true, "__name__":"metric_00000593", "key_5":"k","key_6":"b","key_3":"e", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894983, "value1":0.2879379950878977, "value2":1760029872301364905, "value3":true, "__name__":"metric_00000596", "key_8":"b","key_0":"a","key_4":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404612, "value1":0.5964910019926303, "value2":5352075776414002891, "value3":false, "__name__":"metric_00000595", "key_6":"e","key_1":"h","key_2":"j", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.406123, "value1":0.9911934759109439, "value2":2505572631856818583, "value3":false, "__name__":"metric_00000598", "key_5":"c","key_0":"a","key_4":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098160, "value1":0.29265324156352274, "value2":673331618670269188, "value3":true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317156, "value1":0.5284207406945622, "value2":5801235018377104530, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.278954, "value1":0.8339844494736254, "value2":142783045099103297, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144316, "value1":0.5981739630713123, "value2":4376705426457644175, "value3":false, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.533900, "value1":0.7048057967645066, "value2":7668900249344587055, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655047, "value1":0.9572743468227054, "value2":8615116979377517835, "value3":false, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.164063, "value1":0.12955196466904195, "value2":8720744967395487030, "value3":false, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211413, "value1":0.4645239019949383, "value2":1341011821671689315, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.254073, "value1":0.9282656526373411, "value2":5675449932048560089, "value3":true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939659, "value1":0.3774328021259814, "value2":7756643154892601396, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.789551, "value1":0.7757594275737888, "value2":8309777310097157940, "value3":false, "__name__":"metric_00000617", "key_6":"a","key_4":"c","key_5":"j", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.256014, "value1":0.7599489616826112, "value2":210625827724423434, "value3":true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883947, "value1":0.13842755442158153, "value2":5303830768512577958, "value3":true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516070, "value1":0.17914029301657106, "value2":1056091608874344104, "value3":false, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.446988, "value1":0.27173267141582086, "value2":6180352463990379834, "value3":false, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.622518, "value1":0.8672352642207358, "value2":6931279306288268314, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.002662, "value1":0.25623553544743577, "value2":4114415105877902794, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.250822, "value1":0.1691133676662619, "value2":8315894568290440082, "value3":true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148935, "value1":0.5989068031663785, "value2":916798823552491100, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404397, "value1":0.3854563629426573, "value2":1729425108588138938, "value3":false, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155114, "value1":0.42561573956387977, "value2":3412690494545228649, "value3":false, "__name__":"metric_00000627", "key_6":"f","key_4":"e","key_5":"k", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.185911, "value1":0.7064647650423553, "value2":6268941778110729170, "value3":false, "__name__":"metric_00000628", "key_4":"i","key_7":"b","key_1":"g", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.349779, "value1":0.9255472682273418, "value2":1166345991907977547, "value3":false, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.982678, "value1":0.6891602632892445, "value2":1740831383850752805, "value3":false, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914669, "value1":0.4165113125409203, "value2":1826795354292330747, "value3":true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871167, "value1":0.7086334187759732, "value2":5543325343112219047, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.762690, "value1":0.8505896053437381, "value2":5927978325036218166, "value3":false, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.292162, "value1":0.6672535143315814, "value2":1327915299266347941, "value3":false, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311132, "value1":0.47941008598606394, "value2":7437258572827323548, "value3":false, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523170, "value1":0.7404188223668129, "value2":5505665204112874175, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357326, "value1":0.9538155003965737, "value2":2702198679896121556, "value3":false, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.364597, "value1":0.17680019955559403, "value2":1753707621503560793, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452318, "value1":0.2992929781674707, "value2":8362411304781786581, "value3":true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518625, "value1":0.8647905294441987, "value2":5431066155038766988, "value3":true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.355524, "value1":0.6116936780714586, "value2":1234552975837284329, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542514, "value1":0.8351923165774915, "value2":1733954151005989610, "value3":false, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979827, "value1":0.2892109226113143, "value2":6049795733820800226, "value3":true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883185, "value1":0.011744889551642781, "value2":141159892430595908, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.223860, "value1":0.4073953516179897, "value2":6458792499529133548, "value3":false, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049058, "value1":0.12482723086065665, "value2":9009567363074084595, "value3":true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.298389, "value1":0.4827417824302383, "value2":4841281853843733128, "value3":false, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.629693, "value1":0.6591071306029623, "value2":3355181768485869219, "value3":true, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.147730, "value1":0.34536474665893674, "value2":3191505597112674507, "value3":false, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.179092, "value1":0.7637896583466016, "value2":1179889369576374895, "value3":true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.658439, "value1":0.9209377727035774, "value2":3422449294756383390, "value3":false, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.071622, "value1":0.28143127213081115, "value2":7766511653375780058, "value3":false, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537938, "value1":0.6024534595610256, "value2":8547099742960318907, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646788, "value1":0.20378541750257417, "value2":7978976103929383445, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747803, "value1":0.10859041477345463, "value2":1109588376642858177, "value3":false, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564874, "value1":0.23647658766118684, "value2":6863917686945126643, "value3":false, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042244, "value1":0.3078158837684314, "value2":4419488498667340703, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.284250, "value1":0.9387293705744738, "value2":6769306683790898588, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479123, "value1":0.4851789805412465, "value2":6507016270927900733, "value3":true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025704, "value1":0.8538216321323174, "value2":4020894753483999383, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833612, "value1":0.2626123684082926, "value2":5033990782030407302, "value3":false, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.881634, "value1":0.935752736774716, "value2":6384512508877447191, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474427, "value1":0.8815963618627093, "value2":8929077315195001710, "value3":true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321721, "value1":0.8079053030903706, "value2":7509708901019477626, "value3":false, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.854474, "value1":0.6950357389111533, "value2":6524710922297840763, "value3":false, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229889, "value1":0.6795085741944863, "value2":5609613021607849758, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.988904, "value1":0.5223586661895601, "value2":89977508053442787, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374340, "value1":0.755589597933608, "value2":7955537057313043569, "value3":false, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639189, "value1":0.7408346971613415, "value2":1902500838340597252, "value3":false, "__name__":"metric_00000669", "key_9":"d","key_4":"c","key_5":"e", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481547, "value1":0.5004511492728786, "value2":164083172923357290, "value3":false, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518019, "value1":0.729605886464792, "value2":9068451454110926149, "value3":false, "__name__":"metric_00000671", "key_7":"b","key_1":"c","key_5":"d", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.363805, "value1":0.7382344993077278, "value2":7045722716338732866, "value3":true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.630886, "value1":0.688064557117447, "value2":296768996976239880, "value3":false, "__name__":"metric_00000673", "key_8":"k","key_4":"g","key_7":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.845466, "value1":0.13213135497669065, "value2":4879659946939633689, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660200, "value1":0.2229370900047786, "value2":292379935200020145, "value3":true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073732, "value1":0.8115899146610889, "value2":8577867893469971833, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.019908, "value1":0.49801059455223595, "value2":7683222481005280397, "value3":false, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.347447, "value1":0.9276573673104045, "value2":2091335224644651701, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321409, "value1":0.6126771681057066, "value2":1893407571657392417, "value3":true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.738125, "value1":0.3569881382718092, "value2":4029761814756595416, "value3":false, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.665842, "value1":0.05170172426704011, "value2":8174581603042814007, "value3":false, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.449347, "value1":0.7466823594518665, "value2":6257307241351449210, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.588421, "value1":0.4753472682300776, "value2":4728406440983577982, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.257494, "value1":0.1386821629125743, "value2":332192645116862048, "value3":false, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.728108, "value1":0.43976058351020564, "value2":1985435892430278941, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.209628, "value1":0.8966889425339122, "value2":5787236380684241489, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.638851, "value1":0.9617304712840572, "value2":4707670589409962460, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484422, "value1":0.020940662405038862, "value2":1545134502951937463, "value3":false, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440131, "value1":0.17343525627253645, "value2":6009630316902812876, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229422, "value1":0.02146978522387302, "value2":2111892615484907893, "value3":true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780591, "value1":0.6377996056471382, "value2":8896668703689799570, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.124061, "value1":0.7970190759525996, "value2":7452754965853045737, "value3":true, "__name__":"metric_00000663", "key_2":"i","key_4":"g","key_0":"j", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.450958, "value1":0.8563337518192949, "value2":1606820680288550386, "value3":true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194998, "value1":0.8092586961600053, "value2":1104653518579575826, "value3":false, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746061, "value1":0.888221791516023, "value2":6881584298104407367, "value3":true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.550870, "value1":0.45584434378222843, "value2":7394447363524184561, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.684022, "value1":0.3067771551792505, "value2":1245627384806850593, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289452, "value1":0.4582593754084005, "value2":5465272523370779338, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235677, "value1":0.8840876989101745, "value2":551093905717196985, "value3":false, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.890313, "value1":0.09198263253620918, "value2":6028451346704401787, "value3":true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.168914, "value1":0.40840563117271195, "value2":3931349792536134233, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.717876, "value1":0.7410792258916561, "value2":5169350845698434472, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.202605, "value1":0.5290546996961457, "value2":3475693437398263961, "value3":true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636633, "value1":0.824130818871441, "value2":5903284123492948390, "value3":true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.615008, "value1":0.04083331973299552, "value2":7577674506757874978, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036559, "value1":0.5696805792367668, "value2":4633771544842561674, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367723, "value1":0.355783015228386, "value2":5675601540429514938, "value3":false, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907306, "value1":0.5698956093759964, "value2":9122457118633339454, "value3":true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685953, "value1":0.4276033626449204, "value2":7616472412744453525, "value3":true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587193, "value1":0.9520532910468696, "value2":2039460897796769985, "value3":false, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.322100, "value1":0.9372598841781506, "value2":8025865217929956411, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155211, "value1":0.2725517811852112, "value2":7680103519195376114, "value3":false, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964045, "value1":0.49419337991273576, "value2":8605447057400658561, "value3":false, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.423455, "value1":0.28782941494899417, "value2":7631463815755268967, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433388, "value1":0.6722143467332055, "value2":4743404486443339981, "value3":false, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073044, "value1":0.9919783220125814, "value2":3546009072923452494, "value3":true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060903, "value1":0.09931972482798261, "value2":3396822536822006988, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.952732, "value1":0.3775686488599951, "value2":7121618792612707640, "value3":true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521690, "value1":0.5851273414170087, "value2":4662473906180780029, "value3":false, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227468, "value1":0.11958534947502218, "value2":3788112644660727919, "value3":false, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196785, "value1":0.8966564462732959, "value2":4357533587354590088, "value3":false, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901366, "value1":0.5571789697475874, "value2":7759726578542538969, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.633498, "value1":0.9521589976099354, "value2":3412636563124759184, "value3":true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477196, "value1":0.6345321514502629, "value2":2968047284799701894, "value3":true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.245970, "value1":0.8602524837567829, "value2":239564384927138993, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923521, "value1":0.1090517952569125, "value2":8200798429042044530, "value3":false, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648242, "value1":0.5170091370557403, "value2":2725553204805480592, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812069, "value1":0.8884437199979248, "value2":9154279472010215199, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962857, "value1":0.2140414709499265, "value2":5080056778998450413, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.561764, "value1":0.6433379926959468, "value2":1246414196804721612, "value3":true, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288689, "value1":0.8910605919928337, "value2":8094720078703937078, "value3":true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.419833, "value1":0.6705423885099231, "value2":6631047350002617679, "value3":true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.085812, "value1":0.03831227212231745, "value2":2735730201464302258, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.046896, "value1":0.09940708362873886, "value2":600170391537990700, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.884000, "value1":0.5332485748572247, "value2":5278799395808920103, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.942254, "value1":0.6825570254740958, "value2":3505708958239891309, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010039, "value1":0.007884797092278711, "value2":3745225427592325971, "value3":true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.712360, "value1":0.2908960224193339, "value2":9147580185923316377, "value3":true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109158, "value1":0.3741670076640341, "value2":1005531285446043532, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680722, "value1":0.010722209112304005, "value2":826737562607372966, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.439540, "value1":0.4713372686055574, "value2":6754252699406092830, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392281, "value1":0.9898544758145399, "value2":2353492636354645276, "value3":false, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205843, "value1":0.6788358030320953, "value2":712272724855746801, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003145, "value1":0.7114317643988356, "value2":3716874444298014530, "value3":true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.133514, "value1":0.37586422274898074, "value2":2416361460381490648, "value3":false, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.904105, "value1":0.18756095304630882, "value2":6928964256284052382, "value3":false, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171387, "value1":0.6820596249231228, "value2":172259903859496202, "value3":true, "__name__":"metric_00000736", "key_6":"e","key_0":"i","key_4":"k", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.510204, "value1":0.06734756409519246, "value2":7819819964334132673, "value3":true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422546, "value1":0.1465987153240839, "value2":3085770726305992826, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.039220, "value1":0.39594354666033404, "value2":6309816101603497959, "value3":false, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.739730, "value1":0.8530985778840952, "value2":7379770578925142492, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.672747, "value1":0.7865961192571739, "value2":7774331371169359686, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.769896, "value1":0.802526565041674, "value2":3838614021162591095, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.304485, "value1":0.6949310789221889, "value2":7501685449967872481, "value3":false, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.678592, "value1":0.7504248678928968, "value2":7218481023042482207, "value3":true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764761, "value1":0.20348098581370722, "value2":5988835601145003290, "value3":true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.968921, "value1":0.21895244723583734, "value2":6407608012483731307, "value3":true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480313, "value1":0.3910222274917921, "value2":935230396616102910, "value3":true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680717, "value1":0.7679413590875351, "value2":6737105603635406237, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843792, "value1":0.928935788622385, "value2":8032690240756723112, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544961, "value1":0.3803375288284628, "value2":808064213573953017, "value3":false, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974943, "value1":0.2873027943668844, "value2":420303650121753052, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351352, "value1":0.9011332762961933, "value2":5787708950335762277, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049110, "value1":0.920403506389589, "value2":3542056558848585785, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.473874, "value1":0.20499658001386006, "value2":6473623955229286943, "value3":true, "__name__":"metric_00000749", "key_9":"e","key_5":"f","key_6":"c", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705527, "value1":0.3369786416226785, "value2":5565389573084278676, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977283, "value1":0.5753145958771464, "value2":5509805379215659395, "value3":true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.969055, "value1":0.7944991097395847, "value2":5227490476089768493, "value3":true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.260822, "value1":0.37247993171967786, "value2":9100748121410350084, "value3":false, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392300, "value1":0.9170516775216153, "value2":3337181235028553814, "value3":false, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.917458, "value1":0.7005088931842404, "value2":4249344436835260528, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398079, "value1":0.4412594689419285, "value2":355338602844953023, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605493, "value1":0.14953313696992326, "value2":7522955956227719413, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.947612, "value1":0.24504972733419753, "value2":19900356272350185, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.657164, "value1":0.7554151847082919, "value2":2083009730678034907, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.303410, "value1":0.9047440140278514, "value2":2665949305315254597, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720136, "value1":0.02233304294884983, "value2":2148351116720732280, "value3":false, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605342, "value1":0.06576250199061187, "value2":3149195196917185744, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.249421, "value1":0.10981969871005968, "value2":1489417694228791945, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490002, "value1":0.2365349300258502, "value2":5413143921150744151, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537535, "value1":0.056469545325025704, "value2":2942905392018720189, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748755, "value1":0.5086783291906242, "value2":4857376881146282553, "value3":false, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235928, "value1":0.6784255332464089, "value2":3029395751611067716, "value3":true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.401233, "value1":0.753375539931517, "value2":8169340970187425814, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960595, "value1":0.5301594317579601, "value2":3908827718931366422, "value3":true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113359, "value1":0.7603991477763598, "value2":2119709520548445193, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.875780, "value1":0.49587314465392923, "value2":8685987546300538378, "value3":false, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.938401, "value1":0.9152835192021896, "value2":1076417540165823423, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.869630, "value1":0.7973627027092416, "value2":2126356962780308110, "value3":false, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388566, "value1":0.6382739532774706, "value2":159979324198275701, "value3":false, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.275145, "value1":0.5927046376681012, "value2":7080867181269495541, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523278, "value1":0.5267214411412359, "value2":1728237597853111534, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584632, "value1":0.5083502145633392, "value2":7227399514745308579, "value3":true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964267, "value1":0.6658213068549924, "value2":5519764789585080716, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824564, "value1":0.9264931547852524, "value2":3998899671907293853, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986213, "value1":0.027645492661625626, "value2":3185626691636394685, "value3":true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634127, "value1":0.858418006506526, "value2":5813619502423233874, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.099856, "value1":0.5750664046005645, "value2":7125055104530490351, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685619, "value1":0.3275765619786257, "value2":3507230098472264785, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174236, "value1":0.36236865121536765, "value2":2097215901740320742, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093835, "value1":0.8733207807127527, "value2":5480486627610559582, "value3":false, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687722, "value1":0.42815747753178873, "value2":2463297564216975626, "value3":false, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015930, "value1":0.035417712050281464, "value2":6849011969803492014, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181519, "value1":0.9822679258679019, "value2":408242883289676228, "value3":false, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370465, "value1":0.9916450645571185, "value2":3540916962390370495, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.557960, "value1":0.814086825170977, "value2":975702025360269586, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785667, "value1":0.18117401736664177, "value2":8964384630261548826, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014126, "value1":0.6836156690970117, "value2":5150749937802379773, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310715, "value1":0.8554125170001826, "value2":6356794838014021178, "value3":false, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000193, "value1":0.43591866134299384, "value2":3781675008214883017, "value3":true, "__name__":"metric_00000800", "key_5":"k","key_7":"e","key_3":"d", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310472, "value1":0.20656655062551638, "value2":2155883927623577685, "value3":true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348781, "value1":0.1874457704512197, "value2":3156445090401189095, "value3":true, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737752, "value1":0.4107965308340836, "value2":6831391234474352496, "value3":false, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.864771, "value1":0.7087377270632053, "value2":3852759604751603741, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377175, "value1":0.302211293966416, "value2":3671141882815450126, "value3":true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035362, "value1":0.11868658008033231, "value2":4402958858442974997, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206005, "value1":0.9050640840246325, "value2":1571018380191846388, "value3":false, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607719, "value1":0.5276900307886362, "value2":2938372021759409514, "value3":true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847851, "value1":0.23589306520821895, "value2":8130082209207560079, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181853, "value1":0.3318615047159984, "value2":2814234275379593797, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093707, "value1":0.8735019416998069, "value2":7839357347839415099, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.050872, "value1":0.045154096442812874, "value2":1387216811650422069, "value3":false, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635715, "value1":0.7372871363614325, "value2":2653034997948603272, "value3":true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.807390, "value1":0.06360776205204009, "value2":8705617981394656683, "value3":true, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893750, "value1":0.033458142425902254, "value2":2550969485909611240, "value3":true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599717, "value1":0.9713785482184008, "value2":1528762568291094955, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544670, "value1":0.0942081326101987, "value2":5208646740238503092, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.460630, "value1":0.30899325368963804, "value2":6955190987114158639, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348159, "value1":0.8338319484328468, "value2":382561011167415899, "value3":false, "__name__":"metric_00000820", "key_5":"f","key_4":"d", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.866844, "value1":0.4559148181796855, "value2":4921190547959215054, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.779864, "value1":0.8408779744303566, "value2":5151627244176235100, "value3":false, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087888, "value1":0.9444160436820308, "value2":2636681313922893724, "value3":true, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422949, "value1":0.45533312838966733, "value2":8719456700476032056, "value3":false, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909073, "value1":0.4871022495216044, "value2":7470894518121500308, "value3":true, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526308, "value1":0.6556527723106481, "value2":2061496006850665904, "value3":true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483995, "value1":0.01917969989791312, "value2":6594173174887504625, "value3":false, "__name__":"metric_00000836", "key_9":"g","key_1":"k","key_3":"h", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.704096, "value1":0.7335839167893562, "value2":7683462035683402688, "value3":false, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796245, "value1":0.775721209508694, "value2":1762994312464074505, "value3":false, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635089, "value1":0.08634955165534107, "value2":5435729701416464796, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.146656, "value1":0.904887030211023, "value2":3845389016759808576, "value3":true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.975641, "value1":0.06011625753112877, "value2":2167561119971885977, "value3":true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.027722, "value1":0.7606346737865884, "value2":7806658536996851894, "value3":true, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.255754, "value1":0.8709289771166605, "value2":2946981794368069885, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682006, "value1":0.164502402573321, "value2":5375762287724896423, "value3":false, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901286, "value1":0.32721693073724145, "value2":1600743249293213477, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.844121, "value1":0.4592683330441191, "value2":5355971233932247551, "value3":false, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756958, "value1":0.06556927134836166, "value2":2324918632583888734, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.686437, "value1":0.5613972309334896, "value2":1964277954793319022, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.280286, "value1":0.210108301927216, "value2":4762745887015662113, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300106, "value1":0.3451768742896802, "value2":5756080702879459521, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008833, "value1":0.8860643993351358, "value2":5562038867537669443, "value3":true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964560, "value1":0.5679670350248304, "value2":7270655908716143592, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259598, "value1":0.1779142648395649, "value2":1989791279542261185, "value3":false, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289316, "value1":0.27910110845131275, "value2":9108304266352627509, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.380786, "value1":0.03821154554853608, "value2":5351444755572082025, "value3":false, "__name__":"metric_00000845", "key_3":"d","key_8":"b","key_0":"c", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.619088, "value1":0.014955635682113628, "value2":1506343674187146948, "value3":true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.991863, "value1":0.4507410741879421, "value2":700372855016807730, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.819751, "value1":0.47361005562943986, "value2":397573677309043949, "value3":false, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962980, "value1":0.5040052190830692, "value2":7266525751259633187, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.451541, "value1":0.1442134636404948, "value2":561640599502466121, "value3":false, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.387998, "value1":0.5301085061271325, "value2":9134048967019410288, "value3":true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300013, "value1":0.7373601287356207, "value2":342878676774549894, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008200, "value1":0.7728904842469639, "value2":6539748237741452295, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.697595, "value1":0.02404028274324282, "value2":5913828782152003967, "value3":false, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.726484, "value1":0.45889774925126936, "value2":8474247967864647236, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.696138, "value1":0.8390712125923527, "value2":8640587498190477408, "value3":false, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.053954, "value1":0.3443648011087006, "value2":6899937907932201153, "value3":true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.702864, "value1":0.09419193983638496, "value2":1509375003037818758, "value3":false, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379925, "value1":0.10612710322810476, "value2":7235965724589430822, "value3":true, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.065471, "value1":0.03922963726603652, "value2":5920886218601602154, "value3":false, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087291, "value1":0.4838942531586474, "value2":535199259720805783, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977724, "value1":0.6375345212159459, "value2":4619853255519698247, "value3":false, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566854, "value1":0.21378747510647453, "value2":3467239940916600883, "value3":true, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110862, "value1":0.6797718237586716, "value2":7914262457466633074, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440246, "value1":0.30120840846619557, "value2":3776365109572133674, "value3":false, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.797803, "value1":0.7797287504617298, "value2":5037915665887516284, "value3":false, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.586577, "value1":0.1709193163609849, "value2":5198104224414552394, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879594, "value1":0.7450124911318364, "value2":8773848846938448870, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886367, "value1":0.551654132389125, "value2":358026416808351667, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464288, "value1":0.7461865171369122, "value2":8085418698319569937, "value3":true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197248, "value1":0.8352566024408136, "value2":5751810192674740519, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.217337, "value1":0.19284687675681011, "value2":7864252915572676984, "value3":true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448294, "value1":0.42640546323275524, "value2":168095719577724557, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.591254, "value1":0.037748263913037684, "value2":2863811536714848498, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372170, "value1":0.6660431538229836, "value2":8567505344595141514, "value3":false, "__name__":"metric_00000885", "key_6":"g","key_9":"f","key_3":"k", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317243, "value1":0.18445456220952994, "value2":4074456277125729011, "value3":false, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.267738, "value1":0.2691217808870719, "value2":159261331127552033, "value3":false, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585076, "value1":0.21078242008152992, "value2":1239129613801043878, "value3":false, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979016, "value1":0.5956340156534701, "value2":9183614150234962352, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447286, "value1":0.10494392100402684, "value2":3770086671966420800, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105069, "value1":0.2794799410983503, "value2":3469689160956336636, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.173597, "value1":0.6860882854882017, "value2":6360441874685617414, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.835787, "value1":0.7543174676127191, "value2":6129127352506150313, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.772599, "value1":0.7516717167947417, "value2":8027123646004983290, "value3":false, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.115846, "value1":0.3569203207491387, "value2":8576278949637442153, "value3":true, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551211, "value1":0.9246333110829817, "value2":898669648143321253, "value3":false, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.230459, "value1":0.04953444106043317, "value2":2309701856093198744, "value3":true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060038, "value1":0.7972052364471702, "value2":2490346122315313464, "value3":false, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512371, "value1":0.09068245544499785, "value2":3847395208192505439, "value3":true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783506, "value1":0.7070750057247079, "value2":451775490327949567, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000319, "value1":0.9107891782905447, "value2":4820343479119697886, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628626, "value1":0.016138857727039164, "value2":6452512787294668321, "value3":false, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.063279, "value1":0.39758013318800806, "value2":2380900021298276106, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480402, "value1":0.014403992075819017, "value2":6901656754464212004, "value3":false, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.462954, "value1":0.20570595420817753, "value2":4308695238369802814, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308777, "value1":0.46154273334740287, "value2":533630259579000143, "value3":true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.006531, "value1":0.41769996010310556, "value2":1003539425076610908, "value3":true, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.606413, "value1":0.8392935388898198, "value2":7138000920468723420, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390008, "value1":0.9457032273317953, "value2":8816745864308273066, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.545540, "value1":0.9980077865004171, "value2":3431998048103898458, "value3":true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.771291, "value1":0.9269154791448218, "value2":2868685560372963447, "value3":false, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035244, "value1":0.15635792222626285, "value2":745780555759481875, "value3":true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.802078, "value1":0.49326419263144117, "value2":7061825021138296948, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141863, "value1":0.6394029445292002, "value2":4052750041261099856, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210519, "value1":0.4447889112985626, "value2":4491375706281220187, "value3":true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014644, "value1":0.5845639893426985, "value2":8504751940020912407, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.457311, "value1":0.39232937571361776, "value2":8948083137978870620, "value3":false, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497330, "value1":0.0554441906288732, "value2":8552872946126060990, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778960, "value1":0.3021332369906045, "value2":5664412036675630494, "value3":false, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151648, "value1":0.7180248405530221, "value2":4322565027390602745, "value3":false, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.286112, "value1":0.7877440826847747, "value2":6988799327817384951, "value3":false, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.131604, "value1":0.8048208447034889, "value2":7951591877124931974, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122845, "value1":0.26609349854031233, "value2":1608820578917270500, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.059135, "value1":0.8932421413801345, "value2":6863045131855298597, "value3":false, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.119289, "value1":0.6625474883897694, "value2":3938725492057076257, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.114546, "value1":0.1285766450638473, "value2":3866539491041344116, "value3":true, "__name__":"metric_00000908", "key_7":"e","key_6":"i", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134976, "value1":0.9092879262740383, "value2":3783610158682087147, "value3":true, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.676253, "value1":0.24518334285373874, "value2":5037863892800296388, "value3":true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613311, "value1":0.5221731263503148, "value2":6112150162991672535, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448627, "value1":0.09620166167463585, "value2":7262951598026367151, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370708, "value1":0.9034926573578455, "value2":6264024668154911646, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694827, "value1":0.4562036337293121, "value2":5132858458452108988, "value3":false, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.409213, "value1":0.2400040814231921, "value2":5692228309415552332, "value3":true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.272515, "value1":0.512618446397202, "value2":4245362498789762636, "value3":true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398497, "value1":0.9796136787594169, "value2":2854381927687220104, "value3":true, "__name__":"metric_00000926", "key_3":"b","key_5":"g","key_1":"k", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573421, "value1":0.25717373724181153, "value2":6071818930618476006, "value3":false, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499699, "value1":0.5015171773350958, "value2":6590026308842608922, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390872, "value1":0.15108026004608982, "value2":4054685864574240684, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.187721, "value1":0.11524534470163803, "value2":4129310514528385220, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486124, "value1":0.9297806553036565, "value2":2076740459139610542, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.506484, "value1":0.48473773192721936, "value2":5279382479053069968, "value3":true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095317, "value1":0.5302104024468535, "value2":1006511502199967697, "value3":true, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659995, "value1":0.7173304922658666, "value2":7272260713937588084, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206009, "value1":0.21353139637435428, "value2":4614697991046896838, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049549, "value1":0.9736103016824452, "value2":713014430698092731, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764793, "value1":0.5877828915841818, "value2":2743371147788843703, "value3":false, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095972, "value1":0.7802447999330678, "value2":1247537151507568078, "value3":false, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210056, "value1":0.13206812499303777, "value2":5119237508780620544, "value3":false, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477450, "value1":0.6302618943375251, "value2":3505999540579430287, "value3":false, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.182076, "value1":0.5504514321159135, "value2":2799023815129557532, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262688, "value1":0.9279334258185475, "value2":3960329791115549039, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601942, "value1":0.9333313963421735, "value2":8488152958276516414, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095627, "value1":0.2423663973685341, "value2":3147375434225291522, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325183, "value1":0.9698662748883076, "value2":8348906475599130816, "value3":true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535337, "value1":0.9108408536175355, "value2":548221601502028995, "value3":false, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.465394, "value1":0.20049512890521093, "value2":6918344754347341650, "value3":false, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805179, "value1":0.04912411768594746, "value2":7498908729880858823, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635014, "value1":0.5227012296024662, "value2":6384611575231594987, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481932, "value1":0.2492598145582209, "value2":7063605156324549396, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.458503, "value1":0.3120534424622104, "value2":219352200445992919, "value3":false, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144186, "value1":0.6043714345133836, "value2":5343022569080095368, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148408, "value1":0.7539243961545181, "value2":8912511444056366708, "value3":true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357993, "value1":0.9968001216010738, "value2":3427619790755264753, "value3":true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519356, "value1":0.2148487452898843, "value2":387702087539510555, "value3":true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631948, "value1":0.4990937312036263, "value2":936214783188826913, "value3":true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631191, "value1":0.03578918325727108, "value2":2415177266799716415, "value3":false, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.915725, "value1":0.6672040350791707, "value2":7911702372383761591, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898111, "value1":0.2610115750745007, "value2":6604319492118960080, "value3":false, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871853, "value1":0.700295668840288, "value2":4280817186446527854, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601683, "value1":0.48046941918756864, "value2":4803180873024223807, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036853, "value1":0.059813074330959816, "value2":4421336201693669164, "value3":true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.870282, "value1":0.09570589043315565, "value2":2716621407072678469, "value3":false, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543069, "value1":0.31121787546995405, "value2":1276512731094194139, "value3":false, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367830, "value1":0.09434652170549433, "value2":2256603050688647078, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468827, "value1":0.8402294714187805, "value2":299914447978500025, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910520, "value1":0.6890435117531037, "value2":2224012689460052929, "value3":true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291500, "value1":0.2581942233541333, "value2":94642526795868830, "value3":true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972763, "value1":0.39798358703466546, "value2":711133600064208053, "value3":true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398873, "value1":0.007238280134283497, "value2":414930643350925093, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003583, "value1":0.3709596459020853, "value2":7158538565937127848, "value3":true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288974, "value1":0.0033772263476846164, "value2":3505644260463770547, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.989217, "value1":0.7234413065529468, "value2":6102862603791704696, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151126, "value1":0.962243743917813, "value2":2796295067408464675, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.313076, "value1":0.4725232716261173, "value2":5800110791744456113, "value3":false, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.169414, "value1":0.10364598232077965, "value2":8013370755699143486, "value3":false, "__name__":"metric_00000980", "key_4":"d","key_7":"f","key_0":"d", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641143, "value1":0.3874568519220252, "value2":1865317425913683745, "value3":false, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960408, "value1":0.6444887127846117, "value2":2021509390881885067, "value3":true, "__name__":"metric_00000977", "key_7":"f","key_1":"h", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188655, "value1":0.9671102461325771, "value2":4499002439335266005, "value3":false, "__name__":"metric_00000978", "key_8":"h","key_9":"a","key_7":"e", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438594, "value1":0.23438356733095106, "value2":5880329494012383454, "value3":false, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.043142, "value1":0.8137602822079195, "value2":5072736510183548575, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566884, "value1":0.3037390436862004, "value2":1427943088173652054, "value3":true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.528095, "value1":0.06744248220950605, "value2":2617279385140322030, "value3":false, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.075656, "value1":0.6248261098594748, "value2":3102580216650467270, "value3":false, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644300, "value1":0.11077089127760144, "value2":252971746503923271, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061068, "value1":0.18796880019702752, "value2":1108553918392505010, "value3":true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.996996, "value1":0.5752434742054809, "value2":8128574670569848168, "value3":false, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015917, "value1":0.06928565567804344, "value2":6283279207443485305, "value3":false, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238217, "value1":0.29468486256767473, "value2":4879109468201269125, "value3":true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.302561, "value1":0.16374126420414117, "value2":1267840542849218667, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.879665, "value1":0.06081772906299747, "value2":7735358902544562393, "value3":true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.436414, "value1":0.9145077406345081, "value2":725040121642504252, "value3":false, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845550, "value1":0.6355646354104764, "value2":2412964869373168042, "value3":true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918386, "value1":0.3307533061620177, "value2":4567564826294647319, "value3":true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310744, "value1":0.5825111373150833, "value2":6000767898591093764, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206794, "value1":0.8760021186600452, "value2":3359095928804782209, "value3":true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.237359, "value1":0.7515468979685085, "value2":4507098732286155978, "value3":false, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247625, "value1":0.9212103536180063, "value2":1210374309278913705, "value3":true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.531076, "value1":0.9294917705494895, "value2":6008294356031388873, "value3":false, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258474, "value1":0.4801885775502877, "value2":4809296905208303773, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687251, "value1":0.5503479981842399, "value2":1159057662539665025, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294328, "value1":0.579524047000884, "value2":8591427275775614498, "value3":false, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030329, "value1":0.3897745465299769, "value2":8195510768796559399, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591813, "value1":0.4311576035013457, "value2":57188656194062821, "value3":true, "__name__":"metric_00000013", "key_7":"e","key_3":"f","key_6":"k", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258460, "value1":0.19968970719511664, "value2":6006706732580329556, "value3":true, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153359, "value1":0.6163826197572554, "value2":8075354818413450461, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977991, "value1":0.47088424321623057, "value2":1437620300807340754, "value3":true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.554048, "value1":0.1312819884002532, "value2":2639581932761358442, "value3":true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.829720, "value1":0.9986118788512084, "value2":3816923447231661055, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648184, "value1":0.9987080219677139, "value2":7493304631161246772, "value3":false, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171705, "value1":0.6479656070812448, "value2":940500361907630226, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452962, "value1":0.7379240782521741, "value2":8607242240788556904, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422715, "value1":0.29055682202125643, "value2":2609014884729200526, "value3":false, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997247, "value1":0.03493075047200949, "value2":1341694184124582251, "value3":false, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.454967, "value1":0.4083872840808467, "value2":7327414310312171249, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268085, "value1":0.9598378795022791, "value2":9180787761997570552, "value3":false, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.185765, "value1":0.869366023599698, "value2":6065348312952390593, "value3":false, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954707, "value1":0.29066212545734305, "value2":6500179214577769106, "value3":false, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134999, "value1":0.0746058596733823, "value2":3954294580008112843, "value3":true, "__name__":"metric_00000019", "key_7":"j","key_1":"b","key_4":"h", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.056701, "value1":0.08887774822608922, "value2":2624540757429278414, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503714, "value1":0.7939975169235949, "value2":1543165523760841555, "value3":false, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.407434, "value1":0.5263350258249397, "value2":6207484012298867707, "value3":false, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.216887, "value1":0.6135120862192558, "value2":6706252188296828465, "value3":true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950767, "value1":0.6258842267416237, "value2":6943474392706233335, "value3":true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409499, "value1":0.39019457036616395, "value2":5543165311657950313, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.064967, "value1":0.25118206025688317, "value2":4933404061205221336, "value3":false, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503859, "value1":0.6260829223715964, "value2":2553798156705274576, "value3":false, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.239884, "value1":0.0646597153433716, "value2":5357886478310061690, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.560239, "value1":0.025368350153306324, "value2":1834804451034467956, "value3":false, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021890, "value1":0.4250913620299391, "value2":7901663683744276123, "value3":true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.720024, "value1":0.7102429898685084, "value2":8848853087379978689, "value3":false, "__name__":"metric_00000040", "key_1":"b","key_6":"f","key_0":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721708, "value1":0.267345226146574, "value2":8204105067810932755, "value3":false, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.688701, "value1":0.8124970825519422, "value2":5072280932532136578, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227719, "value1":0.19432503349760097, "value2":2763726347992488568, "value3":false, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398501, "value1":0.6725319245300637, "value2":2029542129845625509, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220707, "value1":0.07663396351127535, "value2":6862161694126124253, "value3":true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464464, "value1":0.8033021261492177, "value2":500776341255510367, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.369494, "value1":0.6874411246839779, "value2":5472077652556256405, "value3":false, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429691, "value1":0.7205162840867915, "value2":2379554410345555228, "value3":true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.003693, "value1":0.5301205012344029, "value2":7339147645001280473, "value3":false, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095495, "value1":0.0031080960946583136, "value2":3948518200809126624, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.649621, "value1":0.9796250725140294, "value2":5076285879574984318, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.701028, "value1":0.4664187080974912, "value2":8061147057085676917, "value3":false, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024773, "value1":0.7030139999067455, "value2":8201442408389043582, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570617, "value1":0.6283295712663881, "value2":7893659755244222642, "value3":false, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780271, "value1":0.6223497445918998, "value2":4413418515678745434, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904547, "value1":0.28169993275009253, "value2":1702651611066126658, "value3":false, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.039402, "value1":0.42131084562328136, "value2":3560241450438323083, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.612707, "value1":0.19465390726157364, "value2":3161717756488318325, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729159, "value1":0.25669937307380997, "value2":1479715553122219203, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813686, "value1":0.049715056995271165, "value2":2079826454687289078, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747376, "value1":0.7275373884923515, "value2":2435962270026982603, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354388, "value1":0.8636816800782944, "value2":6027722589314572513, "value3":false, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505532, "value1":0.1663611987871669, "value2":668594150031699725, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087699, "value1":0.09888054207021492, "value2":8612173434517336899, "value3":false, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.118211, "value1":0.9755664430251321, "value2":3670406131876856616, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918355, "value1":0.2320424600489978, "value2":8375963807463485059, "value3":false, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106082, "value1":0.2496270382815134, "value2":3218140633568902603, "value3":false, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.356200, "value1":0.22905701139904852, "value2":4580449910037652861, "value3":true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272156, "value1":0.08246894655199775, "value2":8520120895086499443, "value3":true, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.445288, "value1":0.13062843592283024, "value2":1544733522378030955, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189215, "value1":0.023728801354414537, "value2":7950194843845547325, "value3":true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525278, "value1":0.8120711270842138, "value2":6926776820889632931, "value3":true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532234, "value1":0.2803691083519268, "value2":2750609942680894663, "value3":false, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169390, "value1":0.721935860273012, "value2":323165999073131824, "value3":true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465184, "value1":0.14065806381420595, "value2":6373433541582043482, "value3":false, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211347, "value1":0.5119890992803398, "value2":437881008737730578, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.326310, "value1":0.8982944342798628, "value2":7550941116286319434, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767054, "value1":0.058056603449873775, "value2":8172199064443221808, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098754, "value1":0.802732462605452, "value2":5531373846445331861, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.626701, "value1":0.9658445273400498, "value2":6144529458195478799, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382019, "value1":0.6200649361393201, "value2":74370737764383341, "value3":true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.162075, "value1":0.773701976515762, "value2":3415021911489973103, "value3":false, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452128, "value1":0.4165931264471384, "value2":1592331302702594637, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546168, "value1":0.9818897837453856, "value2":2770075967419206536, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794037, "value1":0.6329308941123749, "value2":1855788356162220067, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267186, "value1":0.5183944510585616, "value2":2971715621482767583, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.386007, "value1":0.7925028746106562, "value2":5519530031313056293, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194807, "value1":0.9560872102829179, "value2":8899546063621825706, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786449, "value1":0.25289379431092024, "value2":3323685962729555247, "value3":true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722033, "value1":0.791566877264509, "value2":8569419440028565177, "value3":false, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.527726, "value1":0.5123053424894405, "value2":5410088201011323097, "value3":true, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991248, "value1":0.00835285017592764, "value2":7876085652270552879, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971469, "value1":0.7516893928865965, "value2":1615677872423733394, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457203, "value1":0.2802029059377931, "value2":3774880897765407379, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429963, "value1":0.4064794421282188, "value2":7288375695455796287, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807745, "value1":0.1346588472416809, "value2":7295335243976756666, "value3":false, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247446, "value1":0.18437680636683432, "value2":2763564622590822709, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505494, "value1":0.36367826246909607, "value2":279237135102501393, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570322, "value1":0.807767632694814, "value2":3489957523135662213, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.770864, "value1":0.12825646293816148, "value2":559050377523308246, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723654, "value1":0.48061854827645123, "value2":2625015563945376475, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939025, "value1":0.31592719165163524, "value2":1661597397006530804, "value3":true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935809, "value1":0.16925319668616004, "value2":8449044609778795224, "value3":true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336027, "value1":0.7017500784708112, "value2":8428599073105193397, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904159, "value1":0.7924061448124604, "value2":4266452884457119467, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847458, "value1":0.27361557683078364, "value2":3345769475113460399, "value3":false, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378261, "value1":0.7380019020377752, "value2":432938298263940108, "value3":false, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799977, "value1":0.25999970547607815, "value2":7119589132321429408, "value3":true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786435, "value1":0.812994267772834, "value2":2642960787391817015, "value3":true, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.587571, "value1":0.5078453077544904, "value2":7413095241087719577, "value3":true, "__name__":"metric_00000100", "key_2":"a","key_3":"f","key_1":"e", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189572, "value1":0.33029349839366906, "value2":182717731110127548, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128493, "value1":0.3005744554380485, "value2":1813809549706584614, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.751426, "value1":0.43021252312166497, "value2":1048575931846703963, "value3":true, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.824593, "value1":0.9962985612048794, "value2":3451457313304562908, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913497, "value1":0.19667409773488792, "value2":324965724484330012, "value3":true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280371, "value1":0.6558253377067158, "value2":3238482029594886215, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953879, "value1":0.11691624934505847, "value2":3461549771600466582, "value3":true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189513, "value1":0.9019750283572279, "value2":7575759643142018621, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465264, "value1":0.15533711497405195, "value2":8078121216216233303, "value3":true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857857, "value1":0.2731351567904969, "value2":6433553377051334332, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.114548, "value1":0.9868338794536605, "value2":6036587149016739858, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734426, "value1":0.4227117888273388, "value2":1455241552057769722, "value3":true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075679, "value1":0.2701709613339434, "value2":3857058056766739646, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104255, "value1":0.981970010025583, "value2":3682624724640104305, "value3":false, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653572, "value1":0.008103832880105273, "value2":16321050615205215, "value3":false, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.790159, "value1":0.6178089725023603, "value2":2820923101957050608, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274981, "value1":0.28628503444016773, "value2":5532572594206077059, "value3":false, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274235, "value1":0.01138331892670461, "value2":4146948379716629700, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280495, "value1":0.19902196639234795, "value2":7564853449905093412, "value3":false, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272293, "value1":0.017257126721765374, "value2":91860645774780383, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447052, "value1":0.6776141143437516, "value2":5897324288007611430, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.209396, "value1":0.14657117940198489, "value2":3976211106646613048, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850761, "value1":0.5201711080016932, "value2":7354069105129952055, "value3":false, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.300233, "value1":0.1860571279843043, "value2":470481327297902326, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310707, "value1":0.25674808205958144, "value2":2662172304685648211, "value3":false, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766353, "value1":0.3692045132202804, "value2":7757321942348800299, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214351, "value1":0.6936702854905948, "value2":8248415695462939576, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.005940, "value1":0.9564433372149446, "value2":8800540347032950677, "value3":true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009379, "value1":0.5129617748107491, "value2":2230603715203037945, "value3":false, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867507, "value1":0.6850688842908961, "value2":4341430047057401636, "value3":true, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640527, "value1":0.8542276865628686, "value2":6337085201397795074, "value3":true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186612, "value1":0.2520653493154805, "value2":3597439324405314019, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168374, "value1":0.9686674775113324, "value2":5086009893745928969, "value3":true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.930110, "value1":0.9065446675547199, "value2":9084895148279172527, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385337, "value1":0.5310871446244799, "value2":3634604672435791412, "value3":false, "__name__":"metric_00000137", "key_6":"k","key_1":"k","key_4":"a", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273397, "value1":0.6978323941286791, "value2":6270777752769360572, "value3":false, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.644440, "value1":0.7390124331493296, "value2":5633558971491993000, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692014, "value1":0.5735092134024398, "value2":4018757059763515593, "value3":true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307407, "value1":0.9523210688679733, "value2":4664291077378357176, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.864207, "value1":0.2504856063622225, "value2":5125796995165750818, "value3":true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.891946, "value1":0.8905322399035085, "value2":3790600090207208565, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.196262, "value1":0.02637661052612273, "value2":3345947703698603019, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609259, "value1":0.9964561610316119, "value2":4297199572790537149, "value3":false, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.479133, "value1":0.573177630994298, "value2":4352137818057928535, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.212122, "value1":0.9374174976714222, "value2":2883686750556684691, "value3":true, "__name__":"metric_00000146", "key_7":"i","key_1":"c", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143194, "value1":0.5493618690016547, "value2":8675397367930704936, "value3":true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.911116, "value1":0.8323148117854684, "value2":3727739579448171557, "value3":true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.100156, "value1":0.7882098153425944, "value2":6270723454145661182, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159904, "value1":0.5891414895820671, "value2":7892306327256327213, "value3":true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.561342, "value1":0.575909380567069, "value2":2087294011397500886, "value3":true, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.542768, "value1":0.7419123808449162, "value2":5892135507379591161, "value3":false, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977295, "value1":0.2537719509110862, "value2":23083079770681507, "value3":true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.690704, "value1":0.08136832639928646, "value2":8018440792149947537, "value3":true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.681388, "value1":0.6173470752976815, "value2":4725502695033185312, "value3":false, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045833, "value1":0.9052130764103621, "value2":2335561799986333486, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.621313, "value1":0.5578705772605427, "value2":5423853801041319612, "value3":false, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362143, "value1":0.058506006415061124, "value2":1952601720041809520, "value3":true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.019544, "value1":0.2454839257683829, "value2":5439837990435419484, "value3":false, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414507, "value1":0.8898999762410835, "value2":9399980232748751, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.946393, "value1":0.5231814332055934, "value2":8044307915684488670, "value3":false, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640533, "value1":0.1401932575466956, "value2":3489584691123939236, "value3":false, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.641620, "value1":0.7793690564908118, "value2":6745229093735363060, "value3":false, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256954, "value1":0.9792655297711959, "value2":8340241164033802673, "value3":true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.578274, "value1":0.6786422376056035, "value2":2268577288780537910, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168131, "value1":0.7011560783500894, "value2":5754899077173699372, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261168, "value1":0.41241864943729767, "value2":6926145892542625665, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.142353, "value1":0.026897262060108916, "value2":3407492707727317952, "value3":false, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453717, "value1":0.9480329042934568, "value2":5909842310603335433, "value3":true, "__name__":"metric_00000169", "key_6":"d","key_9":"b","key_4":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.498253, "value1":0.2696408414861873, "value2":9093122739488143709, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152193, "value1":0.829008206269259, "value2":7872930012981859271, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147195, "value1":0.40564509426795664, "value2":388387455399110782, "value3":true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.756476, "value1":0.08296241719338777, "value2":8819867312676478909, "value3":false, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472139, "value1":0.10622575406945087, "value2":2389837147036501120, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553362, "value1":0.4640810980824681, "value2":9129773762938561987, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138568, "value1":0.27153851873742235, "value2":3922539618739320056, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629854, "value1":0.06560725045609712, "value2":5242856550347246211, "value3":false, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532533, "value1":0.6685532806932011, "value2":4403464406211940116, "value3":false, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.102100, "value1":0.30269603070417783, "value2":560972484995423105, "value3":false, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.594212, "value1":0.3436785616541505, "value2":6118258015340076117, "value3":true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182345, "value1":0.5829131946786057, "value2":316161802601703652, "value3":false, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865350, "value1":0.9510270552078569, "value2":5815202444060769343, "value3":true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.417157, "value1":0.07805271825644472, "value2":1671886271645798176, "value3":false, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453428, "value1":0.07722736953435244, "value2":5062577774198478117, "value3":true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923875, "value1":0.3869476521576109, "value2":7644772358804477804, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170062, "value1":0.771360835653392, "value2":8029933370964328416, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.341390, "value1":0.538519995660261, "value2":6637857221246165272, "value3":true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267222, "value1":0.4666279704151431, "value2":716146669296183590, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562383, "value1":0.35102191528024995, "value2":8428363533739190009, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767969, "value1":0.6173626174937589, "value2":6290463280158957319, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451339, "value1":0.546379610337649, "value2":1547962669719183472, "value3":true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464199, "value1":0.24671185662577808, "value2":3561721323195761229, "value3":true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.154391, "value1":0.45596856325620033, "value2":6560907831281448338, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.709403, "value1":0.6109692366076535, "value2":3486468545935536219, "value3":false, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570798, "value1":0.7335030585660903, "value2":1572258901091495894, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.198469, "value1":0.7425214958348296, "value2":3904425769094656355, "value3":true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315912, "value1":0.9245437278546984, "value2":3047808486948520796, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186365, "value1":0.8721078635865479, "value2":6305488287785962360, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759050, "value1":0.3487806615217454, "value2":5791750426523247447, "value3":true, "__name__":"metric_00000199", "key_5":"j","key_9":"j","key_2":"f", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991504, "value1":0.5895957831739977, "value2":6282966318152098641, "value3":true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355229, "value1":0.507857444797784, "value2":295494213133771851, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551655, "value1":0.4307208831464999, "value2":7265410065885482832, "value3":true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525491, "value1":0.6404292728685821, "value2":1060635693413562315, "value3":true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008415, "value1":0.6916001000931801, "value2":2353592246165483773, "value3":false, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400680, "value1":0.36661865825780576, "value2":9104767548881126156, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376594, "value1":0.3981424874885838, "value2":7118511131751820911, "value3":true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553143, "value1":0.6460744131761798, "value2":1887476981066696157, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473253, "value1":0.8348910772414322, "value2":7455007366058138980, "value3":true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457579, "value1":0.4708610456466168, "value2":4593338028669686470, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433566, "value1":0.16598645827046257, "value2":5257287213674383265, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.357007, "value1":0.7860515830836307, "value2":2972497673348348004, "value3":true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.461293, "value1":0.35370822925539525, "value2":355204159426372577, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.975565, "value1":0.6491904274322989, "value2":2226145731856801788, "value3":true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951226, "value1":0.8792346577420341, "value2":5756323624906456991, "value3":true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143238, "value1":0.9818710036776283, "value2":4180983964934518595, "value3":true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874423, "value1":0.3287251148209575, "value2":381928105696021842, "value3":false, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648299, "value1":0.1842978254216889, "value2":5567179544229053293, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055528, "value1":0.2649298342733898, "value2":3063386077105020906, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551525, "value1":0.008929798118195076, "value2":1932022596647743318, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078106, "value1":0.6249963076831733, "value2":3775614794809061222, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595048, "value1":0.6556888188689401, "value2":263555282139003693, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270487, "value1":0.15652423199780005, "value2":8005167724994681049, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986526, "value1":0.44168299905409525, "value2":3616684978189935787, "value3":false, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648015, "value1":0.6935400265504532, "value2":3554010850084269714, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082643, "value1":0.4464596047653067, "value2":7393355921025489008, "value3":false, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.297812, "value1":0.8411130245970256, "value2":3060908524323233968, "value3":false, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.023535, "value1":0.8979883834169406, "value2":3955075066530261651, "value3":false, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775710, "value1":0.803115138047979, "value2":1956023105923958421, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276355, "value1":0.35534087510852963, "value2":70554362711904210, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227561, "value1":0.3475937884638202, "value2":5770217876057854108, "value3":false, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.997058, "value1":0.9920212102015171, "value2":4179786645072950325, "value3":true, "__name__":"metric_00000233", "key_9":"i","key_3":"e","key_4":"j", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330001, "value1":0.7117987721576648, "value2":1416148178901813673, "value3":false, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.859492, "value1":0.07088427876105753, "value2":5628499584516778303, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.758514, "value1":0.5315142010423931, "value2":3999275359211756172, "value3":false, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567291, "value1":0.5976846753929923, "value2":3568009536081310459, "value3":true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914132, "value1":0.8979176888984427, "value2":4778372357738152583, "value3":true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.470387, "value1":0.2136647847272661, "value2":4632422950916722092, "value3":true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.180567, "value1":0.7349455061346736, "value2":4238367012667671950, "value3":false, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444766, "value1":0.35029664162897833, "value2":4149159141941833270, "value3":true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.351876, "value1":0.719839928132912, "value2":8311035304279403168, "value3":true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.343283, "value1":0.5403744774398413, "value2":7831018924301288402, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202600, "value1":0.07323861368397915, "value2":7512140161978615708, "value3":true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953729, "value1":0.4072430540967121, "value2":6162477020927495810, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206750, "value1":0.7276384026815652, "value2":3867378392166470570, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313521, "value1":0.19760060640640484, "value2":1845943836170488560, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774075, "value1":0.9632484532837077, "value2":2186178755453106945, "value3":true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228120, "value1":0.1138891785769252, "value2":2119625663561385331, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355712, "value1":0.12321943936422765, "value2":5732137663228356419, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.004820, "value1":0.1110884226197388, "value2":2265343653685159151, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.630751, "value1":0.6269969213669998, "value2":3359610757917670818, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.015500, "value1":0.20581219173107831, "value2":452968813923389028, "value3":false, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646040, "value1":0.14066092038312122, "value2":6343927209062846133, "value3":false, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220902, "value1":0.6865529990306172, "value2":1648919416099165702, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.700811, "value1":0.4621690976315305, "value2":3551937779376072568, "value3":true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607198, "value1":0.8914891421096152, "value2":246252330281719887, "value3":false, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.574035, "value1":0.7865743679720645, "value2":5713754277685792561, "value3":true, "__name__":"metric_00000255", "key_7":"e","key_9":"e","key_1":"i", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473497, "value1":0.13845795308548414, "value2":259488391880499746, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944779, "value1":0.9609062478569093, "value2":5705930854457323614, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775443, "value1":0.1259913054905557, "value2":5080990926603035088, "value3":true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010365, "value1":0.4903104589413477, "value2":3314614042137410585, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913172, "value1":0.7864192987989552, "value2":8344919384469197086, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.796557, "value1":0.037094581426906255, "value2":9011770804417776349, "value3":true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.314769, "value1":0.8877112795540667, "value2":6683425457896895076, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.806104, "value1":0.5962447685862431, "value2":5435268152791547936, "value3":false, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579388, "value1":0.6885293468734471, "value2":9064761548833010698, "value3":false, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.391022, "value1":0.5900734892123236, "value2":4669259037774416678, "value3":false, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762980, "value1":0.7685532202125763, "value2":2315953471058650483, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.135687, "value1":0.32474681440848086, "value2":5755607255133419339, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104934, "value1":0.14589803567481788, "value2":126178294464524762, "value3":true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.052690, "value1":0.593421743400015, "value2":5119197034484564414, "value3":false, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870007, "value1":0.4673411129128983, "value2":2860543686633489085, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629308, "value1":0.4498616996591619, "value2":5328613189051877451, "value3":true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711415, "value1":0.8849559918426787, "value2":5903543494874730971, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901063, "value1":0.5349473283169682, "value2":4797032563086773247, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.651731, "value1":0.3127106327077303, "value2":5700343521566497813, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825358, "value1":0.7283436835171048, "value2":7844311848127264742, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.956544, "value1":0.8853976152911399, "value2":5661939812745981161, "value3":true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235737, "value1":0.26141150008124164, "value2":1680083033155058708, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.537374, "value1":0.4021256295776066, "value2":412217863647126208, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238448, "value1":0.20048610554803628, "value2":6383291537020572135, "value3":false, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506422, "value1":0.3812390668975348, "value2":184701845674968962, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.456772, "value1":0.12479606082790685, "value2":7770738457381111114, "value3":true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.789484, "value1":0.7412786023699602, "value2":492664655073083816, "value3":false, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931307, "value1":0.2536425784942127, "value2":3030103981590386173, "value3":true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610031, "value1":0.5842291904640166, "value2":5028658995770055300, "value3":false, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.068107, "value1":0.6578126156370956, "value2":4352952521986270134, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477620, "value1":0.16120256759137608, "value2":8502715215777515370, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.124418, "value1":0.2689168702571952, "value2":404701828083799050, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.779829, "value1":0.4388821199327946, "value2":4899529018433185166, "value3":false, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800101, "value1":0.2714233939432663, "value2":584157730559703393, "value3":false, "__name__":"metric_00000273", "key_7":"i","key_0":"g","key_2":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860807, "value1":0.4135766617320191, "value2":3872756991712201227, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.917667, "value1":0.2149289211585331, "value2":1070350813804397058, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214604, "value1":0.6792613272879433, "value2":2691750153580462441, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245813, "value1":0.6913342601180135, "value2":6216928639197666035, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087487, "value1":0.9301668438033497, "value2":6232780192940116715, "value3":false, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.069143, "value1":0.6293446581971062, "value2":7108419812354921929, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635591, "value1":0.9539262040674916, "value2":2954917330417044389, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672064, "value1":0.8389645710809651, "value2":3571122666192720690, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818753, "value1":0.0576648702049322, "value2":450051931472461981, "value3":true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270657, "value1":0.28617962738097397, "value2":7674542934415081242, "value3":false, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726745, "value1":0.2340744755556526, "value2":82189729669291091, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.897029, "value1":0.2638345630149327, "value2":8238518585440831118, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721815, "value1":0.7091199604872773, "value2":6366310396364171134, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.130557, "value1":0.49724111456500786, "value2":5852311186811706792, "value3":true, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447829, "value1":0.6230142507141527, "value2":937535691807945647, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.802396, "value1":0.6394461194574114, "value2":7242948197847068875, "value3":false, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.424144, "value1":0.6000896127174374, "value2":8028530164864078540, "value3":true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.426986, "value1":0.7003205100320555, "value2":2322106140405745021, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942361, "value1":0.016353745384231323, "value2":4629676907790639064, "value3":false, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653679, "value1":0.37062297658466353, "value2":8208981100462487617, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.992030, "value1":0.7072168752768998, "value2":3820991654654691772, "value3":false, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.346953, "value1":0.4620611667835753, "value2":4760146202938852105, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321463, "value1":0.9135126280713665, "value2":8417775919956392807, "value3":false, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168908, "value1":0.05729213648419359, "value2":8181061490241314804, "value3":true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.230852, "value1":0.17085551083436254, "value2":3688016011930153595, "value3":true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.066979, "value1":0.33115248245947465, "value2":398734641986366420, "value3":false, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825110, "value1":0.29481497745263574, "value2":7623192343447837676, "value3":false, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.979130, "value1":0.18961515202216345, "value2":6037869636552702023, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692641, "value1":0.4102441559202999, "value2":2079215473977001869, "value3":true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459527, "value1":0.38418621904384953, "value2":8455919021737289595, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924379, "value1":0.831174718392, "value2":8455941435197243693, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433952, "value1":0.6847864876960775, "value2":6021961695231181621, "value3":true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195096, "value1":0.9137826039265785, "value2":1951159443698721147, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.392032, "value1":0.7140691740516873, "value2":7387270195217478014, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711603, "value1":0.30553043110809663, "value2":5447610537284294754, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024988, "value1":0.636153947206049, "value2":6011662688198487566, "value3":true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140076, "value1":0.9829818790209834, "value2":1992400557046890911, "value3":false, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591682, "value1":0.734180708126354, "value2":8165041750078897088, "value3":false, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.254962, "value1":0.4530139927706582, "value2":792701874387399785, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653418, "value1":0.6853469695508817, "value2":2445415548541311714, "value3":true, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.406935, "value1":0.49945705352987035, "value2":4326827190201670587, "value3":true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140154, "value1":0.19200910536315854, "value2":7788114053604406936, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774910, "value1":0.775205817556658, "value2":1778011502106771934, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224005, "value1":0.4109368947961367, "value2":5090636250181538696, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025070, "value1":0.8639998386130291, "value2":4097020029155871346, "value3":true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.110800, "value1":0.46475105418540896, "value2":7930781849523032659, "value3":true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502796, "value1":0.19123829931308983, "value2":4926709577891896623, "value3":false, "__name__":"metric_00000346", "key_8":"g","key_9":"c","key_4":"e", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.483311, "value1":0.21080006327277256, "value2":1394785449775081074, "value3":false, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194479, "value1":0.7780668170411426, "value2":3729639098545975029, "value3":false, "__name__":"metric_00000337", "key_1":"c","key_4":"b","key_0":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385848, "value1":0.4299644347219865, "value2":7191790268699040877, "value3":true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211519, "value1":0.32521468058588077, "value2":2791560527192035681, "value3":false, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228801, "value1":0.83757037869911, "value2":7191574037900046957, "value3":false, "__name__":"metric_00000336", "key_3":"e","key_7":"e","key_0":"k", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.752930, "value1":0.8040696176893675, "value2":1942969218455296906, "value3":true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951018, "value1":0.07747976080254226, "value2":1630225104393013299, "value3":true, "__name__":"metric_00000339", "key_9":"g","key_4":"k","key_5":"b", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.750256, "value1":0.816008410350239, "value2":6724933613410762759, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583564, "value1":0.4508316797935446, "value2":6917556384524072348, "value3":true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697470, "value1":0.8171672522091267, "value2":2060386169134326462, "value3":false, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.974221, "value1":0.7951082158998446, "value2":153021997721922670, "value3":true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134841, "value1":0.655263428147034, "value2":6943977496854991893, "value3":false, "__name__":"metric_00000333", "key_4":"b","key_9":"e","key_1":"f", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.673465, "value1":0.16441674918821836, "value2":3664243165148767961, "value3":false, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152562, "value1":0.26528241288165916, "value2":1119343360475809102, "value3":true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.420604, "value1":0.2336675869017092, "value2":3279324159460647609, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.622036, "value1":0.6846769209119129, "value2":7351022317760422092, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687001, "value1":0.06651364619191985, "value2":4264970580734046753, "value3":false, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106345, "value1":0.7730270909594562, "value2":6259010316309812350, "value3":true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018243, "value1":0.39163099713401783, "value2":2052648194611616106, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576138, "value1":0.9602266945474953, "value2":5355810905366063269, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398686, "value1":0.6503809877595728, "value2":8153069195959796847, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.367754, "value1":0.6043902819598663, "value2":3335160234939115824, "value3":true, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.089576, "value1":0.7877453973247192, "value2":7617166012064737014, "value3":false, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.112529, "value1":0.42004273729990893, "value2":2084869905228279863, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262886, "value1":0.6795217484416415, "value2":8937461554206260283, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652433, "value1":0.15404027924570762, "value2":16114496573329591, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.285612, "value1":0.36648611710364465, "value2":4517711436918873094, "value3":true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.213223, "value1":0.283682948749034, "value2":1960872184236727731, "value3":false, "__name__":"metric_00000374", "key_9":"a","key_5":"k","key_8":"j", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096677, "value1":0.1578537002758255, "value2":7008414275581832390, "value3":true, "__name__":"metric_00000375", "key_6":"a","key_7":"d","key_1":"j", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.900633, "value1":0.993822419434507, "value2":3303474850432526688, "value3":true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862995, "value1":0.2925883436045491, "value2":4890951234376924728, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.038886, "value1":0.18850439286651008, "value2":7964663822924622191, "value3":false, "__name__":"metric_00000378", "key_8":"d","key_0":"e","key_6":"f", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742880, "value1":0.8063312642154098, "value2":8220352304335077837, "value3":true, "__name__":"metric_00000379", "key_7":"a","key_9":"k","key_3":"f", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545517, "value1":0.023471717745134867, "value2":5984820965643364202, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.079979, "value1":0.6088556886509169, "value2":8961288376687773441, "value3":false, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409232, "value1":0.6445899053375183, "value2":992192951010465319, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224358, "value1":0.13573773649022736, "value2":1165195770092489984, "value3":false, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211580, "value1":0.8798345576210722, "value2":2259526712170036956, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.137602, "value1":0.5470546586447269, "value2":2196698895749095736, "value3":true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950011, "value1":0.5109493788687889, "value2":7352128446573662528, "value3":true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098941, "value1":0.972587647595324, "value2":4702835811693986573, "value3":true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036022, "value1":0.7940355478266105, "value2":3489006783771959904, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.257286, "value1":0.6629731808259061, "value2":6607686064851284173, "value3":false, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283256, "value1":0.4735441861177576, "value2":690002256366468336, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729363, "value1":0.5152988869648636, "value2":7191544072945571987, "value3":false, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.671685, "value1":0.7719299111978523, "value2":8611060565403129118, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585682, "value1":0.23795953447079246, "value2":7946332098229864997, "value3":true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.187797, "value1":0.09090886526352324, "value2":470342401691160594, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.617428, "value1":0.5223785659162559, "value2":5891162691146013349, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.053786, "value1":0.9753973493719631, "value2":1732499838549362952, "value3":true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.830052, "value1":0.2400470925507307, "value2":7569204709479310112, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566485, "value1":0.9031315965779712, "value2":1637870392526502935, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104166, "value1":0.8068833505635166, "value2":5557577539110699594, "value3":true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457581, "value1":0.31154508147136156, "value2":3306826589254236098, "value3":false, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740978, "value1":0.6517923386066168, "value2":6929527434345999196, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398554, "value1":0.7163746669385104, "value2":6895927508923594471, "value3":true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174253, "value1":0.37937402384141433, "value2":1418065909790834443, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168613, "value1":0.8552112788507821, "value2":1201668238332263018, "value3":false, "__name__":"metric_00000403", "key_6":"i","key_1":"k","key_2":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376103, "value1":0.7428463858250676, "value2":9210155210384521495, "value3":false, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.798751, "value1":0.16215960908856522, "value2":4916478620663149838, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.588227, "value1":0.7494081995223059, "value2":1969994590907491188, "value3":false, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270909, "value1":0.8815916954125361, "value2":2264317244685656881, "value3":false, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.466310, "value1":0.09496768292246209, "value2":5092383460942749308, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543412, "value1":0.9329719667021983, "value2":5844328151679816255, "value3":true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.834516, "value1":0.9248915367132075, "value2":5869247588035820642, "value3":false, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.916922, "value1":0.052803170907048405, "value2":105752679780838825, "value3":false, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583307, "value1":0.04534276867452472, "value2":1081545276806182474, "value3":false, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.957758, "value1":0.1893001037569564, "value2":4530859851365168030, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723091, "value1":0.22925973916873355, "value2":5896808329469448104, "value3":false, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.719902, "value1":0.5038773612977684, "value2":5428690516490754673, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025587, "value1":0.7184099655587061, "value2":4108913234892685798, "value3":true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907580, "value1":0.24492386825368684, "value2":1156962833669313399, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958121, "value1":0.9818175985414233, "value2":3857893912179537366, "value3":false, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.657555, "value1":0.17952027005330515, "value2":3409773579631767064, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829662, "value1":0.9863896179595923, "value2":7017412361402418209, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382356, "value1":0.13377486891358994, "value2":2261611253942664564, "value3":false, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.735596, "value1":0.3414280491857885, "value2":1656516704640942493, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.365266, "value1":0.08360581432684039, "value2":751089660510969379, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.236047, "value1":0.6527140947106955, "value2":7257801694535836049, "value3":false, "__name__":"metric_00000424", "key_8":"b","key_1":"i","key_2":"d", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.563096, "value1":0.4425122630203769, "value2":8109435442419953734, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870160, "value1":0.7097748830786019, "value2":6394016408878658736, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853311, "value1":0.7865716590956611, "value2":9110725490965710227, "value3":false, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.103882, "value1":0.9153251610630555, "value2":5340001815935707095, "value3":false, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.319691, "value1":0.32338872565142723, "value2":9182777252636363483, "value3":true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971522, "value1":0.5621974159988574, "value2":6803454508761360177, "value3":true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893225, "value1":0.5781000116088169, "value2":3230113190945788558, "value3":true, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.910026, "value1":0.6955188069638236, "value2":3766609848094762934, "value3":true, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.680766, "value1":0.24230859153208487, "value2":5617809008491748996, "value3":true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.493463, "value1":0.1697890841270668, "value2":8556153147754124462, "value3":true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.849699, "value1":0.7552917967316483, "value2":2395400148768820203, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091646, "value1":0.5496353344673094, "value2":8512525001992418076, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104484, "value1":0.4174390270830882, "value2":8422771009020841833, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210780, "value1":0.8954537390033984, "value2":4547239076695924457, "value3":true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.882314, "value1":0.5142219879570834, "value2":7919864591569880176, "value3":false, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.983791, "value1":0.42885136319724737, "value2":7969816758490279476, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747300, "value1":0.4699241249924091, "value2":4491569276746468237, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939701, "value1":0.6800950779540557, "value2":3198358625582737117, "value3":true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261442, "value1":0.4575968148914831, "value2":6477186909414818810, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256111, "value1":0.693918133492274, "value2":1509539897942915838, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893394, "value1":0.31628967903294547, "value2":5501152751179175693, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742786, "value1":0.9162968056386509, "value2":9070857057311832045, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.642040, "value1":0.9183764345745785, "value2":1183434137529208194, "value3":true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045625, "value1":0.9635794700047972, "value2":8991017817592303034, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847013, "value1":0.07723071092189422, "value2":6180620156015906931, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532846, "value1":0.5943541460929466, "value2":2462231763403080171, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514157, "value1":0.7898774054205837, "value2":9059207414611927294, "value3":true, "__name__":"metric_00000452", "key_9":"h","key_0":"i", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.044776, "value1":0.9033139820464094, "value2":6478344621702437696, "value3":false, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043799, "value1":0.8974102840071708, "value2":4685725453168510298, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138572, "value1":0.5940072973710391, "value2":9150962581658325077, "value3":true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.215236, "value1":0.35464434283570645, "value2":2728457088873951162, "value3":false, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545063, "value1":0.4562894763140951, "value2":3160291795997224895, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.277361, "value1":0.9620522510409097, "value2":2528750129940465586, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477478, "value1":0.5247853848955957, "value2":7252619165572966740, "value3":false, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.940911, "value1":0.4288772636544275, "value2":1739941452377564494, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.488981, "value1":0.07801581197604958, "value2":3785496084335402474, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.037817, "value1":0.20985274999200432, "value2":3757407018866875990, "value3":false, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477559, "value1":0.26133928393072436, "value2":4699711244003996922, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.107051, "value1":0.5035109860681579, "value2":2406818127100942026, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692353, "value1":0.44174492856895586, "value2":5849623391254459092, "value3":false, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.948736, "value1":0.6131752371911697, "value2":8355568973826156875, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562536, "value1":0.5314554437999132, "value2":2325922727970832181, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336595, "value1":0.8477764608771982, "value2":2899112579581457318, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609668, "value1":0.7984731186839654, "value2":1035702079046481045, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.446414, "value1":0.8536378880203014, "value2":1450863797869138094, "value3":false, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.345841, "value1":0.005289676671501025, "value2":9055262701472338992, "value3":false, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.665956, "value1":0.5725507407225954, "value2":2388492838265413703, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.972916, "value1":0.6568884959214099, "value2":4110169872571045957, "value3":true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.370968, "value1":0.9171054847132539, "value2":9195948027770005199, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109600, "value1":0.6976295242986092, "value2":3260150427416206815, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722444, "value1":0.756396263144578, "value2":5473984714896425934, "value3":true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.011414, "value1":0.35518020333634837, "value2":5432640933038728748, "value3":false, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.442479, "value1":0.9135774445018765, "value2":1658872104403529722, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860707, "value1":0.1419422977742929, "value2":5745627115489673481, "value3":true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.925502, "value1":0.19711051262279727, "value2":4597556979704859536, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.636414, "value1":0.8593669141574183, "value2":3782467267185587990, "value3":false, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157759, "value1":0.4115235495685266, "value2":3223572108511746336, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399657, "value1":0.24629418245237128, "value2":4491685955930394692, "value3":false, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.941994, "value1":0.17062270403788604, "value2":3592023630547953731, "value3":false, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282191, "value1":0.8843261061566057, "value2":6691527928807377675, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938895, "value1":0.7167094005927336, "value2":1624441217258843945, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492025, "value1":0.09243530358924243, "value2":1564407566445130438, "value3":false, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837030, "value1":0.7918808546682014, "value2":6330206212330805579, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.123530, "value1":0.594358060271596, "value2":1086840755761366155, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.515405, "value1":0.8774866546466452, "value2":6471463500843130242, "value3":false, "__name__":"metric_00000466", "key_7":"h","key_0":"b","key_5":"g", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814527, "value1":0.46973250770262637, "value2":617895520080191082, "value3":true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.712491, "value1":0.8366069393700927, "value2":8124214803409242134, "value3":true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.598705, "value1":0.13529834276697159, "value2":4048137400971693133, "value3":false, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.115099, "value1":0.013374612744122648, "value2":4497239763994228002, "value3":false, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.871918, "value1":0.5477513867264745, "value2":6929880807654898865, "value3":false, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.600230, "value1":0.06263242199896547, "value2":7099533171627178830, "value3":false, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.832053, "value1":0.9159517384120022, "value2":6068004913866310526, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497324, "value1":0.7077689276515862, "value2":8496454581274124817, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912067, "value1":0.9966402570764005, "value2":6508382435383037912, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394175, "value1":0.4056622098754871, "value2":7483291783499825043, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931738, "value1":0.7229091089946994, "value2":6319491958713592651, "value3":false, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.778862, "value1":0.3546846056067059, "value2":5823244617744051085, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.571315, "value1":0.32603322678558494, "value2":617900084765432670, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.022106, "value1":0.09634190796880056, "value2":1538435119318752571, "value3":false, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777759, "value1":0.9991415497234644, "value2":6469274665586859562, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813159, "value1":0.6009731720422214, "value2":5681496788640905646, "value3":false, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990295, "value1":0.866629146873839, "value2":454920947769684349, "value3":false, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577898, "value1":0.4068627360956291, "value2":8354382964548721361, "value3":false, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652554, "value1":0.25976343730256174, "value2":4258649045980411819, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031275, "value1":0.4835922460947118, "value2":1965749848088972278, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535728, "value1":0.0741614372394035, "value2":2365543114658538202, "value3":true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.725451, "value1":0.08897749963143643, "value2":2568717794069312737, "value3":true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517633, "value1":0.31412074219954, "value2":3756199754285644161, "value3":false, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170595, "value1":0.31501680305424884, "value2":835864889280146724, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.839334, "value1":0.8860221287236891, "value2":7263801537900469082, "value3":false, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874807, "value1":0.37435025928000243, "value2":7922739501753662605, "value3":false, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914081, "value1":0.7766285758384013, "value2":8262805777914063347, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.668141, "value1":0.6740415300239384, "value2":5791687526396481300, "value3":false, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.302177, "value1":0.7738583815242437, "value2":4861664383009090324, "value3":false, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711096, "value1":0.314472985957645, "value2":8509002774444373125, "value3":false, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321149, "value1":0.44826985902573663, "value2":5704588376153110492, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.438786, "value1":0.9611261969930821, "value2":8417802456083541652, "value3":true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894152, "value1":0.47571072484930443, "value2":22587301710957954, "value3":true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939556, "value1":0.9226156153333962, "value2":5568759137623866023, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159870, "value1":0.6737683215814319, "value2":23284244914642959, "value3":true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273159, "value1":0.7150778132424469, "value2":7795879578675436289, "value3":true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.116050, "value1":0.7176538032312242, "value2":6236044932175094049, "value3":true, "__name__":"metric_00000526", "key_4":"c","key_0":"k", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274847, "value1":0.2989651835291124, "value2":2096206477458603718, "value3":true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764340, "value1":0.5947263013675924, "value2":949703240129275173, "value3":true, "__name__":"metric_00000507", "key_3":"i","key_1":"a", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.073476, "value1":0.1345576666087966, "value2":2417122739684294792, "value3":true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418445, "value1":0.8647442373572726, "value2":4787011153359227960, "value3":true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721780, "value1":0.31241733378602804, "value2":2882725760348945089, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338609, "value1":0.30106180327338455, "value2":4145007555047407199, "value3":true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109899, "value1":0.7817730958310003, "value2":5914588161487452188, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.547269, "value1":0.29274410504915255, "value2":3689334620249634917, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741862, "value1":0.6895772402009777, "value2":4687778247837546339, "value3":true, "__name__":"metric_00000527", "key_5":"b","key_9":"k","key_3":"a", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.744160, "value1":0.7533050071878981, "value2":183319309026503278, "value3":false, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.880890, "value1":0.8337197335863336, "value2":5434700933804867558, "value3":false, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467450, "value1":0.7739450951598513, "value2":8969122970020127825, "value3":false, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036765, "value1":0.05604892859180446, "value2":5414621267015229038, "value3":false, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458853, "value1":0.3999783567682559, "value2":7084993602991838778, "value3":true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.144253, "value1":0.945939865721221, "value2":9151551595196013169, "value3":false, "__name__":"metric_00000533", "key_3":"b","key_9":"f","key_2":"i", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.519078, "value1":0.28562220040037745, "value2":3466327056358787385, "value3":false, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.425448, "value1":0.1561471042921615, "value2":8555609213379362480, "value3":false, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.071015, "value1":0.867937414704974, "value2":7843796710543352776, "value3":true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936460, "value1":0.8572536817998362, "value2":3953769786680934648, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886358, "value1":0.2289778721765615, "value2":7617650297619769040, "value3":false, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.029905, "value1":0.0015729191075447354, "value2":2294660572707247909, "value3":false, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777761, "value1":0.9646894479758846, "value2":530724906601952281, "value3":true, "__name__":"metric_00000539", "key_6":"e","key_3":"i", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294885, "value1":0.6419477950120894, "value2":3626154680940200317, "value3":true, "__name__":"metric_00000545", "key_6":"g","key_3":"c", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210708, "value1":0.9629469371323777, "value2":1825251501924013732, "value3":true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354794, "value1":0.9560891897559984, "value2":7402084087841622412, "value3":true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.007599, "value1":0.8488357845508476, "value2":9129548229641808509, "value3":true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.949198, "value1":0.3153555717941824, "value2":7434568718980757764, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373802, "value1":0.3277196318709405, "value2":3450473392216255589, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.815211, "value1":0.19640953195392433, "value2":6991753246444170561, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532006, "value1":0.8004824485602414, "value2":6264834568239277372, "value3":true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.745472, "value1":0.20136170467216696, "value2":5514377914099787271, "value3":true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.296097, "value1":0.9552161212557331, "value2":1452145138149816534, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350623, "value1":0.4834905958063239, "value2":3177892637454509046, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894862, "value1":0.26274803670489, "value2":5904879356296841126, "value3":false, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663989, "value1":0.4617229653712207, "value2":5839673609391532826, "value3":true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920142, "value1":0.4202645422145252, "value2":6322617270554106528, "value3":false, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174524, "value1":0.7616258127751694, "value2":465160921360548556, "value3":true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829492, "value1":0.9553481357587528, "value2":1445392053101953093, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.419698, "value1":0.7725216991307152, "value2":4285351248890366822, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235468, "value1":0.9784345273084265, "value2":3668509819811778630, "value3":true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935515, "value1":0.5182554276152802, "value2":3295309486606700899, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539034, "value1":0.37544115144770396, "value2":7306450144895271272, "value3":true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729718, "value1":0.8597111754781319, "value2":3238304594948466970, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538038, "value1":0.32408768170822555, "value2":2186207806521299657, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566784, "value1":0.4892156459829451, "value2":8596226688481136458, "value3":false, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.980628, "value1":0.8327067880954203, "value2":8561137068463144173, "value3":true, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153522, "value1":0.15467129518207473, "value2":8602088790336601513, "value3":false, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748640, "value1":0.18204671422495655, "value2":6797988693835752844, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247682, "value1":0.09249535842888819, "value2":5067988675442272893, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936986, "value1":0.9192193881284415, "value2":5123985662043884348, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378293, "value1":0.815846067412878, "value2":6070693823656037795, "value3":false, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734247, "value1":0.49161995996865615, "value2":624464230830574462, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.661797, "value1":0.2056464067499432, "value2":6156118294329348494, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.033052, "value1":0.35405594685495784, "value2":520776464703947921, "value3":false, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.575933, "value1":0.9335712679806217, "value2":425476291317360309, "value3":true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.732828, "value1":0.4700815629744225, "value2":8550463063060596020, "value3":true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.899613, "value1":0.4385944862087238, "value2":7654768751774889730, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165683, "value1":0.16899343532059313, "value2":5688582074969409894, "value3":false, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986964, "value1":0.253512537787597, "value2":8166110434329136902, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766787, "value1":0.6248495968383331, "value2":8035594691046262470, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338227, "value1":0.6533144293221477, "value2":3922786069578896774, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912364, "value1":0.888149605868496, "value2":7372081699617979857, "value3":true, "__name__":"metric_00000589", "key_8":"f","key_1":"a","key_4":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.223399, "value1":0.21329948027080436, "value2":8415945829680890770, "value3":false, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.922973, "value1":0.6987095535905381, "value2":8304144963505893865, "value3":false, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.484292, "value1":0.6786625244755433, "value2":595838710724334047, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977202, "value1":0.8656898577077133, "value2":3529785897914282727, "value3":true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.231129, "value1":0.9035229811237685, "value2":2477457900157688998, "value3":false, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.565357, "value1":0.40499024658110055, "value2":1356497801667980388, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.691153, "value1":0.5098765403884086, "value2":2450630350992474672, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459909, "value1":0.07976566115765178, "value2":3732709013710166299, "value3":true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104179, "value1":0.3445131016649922, "value2":528513889221012225, "value3":true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764200, "value1":0.1348858138724795, "value2":409727276676244812, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288182, "value1":0.46853874054271116, "value2":6728001446241512599, "value3":true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502298, "value1":0.31577497007213595, "value2":8859554815564990855, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990152, "value1":0.3102588503519357, "value2":5723742327078842798, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520176, "value1":0.15090023272414005, "value2":5438285255049886987, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.155921, "value1":0.6850691286515489, "value2":7357616240486203605, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350044, "value1":0.8819289168133723, "value2":4656926419487785926, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.383040, "value1":0.8254228746494352, "value2":2036646364854508134, "value3":false, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451480, "value1":0.9507122214170446, "value2":4089393907406910121, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516899, "value1":0.7707382433506376, "value2":5274734231531309553, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467177, "value1":0.603964532592086, "value2":1716897544081104224, "value3":false, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264870, "value1":0.9573825263948318, "value2":3412285010274210994, "value3":true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.556842, "value1":0.17542611082882992, "value2":1043370942216175800, "value3":false, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009549, "value1":0.21407922779664104, "value2":5715745607073532896, "value3":false, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462287, "value1":0.8132271496462169, "value2":7467232826614980432, "value3":true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538632, "value1":0.7627133328751289, "value2":2219523491939336599, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.217749, "value1":0.9344741716490049, "value2":8166313022724995390, "value3":true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811552, "value1":0.9190437296914767, "value2":5885018846694177935, "value3":true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814548, "value1":0.8390440310277583, "value2":5493133492272371657, "value3":false, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.817102, "value1":0.6598894636593393, "value2":8537596977440503466, "value3":false, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.067200, "value1":0.263575315223174, "value2":2125629267937428501, "value3":false, "__name__":"metric_00000619", "key_1":"i","key_5":"h","key_0":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082507, "value1":0.6268250434274911, "value2":9058686535109969227, "value3":false, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734195, "value1":0.3167788157158676, "value2":7432987222538757392, "value3":true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.795085, "value1":0.4488172699242242, "value2":4473525864538142538, "value3":false, "__name__":"metric_00000622", "key_6":"b","key_8":"i","key_4":"d", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.332775, "value1":0.8505535842615966, "value2":6047205731470921861, "value3":true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727338, "value1":0.8876718108712484, "value2":1531222095185940701, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.387878, "value1":0.5788599279069843, "value2":8604920509885625807, "value3":false, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.233954, "value1":0.6310172938155442, "value2":8925576745188878375, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.805738, "value1":0.9330396354121838, "value2":1424947937953535481, "value3":true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726267, "value1":0.32286499179099665, "value2":3766446177817631940, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229570, "value1":0.9916323313225612, "value2":3888482529661188248, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167559, "value1":0.08178718223938254, "value2":7520256881259787943, "value3":false, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845474, "value1":0.6808433459458884, "value2":7844247683138918907, "value3":false, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400844, "value1":0.39452808620697233, "value2":7357453563119638881, "value3":false, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206854, "value1":0.9379245513406866, "value2":6598367899784187154, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640341, "value1":0.11560748654575319, "value2":414342029075207599, "value3":false, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497616, "value1":0.26256799403609016, "value2":7366324474884701561, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.822083, "value1":0.6049405008185211, "value2":6275944038502753172, "value3":true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.819954, "value1":0.373947872013572, "value2":3355608188213748317, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739894, "value1":0.06143600864431769, "value2":309096149476303416, "value3":true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.684612, "value1":0.7692595721499597, "value2":2629612954063953162, "value3":true, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958165, "value1":0.6185066606297357, "value2":2824340682692220488, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944290, "value1":0.3864309642344103, "value2":6500366333535522458, "value3":false, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576232, "value1":0.6603219832087598, "value2":7174737056650755203, "value3":true, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.047735, "value1":0.4501836136879949, "value2":7043766590114442160, "value3":false, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282665, "value1":0.3628575018664628, "value2":3673366331802014852, "value3":false, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.590235, "value1":0.30332064856581503, "value2":5473820163155790307, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995108, "value1":0.1854914610326227, "value2":3604467487566582522, "value3":true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270312, "value1":0.9632251653093951, "value2":4816282908135882739, "value3":false, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541752, "value1":0.2581250654809775, "value2":6451610961696076892, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535737, "value1":0.3163976646275719, "value2":2322386485534186745, "value3":false, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610988, "value1":0.49088683208094147, "value2":4479251775302671, "value3":true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.111345, "value1":0.016089752487428322, "value2":4062002565116218346, "value3":true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.380428, "value1":0.8878960076695425, "value2":7482838775142667379, "value3":true, "__name__":"metric_00000652", "key_5":"a","key_9":"i","key_4":"f", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.286191, "value1":0.5281244570210905, "value2":4742826162899756346, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.633864, "value1":0.28655049708989444, "value2":2281246548041472554, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517718, "value1":0.8760337085208192, "value2":6876760617398624261, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275264, "value1":0.8763572333888452, "value2":2040006649964380357, "value3":false, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.133688, "value1":0.03683553631174895, "value2":879315984438783745, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.178775, "value1":0.7921271023115032, "value2":8974452178625132408, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.559621, "value1":0.6842414958291543, "value2":4473030242509416772, "value3":false, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.933147, "value1":0.42656042046254644, "value2":6875784776532081765, "value3":true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.265632, "value1":0.6577298952553573, "value2":4117148991099157160, "value3":false, "__name__":"metric_00000661", "key_6":"e","key_1":"d","key_5":"k", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.291705, "value1":0.35063088013808896, "value2":6022154315116551060, "value3":false, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.486399, "value1":0.9066996869066141, "value2":3890435706555388518, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539080, "value1":0.6597581253474917, "value2":7149394823928068405, "value3":false, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.172124, "value1":0.8972702809236033, "value2":5889011931565346539, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399820, "value1":0.22420487156545466, "value2":3632173066141869984, "value3":false, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857491, "value1":0.658023411947534, "value2":5044439445561970794, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.960983, "value1":0.8072960706443908, "value2":4799697070254932404, "value3":true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672380, "value1":0.6734321372300892, "value2":7080832382541851744, "value3":false, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.699212, "value1":0.7095725169760027, "value2":1829131464356530776, "value3":false, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893830, "value1":0.2302554104612147, "value2":9188062021573390442, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520337, "value1":0.16075557090608333, "value2":219133269848494495, "value3":false, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173477, "value1":0.6892053752715795, "value2":2151318569219964336, "value3":true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229017, "value1":0.9691975492787838, "value2":1366609608627051856, "value3":true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659208, "value1":0.7460794334063994, "value2":817106321355418532, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.361925, "value1":0.05687743883366658, "value2":7921175806067397073, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.523858, "value1":0.7866547024502999, "value2":4071070330904780510, "value3":true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920001, "value1":0.5740949320374887, "value2":487306006337280717, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775196, "value1":0.1341448972829432, "value2":7272783478294294123, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000330, "value1":0.1260519175980482, "value2":9041625229579242321, "value3":false, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313318, "value1":0.4146999970169919, "value2":6937035767989914959, "value3":true, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147057, "value1":0.5790734522844233, "value2":1184517563846638764, "value3":true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.331707, "value1":0.5826950424479324, "value2":7357100402735593295, "value3":true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.476945, "value1":0.09337572406541937, "value2":3528304154285323988, "value3":true, "__name__":"metric_00000684", "key_4":"i","key_5":"d","key_2":"e", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.660357, "value1":0.9211165869524813, "value2":6490630229378478544, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.978786, "value1":0.24445422171222897, "value2":6316462580906113749, "value3":true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031375, "value1":0.9435579557750952, "value2":3775564649397357096, "value3":true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786489, "value1":0.19831604296196023, "value2":4519338814156406008, "value3":false, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845707, "value1":0.0733198408835947, "value2":6043258187561628613, "value3":false, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607379, "value1":0.49574791837525684, "value2":2511607289885086712, "value3":true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541151, "value1":0.9239913172285873, "value2":8906754784907447893, "value3":true, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409187, "value1":0.347121325872609, "value2":4022279358676190140, "value3":false, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179171, "value1":0.9901703571238136, "value2":1971122679978818549, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807000, "value1":0.9794430062314226, "value2":4140173177331652200, "value3":true, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514307, "value1":0.3368836183017325, "value2":6646214711319824003, "value3":true, "__name__":"metric_00000695", "key_8":"d","key_9":"g","key_6":"a", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.430324, "value1":0.6783643243115747, "value2":8700387930430110357, "value3":false, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245716, "value1":0.5777847683942137, "value2":1772788621154859179, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825752, "value1":0.8229486912139793, "value2":3218217186376093534, "value3":true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.550373, "value1":0.44484600976320454, "value2":1214217372893594474, "value3":true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810847, "value1":0.6410630380778536, "value2":1107368725861373561, "value3":true, "__name__":"metric_00000700", "key_7":"e","key_3":"g","key_4":"b", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258640, "value1":0.3699636915738955, "value2":3778046401086414746, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354190, "value1":0.2787621713653213, "value2":7905127146501563870, "value3":true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276300, "value1":0.38144875313061755, "value2":279189270439050245, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030267, "value1":0.09524338073167776, "value2":624319768896255519, "value3":false, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395031, "value1":0.017165936825319472, "value2":1839096275149454289, "value3":false, "__name__":"metric_00000659", "key_3":"c","key_8":"i","key_1":"k", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920079, "value1":0.3695378285006256, "value2":3642800979833317953, "value3":false, "__name__":"metric_00000681", "key_3":"f","key_9":"f","key_1":"e", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055296, "value1":0.5098042037064671, "value2":1508579590160432139, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128994, "value1":0.8793028393199556, "value2":5718919044160675205, "value3":false, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418223, "value1":0.7389480744126634, "value2":5430202156875304420, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458004, "value1":0.7589090833562657, "value2":7035246718540970070, "value3":false, "__name__":"metric_00000682", "key_6":"k","key_0":"f","key_3":"b", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492634, "value1":0.3450062925676948, "value2":289715553515433637, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965681, "value1":0.9053649041017894, "value2":2294309054504906499, "value3":true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234931, "value1":0.0732565141042219, "value2":2755939599945126397, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939989, "value1":0.8451432422468432, "value2":2656355912924482468, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862255, "value1":0.3071521473170743, "value2":4029052487902069438, "value3":true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027859, "value1":0.5418227496728708, "value2":5130775345772381962, "value3":false, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016707, "value1":0.5241900456864456, "value2":1703509093458181611, "value3":true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.129222, "value1":0.45379187899537626, "value2":8646708304907761402, "value3":false, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167400, "value1":0.4118569437426427, "value2":1933032658312744458, "value3":false, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.448154, "value1":0.9880224154363099, "value2":1958598830665944865, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095756, "value1":0.7737922723291526, "value2":1554270443791925823, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.669733, "value1":0.05998044134164805, "value2":4233304263515802857, "value3":false, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360452, "value1":0.021777318888489317, "value2":1462350357926539483, "value3":true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.791525, "value1":0.24698569461323847, "value2":6782513953965205791, "value3":false, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.136814, "value1":0.13312227926717296, "value2":2973379075607635624, "value3":false, "__name__":"metric_00000715", "key_5":"a","key_2":"i", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.611322, "value1":0.06800531937511199, "value2":2253428998305264546, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000702, "value1":0.40505920041369187, "value2":7444308564790803777, "value3":false, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.317483, "value1":0.5921456568045917, "value2":1159233331687655633, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.207714, "value1":0.03623125609776327, "value2":4050612595565072329, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739699, "value1":0.1967627573726683, "value2":5128621003255593133, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938178, "value1":0.8074789236886436, "value2":5329361240090086591, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075581, "value1":0.36183126859676484, "value2":8380824340175200195, "value3":false, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.840434, "value1":0.36564465907617033, "value2":4138989352760782569, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697538, "value1":0.8089117668002023, "value2":2706669265618979267, "value3":true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810618, "value1":0.9822561514547918, "value2":6363847253975708775, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453492, "value1":0.7521891954121415, "value2":3415954419502830788, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.246679, "value1":0.291250645374521, "value2":6787758008675320095, "value3":false, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.717578, "value1":0.21885147116003636, "value2":8249910093492361560, "value3":false, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.584635, "value1":0.267173208281922, "value2":6350566579717368150, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288596, "value1":0.8034133849819789, "value2":3057165042866020348, "value3":false, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043225, "value1":0.8891915956366093, "value2":1172085376591869222, "value3":true, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514557, "value1":0.7993701790896678, "value2":3530709071532758425, "value3":false, "__name__":"metric_00000734", "key_8":"h","key_3":"k","key_4":"d", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727549, "value1":0.7581402483767911, "value2":2882149021889454970, "value3":true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944767, "value1":0.03125902081908963, "value2":5930819025045052350, "value3":false, "__name__":"metric_00000735", "key_9":"c","key_6":"c","key_7":"k", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635131, "value1":0.5563692033064526, "value2":3139117266540547008, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901432, "value1":0.08725832596843655, "value2":2516400281682537507, "value3":false, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965447, "value1":0.24555624361218226, "value2":8420031154067673423, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653005, "value1":0.6015476287649423, "value2":7179718486091286242, "value3":true, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.251270, "value1":0.6303898680513278, "value2":1976335650844570524, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.902243, "value1":0.42829690884430943, "value2":7763758072113593479, "value3":true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748351, "value1":0.5394339139974091, "value2":1947184765867041268, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.496788, "value1":0.0776914163879659, "value2":4120402058111565140, "value3":false, "__name__":"metric_00000742", "key_6":"j","key_4":"a","key_5":"g", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202592, "value1":0.3950986449717796, "value2":7867550414487529994, "value3":false, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.475860, "value1":0.5847506627916105, "value2":1395577688027634835, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307279, "value1":0.66953692642835, "value2":1884810173101069714, "value3":false, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447867, "value1":0.30049434187635843, "value2":2021494635928386331, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371456, "value1":0.9583003775042523, "value2":8408660678843228458, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321951, "value1":0.673208957772375, "value2":4484934316922285119, "value3":false, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.654317, "value1":0.34301031853223307, "value2":1377167639028225502, "value3":false, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258833, "value1":0.5754272342729656, "value2":3672436572873719874, "value3":false, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894800, "value1":0.27153721678184795, "value2":6771295009871301863, "value3":false, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886915, "value1":0.4211098948169063, "value2":327314688390276361, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464071, "value1":0.8401442859431041, "value2":7237906700870505791, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179548, "value1":0.35527262545135796, "value2":6929428209196710992, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348853, "value1":0.7390477554491065, "value2":6017303276519989534, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228799, "value1":0.7190691653435217, "value2":3466847624797872404, "value3":false, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.676189, "value1":0.8701176990093307, "value2":8139200890464740272, "value3":true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780301, "value1":0.31331610868424237, "value2":7970934533227613450, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.763684, "value1":0.6117316228567153, "value2":1342308618516991103, "value3":true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.510130, "value1":0.26253831176449904, "value2":223167711418991037, "value3":false, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.054906, "value1":0.30113096258283606, "value2":9029709188884415357, "value3":true, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.101830, "value1":0.6535137630775729, "value2":8047896116508601877, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.816267, "value1":0.6779088610569585, "value2":402736258268648092, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794465, "value1":0.6385825716360082, "value2":647146760807700438, "value3":true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.342961, "value1":0.08518129482765982, "value2":3079348210933738649, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503723, "value1":0.5000975675164184, "value2":3087069466664576618, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.552740, "value1":0.46623198603217897, "value2":5991311418948024851, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.634526, "value1":0.1623989693910568, "value2":880550310997257962, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409701, "value1":0.18776946541701378, "value2":4672037814003639717, "value3":false, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928628, "value1":0.5544683022868725, "value2":8786609556345104074, "value3":false, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422305, "value1":0.3339761609696324, "value2":2037434470333278554, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497260, "value1":0.21504973222613463, "value2":8744267736203582723, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348792, "value1":0.7267165846871734, "value2":2029452385366977955, "value3":false, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767788, "value1":0.7268443539701483, "value2":5120204611450631315, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313022, "value1":0.3937734460488266, "value2":8789465407671191549, "value3":true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373011, "value1":0.5815734596824621, "value2":7433149153695507174, "value3":false, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.403959, "value1":0.19547385196106162, "value2":4536904833017601954, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702004, "value1":0.787480554646893, "value2":6826871522585433391, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238346, "value1":0.3158260333418349, "value2":4890876749664266809, "value3":false, "__name__":"metric_00000781", "key_7":"h","key_9":"d","key_2":"g", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.184806, "value1":0.1658734659451667, "value2":2103368389172466197, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646125, "value1":0.7611256397426871, "value2":3779878057324979416, "value3":false, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.963683, "value1":0.6784369755611176, "value2":7198319968843792884, "value3":true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264784, "value1":0.6944677799630911, "value2":1361718740081763677, "value3":true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.606235, "value1":0.9767553574110414, "value2":8172694375569168120, "value3":true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.555524, "value1":0.0757100455425729, "value2":6369412142150141094, "value3":true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.683577, "value1":0.23488908846392256, "value2":7496440285542052380, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741203, "value1":0.6152800537206294, "value2":4336041330603516829, "value3":false, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.076957, "value1":0.46349128481126883, "value2":1995149205719248029, "value3":true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360568, "value1":0.18125853395781905, "value2":5023669000017633099, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.127735, "value1":0.2064987900720652, "value2":1090691939971047635, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182440, "value1":0.18035240548503653, "value2":2377567143003176008, "value3":false, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173358, "value1":0.9386364543056086, "value2":6498536821606623122, "value3":true, "__name__":"metric_00000785", "key_9":"k","key_5":"j","key_6":"d", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.749875, "value1":0.9851379736422945, "value2":9068316297781013801, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262923, "value1":0.6215747239294878, "value2":3444654966165617334, "value3":true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.471762, "value1":0.9687744980352465, "value2":4008980207107557770, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.449800, "value1":0.43369424684152247, "value2":1796590317391088733, "value3":false, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799032, "value1":0.7408688395558841, "value2":1701592597588021265, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727561, "value1":0.14048934174685151, "value2":1964669731520342830, "value3":false, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393461, "value1":0.22927403588573142, "value2":6637169452408438633, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577492, "value1":0.9899876418682123, "value2":417258805492276216, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.149873, "value1":0.2451895383714133, "value2":6783012468215477739, "value3":false, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794818, "value1":0.2978983436952165, "value2":6898137216027059682, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826276, "value1":0.03031011408512658, "value2":6243267082953045335, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853432, "value1":0.8319004837974954, "value2":6916103698866916518, "value3":true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336756, "value1":0.6573141098461979, "value2":7552682537825884423, "value3":true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280458, "value1":0.9944062499165491, "value2":4515137518875248587, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723258, "value1":0.8389015105022624, "value2":4385100343250672688, "value3":false, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.518510, "value1":0.262846594625773, "value2":8956848533096958437, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398441, "value1":0.29945983222092276, "value2":7583507905789284927, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169305, "value1":0.3509582334049355, "value2":4486670807718872528, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229006, "value1":0.9454115215233667, "value2":4148482506114005261, "value3":true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.108860, "value1":0.7792992725899739, "value2":6662551108461731004, "value3":false, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270948, "value1":0.22595556811111142, "value2":4858846807835096111, "value3":false, "__name__":"metric_00000808", "key_3":"e","key_6":"b","key_1":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739267, "value1":0.10553288145788361, "value2":3903915559852076766, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.771104, "value1":0.2713630569195371, "value2":9127087478037486061, "value3":true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234974, "value1":0.4512849051156243, "value2":388475201207159224, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021067, "value1":0.8525575906494762, "value2":6112677337472259317, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360331, "value1":0.5466498354493777, "value2":8416483456048971785, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.083431, "value1":0.6268377803161493, "value2":6874204752716819531, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543079, "value1":0.356809340191254, "value2":1479162719217476511, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543913, "value1":0.1823988812868871, "value2":1772497490870724163, "value3":true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697354, "value1":0.6008728382547858, "value2":4808659248566150951, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173629, "value1":0.11694918608631227, "value2":8374732016838634205, "value3":false, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.478338, "value1":0.6523910738528493, "value2":6127719310482278347, "value3":false, "__name__":"metric_00000825", "key_8":"d","key_5":"k","key_7":"j", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195226, "value1":0.11728848022547718, "value2":2692984308355874383, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472792, "value1":0.3849335854410186, "value2":5551198795362400796, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.088581, "value1":0.6479908764028293, "value2":6646669667589759935, "value3":false, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.051281, "value1":0.8063603427465644, "value2":4677280644621004022, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.421107, "value1":0.8098711885881693, "value2":781451797112683390, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702263, "value1":0.7459154166325007, "value2":8763722224980194522, "value3":false, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.105837, "value1":0.6082412929946549, "value2":9076065007808782827, "value3":false, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229179, "value1":0.5208679934438682, "value2":1614625428456495501, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452972, "value1":0.6602375068241643, "value2":3665988082745333789, "value3":true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570733, "value1":0.8539881179697112, "value2":8291534021329215591, "value3":false, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.854105, "value1":0.19867014748060782, "value2":4707537791933933479, "value3":false, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134473, "value1":0.013248508886480496, "value2":8975225813346771480, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.401924, "value1":0.8131686031327187, "value2":1570626633965770939, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462042, "value1":0.33875887576127867, "value2":1531594524235268547, "value3":true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780840, "value1":0.6167438470784343, "value2":3111876176210806342, "value3":false, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157314, "value1":0.024850360062745134, "value2":2155248434369533975, "value3":false, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.892341, "value1":0.29151476400849524, "value2":3929095109066478763, "value3":true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811920, "value1":0.3960145984061985, "value2":8079803277636061346, "value3":true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010049, "value1":0.1522516905011452, "value2":3887799589835913190, "value3":false, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.511030, "value1":0.1096312618999367, "value2":4683382419730786111, "value3":false, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799775, "value1":0.8107638537946521, "value2":1548400788132615580, "value3":true, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.034128, "value1":0.8145920212010633, "value2":4095930474475755353, "value3":false, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506330, "value1":0.8145793344024634, "value2":7248985564172971873, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.632617, "value1":0.530965707458104, "value2":6582545459218253298, "value3":false, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.427368, "value1":0.5488520970953542, "value2":7764537337432663671, "value3":true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128543, "value1":0.339288607631015, "value2":8135936826126620967, "value3":false, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.783911, "value1":0.4072868133390787, "value2":398383887746388268, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030437, "value1":0.40039567718797253, "value2":5009000449434898292, "value3":true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.301665, "value1":0.2851345919898927, "value2":196381052074952545, "value3":false, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740740, "value1":0.481454729816193, "value2":2929970498837276050, "value3":true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867428, "value1":0.8246737181844912, "value2":4477437696490734479, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.708771, "value1":0.44011710908297325, "value2":7995836793967148678, "value3":false, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663309, "value1":0.07002506566825563, "value2":3582676294388926669, "value3":false, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.628344, "value1":0.9801112824460718, "value2":196546046309844748, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330627, "value1":0.6390220685465513, "value2":5252094592319160612, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.014786, "value1":0.4666494400039896, "value2":6746944521794840076, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.736397, "value1":0.1650175517955452, "value2":4725786271252183916, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971200, "value1":0.08835643831453004, "value2":1662264677868065737, "value3":false, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.804675, "value1":0.3583297226255054, "value2":6764608743691309744, "value3":true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577888, "value1":0.7784009799351271, "value2":7389846578443042716, "value3":false, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942762, "value1":0.5281041922180323, "value2":225078423814016880, "value3":true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807869, "value1":0.12159890893033082, "value2":2007944198207870435, "value3":false, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393243, "value1":0.9174108973827193, "value2":5956246567273801920, "value3":true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842305, "value1":0.6881483813172635, "value2":51917468498479892, "value3":false, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.625958, "value1":0.7332068726661788, "value2":1595636028517738482, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.738053, "value1":0.19091798634208768, "value2":2237197021871432008, "value3":true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268129, "value1":0.8406719661444901, "value2":2842521222444292729, "value3":false, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865313, "value1":0.8513846203817129, "value2":3859037819752068385, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762176, "value1":0.0431523780543075, "value2":6794482877795635511, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.658034, "value1":0.12118411547914006, "value2":7727119685187980691, "value3":false, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497498, "value1":0.8736017830767735, "value2":5281383221003217421, "value3":false, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016157, "value1":0.45635245796699264, "value2":6588970499031475943, "value3":false, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434398, "value1":0.5982494402461963, "value2":3451343743898655419, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995659, "value1":0.41609794124038807, "value2":2577003956591436853, "value3":true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.402057, "value1":0.3168156284522602, "value2":7664962841417781702, "value3":false, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874274, "value1":0.8588869341047012, "value2":6471105672211793135, "value3":true, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.193561, "value1":0.7716291581375219, "value2":6758054142720243683, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.298108, "value1":0.8739782093698222, "value2":7600130946423390938, "value3":true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008667, "value1":0.15272944664848967, "value2":28862629936648465, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924564, "value1":0.2262144335029645, "value2":2100662153491567952, "value3":true, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.250052, "value1":0.2945787294595236, "value2":6018457540699249059, "value3":false, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395074, "value1":0.2290685251999151, "value2":4865312816831141048, "value3":false, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414183, "value1":0.2892057090700859, "value2":3284381379197405662, "value3":true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800711, "value1":0.20023612385737624, "value2":5612718418445245977, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096882, "value1":0.6190652975917784, "value2":4456707593278158335, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811596, "value1":0.1974893316007636, "value2":8350883557246751407, "value3":true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.328602, "value1":0.3577823211406552, "value2":3856849717607100839, "value3":true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165294, "value1":0.1558044782512875, "value2":2819862014599071323, "value3":true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.934329, "value1":0.6967191883920082, "value2":8317801698439570519, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826310, "value1":0.5889684966352713, "value2":8904148697557725595, "value3":false, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.544062, "value1":0.3036793591749508, "value2":1172054639314476559, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104925, "value1":0.0006382055160369547, "value2":4527189243213341516, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577496, "value1":0.8262646875704296, "value2":6857852785226312453, "value3":false, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.304163, "value1":0.23448830223910028, "value2":118297432460308257, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.201626, "value1":0.8649386204518231, "value2":2335277383532252247, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.072447, "value1":0.305504173455374, "value2":8879950191665801641, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635410, "value1":0.8646407642266409, "value2":4794888988052345760, "value3":true, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799765, "value1":0.020261599778331852, "value2":6569370350090311777, "value3":true, "__name__":"metric_00000891", "key_7":"i","key_2":"i","key_3":"d", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.643397, "value1":0.25580026537842243, "value2":8727368389050428851, "value3":true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886737, "value1":0.709498290535563, "value2":159638875141661569, "value3":false, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027896, "value1":0.4252750468577108, "value2":5985063599490557923, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659322, "value1":0.8283450566415441, "value2":3829413923759281391, "value3":true, "__name__":"metric_00000893", "key_2":"h","key_1":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.323182, "value1":0.05823156526332208, "value2":158181159624437675, "value3":true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721491, "value1":0.058149447228806335, "value2":4425192040964093188, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.715378, "value1":0.1468510162162874, "value2":957193015456701660, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759517, "value1":0.11145345073332971, "value2":9085390279667186751, "value3":false, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179380, "value1":0.22290378424729548, "value2":4452524214324184228, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018314, "value1":0.08677941054726214, "value2":6248669636718030352, "value3":false, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579864, "value1":0.1843770522783152, "value2":2395237031433429826, "value3":true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907863, "value1":0.19391713305151562, "value2":1831578948753070965, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567206, "value1":0.1200213703180807, "value2":281170079691225837, "value3":true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.908312, "value1":0.16873762365247053, "value2":3349277004368724008, "value3":true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.203346, "value1":0.3461718122656804, "value2":2053673338515188112, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.252421, "value1":0.8594831603279349, "value2":8129734681001143206, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862108, "value1":0.8625875861161819, "value2":266750263503917399, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394606, "value1":0.1284747365418792, "value2":511162186129513778, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104334, "value1":0.348212621993861, "value2":8809834773329001166, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.390817, "value1":0.056538788821091236, "value2":959306944656093342, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371946, "value1":0.0354556141213121, "value2":5315204481196619882, "value3":false, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027057, "value1":0.04051346889125732, "value2":19281603192892033, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315646, "value1":0.29470865205235175, "value2":3350853301006328333, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516736, "value1":0.32421705083072555, "value2":6409364233498955235, "value3":true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409843, "value1":0.8431693191414327, "value2":2309002455167863238, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.290066, "value1":0.769846734088458, "value2":1354615135861452763, "value3":false, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025458, "value1":0.04383062091902626, "value2":6270990659808009038, "value3":false, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422265, "value1":0.4215091845870574, "value2":903506811871753839, "value3":true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954147, "value1":0.1703656776121607, "value2":8625109899283360680, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468022, "value1":0.8656906467989927, "value2":8844353092623218329, "value3":false, "__name__":"metric_00000933", "key_6":"d","key_2":"g","key_4":"j", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.572694, "value1":0.03657142931464268, "value2":7269928716214871698, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.035238, "value1":0.2174690883774512, "value2":2760303323859671187, "value3":true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.500319, "value1":0.8848932111671852, "value2":124147493136831346, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.324572, "value1":0.32024379123233365, "value2":8082835280902735033, "value3":true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585321, "value1":0.18246288567364422, "value2":7470239488455160357, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726932, "value1":0.8209711897357084, "value2":4430579260139045397, "value3":false, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.851627, "value1":0.9988163498901884, "value2":7782482596272329873, "value3":true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126247, "value1":0.7338638289875997, "value2":2035846164975007267, "value3":true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923987, "value1":0.31023714156918913, "value2":7969614988653223555, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.797525, "value1":0.9308157523651853, "value2":2008465742081738773, "value3":true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595532, "value1":0.6306393412084162, "value2":1794355738915834222, "value3":false, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078930, "value1":0.46045211861703167, "value2":2231560481693103072, "value3":false, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.835058, "value1":0.49083826368226746, "value2":947523171682771279, "value3":false, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.757956, "value1":0.5783027668846169, "value2":7939213319454038876, "value3":false, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.431201, "value1":0.249271140930431, "value2":7943702047022612152, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.674800, "value1":0.9050148884978706, "value2":4701267498128748767, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294430, "value1":0.37838796407920183, "value2":6812226255195471591, "value3":false, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837164, "value1":0.41192774227492235, "value2":1535119829477772546, "value3":false, "__name__":"metric_00000946", "key_2":"d","key_1":"j", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464005, "value1":0.1600487895075015, "value2":1434123145157650204, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546806, "value1":0.6061159682210883, "value2":6208205100336254753, "value3":false, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096572, "value1":0.8332138223552116, "value2":7164548384493446541, "value3":true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.325386, "value1":0.6865864080335002, "value2":1198200374614070582, "value3":false, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179713, "value1":0.590117470581001, "value2":8089464430800358371, "value3":true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273193, "value1":0.6452720470474251, "value2":1981356061798226474, "value3":false, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330589, "value1":0.2830579363626061, "value2":6390191335007008364, "value3":true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468768, "value1":0.2265890199157533, "value2":4363523400851143524, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901505, "value1":0.3297489212069802, "value2":7168425838004946986, "value3":false, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850173, "value1":0.7443546478795617, "value2":1215583272150299781, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.968362, "value1":0.6932020029271382, "value2":4461729221210949302, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210225, "value1":0.36249305348248906, "value2":7617198529682822125, "value3":false, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.001816, "value1":0.46528315796295283, "value2":553457115403327004, "value3":false, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.564478, "value1":0.42774941008998635, "value2":6190959325547605268, "value3":false, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126782, "value1":0.6691920311814687, "value2":8190037233750158351, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.125382, "value1":0.3161902537409363, "value2":2607443448109856155, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.782262, "value1":0.25500427404714787, "value2":694236189793698587, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635388, "value1":0.4760006261995292, "value2":7236218797911995877, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.481920, "value1":0.5382100231194429, "value2":6783652184521772255, "value3":true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818398, "value1":0.6906166285200835, "value2":1593698347526099227, "value3":false, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283475, "value1":0.9162718353521024, "value2":131978695455153917, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434017, "value1":0.8865622180830747, "value2":5499929802234516147, "value3":true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838878, "value1":0.8831606268003614, "value2":4413611476125545911, "value3":true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451279, "value1":0.9923758573970507, "value2":7842864722613769419, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362386, "value1":0.09704282298545679, "value2":6868263842667298833, "value3":true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274837, "value1":0.5887807861238907, "value2":1280869300914331508, "value3":false, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.915666, "value1":0.6149407272649627, "value2":2848990718411372775, "value3":true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024227, "value1":0.1623145961115003, "value2":2455593750882693548, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.695944, "value1":0.6887899812661492, "value2":533109273214059474, "value3":false, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.528843, "value1":0.5075436862092302, "value2":515274582595904032, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842262, "value1":0.5447726756017138, "value2":411822948007690047, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444248, "value1":0.5454758220348956, "value2":1028316813231771868, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.604063, "value1":0.46187992572513836, "value2":8158929971855121147, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275994, "value1":0.8154001283675578, "value2":5366649133302343998, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348933, "value1":0.903584155771458, "value2":3421400845314883405, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.041062, "value1":0.1190413277495334, "value2":3602483838893891944, "value3":true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928372, "value1":0.19081223162638847, "value2":4430971211717726323, "value3":false, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.677971, "value1":0.7830672620168732, "value2":4412058601784711710, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152188, "value1":0.6024595453233776, "value2":3457200300695401373, "value3":true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838789, "value1":0.8421167420494099, "value2":690533435719409387, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.578903, "value1":0.6328751611123744, "value2":3916711663864490625, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.896839, "value1":0.9070308716530836, "value2":5545334900650073897, "value3":true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.712901, "value1":0.5273783009751816, "value2":7859089933688216880, "value3":true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884939, "value1":0.07741816375541602, "value2":2787557417309799857, "value3":true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944440, "value1":0.9501843438293957, "value2":4306730282944265532, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814029, "value1":0.12358405285849303, "value2":6299011328560473443, "value3":true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965702, "value1":0.9397419047313924, "value2":2182258508456140601, "value3":false, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045751, "value1":0.6945070980136847, "value2":6892165864647238687, "value3":false, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821781, "value1":0.27908605223075134, "value2":5742625392746876485, "value3":false, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514196, "value1":0.8720271384105083, "value2":6651696455700077053, "value3":false, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.705476, "value1":0.06250737971386458, "value2":1193722866498852150, "value3":false, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.603276, "value1":0.8453600274943766, "value2":31981742286968982, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.682527, "value1":0.2427465956249123, "value2":7462455737298824829, "value3":false, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.605338, "value1":0.7488120008677794, "value2":4273030954599275940, "value3":true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893961, "value1":0.0012093187527627421, "value2":3798553194034218364, "value3":false, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.593870, "value1":0.4389808952680249, "value2":4760750743940894178, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091153, "value1":0.8390848719802647, "value2":4663558817694651969, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.014239, "value1":0.6803598889667898, "value2":4768916054002910461, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478582, "value1":0.8497920155872134, "value2":6014547020899484560, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895949, "value1":0.2900471481778915, "value2":9139898160138156870, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.613215, "value1":0.18238778666189503, "value2":1572248498405458162, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.817510, "value1":0.7988916675957994, "value2":5551320908811649706, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881135, "value1":0.18477673938398376, "value2":5754586889701314224, "value3":false, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128674, "value1":0.2730542267310359, "value2":3723967523716120986, "value3":false, "__name__":"metric_00000015", "key_8":"k","key_0":"a","key_6":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392852, "value1":0.5342690346158169, "value2":2053063244515016190, "value3":false, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931934, "value1":0.712284281615868, "value2":5743978094943818789, "value3":false, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537147, "value1":0.023259389444222435, "value2":5526408740424742448, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.216805, "value1":0.8594083029021016, "value2":5158284428621389401, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837342, "value1":0.5180882037372726, "value2":7808493880824756093, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144558, "value1":0.054388678344141024, "value2":7174534666209181912, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599909, "value1":0.09725816948703289, "value2":4397869661367210515, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.880028, "value1":0.4672046328070093, "value2":7512677570982024303, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271509, "value1":0.8816366985657997, "value2":7431242132698500082, "value3":false, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.752197, "value1":0.6078880045612306, "value2":3606538184952837708, "value3":false, "__name__":"metric_00000025", "key_5":"b","key_0":"a","key_4":"j", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392689, "value1":0.5702352441472875, "value2":327376335260250215, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.201991, "value1":0.3880870455267555, "value2":2539019737033922024, "value3":true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.894353, "value1":0.3785370438906491, "value2":8100099805481858212, "value3":false, "__name__":"metric_00000029", "key_4":"e","key_7":"c","key_1":"f", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.193824, "value1":0.1316387900370704, "value2":2566382490326741291, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.735887, "value1":0.9017592259731758, "value2":3204066872434851855, "value3":true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891488, "value1":0.8557864633508439, "value2":7321454426994500720, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868424, "value1":0.21022389029059355, "value2":4239400393420255730, "value3":false, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.624822, "value1":0.8941824508080144, "value2":4130019486643335780, "value3":false, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.292285, "value1":0.2379306348830461, "value2":19245718469181538, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690737, "value1":0.07804300554340773, "value2":3416010458091576918, "value3":true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404288, "value1":0.7382908404371364, "value2":4673144760418942783, "value3":true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.333079, "value1":0.7119882858599182, "value2":2804563282517597579, "value3":false, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.472391, "value1":0.08143361630646322, "value2":8016722515581298003, "value3":true, "__name__":"metric_00000037", "key_3":"i","key_7":"a","key_0":"c", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.373563, "value1":0.6580401651760573, "value2":6974099035575699967, "value3":true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078367, "value1":0.04245208361261986, "value2":3309677777480755698, "value3":true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.312294, "value1":0.08781290868459149, "value2":410348162808240186, "value3":false, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768785, "value1":0.8804562203614901, "value2":3554533373089833157, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636131, "value1":0.6764401768679541, "value2":5380729408367504508, "value3":false, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041277, "value1":0.013798250982638604, "value2":7112024095753417757, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867267, "value1":0.8747601920631648, "value2":1340507463961878602, "value3":true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776242, "value1":0.08520211967435457, "value2":4488809966272125582, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634645, "value1":0.8571991096706362, "value2":4813432822336330164, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415856, "value1":0.16181998754042196, "value2":2843492259629836672, "value3":true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466761, "value1":0.6961378523412571, "value2":7745491307176492894, "value3":false, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.594554, "value1":0.04099735827317489, "value2":8913619907287518073, "value3":false, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.170785, "value1":0.5648256194335608, "value2":476929125931913132, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.184821, "value1":0.7825477180517281, "value2":4452433023228691120, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906243, "value1":0.48861587452050464, "value2":8430953017453652958, "value3":true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.667172, "value1":0.06605403171722925, "value2":3065000063305702983, "value3":false, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.192260, "value1":0.12827053824062132, "value2":4944252317941317234, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400652, "value1":0.728695995485893, "value2":1367272622265070542, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535047, "value1":0.4567196818143048, "value2":5331291578601537074, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466316, "value1":0.5659706744457615, "value2":5221382170536673746, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487405, "value1":0.8907853085096481, "value2":2583510249445277455, "value3":true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.375762, "value1":0.8536987085513627, "value2":1666025213570716259, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.165432, "value1":0.43405723604720065, "value2":714036789144069944, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175401, "value1":0.9859093721286644, "value2":4617822411248224956, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.059065, "value1":0.378208510942557, "value2":8575694693763974484, "value3":true, "__name__":"metric_00000059", "key_7":"j","key_0":"d","key_6":"c", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.073374, "value1":0.7874709956095631, "value2":3297271830714214500, "value3":true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656159, "value1":0.11136055751977454, "value2":6742238162891439594, "value3":true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102008, "value1":0.8937221842663475, "value2":6378107019082012182, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.177632, "value1":0.36894176860576355, "value2":7704992361178834991, "value3":true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335734, "value1":0.6980181667187008, "value2":260944621673155935, "value3":false, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.855348, "value1":0.9199066328884838, "value2":4741986190169422412, "value3":false, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965490, "value1":0.12178040424602839, "value2":8326386976538257273, "value3":false, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090240, "value1":0.23602165280654655, "value2":9043605027949953616, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.028464, "value1":0.05960386743156977, "value2":3549887217756566974, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.956988, "value1":0.5498920550278433, "value2":6597949147890816377, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.171316, "value1":0.0904948299110727, "value2":3471748119473561940, "value3":true, "__name__":"metric_00000083", "key_6":"b","key_9":"a","key_1":"g", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.523091, "value1":0.4454856417468202, "value2":3973537432672533989, "value3":false, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453397, "value1":0.7881010562880141, "value2":448816938066732131, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384576, "value1":0.9045802469513693, "value2":4349360112470402658, "value3":true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.929660, "value1":0.1904824371420524, "value2":4775599930437109515, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642052, "value1":0.9792443542954197, "value2":6092407690974464405, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194609, "value1":0.8726282980309354, "value2":7591391705732812177, "value3":true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095352, "value1":0.384168214137503, "value2":7516656275093877057, "value3":false, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868497, "value1":0.924966560225317, "value2":4429888702948279946, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.146490, "value1":0.7150691128822396, "value2":205950019316022568, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009297, "value1":0.4940665543165739, "value2":924283179287111091, "value3":false, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904057, "value1":0.23580267125603968, "value2":531910251609622124, "value3":true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728011, "value1":0.8321936268339676, "value2":2170557262229675184, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944000, "value1":0.7690990294985828, "value2":8227626091745901, "value3":true, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.871354, "value1":0.21098039407901223, "value2":4374012870679696344, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.538323, "value1":0.681484837485245, "value2":6775701390216802286, "value3":false, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230276, "value1":0.36173491575946426, "value2":8235898214428447094, "value3":false, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552468, "value1":0.8892513220522287, "value2":612692529250917070, "value3":true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782746, "value1":0.3267438552083453, "value2":5821122346950557164, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908039, "value1":0.8750317891455761, "value2":6904198848902253780, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124465, "value1":0.27489616104124, "value2":1813939809215341473, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.739988, "value1":0.11884426917626255, "value2":6375710418065432574, "value3":false, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246847, "value1":0.065994349338974, "value2":4501641519889306838, "value3":true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.675544, "value1":0.4346958284003153, "value2":8656554552944846961, "value3":false, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907951, "value1":0.3861999534665598, "value2":7376195997991690698, "value3":false, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.601240, "value1":0.09029192218287085, "value2":4084656981993173700, "value3":false, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327311, "value1":0.46909482167142375, "value2":6484192245523357545, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453608, "value1":0.634255139121383, "value2":8880783116370612057, "value3":true, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562417, "value1":0.4260014133745824, "value2":1052849307102914712, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708174, "value1":0.05688677429464904, "value2":5475041244330586198, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.774531, "value1":0.8132424157243254, "value2":1362266175582132598, "value3":true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.106500, "value1":0.7014281430749205, "value2":7134854011640912176, "value3":false, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.486559, "value1":0.28063120575400446, "value2":3192436055979504417, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.864997, "value1":0.3477928996994262, "value2":735248433592549787, "value3":false, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.657367, "value1":0.65265411185833, "value2":2443852584375783883, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.172271, "value1":0.4417461489558258, "value2":7942690633839505457, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419664, "value1":0.16011001601248667, "value2":6549588207213967629, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416761, "value1":0.6429956000093604, "value2":8493895113638046625, "value3":false, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564282, "value1":0.4457588529183665, "value2":6725016002434781716, "value3":true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430750, "value1":0.3037956469116298, "value2":7998283396935875899, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827125, "value1":0.5250484976450595, "value2":6524724358656282389, "value3":false, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395896, "value1":0.12287086144958013, "value2":1930681199692368788, "value3":false, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851159, "value1":0.23103414004696357, "value2":4914748620478090577, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.736709, "value1":0.6408817571256187, "value2":9105122753792897591, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984808, "value1":0.2905624937515174, "value2":7851451601162680954, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487306, "value1":0.605006354442018, "value2":4276465105843409598, "value3":false, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790850, "value1":0.6268022197129945, "value2":9220780769142386741, "value3":true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185700, "value1":0.566519118535341, "value2":3705647574979636584, "value3":false, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451877, "value1":0.0460137249105004, "value2":7915985022414197030, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150718, "value1":0.35693559742029707, "value2":5014219743291793375, "value3":false, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.294260, "value1":0.9247064360015096, "value2":5127524279022159080, "value3":false, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995623, "value1":0.21178547000119013, "value2":2431494318968387519, "value3":true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.256959, "value1":0.8916548214121515, "value2":1002635579417294841, "value3":true, "__name__":"metric_00000095", "key_5":"c","key_1":"c","key_2":"g", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110758, "value1":0.9737478776363947, "value2":7247818685878473810, "value3":false, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825871, "value1":0.2917878380844755, "value2":6516114778766465158, "value3":true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910566, "value1":0.3957632746819297, "value2":7166657573372241684, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.178800, "value1":0.5649796089766207, "value2":7775035464817162126, "value3":false, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634489, "value1":0.6090054970934514, "value2":6647498876825886588, "value3":true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610499, "value1":0.5292322793488414, "value2":8830154316280741019, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574572, "value1":0.5304362659437236, "value2":3949756347058852883, "value3":false, "__name__":"metric_00000132", "key_3":"a","key_9":"i","key_0":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380651, "value1":0.956019198700713, "value2":9135146156795796096, "value3":true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778218, "value1":0.2121518375707717, "value2":8667556184145489676, "value3":true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078961, "value1":0.06325131155030916, "value2":8555129638818277403, "value3":true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.047017, "value1":0.1027203958612216, "value2":5246272405737501952, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439829, "value1":0.7417681560989317, "value2":7749536771402273968, "value3":true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203072, "value1":0.5621215132192839, "value2":6118954075279232029, "value3":true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758671, "value1":0.48389569392567205, "value2":2705225096795559234, "value3":false, "__name__":"metric_00000140", "key_2":"g","key_1":"c", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665522, "value1":0.8426429373047708, "value2":8279168363918064894, "value3":true, "__name__":"metric_00000141", "key_6":"b","key_8":"h","key_1":"a", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.345535, "value1":0.20088437097171843, "value2":4950747031440638845, "value3":false, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124655, "value1":0.41827026587025645, "value2":3430038763553236823, "value3":true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977760, "value1":0.10421019819450697, "value2":733644826366835807, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118861, "value1":0.7989242448499801, "value2":7548777360232504083, "value3":true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318842, "value1":0.009939081834750191, "value2":8375328698470876119, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590619, "value1":0.03636791737366475, "value2":7582770650474370737, "value3":true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045468, "value1":0.7763100607726895, "value2":2803435127241577664, "value3":true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.553650, "value1":0.9987121300161536, "value2":3948733587296676032, "value3":true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790022, "value1":0.793158890727088, "value2":1174928966366390949, "value3":true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.232763, "value1":0.07728571888784644, "value2":3240453509192284959, "value3":false, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074815, "value1":0.5297469977127046, "value2":2019972453697918898, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.126329, "value1":0.8018605847489219, "value2":4857597305010910280, "value3":true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.465083, "value1":0.1752053660866726, "value2":7163605506157251603, "value3":true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.714319, "value1":0.8220755414438086, "value2":876552025810846680, "value3":true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392782, "value1":0.6635667135366609, "value2":6624504880778850428, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404283, "value1":0.5416570514867133, "value2":8266175224671672272, "value3":false, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.734325, "value1":0.7575887007274229, "value2":8864761125492843547, "value3":true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.164845, "value1":0.7328611642697666, "value2":1939960544899646141, "value3":true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.087713, "value1":0.6472001702780127, "value2":7064434641814632478, "value3":false, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636375, "value1":0.263542227747472, "value2":5091288498651627633, "value3":true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439658, "value1":0.7518898990734011, "value2":6126681364245747402, "value3":true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580431, "value1":0.28444432068121084, "value2":8147884504685238694, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408258, "value1":0.5563100440073113, "value2":4573231178010278468, "value3":true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923759, "value1":0.7035415767681118, "value2":4284377617122095281, "value3":false, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069625, "value1":0.7755797987036898, "value2":7005063200108318654, "value3":true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475808, "value1":0.7602915679203158, "value2":1889690102234767528, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045338, "value1":0.4860219702134258, "value2":5422255506416156729, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.985908, "value1":0.7866996153890412, "value2":4784461751340030748, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937856, "value1":0.5239212561479751, "value2":3109818040142871590, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.376519, "value1":0.2587179526287473, "value2":1263076833301844915, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943956, "value1":0.6806333311736701, "value2":6002705815114995201, "value3":true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.653771, "value1":0.7947376720839073, "value2":2247611778314354742, "value3":true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290979, "value1":0.13605627013098143, "value2":859676226210675359, "value3":false, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166492, "value1":0.4455170530582711, "value2":3676647784290217622, "value3":false, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.625732, "value1":0.26531203043868007, "value2":8360465694110411228, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033924, "value1":0.027884795524999093, "value2":4290921120785429469, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.199865, "value1":0.6930021146351104, "value2":8033400918877373777, "value3":true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.387257, "value1":0.5315645824113636, "value2":8487597923574953049, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.385317, "value1":0.687478534046926, "value2":5788685556231917492, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.632370, "value1":0.4013559424653022, "value2":8640639439561304540, "value3":true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226297, "value1":0.2440055184343977, "value2":5133913946019355967, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.763037, "value1":0.20026459422324144, "value2":8670140335638909516, "value3":false, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695304, "value1":0.16276770698378185, "value2":1188100569829740721, "value3":false, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595366, "value1":0.4506718132549764, "value2":5734081257378041711, "value3":true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.833543, "value1":0.5734402814385565, "value2":922425738240504967, "value3":false, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709630, "value1":0.8692041834415107, "value2":1960359817596413212, "value3":true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.732788, "value1":0.23444382672590003, "value2":2854071547092864254, "value3":true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.412896, "value1":0.19053537293020936, "value2":1082762509249072310, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988462, "value1":0.4293280866481845, "value2":4780102507614458134, "value3":true, "__name__":"metric_00000198", "key_8":"i","key_0":"h", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823600, "value1":0.01981381195792859, "value2":8743022558864727512, "value3":false, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.092886, "value1":0.9862388684646872, "value2":3622022818199205736, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.018231, "value1":0.4242288910229569, "value2":880886547836969584, "value3":true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062605, "value1":0.9179292075855371, "value2":3743117943661894467, "value3":true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.759025, "value1":0.5166884948154693, "value2":739459076946893696, "value3":true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907932, "value1":0.2882739675567565, "value2":3650155370836487214, "value3":true, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068172, "value1":0.4347373436897239, "value2":1372308133830179280, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514459, "value1":0.3578934602691126, "value2":1369826174473414234, "value3":false, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282115, "value1":0.4836091160007032, "value2":7304317831220994830, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652653, "value1":0.7715198420930932, "value2":1854799303037917400, "value3":true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.873426, "value1":0.6674125872681426, "value2":1713253808702182629, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.253882, "value1":0.35618452262868266, "value2":1451362150616905038, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058435, "value1":0.11619135407507769, "value2":729304989898004351, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711979, "value1":0.19795958240207612, "value2":5044852283450257784, "value3":true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.620015, "value1":0.7494674771784868, "value2":9072157623838715380, "value3":false, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265268, "value1":0.6332467233229876, "value2":7660280405186635297, "value3":false, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471797, "value1":0.9586561875915051, "value2":1400450692063171507, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715350, "value1":0.28992759308570154, "value2":4420882374996204645, "value3":true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447264, "value1":0.32845931032025727, "value2":3764283351468138749, "value3":true, "__name__":"metric_00000209", "key_7":"g","key_1":"f", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799817, "value1":0.8344281598160882, "value2":3194977322046448615, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225092, "value1":0.832673789719215, "value2":8242369633095170423, "value3":true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261842, "value1":0.1783779388932828, "value2":7591102613464952913, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885476, "value1":0.33420965888644627, "value2":2231088099862071194, "value3":true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015564, "value1":0.23747285224855552, "value2":2211582269309113296, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415709, "value1":0.43459757217905465, "value2":2297522408220839681, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671302, "value1":0.5521783843346304, "value2":8797575493863836447, "value3":true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809774, "value1":0.3232411464090472, "value2":689408551095099282, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391492, "value1":0.44145120690127054, "value2":741707985323247529, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593752, "value1":0.5004004561210201, "value2":7899920698831884885, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090704, "value1":0.9099972336756517, "value2":8993930752293016892, "value3":false, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.365212, "value1":0.17134030372562564, "value2":6300597784300096913, "value3":true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.458419, "value1":0.03537414872175956, "value2":4931151600790868120, "value3":true, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.266288, "value1":0.2710772740926763, "value2":5168762762336913335, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.846075, "value1":0.8530137052878358, "value2":5846881912829979057, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.441246, "value1":0.895274800378327, "value2":6555884777228809545, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181487, "value1":0.09635807954147779, "value2":5139650932097495606, "value3":true, "__name__":"metric_00000231", "key_7":"g","key_3":"a","key_4":"k", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.283526, "value1":0.6765717041541169, "value2":717191993149931255, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690428, "value1":0.5868291690096701, "value2":4142969563289744689, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461183, "value1":0.7388746066577521, "value2":3907321818804232131, "value3":true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078559, "value1":0.5509982085563974, "value2":8489285868587130769, "value3":true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.006442, "value1":0.17964435576390814, "value2":6385046429151027537, "value3":true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803653, "value1":0.007944203568736737, "value2":8630069900982836811, "value3":true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843384, "value1":0.5245784602574561, "value2":1611187771800002701, "value3":false, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.322734, "value1":0.26583004470195515, "value2":5891979565833996742, "value3":false, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019366, "value1":0.9471429425977943, "value2":3105898999363337716, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583535, "value1":0.3000091753953168, "value2":1671214988273742274, "value3":true, "__name__":"metric_00000220", "key_9":"d","key_5":"g","key_6":"h", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.528263, "value1":0.6167724953698004, "value2":3571326627931246323, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.353615, "value1":0.09339691407017821, "value2":8955717188172767409, "value3":false, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.638995, "value1":0.5498958167694153, "value2":6537844553335776020, "value3":true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327054, "value1":0.5183869093313511, "value2":4021636019817176776, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391759, "value1":0.6107482510959784, "value2":6639243589639002168, "value3":false, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.083515, "value1":0.6947157647744896, "value2":3642097726125624072, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.586405, "value1":0.683079171453159, "value2":414358593011525856, "value3":false, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847208, "value1":0.004363913601366179, "value2":6293076524070827579, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583066, "value1":0.38103391426791644, "value2":7168389465606666058, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.361829, "value1":0.0757878691848108, "value2":18451857992724418, "value3":false, "__name__":"metric_00000246", "key_5":"i","key_0":"k","key_3":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481695, "value1":0.5254185806253874, "value2":3155631786417589238, "value3":true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.072077, "value1":0.8787002152127282, "value2":5274515234654561765, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.267961, "value1":0.8648926070157235, "value2":4125897469271708547, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.245076, "value1":0.06458749862097861, "value2":3727526843973668028, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602905, "value1":0.9902842013208221, "value2":3395310217472956562, "value3":true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776585, "value1":0.7219469790990183, "value2":77676005392087147, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741224, "value1":0.9110757287517638, "value2":3235394638378837669, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.616438, "value1":0.32846413845355765, "value2":5558119133284488407, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.617955, "value1":0.12562192604901234, "value2":603576184957547937, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.651221, "value1":0.8976398132733482, "value2":7308768939039920124, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610616, "value1":0.3225584230939945, "value2":8416609441712941807, "value3":true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.688197, "value1":0.39642287994645425, "value2":7290339184191009077, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599842, "value1":0.6105446645640619, "value2":1208112926522863577, "value3":false, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867582, "value1":0.1840371797963492, "value2":2177170750841832105, "value3":true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098316, "value1":0.966593809823329, "value2":2548324970162406510, "value3":false, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694537, "value1":0.3376604030915601, "value2":607328136959332742, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748191, "value1":0.9957584594865343, "value2":67877367036436991, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.064163, "value1":0.7248743828192261, "value2":2749527480847272451, "value3":false, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502667, "value1":0.16495830272715267, "value2":9030464128428456991, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990661, "value1":0.3949843558188797, "value2":7201429976848106189, "value3":false, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987854, "value1":0.29870559261727064, "value2":348149959926675176, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481753, "value1":0.06548607806382464, "value2":1031140396971019018, "value3":false, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.630820, "value1":0.8992403438153617, "value2":5146298573769468519, "value3":false, "__name__":"metric_00000274", "key_7":"g","key_9":"h","key_3":"k", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804082, "value1":0.2519724232639516, "value2":1298310485258981735, "value3":true, "__name__":"metric_00000278", "key_4":"h","key_5":"j","key_1":"c", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368623, "value1":0.647450377107345, "value2":1284622928379296522, "value3":true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.541781, "value1":0.3261143352667153, "value2":130140210100844559, "value3":false, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543424, "value1":0.06226681703581004, "value2":726618431953897465, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988533, "value1":0.49745860881748044, "value2":3406264037344093572, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080362, "value1":0.7895574305241833, "value2":152509341700687015, "value3":false, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150372, "value1":0.9343525151823494, "value2":6076357966504332576, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489577, "value1":0.16062395564859527, "value2":3405065082227979672, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225570, "value1":0.28122658630857733, "value2":8458136042003760574, "value3":false, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862952, "value1":0.7649680950314343, "value2":1756495878088169871, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009186, "value1":0.7999050849241861, "value2":3718209620537893555, "value3":true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327636, "value1":0.23708513628211542, "value2":1522368463913294016, "value3":false, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.551176, "value1":0.009439114458257377, "value2":2361018304142309387, "value3":false, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.633123, "value1":0.2821310200838197, "value2":6052402151738260293, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128752, "value1":0.3359757178563009, "value2":7472471602725669166, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.456386, "value1":0.46133506790277695, "value2":2296093836300686089, "value3":true, "__name__":"metric_00000277", "key_3":"e","key_4":"d","key_1":"g", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420349, "value1":0.36664549905260974, "value2":3546467223240204480, "value3":true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.938528, "value1":0.007977829048726685, "value2":4244809297577031047, "value3":true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226579, "value1":0.6725646472034443, "value2":913735509091846801, "value3":true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.850850, "value1":0.23068766044782393, "value2":2300681752566773332, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285218, "value1":0.6989337075616671, "value2":2607116938388175821, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.221331, "value1":0.11523512208048978, "value2":8241016534243475750, "value3":false, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886294, "value1":0.8815954846610522, "value2":1129122380251841667, "value3":true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010095, "value1":0.4468942822052692, "value2":7305659252479204819, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007461, "value1":0.3287186449948172, "value2":8533612311279585337, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841878, "value1":0.06768912562638665, "value2":3915275733365598237, "value3":false, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239318, "value1":0.2752287175449378, "value2":6104494420782937095, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364905, "value1":0.7269412172343553, "value2":7167814392288649336, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030862, "value1":0.35386987370196926, "value2":568029767114417610, "value3":false, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945737, "value1":0.8998415941144967, "value2":2346717810373152754, "value3":false, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.093117, "value1":0.8244480275524787, "value2":3248267576826263944, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.422568, "value1":0.5719502852011219, "value2":8260911644483203383, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823166, "value1":0.2849312986012331, "value2":3755798306625123375, "value3":true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268641, "value1":0.33070121758142046, "value2":3503668812330101597, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564550, "value1":0.4481321012388454, "value2":2552310395319729245, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417743, "value1":0.047809954457076996, "value2":1081553625885385604, "value3":true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595860, "value1":0.2921995022802213, "value2":5059986207137621431, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.505814, "value1":0.9601390700122682, "value2":1325398133306866350, "value3":true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357820, "value1":0.3304365801280413, "value2":5155828582049823863, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963899, "value1":0.8305280855606907, "value2":2952790462431625685, "value3":true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395768, "value1":0.05940274489209035, "value2":2206736622008483109, "value3":true, "__name__":"metric_00000311", "key_2":"f","key_5":"d","key_0":"b", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.994837, "value1":0.28131858851364544, "value2":6377323293251471217, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825885, "value1":0.9524553792547288, "value2":5686625986396675519, "value3":true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962844, "value1":0.7824414856459302, "value2":3176085690721005555, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525393, "value1":0.2894500361431452, "value2":9170274733300331931, "value3":true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066894, "value1":0.9750264781896856, "value2":6481943120958967565, "value3":true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118777, "value1":0.49186253332263474, "value2":196535145535309391, "value3":false, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.011455, "value1":0.25527316956832247, "value2":23216573243095801, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.516739, "value1":0.7031419417768016, "value2":2051319192678332676, "value3":false, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621518, "value1":0.7948563798954263, "value2":8394265809911073481, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.008794, "value1":0.671348308630362, "value2":8908504416251388849, "value3":false, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804142, "value1":0.34185386259040573, "value2":7569056971355045232, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971215, "value1":0.795050202357287, "value2":7291081553392495608, "value3":false, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285984, "value1":0.6233623903797153, "value2":6444347877884163801, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.436168, "value1":0.122788509000057, "value2":2542694660610691923, "value3":true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545038, "value1":0.897272884042895, "value2":3518180018317716924, "value3":true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.377957, "value1":0.26500726028503324, "value2":918021549502007270, "value3":true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186659, "value1":0.8861606285044316, "value2":7428746411605199070, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.706851, "value1":0.8539941323885553, "value2":5095874822126979293, "value3":false, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552666, "value1":0.8209937326987259, "value2":7918997962757156260, "value3":false, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708678, "value1":0.20172873469302396, "value2":473140516571282958, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.588929, "value1":0.8992364975321605, "value2":1107792164310220135, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.396543, "value1":0.9159358105409485, "value2":3240110351377657654, "value3":true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949484, "value1":0.3395753649788759, "value2":1556081794159850294, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627633, "value1":0.6064813434025144, "value2":504106831033400051, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760461, "value1":0.027052105627838302, "value2":2547821024594975963, "value3":false, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690642, "value1":0.29842739674487334, "value2":4347369482397818828, "value3":true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271622, "value1":0.9232891397814524, "value2":4902825584314369050, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678674, "value1":0.8612335750248717, "value2":2676726884600182684, "value3":false, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.789269, "value1":0.7153455595388135, "value2":8629094676547921616, "value3":true, "__name__":"metric_00000348", "key_4":"k","key_5":"e","key_3":"h", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.274447, "value1":0.04852479796186674, "value2":520491486142586388, "value3":false, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517924, "value1":0.5065900646461697, "value2":1305736770624753229, "value3":false, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440510, "value1":0.8079290707038177, "value2":745163355638334416, "value3":true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.992658, "value1":0.33022695610085456, "value2":3745647814397920317, "value3":false, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.142855, "value1":0.7836474840216356, "value2":1701655973854226818, "value3":false, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.718918, "value1":0.9106879340813463, "value2":2642697174847166539, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358697, "value1":0.6889718382775613, "value2":2396285724783610411, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782243, "value1":0.10018629733131312, "value2":3557685057170687759, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290843, "value1":0.5213600910147824, "value2":5522593432675792373, "value3":true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243393, "value1":0.9922299657128142, "value2":267617765686382025, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827775, "value1":0.17424595518809072, "value2":3247150599498567700, "value3":true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954439, "value1":0.6395234710891515, "value2":3700591746218368196, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543518, "value1":0.7123005069599985, "value2":8170425126276779400, "value3":false, "__name__":"metric_00000361", "key_9":"c","key_3":"h","key_7":"j", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862097, "value1":0.7928466963741031, "value2":1388049342872581158, "value3":false, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861104, "value1":0.44830540845614686, "value2":3638392887423846691, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996353, "value1":0.9606274415719652, "value2":6260542395914742363, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269620, "value1":0.27973905679528643, "value2":1556926452504115363, "value3":false, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118090, "value1":0.5428339606791369, "value2":1853095038687144617, "value3":false, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843203, "value1":0.15162943121847733, "value2":978719322151022878, "value3":true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.168392, "value1":0.30548225941021784, "value2":7137186554911304535, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.197151, "value1":0.3975385059702385, "value2":5863175014471717006, "value3":false, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565354, "value1":0.9234247531019719, "value2":812121677319261736, "value3":false, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030909, "value1":0.8792093662826526, "value2":8559138722753842843, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.484600, "value1":0.23944552081323173, "value2":7751158766892162613, "value3":true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.320620, "value1":0.12230291625693078, "value2":6192516194919614511, "value3":true, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.935691, "value1":0.28006447226692, "value2":6784015217669799303, "value3":true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.336215, "value1":0.23094104388429443, "value2":5569809866177597446, "value3":false, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988466, "value1":0.5462607441657077, "value2":8311104273733417101, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447804, "value1":0.4867937694484817, "value2":5062168010270449210, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.190570, "value1":0.8283372201936341, "value2":5874711067636169206, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904281, "value1":0.8553852749887187, "value2":3311030506444553065, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.751439, "value1":0.4266991113414808, "value2":4999173498909581042, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.125309, "value1":0.5563950948301023, "value2":1187994145431678821, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.973142, "value1":0.11588338763957572, "value2":8318570385943791860, "value3":true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.116231, "value1":0.2526143185693312, "value2":782968109827902547, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.187535, "value1":0.27872170170772304, "value2":8766589428595489268, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945637, "value1":0.08854405995381304, "value2":3412460749402337910, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.270152, "value1":0.3638550762746485, "value2":2218032724303888987, "value3":true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051113, "value1":0.40794935925664466, "value2":7831097119693907093, "value3":true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265496, "value1":0.23954206323753308, "value2":63253225412010244, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416624, "value1":0.17710696129812786, "value2":3803821152719869396, "value3":false, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607195, "value1":0.28582291116541503, "value2":7284275038719896556, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.326362, "value1":0.17094485896565303, "value2":4663361033102261072, "value3":false, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438839, "value1":0.3486395787421312, "value2":8809206276934706557, "value3":false, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033597, "value1":0.8575270834812074, "value2":1940304743017854051, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.065029, "value1":0.9427613360719956, "value2":271108060306002877, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.488385, "value1":0.12779971800108084, "value2":3316011689829646532, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961814, "value1":0.5428137306648136, "value2":1514601534044579978, "value3":false, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263207, "value1":0.6949685012843101, "value2":895006576805426648, "value3":true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282635, "value1":0.10777955836055363, "value2":564953042266607985, "value3":false, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.001269, "value1":0.042927028237879465, "value2":1466958494603226347, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682457, "value1":0.8478178196386861, "value2":107121152123493591, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.159543, "value1":0.9604877997752348, "value2":7593923445754669936, "value3":false, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015347, "value1":0.7626139882833062, "value2":5647986907617063275, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209254, "value1":0.8450839759887067, "value2":1886374721985796728, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066244, "value1":0.07791428229072243, "value2":919090579972011287, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.699052, "value1":0.06643124534628293, "value2":2711140899074023727, "value3":false, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.631133, "value1":0.143383362490665, "value2":971805873892273782, "value3":false, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426673, "value1":0.7738540415601494, "value2":4397130911534361286, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.389568, "value1":0.33397631468501043, "value2":4052528315967262589, "value3":false, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.622610, "value1":0.24246003195798918, "value2":5479722481130894419, "value3":false, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904392, "value1":0.9774495314687007, "value2":8793165354063487461, "value3":true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.793581, "value1":0.4190858782174469, "value2":4926610192238705598, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110855, "value1":0.2884411940593678, "value2":5227068360933879219, "value3":false, "__name__":"metric_00000397", "key_7":"d","key_0":"k","key_2":"f", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135653, "value1":0.402267966757566, "value2":6977955000760069723, "value3":false, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.307152, "value1":0.8927574650950808, "value2":5803177489389723115, "value3":false, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.737685, "value1":0.21404400943023025, "value2":6732954103348577362, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.121533, "value1":0.18872391644762126, "value2":8989825709899786188, "value3":true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.972223, "value1":0.023522846364320095, "value2":7146680715055172496, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923388, "value1":0.8860383778342971, "value2":8303048822044107475, "value3":true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851782, "value1":0.32631560891796313, "value2":1989916996789168449, "value3":false, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391839, "value1":0.002802043190783618, "value2":8629895271323501696, "value3":false, "__name__":"metric_00000400", "key_1":"c","key_6":"c","key_0":"f", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066714, "value1":0.031995170191600183, "value2":3319906268065732292, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421535, "value1":0.635220505499552, "value2":7200157785718775016, "value3":false, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287471, "value1":0.8948051177891431, "value2":2176610475830643902, "value3":true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804659, "value1":0.28415985878229416, "value2":33052675971427371, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996360, "value1":0.6152217383986905, "value2":544588703505765788, "value3":false, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.798458, "value1":0.8665558307064753, "value2":8115268483760058936, "value3":true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448744, "value1":0.3917339620374744, "value2":9198148727913546136, "value3":false, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861092, "value1":0.9826532618696505, "value2":4765520374138069290, "value3":false, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.202532, "value1":0.0052591320469092834, "value2":336652236719043103, "value3":false, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.524950, "value1":0.8143167251197552, "value2":6529413407694528834, "value3":false, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.485366, "value1":0.5907312441951306, "value2":3337798525793799244, "value3":false, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558504, "value1":0.3641804008571667, "value2":4575793560178915745, "value3":true, "__name__":"metric_00000432", "key_3":"b","key_2":"k", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642936, "value1":0.623697886602654, "value2":7776504717328671778, "value3":true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043344, "value1":0.9623993994613507, "value2":6602176914068250275, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.241555, "value1":0.23253929846791072, "value2":685349515600255534, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926727, "value1":0.21223983767190985, "value2":3951472227334441759, "value3":false, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.131046, "value1":0.2851063666145672, "value2":6180530959986721149, "value3":true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.247299, "value1":0.5130082295270245, "value2":8139956405448602641, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758053, "value1":0.49876246690774434, "value2":3786706615107859566, "value3":false, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323322, "value1":0.4313958567662574, "value2":8854072512884634153, "value3":false, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748573, "value1":0.9590027921453678, "value2":3839028144294969465, "value3":false, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384213, "value1":0.010562139959872605, "value2":6878855550977960785, "value3":true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180644, "value1":0.6131439659014073, "value2":2706285599198878081, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593573, "value1":0.4120750238333436, "value2":1200854625465685501, "value3":false, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995840, "value1":0.297035809872462, "value2":6168541710158809840, "value3":false, "__name__":"metric_00000425", "key_9":"k","key_3":"e","key_8":"d", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.511756, "value1":0.006206978177219643, "value2":1958577823681147315, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565859, "value1":0.4442276087687805, "value2":39986647781639917, "value3":true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715853, "value1":0.9381008994826828, "value2":3556376195988394411, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236468, "value1":0.20999691756150096, "value2":9161072742831597145, "value3":true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.161371, "value1":0.738745986035695, "value2":5138216362948975345, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445330, "value1":0.14739850543555172, "value2":8667736730671884444, "value3":true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.982188, "value1":0.40813076015564426, "value2":3022991256121544350, "value3":true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984748, "value1":0.6214792221342914, "value2":5933123653156672217, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562273, "value1":0.8568295281307874, "value2":5414869267292754213, "value3":true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476811, "value1":0.49675643916102746, "value2":5664619325942193065, "value3":false, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891490, "value1":0.8866201621688181, "value2":8214607251036840472, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931525, "value1":0.8288823187078326, "value2":3069955632009854668, "value3":false, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455511, "value1":0.8774391931652025, "value2":3046641762023967525, "value3":true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400651, "value1":0.5254465108261448, "value2":4999803900412377548, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666971, "value1":0.9993076586927376, "value2":4982431527333428358, "value3":true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.8618210720028192, "value2":761627529379782309, "value3":false, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.915374, "value1":0.37053901712348225, "value2":3944125570135845867, "value3":true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.960156, "value1":0.9374154078802892, "value2":4747973404867949779, "value3":true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210720, "value1":0.8468677886051782, "value2":4852924773263065270, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.332500, "value1":0.6465642869647131, "value2":1008458018130353532, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.788169, "value1":0.3773654642642006, "value2":4191084511110280464, "value3":false, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545303, "value1":0.25852749290398136, "value2":2783025552475099417, "value3":false, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.143623, "value1":0.36024800648413563, "value2":4932311496806548620, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895647, "value1":0.0053042457769262655, "value2":6923819500937313799, "value3":false, "__name__":"metric_00000447", "key_5":"g","key_8":"i","key_0":"e", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932434, "value1":0.659079228202596, "value2":4313811336647625065, "value3":true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.260771, "value1":0.403057123436948, "value2":4416521488550693116, "value3":false, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238541, "value1":0.2952177348030342, "value2":1538460010819351929, "value3":true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958540, "value1":0.5291106177059647, "value2":1249052435364791122, "value3":true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430589, "value1":0.8147197427498966, "value2":5526371695265553363, "value3":false, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431544, "value1":0.9355303369766881, "value2":4469095563720030593, "value3":true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640478, "value1":0.48586313377391904, "value2":7959068707072142404, "value3":false, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144376, "value1":0.00848981726968387, "value2":5068460906328461885, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416953, "value1":0.6734644377643653, "value2":2766510942957347250, "value3":false, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885086, "value1":0.21498485778798626, "value2":8278007933868416694, "value3":false, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537244, "value1":0.3154976316557619, "value2":2639365305979929980, "value3":true, "__name__":"metric_00000471", "key_8":"b","key_1":"b","key_3":"f", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.020210, "value1":0.453949847740753, "value2":2024272373950233485, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514022, "value1":0.3475698755294227, "value2":1372329110659095675, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806258, "value1":0.9274447352712736, "value2":3920648533163431087, "value3":false, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209427, "value1":0.014587282647209867, "value2":8695997195066251972, "value3":false, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971326, "value1":0.2797692384280631, "value2":6419079357930277367, "value3":false, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652286, "value1":0.5905911533568637, "value2":8712000656540499616, "value3":false, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878040, "value1":0.48362174266483915, "value2":2926563391771284063, "value3":true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990329, "value1":0.5049151964579438, "value2":3650817862757599534, "value3":true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049823, "value1":0.838067458662296, "value2":8854264061775325606, "value3":false, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017442, "value1":0.8431632474037919, "value2":8852347705624487512, "value3":true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.544821, "value1":0.005649842126714668, "value2":2282778264986625535, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.597518, "value1":0.2045781468759336, "value2":8575639431654539646, "value3":true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612206, "value1":0.3075892336013964, "value2":7673450384556490732, "value3":true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471060, "value1":0.40918086270080856, "value2":6747291739813622106, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.513402, "value1":0.7422854995723118, "value2":4083993911078701361, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965008, "value1":0.6372798813181509, "value2":1707224457136697946, "value3":false, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.354283, "value1":0.9108473180297934, "value2":1020266209800871174, "value3":false, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806140, "value1":0.14778437030634853, "value2":6105158525255417708, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194962, "value1":0.3473291379662418, "value2":3795992189004973455, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.974776, "value1":0.09118542426518424, "value2":5134289787707010070, "value3":false, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.339938, "value1":0.13774510006333723, "value2":562171884148417195, "value3":false, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.550551, "value1":0.384009626532502, "value2":8609730106908869554, "value3":false, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.654960, "value1":0.2639517045663951, "value2":1862456670074602807, "value3":true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030932, "value1":0.5143312749002444, "value2":1494847764422163119, "value3":true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695169, "value1":0.2423970054677799, "value2":8509443740065644494, "value3":true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713656, "value1":0.34108742339653203, "value2":4598279459368093139, "value3":true, "__name__":"metric_00000497", "key_2":"f","key_0":"d","key_1":"e", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.794261, "value1":0.9458347874797615, "value2":7302724440323717945, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364772, "value1":0.2365974372946532, "value2":3654435595389536462, "value3":false, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395099, "value1":0.15838919107992194, "value2":1939247105684031089, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335629, "value1":0.39545025723069543, "value2":2672917998276624419, "value3":false, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074949, "value1":0.7876350399335282, "value2":625303995684394443, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536138, "value1":0.5616610785403638, "value2":8164874977498368721, "value3":false, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697782, "value1":0.23834340441765955, "value2":3951692500137347815, "value3":false, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671649, "value1":0.6501720201004972, "value2":2955880776784904551, "value3":true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.787079, "value1":0.378893964304627, "value2":3193906322405958964, "value3":false, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.031359, "value1":0.29145414339441095, "value2":5615459484207384702, "value3":false, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.306294, "value1":0.34557951146905197, "value2":2931839249030464715, "value3":true, "__name__":"metric_00000512", "key_4":"d","key_7":"e","key_1":"k", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357655, "value1":0.3923960009663459, "value2":8112783273863341075, "value3":true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961896, "value1":0.23361672067408637, "value2":432132564805318676, "value3":false, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098797, "value1":0.18420600660031175, "value2":4984332870487251645, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.071876, "value1":0.6659197757567412, "value2":1553450733606247594, "value3":false, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186866, "value1":0.7000117639690743, "value2":7329872261968800728, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406584, "value1":0.13046586013887515, "value2":1622087582581823179, "value3":false, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440301, "value1":0.11646408872647017, "value2":1011606322554570900, "value3":true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.136166, "value1":0.4328392754883894, "value2":2919499144416670917, "value3":false, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318957, "value1":0.7075576973107754, "value2":398072267285532828, "value3":false, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.012179, "value1":0.5668221134322494, "value2":6110074131405385081, "value3":true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830949, "value1":0.3334949924022553, "value2":7572722630631231675, "value3":false, "__name__":"metric_00000528", "key_7":"c","key_8":"g","key_0":"k", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145965, "value1":0.365420169950548, "value2":2535895288587113465, "value3":false, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.649060, "value1":0.16445860625759065, "value2":6921600715877525051, "value3":false, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517227, "value1":0.07192874427493294, "value2":6297114124078920008, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832587, "value1":0.5150936291796077, "value2":181711691089923280, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291455, "value1":0.5623557100802268, "value2":8762626146094505466, "value3":true, "__name__":"metric_00000508", "key_5":"h","key_8":"a","key_2":"j", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.149447, "value1":0.5534139568709165, "value2":2510872908080100775, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476658, "value1":0.4486907399118968, "value2":8992668719445662670, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236318, "value1":0.49075032225313237, "value2":1924725427300127433, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.561765, "value1":0.9668124792198439, "value2":990391456732695590, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884668, "value1":0.0011015591734681801, "value2":5327656656973278324, "value3":false, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175200, "value1":0.9826652864577513, "value2":2876633651705447757, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379413, "value1":0.15362330796009416, "value2":7353421640256094445, "value3":false, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.319517, "value1":0.5202924305313171, "value2":7456692117889227019, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.122103, "value1":0.7187092003738956, "value2":7131563361103346240, "value3":true, "__name__":"metric_00000534", "key_3":"a","key_9":"k","key_1":"a", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678103, "value1":0.7543751799594591, "value2":8526553775473008293, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.643972, "value1":0.19798980984364278, "value2":5929329615236343098, "value3":true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.831121, "value1":0.1528427519418499, "value2":6184723596853858850, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432219, "value1":0.5515160206615604, "value2":23121424938113426, "value3":false, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058751, "value1":0.6338015087216923, "value2":7816911191202995864, "value3":true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790739, "value1":0.8762741406918414, "value2":4555556160832468314, "value3":false, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797544, "value1":0.8139645999303083, "value2":2761547225024668528, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404353, "value1":0.9860244138424634, "value2":2908826677832473108, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016900, "value1":0.053160518991546266, "value2":8380107749249883562, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.407973, "value1":0.7843730354485248, "value2":6824583973593836053, "value3":true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987174, "value1":0.7619687243629265, "value2":3319925462827235787, "value3":false, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577835, "value1":0.20813952139985942, "value2":6437982890007555330, "value3":false, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755320, "value1":0.3636597821442561, "value2":5470963049552133157, "value3":false, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.897987, "value1":0.07151356373469021, "value2":1477083928227057783, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388580, "value1":0.20932748154080844, "value2":3392543048891869159, "value3":false, "__name__":"metric_00000548", "key_9":"c","key_4":"i", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.025802, "value1":0.38577745314564454, "value2":868566582237921681, "value3":false, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941032, "value1":0.17111301155402217, "value2":6961085515450408974, "value3":false, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947055, "value1":0.9780343555878157, "value2":133683558419674847, "value3":true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954822, "value1":0.08800460018752372, "value2":7497584262448035095, "value3":false, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063420, "value1":0.8100121364093217, "value2":3755200469512293547, "value3":true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438294, "value1":0.3424655143965952, "value2":4316249625206240732, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809627, "value1":0.9673131880891396, "value2":4132748723398010856, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439925, "value1":0.8075664782512888, "value2":864263057207692643, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.084104, "value1":0.3818348864598292, "value2":2553923579282526899, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.647912, "value1":0.46216103925581126, "value2":4639093581287452200, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369627, "value1":0.3422458784570339, "value2":4254456412295026385, "value3":false, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.614963, "value1":0.5220645586983701, "value2":6072161543938690608, "value3":false, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.321895, "value1":0.3049038907635511, "value2":6325538472569574229, "value3":true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464290, "value1":0.22336524838033933, "value2":9076126601794953657, "value3":true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166932, "value1":0.3553203900309477, "value2":5982106259976109422, "value3":true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803898, "value1":0.5280548625824253, "value2":4063002262098608883, "value3":true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.927974, "value1":0.17895618154031318, "value2":547646730434307817, "value3":false, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.303832, "value1":0.5511007572679634, "value2":2862350870063686751, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520504, "value1":0.39724544886891017, "value2":6931266301104819935, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941095, "value1":0.14786172741452286, "value2":3205668266543533832, "value3":false, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656997, "value1":0.32075936991205234, "value2":7391478133987747835, "value3":false, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120921, "value1":0.3939864396610821, "value2":2909689659050189847, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958790, "value1":0.3269314306247572, "value2":6783238272107071770, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.386136, "value1":0.861947664801586, "value2":5805659227499250652, "value3":true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.449365, "value1":0.5799629274092946, "value2":9193070830904141987, "value3":true, "__name__":"metric_00000581", "key_3":"e","key_9":"h","key_2":"f", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.785656, "value1":0.3165816948400473, "value2":3116393083198778433, "value3":true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520711, "value1":0.6656585396817016, "value2":1017823568348313071, "value3":true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.650203, "value1":0.22215575308437716, "value2":1902945447598147226, "value3":false, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249002, "value1":0.2961338302748422, "value2":1415432970559418672, "value3":true, "__name__":"metric_00000586", "key_5":"k","key_7":"f","key_1":"e", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694015, "value1":0.7506751146797324, "value2":6142650591148786794, "value3":false, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.304466, "value1":0.37747841635497736, "value2":5375036774405769501, "value3":false, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.641004, "value1":0.46446471911344284, "value2":5955856288590241375, "value3":false, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.684973, "value1":0.907579891922439, "value2":5346005034595738366, "value3":true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.111153, "value1":0.49826719629056415, "value2":7497650141562337241, "value3":true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830722, "value1":0.28960521156096847, "value2":6154503575103623089, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392721, "value1":0.6834318849474948, "value2":2628637152649849276, "value3":true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969704, "value1":0.15429907011430974, "value2":5405475101757554186, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536252, "value1":0.7000830425153579, "value2":406966410660924195, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.807572, "value1":0.2012419938894414, "value2":4358282132524848204, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.820107, "value1":0.5076022682087846, "value2":4017461578732969062, "value3":false, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996611, "value1":0.7422015873263831, "value2":436292068764222211, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.922123, "value1":0.051072363737232486, "value2":268646986926660655, "value3":false, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.729895, "value1":0.8676962023633001, "value2":2065960180781249750, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636047, "value1":0.4996316174329004, "value2":3156670443682802457, "value3":false, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431520, "value1":0.5204212362312931, "value2":8019726224533633908, "value3":true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.903149, "value1":0.5859452724568418, "value2":2288060944551638278, "value3":false, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.070771, "value1":0.9932429756726906, "value2":6690408728176272233, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.314453, "value1":0.3472880048665214, "value2":58548757977887750, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627243, "value1":0.7139819549630214, "value2":7080915148891558502, "value3":false, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755294, "value1":0.05408051807422248, "value2":6931097273017939069, "value3":false, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282331, "value1":0.3203397893756164, "value2":8256062300842673383, "value3":false, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.034396, "value1":0.9531256831279978, "value2":7765396458031448915, "value3":false, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.115811, "value1":0.13116244832703935, "value2":3543488806097224100, "value3":false, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666851, "value1":0.6887850450312126, "value2":5462261380989469479, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525431, "value1":0.3139759113605816, "value2":8160395548303928540, "value3":true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036488, "value1":0.7437550776292858, "value2":905546975072552710, "value3":true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246868, "value1":0.7173148070188747, "value2":7273545640281703287, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840343, "value1":0.20321210783475557, "value2":336636579624758589, "value3":true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100735, "value1":0.8393194019944642, "value2":291716047034557387, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853981, "value1":0.5202422200638643, "value2":5204943321202939039, "value3":false, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512132, "value1":0.018314329801792806, "value2":961979132289563000, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487612, "value1":0.7033581894769818, "value2":4186345519226619201, "value3":true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778169, "value1":0.7839413953257911, "value2":7874569661704504858, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.587363, "value1":0.24066050930994098, "value2":2238921997040090404, "value3":false, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.863823, "value1":0.22725313032865457, "value2":4922946319365439720, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.462195, "value1":0.6013044688497777, "value2":137484525088955387, "value3":false, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.967535, "value1":0.21528129199568857, "value2":5907496598149934999, "value3":false, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263462, "value1":0.3398984210102334, "value2":3627872884296903263, "value3":false, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778010, "value1":0.2649025290720317, "value2":2665113842043817940, "value3":true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895866, "value1":0.8505969892692369, "value2":1773042500322059709, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475234, "value1":0.08631156102550058, "value2":519973765404949203, "value3":false, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996762, "value1":0.6378946736901862, "value2":9195196376217348512, "value3":false, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537168, "value1":0.2006763700535789, "value2":9067766312454029883, "value3":true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.585179, "value1":0.8887141585928169, "value2":1447763822787404407, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937668, "value1":0.18798426398408305, "value2":5531072459849624180, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417573, "value1":0.018969474714960195, "value2":8739232693787930523, "value3":true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432784, "value1":0.6772697684929758, "value2":3670574248400915486, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945707, "value1":0.2017640539145947, "value2":6866599884426415644, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715992, "value1":0.2770810284692845, "value2":6065196798538107388, "value3":false, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724512, "value1":0.7694937233145257, "value2":5461354547379838063, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555480, "value1":0.4116895522673444, "value2":6999095526200651536, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357696, "value1":0.9694808723879291, "value2":2308259417403012034, "value3":true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.925321, "value1":0.9804626384591183, "value2":1344780667969714366, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340972, "value1":0.8564397089342566, "value2":3335346544187347307, "value3":true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634021, "value1":0.2824134389967874, "value2":827846054793909627, "value3":true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535849, "value1":0.8383622181777869, "value2":3210193933415043724, "value3":false, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949231, "value1":0.3423558587998688, "value2":8018588983650318493, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.604636, "value1":0.6545573982070936, "value2":8784079580099061471, "value3":false, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041080, "value1":0.6815494814472329, "value2":2413918152267969152, "value3":true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206623, "value1":0.7407991857310414, "value2":8688412282907699410, "value3":false, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697192, "value1":0.9920334806937902, "value2":4058886921829417026, "value3":true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748394, "value1":0.3435945058384803, "value2":6002972956903494678, "value3":true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421611, "value1":0.8151473391165547, "value2":261810376504255149, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.027674, "value1":0.32343755952478886, "value2":8467224123375485048, "value3":true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205103, "value1":0.26583420276460434, "value2":912318083515976688, "value3":false, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362886, "value1":0.6918507204150298, "value2":3042551716463426665, "value3":false, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181414, "value1":0.9588817871484018, "value2":5500996639169097947, "value3":true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607786, "value1":0.1555201931458164, "value2":3667544046344919475, "value3":false, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447991, "value1":0.6322575842993106, "value2":6581489005452708007, "value3":true, "__name__":"metric_00000656", "key_6":"k","key_7":"k","key_4":"f", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602539, "value1":0.3803193364932477, "value2":501846031619014460, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224149, "value1":0.14287189157740537, "value2":4390766248301994532, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019349, "value1":0.9275903306872896, "value2":6065489708075344215, "value3":true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.753905, "value1":0.04103970388034392, "value2":3647389822252762296, "value3":true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.808552, "value1":0.6912948158348138, "value2":7168224379303111769, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.383522, "value1":0.7213839418172744, "value2":8917533656021329315, "value3":false, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988681, "value1":0.4652567645908903, "value2":6088381044289659824, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426511, "value1":0.8194664026578082, "value2":3767723603035206676, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291449, "value1":0.11519619446576408, "value2":7042710433748403097, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.495051, "value1":0.7541019265512129, "value2":4805842413127851386, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.498957, "value1":0.7657156651926107, "value2":4284045357936772257, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.920671, "value1":0.557277837350798, "value2":7618404853124250629, "value3":false, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.325130, "value1":0.8555228162118138, "value2":5568850179022711136, "value3":false, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619588, "value1":0.8294092067729972, "value2":1498942661428438729, "value3":true, "__name__":"metric_00000669", "key_5":"e","key_9":"d","key_4":"c", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932410, "value1":0.9716095495296341, "value2":6631794618463446087, "value3":false, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.257618, "value1":0.6931656613099204, "value2":4999381616225061323, "value3":false, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.328148, "value1":0.3114535269637382, "value2":7689977371490904120, "value3":false, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.280989, "value1":0.9111540107476489, "value2":2001844404871692409, "value3":true, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722254, "value1":0.17067710980514048, "value2":9024628709036843159, "value3":true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367790, "value1":0.7784645051385055, "value2":4706281866949049190, "value3":true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368237, "value1":0.6741959290593132, "value2":2438020324428284656, "value3":true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885688, "value1":0.4581933301264591, "value2":977306309852850973, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.856287, "value1":0.24855090559153642, "value2":5403228793051986733, "value3":false, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.305675, "value1":0.6170232598989743, "value2":274658269233518106, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908698, "value1":0.5064010761390361, "value2":5817465185742290417, "value3":true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757988, "value1":0.04843336165427696, "value2":5801707099278996481, "value3":false, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.933565, "value1":0.22461787050531695, "value2":6104462847804923186, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.720615, "value1":0.12587001891222954, "value2":7755893877648671182, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642145, "value1":0.4752800242516813, "value2":1097468219850109153, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975649, "value1":0.9950222857947861, "value2":323860279126996239, "value3":false, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.571701, "value1":0.13176848875241834, "value2":8974250013315108425, "value3":false, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841017, "value1":0.5108584547409708, "value2":1015654371149607255, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.499539, "value1":0.7323245432905996, "value2":866168553464820252, "value3":false, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.137678, "value1":0.9837633616038727, "value2":1980815750999371252, "value3":true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713500, "value1":0.6844141285754438, "value2":5215363796900492290, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564182, "value1":0.6959446753092476, "value2":2124568431454851658, "value3":true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.548476, "value1":0.8904291651425387, "value2":8960234619080614404, "value3":true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.334127, "value1":0.5777624482637982, "value2":3890652700139772457, "value3":true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.141095, "value1":0.3019838758248612, "value2":1159300889479734121, "value3":true, "__name__":"metric_00000686", "key_8":"g","key_1":"d","key_4":"a", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800239, "value1":0.4131750973298026, "value2":2344128018741531119, "value3":true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369249, "value1":0.1589896462920189, "value2":4160545898250072075, "value3":false, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.198713, "value1":0.08298578373908365, "value2":50980726393447907, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.773734, "value1":0.7496268790368541, "value2":7804146534404388324, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237426, "value1":0.27543463085841935, "value2":3694238253557813150, "value3":false, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577321, "value1":0.6586006530161179, "value2":641077213271202791, "value3":false, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368897, "value1":0.6785289242869411, "value2":2009007121053273154, "value3":true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.801586, "value1":0.7989756868311243, "value2":3470745118950983568, "value3":true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.035889, "value1":0.4089400550263047, "value2":3320771957585601823, "value3":false, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760511, "value1":0.7386595086100225, "value2":3332011099556778058, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.359596, "value1":0.8203562587923371, "value2":574506838747954548, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063436, "value1":0.2579978917934866, "value2":6993045686428025271, "value3":true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369331, "value1":0.4769488047439803, "value2":3318100445225134934, "value3":false, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795599, "value1":0.9012136284147806, "value2":5770204186306544532, "value3":false, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.219070, "value1":0.20974721576307698, "value2":2356228020734101377, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537221, "value1":0.1725299480275798, "value2":5073957380468101443, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015458, "value1":0.4025573823516011, "value2":3282394548862748845, "value3":true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580078, "value1":0.9823965911617399, "value2":8476500551084420415, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.979266, "value1":0.5236855845499949, "value2":680393934209420766, "value3":true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682327, "value1":0.844770708190905, "value2":8298068650764370173, "value3":false, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350010, "value1":0.5806359147839931, "value2":1527219564263667157, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206279, "value1":0.4978598735396454, "value2":8931241509064706089, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135137, "value1":0.7138645301142236, "value2":2633600346218814000, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.981064, "value1":0.7353726130780065, "value2":268395459435449482, "value3":false, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.526357, "value1":0.3584205029962212, "value2":7926609514114360701, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145680, "value1":0.817655724537724, "value2":2809466879616878308, "value3":false, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.877503, "value1":0.17147641272949588, "value2":1475184649319875768, "value3":true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049475, "value1":0.3904708182990629, "value2":8178189937335677971, "value3":true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419221, "value1":0.2498991121797316, "value2":2019493757980421565, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646852, "value1":0.2981893785163364, "value2":4127223489573331187, "value3":true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194796, "value1":0.9706301430727536, "value2":3907775771119307356, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135232, "value1":0.952467511486142, "value2":6824235396463011996, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392387, "value1":0.19563387556471776, "value2":6531634041022343859, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.133702, "value1":0.7965289882627238, "value2":6086508658030257940, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.691192, "value1":0.21849926406658296, "value2":2500546592704430602, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238250, "value1":0.8547288670113503, "value2":1751658935211452093, "value3":true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.259613, "value1":0.7399127039208734, "value2":3758431842842501346, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.959851, "value1":0.06348104065773419, "value2":5803029266892181407, "value3":false, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350885, "value1":0.07399715612950311, "value2":9184522919237034506, "value3":false, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821915, "value1":0.9143789840874811, "value2":3150076904171453287, "value3":false, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.554775, "value1":0.406185744411068, "value2":4390857727102845864, "value3":false, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.026241, "value1":0.576960195810922, "value2":864775792792522430, "value3":true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118029, "value1":0.6740072102143805, "value2":2738633295377346249, "value3":false, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.669898, "value1":0.6063573387513779, "value2":8118600251510115903, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406027, "value1":0.5408212514385392, "value2":836959541970766944, "value3":false, "__name__":"metric_00000740", "key_3":"j","key_0":"f","key_2":"b", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205752, "value1":0.9617267357194531, "value2":2302570688726041304, "value3":true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.683346, "value1":0.7764943186065553, "value2":1516613683034572821, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535753, "value1":0.8501812602308251, "value2":3781935852497505404, "value3":false, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562975, "value1":0.5762183849456854, "value2":8803839397694965730, "value3":true, "__name__":"metric_00000744", "key_9":"j","key_1":"c","key_8":"i", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224509, "value1":0.6945846689247714, "value2":8790065402136541035, "value3":true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120301, "value1":0.5693297769803048, "value2":8817337009559896419, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535111, "value1":0.7168638232585879, "value2":1459446787633025168, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590881, "value1":0.052466731276145914, "value2":6984133752001938547, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800340, "value1":0.19161367226547096, "value2":8470840993920678933, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797252, "value1":0.22382372006161758, "value2":960932677896506756, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502545, "value1":0.4567752766501535, "value2":9200307528794857259, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.919995, "value1":0.3074462546242729, "value2":1729230620360441809, "value3":true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.273301, "value1":0.5542454650469373, "value2":2007525431527262950, "value3":false, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249396, "value1":0.30289020495328595, "value2":7647941374943306916, "value3":false, "__name__":"metric_00000754", "key_5":"i","key_8":"h","key_1":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847062, "value1":0.8617179899648977, "value2":2309423222399622901, "value3":false, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.770003, "value1":0.8185102377578417, "value2":8900535665947339098, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612076, "value1":0.2520488653737615, "value2":534199497930559417, "value3":false, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867000, "value1":0.3056461531330615, "value2":8074358922929649800, "value3":true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543352, "value1":0.1267819654500433, "value2":2282337611780682090, "value3":false, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771337, "value1":0.3860477352422603, "value2":6614019503509668230, "value3":false, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724939, "value1":0.35759761476037216, "value2":7730807476801277254, "value3":false, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.769189, "value1":0.06006911990731757, "value2":6135075757488436084, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.750837, "value1":0.22593301727381723, "value2":815096043795563480, "value3":false, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.733823, "value1":0.37434887115879495, "value2":8767359811379324612, "value3":true, "__name__":"metric_00000761", "key_7":"b","key_4":"g","key_5":"a", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110462, "value1":0.7613769242381963, "value2":679247224492133418, "value3":false, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.207493, "value1":0.26270378513285625, "value2":6127960652597929191, "value3":true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757551, "value1":0.17246010749021554, "value2":3746123411387113183, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476665, "value1":0.530111421321598, "value2":7929787788154212077, "value3":true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080232, "value1":0.24610687758058403, "value2":4382073893358318099, "value3":false, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.518795, "value1":0.13825391880428892, "value2":6023983814814871189, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.411487, "value1":0.7443212678339678, "value2":3163265105788950613, "value3":true, "__name__":"metric_00000763", "key_8":"g","key_4":"b","key_6":"k", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230729, "value1":0.6799575270403697, "value2":6569848391065499405, "value3":false, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.251807, "value1":0.23368831551353111, "value2":9121098592064787356, "value3":true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450371, "value1":0.8049793990951134, "value2":6271606339009618863, "value3":true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268506, "value1":0.7393978258409317, "value2":3084814221880218023, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453066, "value1":0.7167056491982889, "value2":8359177712079217325, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840677, "value1":0.3999771937764131, "value2":5907070339651841877, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853610, "value1":0.4459636741761099, "value2":2591090489482088505, "value3":true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.050602, "value1":0.9347582889921432, "value2":3586868729035531903, "value3":true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.460673, "value1":0.8573683555697056, "value2":77436103148660020, "value3":false, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.791642, "value1":0.7214035249481627, "value2":4712458830503182070, "value3":true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711339, "value1":0.7704360355595858, "value2":4818602735855589639, "value3":true, "__name__":"metric_00000774", "key_8":"e","key_2":"g","key_4":"c", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709838, "value1":0.980850892357579, "value2":5814687934592979738, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448259, "value1":0.6911929910350553, "value2":2331566896810690473, "value3":false, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069070, "value1":0.7715792084831908, "value2":1237753144790060173, "value3":false, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728905, "value1":0.6453677424318518, "value2":4827712214727464044, "value3":false, "__name__":"metric_00000783", "key_8":"e","key_9":"b","key_7":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.581880, "value1":0.2677360178055146, "value2":8844278477287710309, "value3":true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969799, "value1":0.5783967942752313, "value2":5905445599124832757, "value3":false, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741372, "value1":0.012956038256400619, "value2":914084143172531236, "value3":true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862619, "value1":0.5947373167351414, "value2":3630517863485151782, "value3":true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.139536, "value1":0.8372408825864636, "value2":6952590048486505005, "value3":false, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803976, "value1":0.43722974429572603, "value2":2176830109855416805, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.534405, "value1":0.7241541542820893, "value2":4511693404596905586, "value3":true, "__name__":"metric_00000789", "key_6":"d","key_5":"a", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574402, "value1":0.7055278131957177, "value2":1238394241461479434, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.06494326977178205, "value2":3563720913525659633, "value3":true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878052, "value1":0.8371531564279497, "value2":587508487226380908, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.672424, "value1":0.26126599404383394, "value2":3460643419757907518, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408361, "value1":0.5841391902007207, "value2":8050508010125067469, "value3":true, "__name__":"metric_00000775", "key_6":"a","key_0":"e", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.743507, "value1":0.5292956990399417, "value2":5743834222350004253, "value3":true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.024743, "value1":0.5111340297086123, "value2":4739422872896352154, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053127, "value1":0.41791773967804086, "value2":5388222563610877961, "value3":false, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489667, "value1":0.6344084078946665, "value2":6127655871996983541, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.876031, "value1":0.49328901667737785, "value2":3143559155074364531, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.278465, "value1":0.23241882776321474, "value2":805438734033729133, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380252, "value1":0.6353489769585325, "value2":1619270428062881618, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512159, "value1":0.8152126527698539, "value2":1368040827327524990, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.317649, "value1":0.9550761539015656, "value2":8914758126172814906, "value3":false, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.723807, "value1":0.7854700255052053, "value2":7095402949367847209, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.660856, "value1":0.4053530100702817, "value2":2145190666873464855, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.659680, "value1":0.12646331821035406, "value2":1576238191679234703, "value3":false, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.993554, "value1":0.8499325393155558, "value2":5900620451042065379, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091338, "value1":0.41503145937749936, "value2":2042239402577329647, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.299870, "value1":0.687218145400586, "value2":7560464390267142060, "value3":false, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.731227, "value1":0.15009967640134209, "value2":1886018024243215805, "value3":false, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.348963, "value1":0.6737669308977404, "value2":5404336119559355867, "value3":true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162168, "value1":0.813527799284827, "value2":4016392262793002099, "value3":true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086162, "value1":0.7399808700194177, "value2":8264192680187392458, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975595, "value1":0.11433087569708594, "value2":4671281305443756113, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799363, "value1":0.3067546199025452, "value2":225546722481809384, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.113667, "value1":0.6424657658728049, "value2":8718201548571703569, "value3":false, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904642, "value1":0.5465983180036486, "value2":4877073522652743329, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.467203, "value1":0.8662610389904551, "value2":399387690820391383, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082862, "value1":0.5088605240671744, "value2":1531739105872671512, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.039708, "value1":0.10189606169579954, "value2":5185895909677369244, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068495, "value1":0.2805843145287995, "value2":486469505433183294, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357082, "value1":0.75740796150096, "value2":2690075476895328471, "value3":false, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492715, "value1":0.3004718942084548, "value2":7383744941741065497, "value3":false, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.968147, "value1":0.5004304919533921, "value2":1731582924085558465, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.991776, "value1":0.7394075904771168, "value2":4141740040514370033, "value3":false, "__name__":"metric_00000829", "key_4":"k","key_6":"a","key_3":"i", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420254, "value1":0.45767543208547934, "value2":1964890506496503777, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285702, "value1":0.1366965878727093, "value2":2681549871870198885, "value3":true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492204, "value1":0.9414045318691766, "value2":1575036830396652846, "value3":true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086526, "value1":0.5271825907095232, "value2":3742001028067515529, "value3":false, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.296194, "value1":0.6916437451586811, "value2":384054583246095269, "value3":true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478834, "value1":0.5030459095566234, "value2":3918028956922111738, "value3":false, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.029769, "value1":0.2164261260808611, "value2":4102211333190644591, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102553, "value1":0.42618668443550417, "value2":7568518633836620405, "value3":true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.681434, "value1":0.7225843033115362, "value2":8160060706717748819, "value3":true, "__name__":"metric_00000838", "key_9":"b","key_5":"j","key_6":"h", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210135, "value1":0.21812369739137172, "value2":5702739804858144278, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.786060, "value1":0.33904050622912296, "value2":1017560684015420231, "value3":true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558541, "value1":0.14437656270299287, "value2":7701298697010262063, "value3":false, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.818721, "value1":0.15017675533914024, "value2":7186230665665602451, "value3":true, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910629, "value1":0.09672885609565206, "value2":6203887741849145968, "value3":true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069131, "value1":0.6075489323724108, "value2":9025102416378601029, "value3":false, "__name__":"metric_00000813", "key_8":"h","key_1":"j","key_7":"g", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362866, "value1":0.8018878313034695, "value2":8037732655719146713, "value3":false, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682018, "value1":0.5433657256031907, "value2":6040793575397278438, "value3":false, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797323, "value1":0.5032038038329079, "value2":3803498450979968327, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.829279, "value1":0.605237212493161, "value2":4367127372886022859, "value3":false, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.999162, "value1":0.5082854850214861, "value2":5031512432939201812, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269625, "value1":0.0007161270371379666, "value2":6545591527308584857, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180859, "value1":0.6040920298819816, "value2":8926725137631299725, "value3":false, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090602, "value1":0.5326464117623615, "value2":7752649715252147157, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.191121, "value1":0.12836456615806674, "value2":7198874487475192980, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.686863, "value1":0.708342981200594, "value2":5465607373577957408, "value3":false, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.527995, "value1":0.10447991595171255, "value2":5974156320762011232, "value3":true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.123930, "value1":0.6200031138885586, "value2":5406077264893957847, "value3":true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.356501, "value1":0.44396993176396354, "value2":6255319076363219685, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.048169, "value1":0.08153644488046811, "value2":8582968949378456970, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.112743, "value1":0.6933778275610673, "value2":4587548588265206284, "value3":false, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323148, "value1":0.20142008973704545, "value2":4527540178335323769, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.934744, "value1":0.08093337102458979, "value2":7857820814429250360, "value3":false, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962496, "value1":0.6124995000870105, "value2":6489203228482067372, "value3":true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.160517, "value1":0.5248710269553188, "value2":5026165991297404062, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932200, "value1":0.922378624964946, "value2":7930339398930216623, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.900245, "value1":0.1182228551739615, "value2":1373671224306855417, "value3":true, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790920, "value1":0.31880287032595506, "value2":6084131296995295782, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237291, "value1":0.13557228914344616, "value2":4432801334887160283, "value3":false, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.914465, "value1":0.02040606137133301, "value2":1808915457173232015, "value3":false, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941296, "value1":0.9439262294487866, "value2":6574505025862591303, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.822459, "value1":0.9942489150539108, "value2":3506354754330173943, "value3":true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239733, "value1":0.4989511485969244, "value2":7173139334014716167, "value3":false, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.444066, "value1":0.7594335589002584, "value2":3216271266780228569, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866271, "value1":0.5931978644398052, "value2":3444180839017636171, "value3":false, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421087, "value1":0.6530151828739431, "value2":19805620992839207, "value3":false, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771459, "value1":0.8073198535663951, "value2":4810887667402700825, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832482, "value1":0.734748250215754, "value2":1123934068717543450, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450110, "value1":0.15409548849312346, "value2":418445000125934927, "value3":true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.234597, "value1":0.3684594320773471, "value2":3167333011558865991, "value3":true, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.403637, "value1":0.891873955682516, "value2":6523321617886226804, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017496, "value1":0.5380231504237305, "value2":1565703384693902706, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577818, "value1":0.942234653476486, "value2":6854337928979531257, "value3":false, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886845, "value1":0.561600684880549, "value2":6700870407358100744, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.493299, "value1":0.1842060640300563, "value2":7660717134967306529, "value3":true, "__name__":"metric_00000872", "key_8":"j","key_5":"i","key_6":"b", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043840, "value1":0.2702623264568812, "value2":7040356564551372493, "value3":true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051963, "value1":0.5871637139837996, "value2":4534754735986232109, "value3":false, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.568830, "value1":0.08173891497184316, "value2":408699021018747548, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694578, "value1":0.8818619724910839, "value2":5720963057839829442, "value3":true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095000, "value1":0.9081401378248263, "value2":8656658880665915661, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977416, "value1":0.9899458292085819, "value2":665964700424941220, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.670945, "value1":0.6733933124715158, "value2":2460152217631000428, "value3":false, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636539, "value1":0.6931186653224365, "value2":7451786524160575556, "value3":true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.398486, "value1":0.8808674761773307, "value2":6001178466503449076, "value3":true, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.085292, "value1":0.9258669929119259, "value2":4480969611449139348, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451321, "value1":0.8902429562501767, "value2":2399440975743970404, "value3":false, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100110, "value1":0.4539977466656097, "value2":4906418767346623646, "value3":true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.725438, "value1":0.8779089763048636, "value2":7986118613837183355, "value3":true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.559619, "value1":0.42482197030409785, "value2":921145061767776729, "value3":true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885834, "value1":0.32962036175845005, "value2":5608652380611657042, "value3":true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619049, "value1":0.4697848554943135, "value2":6930715352479440739, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768456, "value1":0.55002846829011, "value2":6856343919804361537, "value3":true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.052488, "value1":0.9193313692023829, "value2":8075338150518594907, "value3":false, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.346748, "value1":0.7969202775153313, "value2":3854682510572365821, "value3":true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.668529, "value1":0.8901520814855493, "value2":3780299539456604836, "value3":true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.680780, "value1":0.9128377273315654, "value2":655380872616480476, "value3":false, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.298499, "value1":0.8901980009691262, "value2":4124662751920860689, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.061822, "value1":0.922218588720657, "value2":6935461410888938736, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.220215, "value1":0.8707922044422206, "value2":6415853705476229931, "value3":false, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825872, "value1":0.6145357419122679, "value2":3712577567679334743, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287572, "value1":0.7153750220855114, "value2":6766565708182723447, "value3":true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.258723, "value1":0.8623529521546892, "value2":3906675258214843555, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417686, "value1":0.6071420867010977, "value2":8210404273898330224, "value3":false, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.939674, "value1":0.018125841559591738, "value2":2686797765813242922, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062829, "value1":0.19220806032635843, "value2":4009115073519468001, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.761409, "value1":0.282044445283748, "value2":2659364730071661456, "value3":true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340677, "value1":0.7867304449409817, "value2":4726626950116268051, "value3":true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.848210, "value1":0.7451256645145572, "value2":6540697575244319830, "value3":false, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082209, "value1":0.08477415388397415, "value2":5473839595744351618, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562389, "value1":0.08808192976810916, "value2":1578232710018420800, "value3":true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445878, "value1":0.7573342205320885, "value2":3400644244684606362, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.603775, "value1":0.183625034999939, "value2":7793796792648526034, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036048, "value1":0.8359424936170327, "value2":5865912253243965224, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866722, "value1":0.3949434009829113, "value2":3290170127727456353, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697995, "value1":0.8423729542207233, "value2":4861550929219709907, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.211200, "value1":0.36512924472577757, "value2":2972205895387325421, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227557, "value1":0.22341834017020207, "value2":4806018666421767946, "value3":false, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574952, "value1":0.6007129713703638, "value2":3849253119017333239, "value3":false, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741721, "value1":0.5702349573898813, "value2":897876839681174700, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379915, "value1":0.6564162460066056, "value2":3131015984288737246, "value3":true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.229369, "value1":0.456646560158155, "value2":4778476677749084799, "value3":false, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.242697, "value1":0.4142421381714439, "value2":7101584169529051655, "value3":false, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.212755, "value1":0.39066964505760526, "value2":4253776777664568253, "value3":false, "__name__":"metric_00000922", "key_7":"h","key_9":"k","key_5":"b", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837272, "value1":0.15824518985564762, "value2":6812701504471384455, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.104360, "value1":0.9940983043766688, "value2":2203777894935310413, "value3":true, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391080, "value1":0.5579038729371453, "value2":6131596744931664143, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265096, "value1":0.7511430046090084, "value2":4283281105409640096, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881500, "value1":0.8197410181139487, "value2":2167116001911138380, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964911, "value1":0.6839484284574476, "value2":2392821775915043629, "value3":false, "__name__":"metric_00000936", "key_4":"f","key_9":"c","key_0":"j", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906639, "value1":0.12232760875324074, "value2":9162174110871211714, "value3":true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562264, "value1":0.9910780181559037, "value2":1543791401715243110, "value3":true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016345, "value1":0.3277590263870748, "value2":3851358761838506924, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963516, "value1":0.13686033235694836, "value2":2786944428195217311, "value3":false, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.446494, "value1":0.051498079678560585, "value2":1973784887996536653, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350210, "value1":0.13860925037657423, "value2":1354426546825223222, "value3":true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358378, "value1":0.28813722807183184, "value2":4138306657967174744, "value3":false, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455527, "value1":0.9787455514136848, "value2":6829598111029855692, "value3":false, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.208210, "value1":0.4291478612877578, "value2":749843861604187954, "value3":true, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.330867, "value1":0.6408478280255169, "value2":8395361043273182125, "value3":true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.176999, "value1":0.9117058417477248, "value2":8428888804090519109, "value3":false, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091618, "value1":0.8122911740955318, "value2":1478086552909570392, "value3":true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243584, "value1":0.06727976096288107, "value2":4178958373440436949, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.127002, "value1":0.8012772426876036, "value2":2149729620077994596, "value3":true, "__name__":"metric_00000923", "key_5":"j","key_6":"e","key_3":"h", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461395, "value1":0.21653382210662625, "value2":2558734709934645897, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932379, "value1":0.43744123497894677, "value2":5779172669858974850, "value3":false, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180562, "value1":0.6091002318007619, "value2":2233652952119987641, "value3":true, "__name__":"metric_00000925", "key_3":"h","key_9":"c","key_0":"e", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.738334, "value1":0.8147115963605253, "value2":1038206878786066414, "value3":true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943138, "value1":0.1805265868912326, "value2":8231024461542686979, "value3":false, "__name__":"metric_00000931", "key_7":"g","key_0":"j", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007238, "value1":0.30166077327328183, "value2":5211541986455490237, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926840, "value1":0.7197646392974351, "value2":6170887305282807365, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.976392, "value1":0.02561261869316712, "value2":8003552327575750503, "value3":false, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665296, "value1":0.38463860662667215, "value2":702575998172013860, "value3":false, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.957627, "value1":0.8365108478779814, "value2":3172439569974822855, "value3":true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.779959, "value1":0.005457784803842251, "value2":1217373846044030405, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814440, "value1":0.03586677782979768, "value2":8668868565483927092, "value3":false, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590538, "value1":0.7998407978822464, "value2":5953384113011158992, "value3":true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.381146, "value1":0.6861901618066985, "value2":806723358919392729, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.324953, "value1":0.09051509245266014, "value2":930835793453842933, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.309761, "value1":0.44326721766264904, "value2":887788347593465760, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722069, "value1":0.08849038862609183, "value2":7352758650578649855, "value3":false, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558827, "value1":0.3337734956738152, "value2":8685182570627965041, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715649, "value1":0.23109099835442082, "value2":1083379478056182665, "value3":true, "__name__":"metric_00000961", "key_9":"h","key_7":"b","key_8":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964540, "value1":0.6330159342037405, "value2":1167652807667781397, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795126, "value1":0.5778697926785736, "value2":9140994998625427103, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236481, "value1":0.5802899488963301, "value2":2895083535971839167, "value3":true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996259, "value1":0.2785697344481934, "value2":4913023416457428502, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.477938, "value1":0.484568023326054, "value2":3495500670092878690, "value3":true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261065, "value1":0.11116858735231369, "value2":5869730879322934574, "value3":false, "__name__":"metric_00000966", "key_2":"f","key_1":"a", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185846, "value1":0.7126003929744608, "value2":3434450920636329132, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841676, "value1":0.3736127841617722, "value2":7802733756976416976, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350997, "value1":0.582318689502649, "value2":372190874054145543, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555394, "value1":0.12831996488184216, "value2":4582628444562419674, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.155140, "value1":0.3906336986690914, "value2":8237405956495459279, "value3":true, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367910, "value1":0.741845875474543, "value2":8082126523839106878, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.950768, "value1":0.6652471104213182, "value2":8268690822588065094, "value3":true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.433269, "value1":0.7980991574588373, "value2":8513183309513339766, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.874369, "value1":0.5464494520072651, "value2":5885072115794893058, "value3":false, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537599, "value1":0.7405427933790399, "value2":6635799236257663587, "value3":true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.955763, "value1":0.8778283555530513, "value2":6592488434360400176, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.582015, "value1":0.879034635958457, "value2":7219576376735925925, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388984, "value1":0.53843340060884, "value2":2254745170299712210, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646131, "value1":0.5101339977981793, "value2":3292039546456501555, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033442, "value1":0.6456192738711094, "value2":4906576027749942202, "value3":false, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.836645, "value1":0.48635849410277077, "value2":1404791026873124424, "value3":false, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464295, "value1":0.5617144285499786, "value2":7990958033367681704, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010923, "value1":0.9807071857311617, "value2":5683359887238098850, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438005, "value1":0.5935141575872156, "value2":4865067817966075444, "value3":true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227310, "value1":0.6681961275740043, "value2":8551103550465421536, "value3":false, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186394, "value1":0.25048622317521113, "value2":2888181774636358576, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053227, "value1":0.10049698552895067, "value2":4557545020451680363, "value3":true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621258, "value1":0.9388516977567649, "value2":4055003743201350923, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227442, "value1":0.7022685194913532, "value2":3740668486633940756, "value3":false, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640300, "value1":0.06434066744840804, "value2":1646848528935407928, "value3":false, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162609, "value1":0.71410216780028, "value2":7370378990581242781, "value3":true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512792, "value1":0.19345496541059937, "value2":4424425935560475985, "value3":true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203787, "value1":0.9476247108925144, "value2":264480497401916863, "value3":true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.076408, "value1":0.13927961349104323, "value2":1149972870879073163, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806676, "value1":0.6172364818942355, "value2":4996985075367830737, "value3":false, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.313484, "value1":0.6715999817727264, "value2":5237271908928184120, "value3":false, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947617, "value1":0.1400224644800571, "value2":4212016629330280660, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.695969, "value1":0.13461968177274677, "value2":8209146459721882476, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466156, "value1":0.7851823238051198, "value2":4609596535178738402, "value3":false, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047192, "value1":0.6615735622970099, "value2":3688971736191236071, "value3":true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179244, "value1":0.5525116533205023, "value2":6958696123355987767, "value3":false, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.836535, "value1":0.8752088579621973, "value2":1772355477524999885, "value3":false, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.385661, "value1":0.6669392703121872, "value2":5026611864593335422, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.122798, "value1":0.5624470316374681, "value2":973613507611825241, "value3":false, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663323, "value1":0.2836760055083927, "value2":8274369123408107345, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988003, "value1":0.07587743066420682, "value2":4947294410032829574, "value3":false, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.240844, "value1":0.4161174869019418, "value2":4077015391953706312, "value3":false, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.945230, "value1":0.14944194749229664, "value2":7660491006096000021, "value3":true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890111, "value1":0.20296484407777726, "value2":1824051875710962915, "value3":true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522740, "value1":0.021855326277136235, "value2":3218235817889269142, "value3":true, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.679843, "value1":0.5055217891288131, "value2":4822706751533920431, "value3":false, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191340, "value1":0.9959364873278129, "value2":3932069232230299078, "value3":false, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771462, "value1":0.4557053136591391, "value2":184726250007256859, "value3":false, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581655, "value1":0.689124829253459, "value2":1817463261736430161, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.894089, "value1":0.9069576891707435, "value2":7318133204837024663, "value3":false, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.307941, "value1":0.4227283465946924, "value2":356860997794924081, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085219, "value1":0.7548908197263309, "value2":2736908468355530081, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510864, "value1":0.6045243894505045, "value2":3203823484629938988, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194038, "value1":0.2945791412290687, "value2":8542013145729544847, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.822784, "value1":0.7046664475635507, "value2":6531211714773873879, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958034, "value1":0.6210073688384752, "value2":5171125204919187937, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245514, "value1":0.07787807249020082, "value2":4796876093300411441, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922409, "value1":0.026305095785565266, "value2":4204774996273721582, "value3":false, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618949, "value1":0.8021374519845728, "value2":7055530836155104119, "value3":true, "__name__":"metric_00000026", "key_2":"c","key_7":"d","key_1":"f", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.842155, "value1":0.9078167657864898, "value2":659406508816846003, "value3":false, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.429827, "value1":0.6861191669827441, "value2":3716781946259346478, "value3":true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.721875, "value1":0.7721427864001064, "value2":1559707715553693565, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.667085, "value1":0.41573209008012846, "value2":770484632734171477, "value3":false, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567897, "value1":0.2688265887173671, "value2":8731672977339492909, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716882, "value1":0.20533155732399708, "value2":8158121924487423466, "value3":true, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.450278, "value1":0.45425248453866135, "value2":7472174068145252016, "value3":false, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700766, "value1":0.38900719316872584, "value2":9070338695929499459, "value3":true, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296131, "value1":0.7588298197955197, "value2":1078200536047373074, "value3":true, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644620, "value1":0.22063287558750297, "value2":1455208095653529502, "value3":true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826663, "value1":0.9059127191998474, "value2":163331002989197798, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.375920, "value1":0.888489428778294, "value2":3478637008435710026, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795211, "value1":0.7169431218180555, "value2":5240161460250645540, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686886, "value1":0.1615738416501897, "value2":4522663398730909152, "value3":true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.641311, "value1":0.5919216049468233, "value2":4264524771451405531, "value3":true, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415500, "value1":0.8013193758641248, "value2":4655946330615841665, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310579, "value1":0.03464731335525151, "value2":2648033740620709930, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.135452, "value1":0.663217728481436, "value2":6656249157262608643, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036458, "value1":0.7592021704654797, "value2":4708550885998497867, "value3":true, "__name__":"metric_00000054", "key_7":"j","key_3":"k","key_6":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196106, "value1":0.9109556832409251, "value2":1182178953302502536, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897060, "value1":0.30582754840738535, "value2":5110613128180106106, "value3":false, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.386690, "value1":0.17482144522900334, "value2":208181032234291542, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770888, "value1":0.5775727956988672, "value2":7308314003124624706, "value3":true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710417, "value1":0.9182102368115501, "value2":8001324021984156096, "value3":false, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.133639, "value1":0.8132459830268748, "value2":6362172937464044603, "value3":true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313132, "value1":0.6425828291871425, "value2":3812023207458774027, "value3":true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816183, "value1":0.12876194798023052, "value2":353578462778793882, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.537076, "value1":0.3677093772206428, "value2":3410314947529692873, "value3":false, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.785801, "value1":0.5650444568188182, "value2":2518267372616726657, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793640, "value1":0.36840754337253134, "value2":6490561754060041624, "value3":false, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277029, "value1":0.5742325627742796, "value2":4576556191237422408, "value3":true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086053, "value1":0.635715694469195, "value2":3276323627400814300, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622335, "value1":0.6664883465636151, "value2":6846298203429885729, "value3":true, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784935, "value1":0.8345083778637773, "value2":2989860444276517490, "value3":false, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394440, "value1":0.3881444024090245, "value2":211938452740442471, "value3":true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.150923, "value1":0.2049623118713732, "value2":7830583450265904275, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334186, "value1":0.8877162218809062, "value2":5530254516648658024, "value3":true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919131, "value1":0.645089444938841, "value2":2752404297334193415, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.550415, "value1":0.8672407553696915, "value2":7296354502501329771, "value3":false, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.251810, "value1":0.12582294112213796, "value2":3850461942759838134, "value3":true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.768322, "value1":0.5089660327127362, "value2":4637000702345102317, "value3":true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.5790505163445078, "value2":2746113057580625956, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310041, "value1":0.7419516660136578, "value2":8573283812679768621, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567732, "value1":0.20310332375311568, "value2":1684484347666739261, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.149277, "value1":0.8537753954365711, "value2":3792414057901721222, "value3":true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019813, "value1":0.3193924681887993, "value2":4416687297621574996, "value3":false, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555536, "value1":0.2678716255116121, "value2":1978875629658477083, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.676000, "value1":0.268194488154906, "value2":9208242650214533822, "value3":false, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603232, "value1":0.7475788122275303, "value2":6003796300446172845, "value3":true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272526, "value1":0.7633068823447601, "value2":5387009257178750855, "value3":true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288629, "value1":0.0999015536587548, "value2":5230082044659723490, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198708, "value1":0.5411261720205245, "value2":1324824529248190153, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754319, "value1":0.5181119376899533, "value2":7586548988147520888, "value3":true, "__name__":"metric_00000079", "key_6":"c","key_0":"d","key_3":"d", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.974535, "value1":0.6605961675334583, "value2":4149492144268522705, "value3":false, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620393, "value1":0.0400929610839578, "value2":2611761419308325853, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489379, "value1":0.20432994152077563, "value2":4859071163254701004, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.497240, "value1":0.21587046902788842, "value2":1674028941320053082, "value3":true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227611, "value1":0.31207060679536985, "value2":3782150896328417443, "value3":false, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.813479, "value1":0.9346689305684607, "value2":4170814668135359561, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832568, "value1":0.40374034609384823, "value2":4890680963042405726, "value3":true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.868761, "value1":0.7456764740945415, "value2":7985095078472385264, "value3":false, "__name__":"metric_00000087", "key_5":"k","key_2":"f","key_3":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566095, "value1":0.11155632539641938, "value2":1650668369012843098, "value3":true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516140, "value1":0.2921171335805605, "value2":699561345288206492, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.009998, "value1":0.0698415110112407, "value2":2175710360877139700, "value3":true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472788, "value1":0.6894980351369026, "value2":5185256568181045963, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013027, "value1":0.47916067130802836, "value2":8339740169449446777, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947875, "value1":0.15641692100908322, "value2":1874863435098609183, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834625, "value1":0.4725529822941945, "value2":3313442976129641776, "value3":true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032027, "value1":0.11526706346113103, "value2":4474958127457336496, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.643426, "value1":0.2365209546681749, "value2":2452872778362311452, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399009, "value1":0.10587590188915917, "value2":5343568589706680846, "value3":false, "__name__":"metric_00000096", "key_2":"j","key_6":"c","key_1":"a", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464026, "value1":0.23089518212237822, "value2":6365432144590215713, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.573860, "value1":0.41263343273878106, "value2":7100420790565954895, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575295, "value1":0.15635829479176244, "value2":972701597090717176, "value3":false, "__name__":"metric_00000099", "key_6":"f","key_8":"j","key_5":"b", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685505, "value1":0.3500577640159365, "value2":5296267831822852754, "value3":true, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382859, "value1":0.8962437052233099, "value2":5000561412494049933, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793198, "value1":0.03594880833972008, "value2":5558834996866286026, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296341, "value1":0.6801030998229524, "value2":9104706563374398078, "value3":false, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.044529, "value1":0.43868239396683095, "value2":1119681077532090572, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977458, "value1":0.7636375641015349, "value2":3738349085487316798, "value3":true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.769690, "value1":0.38941161495083887, "value2":4119290182206919942, "value3":false, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604117, "value1":0.06703731437988915, "value2":2776279989058320233, "value3":false, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.107061, "value1":0.9783306976372779, "value2":6724603539014036116, "value3":false, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.243947, "value1":0.3058624284281911, "value2":8607064800338196111, "value3":true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784300, "value1":0.47413357934186534, "value2":6062775812546347462, "value3":true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053929, "value1":0.8033894774680731, "value2":3798712131176287721, "value3":true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997633, "value1":0.3495480156862711, "value2":6358034341550557079, "value3":false, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399888, "value1":0.5577617871085867, "value2":7606548109917229902, "value3":true, "__name__":"metric_00000113", "key_4":"k","key_2":"k","key_3":"i", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.343972, "value1":0.37033984748970067, "value2":3823813661350967339, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741859, "value1":0.15725835002679114, "value2":7931796757818180793, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118790, "value1":0.8710092300018802, "value2":8432660813300591545, "value3":false, "__name__":"metric_00000117", "key_6":"k","key_8":"f","key_5":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.514050, "value1":0.3589657867466283, "value2":2807064518724813388, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105213, "value1":0.48313119234822866, "value2":9012983220702483909, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.289587, "value1":0.28297188108556603, "value2":8293007746329622374, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.682523, "value1":0.2601723288341452, "value2":5132563496112960923, "value3":true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.521609, "value1":0.17923659255236565, "value2":9080059564709771845, "value3":false, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.866470, "value1":0.8472875420381546, "value2":7662519070278706881, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139688, "value1":0.701824998288288, "value2":8626519461512263205, "value3":false, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693655, "value1":0.3931946923075373, "value2":4088184802361327943, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.733184, "value1":0.33562267701400733, "value2":5057125078949459009, "value3":false, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530556, "value1":0.8546701921585885, "value2":7172802034424433024, "value3":false, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890826, "value1":0.9156630138494082, "value2":1106383994017980379, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736946, "value1":0.28531565085928856, "value2":7492657224478970683, "value3":true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184271, "value1":0.66321078827276, "value2":6482907214355252145, "value3":true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771765, "value1":0.8506763394943703, "value2":2997501176379232751, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621435, "value1":0.4219907917118007, "value2":2371448704479482567, "value3":true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522551, "value1":0.4559374479237847, "value2":8705737330976472895, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579601, "value1":0.3374291561460269, "value2":6118047213487455086, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.911838, "value1":0.24595613407282738, "value2":8543292459890755038, "value3":true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.461163, "value1":0.20032563894255778, "value2":8271655059958021803, "value3":false, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480016, "value1":0.537301255549281, "value2":131039455325957900, "value3":true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264510, "value1":0.8207403430077567, "value2":448751349067313394, "value3":true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.908904, "value1":0.45150207048162533, "value2":1484572129461087565, "value3":true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279527, "value1":0.48678886025337537, "value2":4675520724179222107, "value3":true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.983786, "value1":0.5074627713755523, "value2":6761505393797376999, "value3":true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407769, "value1":0.8974462165990488, "value2":4427875231986566903, "value3":false, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906297, "value1":0.32841731679412833, "value2":4853751460801403413, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.910866, "value1":0.22248621009204866, "value2":6153217292256934376, "value3":false, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481193, "value1":0.07256346779714405, "value2":8357563464914791440, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819751, "value1":0.7821155606881222, "value2":6350148722815418199, "value3":false, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925016, "value1":0.08017277463129999, "value2":8127038680866850197, "value3":true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380866, "value1":0.4236203787530011, "value2":6102770882422449664, "value3":false, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.687520, "value1":0.18833602677282602, "value2":2416669714696967955, "value3":false, "__name__":"metric_00000128", "key_7":"j","key_3":"j","key_4":"i", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392864, "value1":0.12162108913858492, "value2":1037740633966701101, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555618, "value1":0.5726263308688928, "value2":5340970336126619965, "value3":true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041150, "value1":0.1866697214008993, "value2":6089316958837860364, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.848709, "value1":0.5912728689790879, "value2":8374928084508664533, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.633435, "value1":0.34567925121735993, "value2":6074382604796755677, "value3":false, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615342, "value1":0.43884173283368255, "value2":2552875653507489786, "value3":true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.420473, "value1":0.024593195105800023, "value2":6998883295968298787, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.493473, "value1":0.517385578986543, "value2":7898967292563966468, "value3":false, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398409, "value1":0.9494831815326669, "value2":1043777551847385399, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.261653, "value1":0.605095923821751, "value2":5798307460445919671, "value3":true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734130, "value1":0.17339556093076888, "value2":3888875155355529135, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.234024, "value1":0.07147953681129654, "value2":2220989894337620932, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.552276, "value1":0.46428310479897317, "value2":5429095800661644988, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.547014, "value1":0.9160534111766008, "value2":4258800689197649539, "value3":true, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084253, "value1":0.06651726425775369, "value2":8202506388672279723, "value3":false, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137868, "value1":0.6230337836484641, "value2":8788058834810483014, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.168246, "value1":0.1571304566786965, "value2":2933470821094626337, "value3":true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043250, "value1":0.3445304512204906, "value2":8908780418296865224, "value3":false, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542269, "value1":0.007638770771913262, "value2":975662304048103149, "value3":true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016306, "value1":0.44304617075801217, "value2":4700290207925657878, "value3":false, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967184, "value1":0.9943193022594318, "value2":7141028805491571993, "value3":true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522277, "value1":0.8578463560518583, "value2":6372322792910116627, "value3":false, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541009, "value1":0.5200416543540609, "value2":5687465452183940942, "value3":true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742322, "value1":0.4918651213341637, "value2":7530197332852248825, "value3":true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.478705, "value1":0.02234257118225825, "value2":381364684161431163, "value3":true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.401319, "value1":0.36716744081601327, "value2":3165286506094547515, "value3":false, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.154587, "value1":0.8570612240935895, "value2":440607090836829314, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268457, "value1":0.19708890772544582, "value2":716490672588175731, "value3":true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976461, "value1":0.37921805973698547, "value2":5187015251060220885, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.291361, "value1":0.0347133343718353, "value2":4081610772727292868, "value3":false, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.831910, "value1":0.20329085836851984, "value2":1212790782212732680, "value3":true, "__name__":"metric_00000189", "key_3":"j","key_9":"k","key_2":"g", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045270, "value1":0.742873126094037, "value2":1857680286082818546, "value3":false, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719581, "value1":0.8427487279865438, "value2":1584262554399141329, "value3":true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.878973, "value1":0.7217975560465744, "value2":6936890459625848218, "value3":true, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.786254, "value1":0.7388730574867226, "value2":3591226182562159088, "value3":true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719189, "value1":0.6138101273145392, "value2":4767024689432495577, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335758, "value1":0.6941935445321573, "value2":1310712364678635982, "value3":false, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333397, "value1":0.7084933562313812, "value2":2679977702860384198, "value3":true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816868, "value1":0.5704940444863804, "value2":3982836666436033897, "value3":true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.806625, "value1":0.7550148139027509, "value2":2819861429300922010, "value3":true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155981, "value1":0.3978060292285169, "value2":2375324886880679840, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.172764, "value1":0.8807292451628046, "value2":1585412394695366401, "value3":false, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.127700, "value1":0.8691901922846396, "value2":4006452105841009084, "value3":false, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.937137, "value1":0.6079865671377581, "value2":9014517461395944209, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601939, "value1":0.5587486969852118, "value2":896670201927694097, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147910, "value1":0.7860716932621791, "value2":8828747393971238771, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271640, "value1":0.8634829980251869, "value2":7534202338930685599, "value3":true, "__name__":"metric_00000177", "key_7":"c","key_8":"d","key_4":"i", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510609, "value1":0.273735584071857, "value2":7995135221543970576, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.424702, "value1":0.6570665232819806, "value2":7130224512963405605, "value3":true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.120070, "value1":0.4366525687584962, "value2":8321790572084844233, "value3":true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.355286, "value1":0.9376756075719435, "value2":7612697762460955574, "value3":true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057269, "value1":0.9603884122588556, "value2":5675558649074747456, "value3":true, "__name__":"metric_00000182", "key_9":"b","key_3":"a","key_7":"f", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.396116, "value1":0.3241173308406254, "value2":2149479689060994101, "value3":false, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271843, "value1":0.5095285278231975, "value2":5564561021464735924, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389082, "value1":0.7189491284948987, "value2":8539354123799965742, "value3":true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780310, "value1":0.7959419420056967, "value2":791327040768245285, "value3":false, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496100, "value1":0.7137676161743793, "value2":7249423453544847575, "value3":false, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.320840, "value1":0.7088322490051288, "value2":2259284238716415015, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.588238, "value1":0.2616266339940352, "value2":7616111814012126210, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510231, "value1":0.34548313055527086, "value2":9146415948654460714, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725142, "value1":0.832268560021817, "value2":8449163586612650016, "value3":false, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722343, "value1":0.5127452250176201, "value2":8372677466999750797, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455239, "value1":0.7274771017691084, "value2":8558710521155328448, "value3":true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070961, "value1":0.5108403289338641, "value2":6822737602377250548, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.799042, "value1":0.6579145757389928, "value2":6367849243718047663, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997752, "value1":0.29826795257630173, "value2":7963803897292403489, "value3":false, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019959, "value1":0.5329624946388745, "value2":6166072735161874859, "value3":false, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183220, "value1":0.1192005017394279, "value2":1414525889342507585, "value3":true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.216504, "value1":0.10069780302425961, "value2":4147229267124117622, "value3":false, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184537, "value1":0.9674252449470503, "value2":7196918589322334021, "value3":true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.982196, "value1":0.5121645885657403, "value2":1331574421408029626, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.807710, "value1":0.6453473320699041, "value2":5880434997959490048, "value3":false, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139645, "value1":0.45151332154483154, "value2":5206983362834731524, "value3":true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202627, "value1":0.5085399708189182, "value2":229347926887383465, "value3":true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703675, "value1":0.1410911985436997, "value2":1995545208705825638, "value3":false, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.916538, "value1":0.9588604837943727, "value2":3014447442999476863, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174096, "value1":0.8442896590468587, "value2":1669527921851537286, "value3":true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268618, "value1":0.05460739672623512, "value2":6145595254763691216, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184210, "value1":0.08111723273128417, "value2":2941741438359406510, "value3":true, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419569, "value1":0.11197378055505244, "value2":7190302664501414682, "value3":false, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.080457, "value1":0.46189706497698463, "value2":3739595486194011632, "value3":false, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685402, "value1":0.5911751311740201, "value2":375795342652216999, "value3":false, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584152, "value1":0.842391293090085, "value2":3305999218832830126, "value3":true, "__name__":"metric_00000241", "key_9":"d","key_4":"e","key_8":"a", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719567, "value1":0.7117389665869482, "value2":4436375112558311543, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.599224, "value1":0.7507823724841362, "value2":5289861475224212810, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298087, "value1":0.03464563720261393, "value2":5487979858306400729, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084762, "value1":0.4408096997046924, "value2":8782838705036394284, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153226, "value1":0.8396685378218853, "value2":2305238685769866566, "value3":false, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875795, "value1":0.6283427563062458, "value2":1424824369894963651, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.841870, "value1":0.2951891910717211, "value2":1229301210242052079, "value3":true, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036891, "value1":0.10272064811801765, "value2":8629352909535519019, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.475886, "value1":0.47884350384173996, "value2":2002152937943396093, "value3":false, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286578, "value1":0.5511122934004854, "value2":6456656081230025536, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180709, "value1":0.591154696871619, "value2":1085648256858149575, "value3":true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.549035, "value1":0.9073632737596287, "value2":441521694946529713, "value3":false, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644644, "value1":0.42900060724414874, "value2":7252379057595536961, "value3":false, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.185531, "value1":0.28679213225919725, "value2":3666625025829964686, "value3":false, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858171, "value1":0.07899073830408582, "value2":8847544646507443860, "value3":false, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.9086649445849005, "value2":7300771702176978507, "value3":false, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.217129, "value1":0.8286760370555245, "value2":437314161092307860, "value3":true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.092457, "value1":0.3257453112147972, "value2":1241192378749271122, "value3":false, "__name__":"metric_00000225", "key_4":"a","key_2":"e", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.318459, "value1":0.6218522470963161, "value2":2601033448014127071, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543632, "value1":0.47194572326020234, "value2":330858668685674119, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.494889, "value1":0.42021023869128465, "value2":7698787812508472546, "value3":false, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173188, "value1":0.6333586665476976, "value2":1031119021359267335, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392225, "value1":0.13669516747392368, "value2":2722787133031533732, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.418321, "value1":0.8852713566769926, "value2":1145580014369872247, "value3":true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499498, "value1":0.37652715283253, "value2":6023088039348052644, "value3":true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049271, "value1":0.7428393913560761, "value2":4074778390717770530, "value3":false, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344096, "value1":0.618674195595606, "value2":8432036346015765748, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.504726, "value1":0.8381168181634259, "value2":7490362504268179930, "value3":false, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130009, "value1":0.04787900547347569, "value2":8930977988943996511, "value3":false, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517353, "value1":0.9123390757414803, "value2":5406757776090744464, "value3":true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.159780, "value1":0.2711186767871672, "value2":6650537343113966916, "value3":false, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601454, "value1":0.036596921749510625, "value2":1801819188066798655, "value3":false, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740877, "value1":0.9080583731663923, "value2":6623419932438799475, "value3":true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.743094, "value1":0.9900165531318826, "value2":383393257357043043, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.939900, "value1":0.8769075090910828, "value2":487693007619851718, "value3":true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.102543, "value1":0.4399163118191967, "value2":5888144134235313914, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917706, "value1":0.6339047693941521, "value2":6760479052672151531, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108610, "value1":0.06502295112846754, "value2":8902773220268340372, "value3":true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.756394, "value1":0.7104082109611765, "value2":3854497302576556911, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914430, "value1":0.39800381262255646, "value2":1413572596816022205, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.046273, "value1":0.9197740465700138, "value2":5994606503835738604, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.042126, "value1":0.35565383574544296, "value2":206374188969693152, "value3":true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.753926, "value1":0.14736292648599036, "value2":5720098218146494963, "value3":false, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138925, "value1":0.14592833793919333, "value2":5845534364661717380, "value3":false, "__name__":"metric_00000284", "key_7":"e","key_0":"b","key_3":"b", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966713, "value1":0.5691036450918006, "value2":5571037157834825483, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128296, "value1":0.522798351805377, "value2":7845314774564626095, "value3":true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.374869, "value1":0.16619436230032464, "value2":2683834913042811436, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349899, "value1":0.0806742622823487, "value2":5654997107573600799, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621137, "value1":0.6389883746416457, "value2":6718840923740903865, "value3":false, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505989, "value1":0.5736885226925782, "value2":7660274947298694696, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245218, "value1":0.06067210631905806, "value2":7481373150924938933, "value3":true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.961402, "value1":0.40476409107709094, "value2":1656152426641438063, "value3":true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064546, "value1":0.7710994608894077, "value2":5484900997853498427, "value3":false, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686047, "value1":0.46617534097616353, "value2":1998746207850635494, "value3":true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195207, "value1":0.8036664106551048, "value2":21823515460647768, "value3":false, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890318, "value1":0.49246668518608533, "value2":4290226524505777397, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.662549, "value1":0.0648629447687713, "value2":2600958480076392029, "value3":true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867346, "value1":0.6103284095540984, "value2":959086414558873258, "value3":false, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036038, "value1":0.22211264871806694, "value2":2970074654495356013, "value3":false, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772742, "value1":0.47254264158251036, "value2":5967291460715844391, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.765254, "value1":0.59761349012731, "value2":5045361763860538939, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684477, "value1":0.3727521458297325, "value2":7120290254833886255, "value3":true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782498, "value1":0.5592140587339981, "value2":1560278611367944657, "value3":false, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472665, "value1":0.4451202433666956, "value2":7528293771469761549, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957028, "value1":0.6722101190310112, "value2":3256843532996983591, "value3":true, "__name__":"metric_00000298", "key_4":"j","key_9":"b","key_3":"g", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.558411, "value1":0.33974258843823785, "value2":9219292136379640073, "value3":true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686456, "value1":0.662252940687323, "value2":2872679473486076312, "value3":false, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.664702, "value1":0.13252769744708778, "value2":3782836160183678190, "value3":false, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340522, "value1":0.321610730663863, "value2":1993266611063130503, "value3":true, "__name__":"metric_00000301", "key_7":"i","key_9":"k","key_4":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738508, "value1":0.03455404630849952, "value2":5265512962120119813, "value3":true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722717, "value1":0.039393801449424636, "value2":8411289366884053387, "value3":false, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738985, "value1":0.4374781921483304, "value2":4532229627185520929, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.959594, "value1":0.06784833824743756, "value2":2235113542571170507, "value3":true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512001, "value1":0.45518592645372824, "value2":1219663694779720192, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699152, "value1":0.9199280757111055, "value2":920010063978439254, "value3":false, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870981, "value1":0.7153328294012858, "value2":2241596700674662533, "value3":false, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713651, "value1":0.1871397130196525, "value2":5525165593287246022, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608421, "value1":0.2007302921644417, "value2":7082345286303554347, "value3":true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.898979, "value1":0.4326799832087361, "value2":4510789740667402710, "value3":true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542323, "value1":0.5916017742256197, "value2":7802920355385118767, "value3":true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245771, "value1":0.3115277557229622, "value2":6710572841494695733, "value3":true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772755, "value1":0.17023256633885028, "value2":4693930877366777744, "value3":true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.247931, "value1":0.4740059660910414, "value2":1391972395280563550, "value3":true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394287, "value1":0.02366479635408087, "value2":7477007108410228541, "value3":false, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108172, "value1":0.971283160522281, "value2":6224196629196175206, "value3":true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932465, "value1":0.6431474969443486, "value2":6996302573266425119, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070979, "value1":0.4324217725560448, "value2":9202735673442875059, "value3":false, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862734, "value1":0.5425012681866345, "value2":8705521243115097026, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207036, "value1":0.7438270358882513, "value2":1482607831466689197, "value3":false, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345448, "value1":0.3875837956213489, "value2":2480267181897079015, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553288, "value1":0.3343119844132966, "value2":7890867607759034801, "value3":true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559986, "value1":0.2851751818045875, "value2":3231450807250993029, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.008285, "value1":0.11082046459395005, "value2":5670330804396842703, "value3":true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620145, "value1":0.27733408115115116, "value2":4537754734930200229, "value3":false, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.861443, "value1":0.7565577041788646, "value2":216193021440411568, "value3":true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584594, "value1":0.7717911844132257, "value2":7036437829052540741, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.066975, "value1":0.07703464444888349, "value2":8332259918814404840, "value3":true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272133, "value1":0.2064168890828292, "value2":1598424380451243838, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725661, "value1":0.5107106855646196, "value2":2095325894892271000, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.992271, "value1":0.04606905650348332, "value2":3458383290691046043, "value3":true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988671, "value1":0.9012802494548691, "value2":3265213363984040214, "value3":true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344354, "value1":0.6315603617727772, "value2":1819854718616345426, "value3":false, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198753, "value1":0.2443944990728996, "value2":3740319967339712109, "value3":true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294599, "value1":0.7030675431685813, "value2":5821390349436487736, "value3":false, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.233177, "value1":0.5520329333372563, "value2":2208761416514593046, "value3":true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.369614, "value1":0.2829108305094591, "value2":2978362585536403835, "value3":false, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.484777, "value1":0.3351273859753424, "value2":1348038406704883991, "value3":true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511345, "value1":0.6897076794113861, "value2":3140482719657458044, "value3":true, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780083, "value1":0.056509953898858746, "value2":4983435501891849142, "value3":false, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.023345, "value1":0.08633016397064507, "value2":439606973167008387, "value3":true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013043, "value1":0.8750924151560185, "value2":2563435998029443275, "value3":true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929738, "value1":0.6942458871164995, "value2":6742189975038174280, "value3":false, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348725, "value1":0.6477096282239411, "value2":614033611645126767, "value3":false, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.148404, "value1":0.3122137082601719, "value2":5921845943066980452, "value3":false, "__name__":"metric_00000355", "key_9":"e","key_2":"d","key_3":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.892407, "value1":0.0024857877019035433, "value2":78557787569325314, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770110, "value1":0.4854945232954409, "value2":4441111957237018619, "value3":false, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969650, "value1":0.028054363655575613, "value2":6620021166465815802, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096277, "value1":0.7642864471977222, "value2":7079078098912113925, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104248, "value1":0.17068081288522707, "value2":2914828831179785715, "value3":false, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.033230, "value1":0.6691862134057828, "value2":5879037419816229961, "value3":true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442293, "value1":0.918687680755694, "value2":3522143514446827859, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340520, "value1":0.3316554635970253, "value2":3298849581857165314, "value3":true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.005701, "value1":0.22902534413133244, "value2":4040792449962896607, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.101103, "value1":0.4459003073823555, "value2":8364403783112020861, "value3":false, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.269361, "value1":0.9829136286924077, "value2":305552246857799140, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002507, "value1":0.10735284014448321, "value2":2548523454772398525, "value3":false, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516501, "value1":0.3420030652884641, "value2":449036192165412633, "value3":true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914771, "value1":0.6948526439267081, "value2":3274351745805138467, "value3":true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151382, "value1":0.9010351248418295, "value2":6225882894853282790, "value3":false, "__name__":"metric_00000323", "key_7":"d","key_2":"c", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.797287, "value1":0.450517653615541, "value2":6406142276072623664, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575226, "value1":0.6947969923557978, "value2":4732050542115525893, "value3":true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.256780, "value1":0.29401472322197164, "value2":8790173514967991405, "value3":true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791599, "value1":0.6204111480891676, "value2":1162220986630177027, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615289, "value1":0.17893814852804246, "value2":3187373261649831292, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738034, "value1":0.9979754782079555, "value2":6006267840987043849, "value3":true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735591, "value1":0.011768322537597629, "value2":7974963296156276842, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.751213, "value1":0.43700619365470833, "value2":6294325004534159296, "value3":true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781793, "value1":0.6112369782572161, "value2":2263959578091534198, "value3":false, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118524, "value1":0.4979095180792778, "value2":5142183829715514199, "value3":false, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.574787, "value1":0.8076799963798312, "value2":1975720601465651847, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814750, "value1":0.45891950977715895, "value2":4074714357288466936, "value3":false, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579803, "value1":0.25446438731696874, "value2":7191932871100542216, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722158, "value1":0.19082711905600802, "value2":1422914021938617784, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342798, "value1":0.5097927561705997, "value2":2281093093995561768, "value3":false, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164991, "value1":0.06555881689103157, "value2":7152427537795186892, "value3":true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.316516, "value1":0.1580165967231405, "value2":8769573796829946768, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548667, "value1":0.8164945933097875, "value2":1223332369807607717, "value3":false, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174435, "value1":0.5629514369055348, "value2":2359802095953741925, "value3":false, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.561139, "value1":0.9847446485189013, "value2":8748242508292199474, "value3":true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604206, "value1":0.7084164989246987, "value2":5662112818525056068, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.069641, "value1":0.9392432577384211, "value2":7977134301111651930, "value3":false, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977024, "value1":0.8770253339233329, "value2":114419076017958040, "value3":true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.981104, "value1":0.0262761349528985, "value2":3404389245833544901, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.309389, "value1":0.6700115572733264, "value2":1746800498285488694, "value3":true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907890, "value1":0.3847355476155844, "value2":1062772964889350348, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.557849, "value1":0.8423014863985597, "value2":8693557035023467587, "value3":false, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174807, "value1":0.14044078429336324, "value2":2899359413194652810, "value3":true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907265, "value1":0.7213413890977288, "value2":711482054709798828, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730322, "value1":0.010569544047203483, "value2":2272852869521847556, "value3":true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047968, "value1":0.8749328675131659, "value2":3313417752475725526, "value3":false, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548996, "value1":0.13476734433713453, "value2":5686844894034380593, "value3":false, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090199, "value1":0.47038291207998856, "value2":6737581800965808102, "value3":true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372121, "value1":0.4982647921591242, "value2":2960213658998005236, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.732328, "value1":0.6794810773912932, "value2":5614573251611720393, "value3":true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195822, "value1":0.7817234737401128, "value2":6738900276029001186, "value3":true, "__name__":"metric_00000399", "key_9":"g","key_4":"h", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173099, "value1":0.7620323539766012, "value2":6195639668903126539, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739085, "value1":0.8805144351999263, "value2":6990390850908433, "value3":true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851839, "value1":0.19515034480983603, "value2":5022120185556407326, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.764811, "value1":0.07516380443788656, "value2":4953415333432850814, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.954192, "value1":0.6068771093686028, "value2":1921210459347583907, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.501412, "value1":0.20889031917804166, "value2":7266160086492287738, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798358, "value1":0.44429041626860627, "value2":7311639409429341020, "value3":false, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200103, "value1":0.5998566082715615, "value2":3968426630272414916, "value3":false, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443243, "value1":0.7996897987041263, "value2":2955385724626539485, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808129, "value1":0.12934958318030565, "value2":8640009016586784249, "value3":true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890136, "value1":0.395437996374284, "value2":1375026123479284573, "value3":false, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.726245, "value1":0.8383327509398333, "value2":1069495836938595505, "value3":true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464465, "value1":0.7522623057444222, "value2":3745372652823842076, "value3":false, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958198, "value1":0.6062226980863115, "value2":692634818001600691, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336639, "value1":0.3569530703633769, "value2":5114963101620697516, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153507, "value1":0.29997973310668347, "value2":35895264299031809, "value3":false, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496206, "value1":0.16796883917469935, "value2":2457448015461697922, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699773, "value1":0.1320495753882598, "value2":2684330380511330426, "value3":false, "__name__":"metric_00000416", "key_6":"i","key_9":"b","key_3":"h", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086489, "value1":0.01686874394925538, "value2":7266008503964487472, "value3":true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.755161, "value1":0.12039999679109185, "value2":7441080332932356910, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481861, "value1":0.21023924595248417, "value2":7215904118134021296, "value3":false, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141970, "value1":0.6959505249259283, "value2":5593000831636710131, "value3":false, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473262, "value1":0.20484780325876686, "value2":1515840223895186260, "value3":true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.610674, "value1":0.37794171859625975, "value2":7210398121561053631, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784669, "value1":0.8314547953224727, "value2":6889739271959092821, "value3":false, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348279, "value1":0.9806070629868892, "value2":6665537531662525499, "value3":true, "__name__":"metric_00000427", "key_7":"h","key_2":"i","key_5":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342228, "value1":0.8292222808866697, "value2":8140150785342027171, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049991, "value1":0.20259901011770426, "value2":860224417352173376, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442752, "value1":0.9271842959401567, "value2":8958731671751036609, "value3":false, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288552, "value1":0.19606129029896613, "value2":3803062421310719257, "value3":true, "__name__":"metric_00000436", "key_4":"c","key_5":"c","key_2":"h", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851767, "value1":0.8040030606139774, "value2":5022190100385685639, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258187, "value1":0.06995124309795013, "value2":3290080872961653036, "value3":false, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967676, "value1":0.08273649902951419, "value2":5115999717780317436, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.972416, "value1":0.3332024948916442, "value2":3195696687672274736, "value3":true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273980, "value1":0.9801180643314533, "value2":7124826086691987600, "value3":true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962051, "value1":0.7078535280449856, "value2":2450804349449720576, "value3":true, "__name__":"metric_00000442", "key_2":"e","key_6":"f","key_1":"i", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342560, "value1":0.21100802753268208, "value2":5026066029722000095, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582068, "value1":0.9724895772631098, "value2":9045330876857967867, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716662, "value1":0.5239131397726235, "value2":6699936731993788174, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.630988, "value1":0.8757044551570439, "value2":6009726873690078427, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.617422, "value1":0.029531083291305976, "value2":7932542565063790762, "value3":true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389736, "value1":0.40122875818931175, "value2":7350974819717892233, "value3":true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.470743, "value1":0.286118127257554, "value2":2903264335442728018, "value3":true, "__name__":"metric_00000449", "key_6":"j","key_1":"i","key_4":"i", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118668, "value1":0.42900363662380264, "value2":2786819190490835132, "value3":false, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819243, "value1":0.7832376096990535, "value2":6768946688624968401, "value3":true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519924, "value1":0.1784273794737174, "value2":606279348173747859, "value3":false, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575499, "value1":0.8580592683181654, "value2":6065349113607731318, "value3":false, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.952434, "value1":0.8198422458926743, "value2":6520729939047239531, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180605, "value1":0.504162836348111, "value2":8107251100494910096, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.680620, "value1":0.9192990596381249, "value2":7916466116307364506, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421505, "value1":0.6014385702769156, "value2":6172295401410316312, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581350, "value1":0.2695750525680962, "value2":6289934971199391350, "value3":false, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.244031, "value1":0.7392367672457897, "value2":7438364572557990950, "value3":true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801331, "value1":0.014860008784846704, "value2":6873217646463864251, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875352, "value1":0.3975554355639885, "value2":4390005070727485119, "value3":true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929440, "value1":0.40393323123767977, "value2":1148796038957803851, "value3":false, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.817616, "value1":0.5647949954501486, "value2":1024684537861148034, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.990318, "value1":0.25287778647718107, "value2":7093365367237549470, "value3":true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741736, "value1":0.023580263258422104, "value2":1518420105714291210, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463012, "value1":0.7888469551975262, "value2":6786560784050350679, "value3":true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449551, "value1":0.2636704318808076, "value2":7884141870151198099, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419454, "value1":0.6010995928923429, "value2":224905435491328214, "value3":true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.322105, "value1":0.1331880379524159, "value2":7369022064377940075, "value3":false, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581063, "value1":0.603596835974457, "value2":4386876721702326915, "value3":false, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.857080, "value1":0.5842046650044923, "value2":1066043362420191253, "value3":true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191051, "value1":0.4639324153416964, "value2":8874234446483833192, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298751, "value1":0.20423387563647288, "value2":8774956872647827067, "value3":true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924593, "value1":0.0659156088635184, "value2":3634318708597769338, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404558, "value1":0.7799254034556268, "value2":4510574800615418821, "value3":true, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962467, "value1":0.8168896478129019, "value2":7986325005390816690, "value3":false, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622579, "value1":0.0904343638172895, "value2":8283142617455012372, "value3":false, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710038, "value1":0.6909066964841821, "value2":6203472414968382073, "value3":false, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032660, "value1":0.21417322011190834, "value2":1054486238769744890, "value3":true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801225, "value1":0.2765954328390815, "value2":442385272245544502, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116566, "value1":0.6031158754932586, "value2":3770751187916956934, "value3":true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.284086, "value1":0.055471202806315845, "value2":7266763851735180811, "value3":false, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559271, "value1":0.28967037620921054, "value2":6147439427772857051, "value3":true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224654, "value1":0.5513198770455077, "value2":7761756752100166184, "value3":false, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575799, "value1":0.07079456843554317, "value2":4847399039818291257, "value3":false, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819125, "value1":0.4806719127445899, "value2":1129446367168224920, "value3":true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730720, "value1":0.6013248235298089, "value2":1610990221268346838, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.593103, "value1":0.8196760148078414, "value2":6644540894797627889, "value3":false, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.218509, "value1":0.5227005302649587, "value2":2525326877519094680, "value3":false, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.292767, "value1":0.09698193120132978, "value2":8960471678202836654, "value3":false, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858612, "value1":0.675005592095046, "value2":6028830951911645471, "value3":true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736558, "value1":0.24026583264924886, "value2":1360237166500303266, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826970, "value1":0.3893255587955024, "value2":2292740904231475335, "value3":true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128577, "value1":0.3146179907253604, "value2":164469213437964556, "value3":true, "__name__":"metric_00000485", "key_3":"i","key_4":"a","key_0":"g", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003953, "value1":0.33225441021352176, "value2":5713576701192138110, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352438, "value1":0.12247814806052679, "value2":976940527664803795, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.434520, "value1":0.6138506637398143, "value2":1920264902732481602, "value3":false, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.731843, "value1":0.8305061213621728, "value2":4607426852903927950, "value3":false, "__name__":"metric_00000489", "key_7":"h","key_0":"c","key_5":"a", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032970, "value1":0.8273657553543087, "value2":3103821801573462769, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833737, "value1":0.3431117940438562, "value2":5529881494203381864, "value3":false, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.206561, "value1":0.03180903277463739, "value2":1534192978519074976, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598146, "value1":0.7943659660871121, "value2":2512296846803666990, "value3":false, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.830318, "value1":0.9077939825338383, "value2":1289520021940183737, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409047, "value1":0.694416059529231, "value2":520453313543014186, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.949791, "value1":0.6369046983298985, "value2":371741091848582403, "value3":true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354931, "value1":0.7633140731862355, "value2":8981576909845465787, "value3":true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.061767, "value1":0.7699262325323719, "value2":79439898840104021, "value3":false, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.941133, "value1":0.26448399729496197, "value2":6448749789711445941, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.339821, "value1":0.5246395493337294, "value2":44122001398655312, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359654, "value1":0.5954751506379091, "value2":1668054016792098588, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.304719, "value1":0.17854657722051143, "value2":5240836642486639588, "value3":true, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533350, "value1":0.9102023121053504, "value2":5223044570382986189, "value3":true, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.495583, "value1":0.8380393694454433, "value2":2826946501048985900, "value3":true, "__name__":"metric_00000504", "key_6":"c","key_0":"f","key_2":"a", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925864, "value1":0.06057194714527472, "value2":4827880370625631466, "value3":true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200061, "value1":0.3196222334980913, "value2":5536801387533970020, "value3":true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.635607, "value1":0.5926951079597541, "value2":4317404702469315811, "value3":false, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041722, "value1":0.2038750977962134, "value2":8603762180622472, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.565080, "value1":0.5851130292640377, "value2":8612148593533709646, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090894, "value1":0.12207888531940794, "value2":435323305893709131, "value3":true, "__name__":"metric_00000510", "key_9":"e","key_4":"c","key_8":"j", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.071849, "value1":0.7914473708034223, "value2":4833048612300779383, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202326, "value1":0.6798241827808086, "value2":5321585738808934000, "value3":false, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116141, "value1":0.8646385559973755, "value2":2431723757311861306, "value3":false, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.074497, "value1":0.2771958266692314, "value2":2948732814035791763, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.453427, "value1":0.9750934919782419, "value2":7598695164797163349, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517105, "value1":0.6447380734003394, "value2":5945582855956594632, "value3":false, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969670, "value1":0.8792534113647463, "value2":9118825598855667812, "value3":true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064653, "value1":0.7641915057577142, "value2":376649068007551056, "value3":true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.299983, "value1":0.5328541897726483, "value2":7848443219053578824, "value3":false, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814787, "value1":0.5822532477986045, "value2":3517444920674306021, "value3":true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645109, "value1":0.6788316811493683, "value2":2399412759017043209, "value3":false, "__name__":"metric_00000521", "key_5":"d","key_6":"f","key_3":"h", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104970, "value1":0.5150104887448738, "value2":39162375392832614, "value3":true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124629, "value1":0.9415126690810315, "value2":2487867259401435813, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847494, "value1":0.8236014719422879, "value2":8522949906604300533, "value3":true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.694932, "value1":0.5232173520376328, "value2":2865775556504465271, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.824405, "value1":0.33768880558190767, "value2":1449467820646971158, "value3":true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263091, "value1":0.800562879218074, "value2":4640064437824793404, "value3":true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.545676, "value1":0.6991321506153548, "value2":504604265126996830, "value3":true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685750, "value1":0.256315076419413, "value2":7805803213074768353, "value3":false, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.955572, "value1":0.12777100392825705, "value2":4697380262379050863, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.052725, "value1":0.24897441126326256, "value2":2806761823519892967, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832231, "value1":0.7123036010256201, "value2":5558652604185663810, "value3":false, "__name__":"metric_00000541", "key_7":"c","key_3":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.779475, "value1":0.42379979395746137, "value2":5877566589967590752, "value3":false, "__name__":"metric_00000542", "key_1":"a","key_2":"j","key_0":"c", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.056823, "value1":0.7557073972848881, "value2":1462009193456216531, "value3":true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179551, "value1":0.33562095061974395, "value2":2905800790209406415, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.946943, "value1":0.7895726694044407, "value2":4556871779015432465, "value3":false, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.792955, "value1":0.9478490399693646, "value2":6469796413879524879, "value3":true, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302294, "value1":0.40539394874343054, "value2":1479528610985409190, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.859086, "value1":0.8533535931566633, "value2":8328258195824345992, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.912801, "value1":0.4080605346123484, "value2":921699075685228177, "value3":false, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026911, "value1":0.5160934551500268, "value2":4192638354189389027, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.145578, "value1":0.4697942761887252, "value2":1007860316702817500, "value3":true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224036, "value1":0.21446416617045144, "value2":2612501104515894392, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.083418, "value1":0.3800566396023636, "value2":1387864613630168710, "value3":true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.483137, "value1":0.9606563644656662, "value2":2761802265245174240, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.855910, "value1":0.606558104786761, "value2":842530041019109994, "value3":true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332745, "value1":0.01260394609809363, "value2":2781673523626108476, "value3":false, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198691, "value1":0.0952542911286906, "value2":4639530496495690336, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207605, "value1":0.29598976404510996, "value2":149815254996856780, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.219100, "value1":0.7173110134058144, "value2":207519709019999249, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.065034, "value1":0.7683750910989782, "value2":6262446867113607884, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362737, "value1":0.5652820007764047, "value2":4138085582551770816, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.790498, "value1":0.45469936210178996, "value2":3874789041764013688, "value3":false, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581214, "value1":0.8566672554079884, "value2":5701861631669426000, "value3":false, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741360, "value1":0.7962411718913905, "value2":3357161583638308077, "value3":true, "__name__":"metric_00000534", "key_9":"k","key_1":"a","key_3":"a", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.934536, "value1":0.07212743130410183, "value2":6319820377917318485, "value3":true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258971, "value1":0.09357358589212993, "value2":5081900353713330302, "value3":false, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.436226, "value1":0.5553015007542973, "value2":6124312980620622172, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566508, "value1":0.7132604990750537, "value2":8485961090508027563, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298396, "value1":0.4460940570451377, "value2":5709626152776289179, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596697, "value1":0.3774598732286396, "value2":1819820653928002535, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.758549, "value1":0.49680425119422045, "value2":1371518397741898898, "value3":false, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151965, "value1":0.9897517149153644, "value2":3974511699577597667, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.223787, "value1":0.9859897177824236, "value2":6368178524733411270, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699805, "value1":0.9240609624872184, "value2":4091684677817576336, "value3":true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.640771, "value1":0.3605713604653968, "value2":2196974552050802581, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917234, "value1":0.3769625716544593, "value2":7094294232648752669, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541154, "value1":0.07721056009394252, "value2":2450341303960944649, "value3":false, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188174, "value1":0.511305577711157, "value2":4677738772360990938, "value3":false, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277439, "value1":0.006756936522771483, "value2":2736398221812059244, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847832, "value1":0.23362336838695744, "value2":6292362019324286334, "value3":false, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.840677, "value1":0.6137390254843741, "value2":6506004284907452986, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.397331, "value1":0.5291020730916409, "value2":766558029100390419, "value3":false, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.668711, "value1":0.6677953904066025, "value2":2813624786174976455, "value3":true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.728127, "value1":0.9455687108064199, "value2":2598623203001035108, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.260429, "value1":0.5485107722176485, "value2":8238880717060307339, "value3":true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407939, "value1":0.7503222721986553, "value2":8088756690439902034, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016860, "value1":0.6281421773683825, "value2":4463756144226416929, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.592662, "value1":0.7215969122220701, "value2":2957728488172560734, "value3":true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596916, "value1":0.6365196560861767, "value2":3939747841173152393, "value3":true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.650813, "value1":0.33343709461482135, "value2":8822031185214434530, "value3":true, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543088, "value1":0.5660730417813042, "value2":6651394201039088708, "value3":false, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.416422, "value1":0.2907903630149851, "value2":5811650096165674198, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505086, "value1":0.44660978158387593, "value2":6270587282710629926, "value3":false, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.055558, "value1":0.394254430161103, "value2":9153429600073084016, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.578079, "value1":0.6160395423413576, "value2":7388869261196091596, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.347142, "value1":0.28357580684654937, "value2":7651284402887473938, "value3":false, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089369, "value1":0.24785364470389606, "value2":2827626134616355884, "value3":true, "__name__":"metric_00000563", "key_2":"f","key_5":"a","key_0":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922968, "value1":0.9908293125287131, "value2":5006327563319926460, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489701, "value1":0.9764768678976257, "value2":3780590525108581330, "value3":false, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700790, "value1":0.8524762188330061, "value2":2298426148247437198, "value3":false, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.884155, "value1":0.17838161460811935, "value2":707461345329966226, "value3":false, "__name__":"metric_00000592", "key_1":"e","key_7":"b","key_0":"k", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.877319, "value1":0.597182710013635, "value2":66780576193296735, "value3":false, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906533, "value1":0.44570088765160515, "value2":1159099942400157753, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555879, "value1":0.6225620185948739, "value2":7381136800707660104, "value3":false, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.498993, "value1":0.9016213008618695, "value2":8432726482609273880, "value3":false, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349195, "value1":0.16850895992044404, "value2":6332393888509943818, "value3":true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530337, "value1":0.8558953203814225, "value2":76710422944574508, "value3":true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345065, "value1":0.8230787557550705, "value2":4201898391845439571, "value3":true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947222, "value1":0.9688974381514411, "value2":2441269560109987314, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.287425, "value1":0.20362983597374928, "value2":5349125652009205097, "value3":true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519644, "value1":0.3548258293385043, "value2":1594647307470342039, "value3":true, "__name__":"metric_00000602", "key_5":"i","key_0":"j","key_1":"f", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.027626, "value1":0.9120926865897546, "value2":356446699135295995, "value3":true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.054561, "value1":0.22396147232559485, "value2":3930608769790828524, "value3":false, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.388931, "value1":0.5101134733621842, "value2":6568513065035568570, "value3":false, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567065, "value1":0.79529931520039, "value2":6002480040724485306, "value3":false, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449050, "value1":0.9291008790205846, "value2":741995070657009132, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798788, "value1":0.9907520697579564, "value2":2343010721096789225, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.927945, "value1":0.8814555609090619, "value2":6729196069806990782, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917877, "value1":0.15366203128840322, "value2":2435285873402812217, "value3":false, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.301622, "value1":0.20876721045801763, "value2":1403445590939959710, "value3":false, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.745719, "value1":0.6060177757203955, "value2":2370675136825151538, "value3":true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156169, "value1":0.40315211694606023, "value2":5606845635294528121, "value3":false, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.651978, "value1":0.02572219217737101, "value2":6216284509197748039, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.395139, "value1":0.4352070266196555, "value2":2437710317158637331, "value3":false, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.132661, "value1":0.3845242564807311, "value2":1179550652058134811, "value3":true, "__name__":"metric_00000616", "key_7":"j","key_1":"k","key_5":"d", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.088075, "value1":0.27124710684397163, "value2":5954367646517477044, "value3":false, "__name__":"metric_00000618", "key_3":"g","key_0":"e","key_1":"j", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213664, "value1":0.9844242730115798, "value2":1430390936289238362, "value3":false, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070775, "value1":0.6998833636894136, "value2":8373393772820367148, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213733, "value1":0.44329246485521806, "value2":6509431828213171835, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608249, "value1":0.15493446331886743, "value2":4384868943457502322, "value3":true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.956693, "value1":0.9523530599130996, "value2":365504939691815078, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.432058, "value1":0.16692183631193722, "value2":4864357898977097624, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.303592, "value1":0.934106512625592, "value2":7515104000227729499, "value3":false, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333787, "value1":0.6736717480454248, "value2":5830960931165083327, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359385, "value1":0.922695683809537, "value2":6703804740319670530, "value3":true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.211845, "value1":0.13745320161077518, "value2":8688931964841266452, "value3":false, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600304, "value1":0.365028686112319, "value2":4172510140835977605, "value3":true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045527, "value1":0.1293058694163136, "value2":4396801118892738860, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124929, "value1":0.2927998769649342, "value2":7840410733128704268, "value3":true, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.708707, "value1":0.42917010198000677, "value2":190549078586280269, "value3":false, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543921, "value1":0.45395915446447055, "value2":4038857743678185292, "value3":true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001958, "value1":0.8799882402246043, "value2":8959405737069039234, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834357, "value1":0.9787241750393116, "value2":4713255732772978232, "value3":true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398402, "value1":0.3356690698861162, "value2":8600435717396946203, "value3":true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.532477, "value1":0.5790086501753019, "value2":585720405314800565, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995284, "value1":0.019519382747907175, "value2":561302586535494029, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.492003, "value1":0.22729751026433825, "value2":2059445045999328988, "value3":false, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.357595, "value1":0.2653305071097244, "value2":7325567281409701416, "value3":false, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423155, "value1":0.8492553718976581, "value2":5551377313853459727, "value3":true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582438, "value1":0.9616765075659243, "value2":1026983801153932760, "value3":false, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409214, "value1":0.1250667617190597, "value2":5921255804215527024, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744368, "value1":0.8513042177093529, "value2":6603153879156409175, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970658, "value1":0.2306199750722779, "value2":2906238010168281574, "value3":false, "__name__":"metric_00000643", "key_6":"c","key_1":"e","key_4":"d", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520199, "value1":0.9665587748785213, "value2":2458054282507819610, "value3":true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.098618, "value1":0.12617188969368395, "value2":2230070381051915192, "value3":false, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279517, "value1":0.4244894688924968, "value2":7931154210918628962, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394850, "value1":0.011258876506077731, "value2":3335326787820059838, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.672924, "value1":0.7327755721792956, "value2":3880440053684337972, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.431109, "value1":0.6741923566632704, "value2":2327920765233612965, "value3":true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.659350, "value1":0.11708731624685002, "value2":276841347975235584, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.115617, "value1":0.43754612855223973, "value2":3365515217954889202, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533611, "value1":0.8399047779975791, "value2":2365721152492621420, "value3":false, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265333, "value1":0.7911288127976842, "value2":5955361860868770714, "value3":true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966474, "value1":0.3536325699896996, "value2":7032196306810796442, "value3":true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684964, "value1":0.8626527282208771, "value2":7197488766581698381, "value3":true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559000, "value1":0.010659679933132453, "value2":4115322864019466158, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455110, "value1":0.6204915686658947, "value2":6428809732964624602, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625937, "value1":0.9727096027334052, "value2":6229477409536583105, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.113032, "value1":0.5903167177404812, "value2":4156394072465021166, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567689, "value1":0.10316863931938577, "value2":8418749122094896416, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.987774, "value1":0.7873883846984865, "value2":2030919041886568670, "value3":false, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.811061, "value1":0.4992724231992665, "value2":3986295112752729017, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194651, "value1":0.2743659705657923, "value2":583251715501918289, "value3":false, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950390, "value1":0.5427606927000886, "value2":2986321212639280881, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598120, "value1":0.20329762636615106, "value2":5612645284463886443, "value3":true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950214, "value1":0.4515603514606786, "value2":259527570321604761, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645723, "value1":0.13300625888317788, "value2":9108039921133288290, "value3":false, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001569, "value1":0.01777574006312956, "value2":7180543411595530153, "value3":true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036902, "value1":0.19075245150411826, "value2":5889621655868191557, "value3":false, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929644, "value1":0.8218270153578531, "value2":6338216490369302155, "value3":false, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036799, "value1":0.8121072864023562, "value2":4644698086978403677, "value3":false, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099804, "value1":0.41836860978181317, "value2":2663944418369780756, "value3":true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.607018, "value1":0.04603283781595935, "value2":7178332091114199020, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075733, "value1":0.9728379954214806, "value2":4310309256078899557, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795422, "value1":0.6466237769859894, "value2":344630441945165516, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924050, "value1":0.5005809462467858, "value2":4504593684309524057, "value3":false, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352892, "value1":0.9539405805618134, "value2":996952878654521350, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141322, "value1":0.01789552453746491, "value2":5938050187519951412, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.783153, "value1":0.1436758434821978, "value2":6441131001960903605, "value3":false, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473891, "value1":0.3495022243472323, "value2":4306013569645375103, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814977, "value1":0.38372259399528474, "value2":436380091488564183, "value3":true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716628, "value1":0.6699570981955947, "value2":3523603923463352771, "value3":true, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735953, "value1":0.6776358805955892, "value2":5856380565120662945, "value3":false, "__name__":"metric_00000693", "key_5":"e","key_8":"b","key_0":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265185, "value1":0.11607380521583967, "value2":5582686822797693437, "value3":false, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516162, "value1":0.2994616004522488, "value2":2261467086953311820, "value3":false, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334873, "value1":0.14715552224004888, "value2":8821110853892469057, "value3":false, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.400930, "value1":0.8143295386608986, "value2":1312967392198736440, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196057, "value1":0.5352481987346313, "value2":3859881997560184862, "value3":false, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925763, "value1":0.823842231642209, "value2":7076092787875459652, "value3":true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.012976, "value1":0.08033889071311322, "value2":2194666423113168625, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656711, "value1":0.979250223824098, "value2":295513105593707440, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542943, "value1":0.34216024465522854, "value2":8482137672426222632, "value3":false, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738294, "value1":0.6164314293888586, "value2":7863948057381466211, "value3":false, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957221, "value1":0.4133406835193543, "value2":3905154346081616319, "value3":true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047765, "value1":0.9114235810126055, "value2":8926222646986865918, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423884, "value1":0.30572384015704446, "value2":2339988500116190675, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209371, "value1":0.07849561730590998, "value2":3882974907897625962, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310209, "value1":0.2535587414218041, "value2":7470349035360557369, "value3":false, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988088, "value1":0.2879292667105676, "value2":5363951038000132107, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947720, "value1":0.958399617505033, "value2":5218176225849015970, "value3":false, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.674130, "value1":0.9617762098247892, "value2":5270132257081817867, "value3":true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279827, "value1":0.8686925044217091, "value2":7240549587345744089, "value3":true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.257178, "value1":0.28758053646406956, "value2":7938343410176527229, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870350, "value1":0.16917076304978226, "value2":9111760754875505635, "value3":true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.763299, "value1":0.8896924750809221, "value2":6658856857804173307, "value3":true, "__name__":"metric_00000690", "key_8":"k","key_1":"g","key_5":"h", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734200, "value1":0.44507702947189576, "value2":4594583173060659683, "value3":true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155510, "value1":0.8897408774019222, "value2":3124062956444214018, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.363722, "value1":0.07362703906856644, "value2":6124606823090260320, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336581, "value1":0.7586482338064128, "value2":2691495825421343596, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075863, "value1":0.7294045132994561, "value2":8264827508243049447, "value3":false, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108381, "value1":0.45788844677640256, "value2":2616045646790504337, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901873, "value1":0.08770240902249962, "value2":6640573405482547401, "value3":true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.438350, "value1":0.25439050102218247, "value2":7286046839968613612, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.314379, "value1":0.33872622756746645, "value2":2585280362910555462, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736079, "value1":0.12615475607591273, "value2":8133018761204018534, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130003, "value1":0.7963354396181769, "value2":6450859077912673686, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.485449, "value1":0.39497609142163287, "value2":170013329412671465, "value3":true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.06350907823991539, "value2":18163724507539892, "value3":false, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.6759106102023759, "value2":8613373636446320004, "value3":true, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522262, "value1":0.9415564313481927, "value2":183076459384497822, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.456571, "value1":0.3755600914749928, "value2":1317951286539480179, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713142, "value1":0.3684027597862527, "value2":6666246661014977211, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332747, "value1":0.2712525974797622, "value2":6685862283475642897, "value3":false, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622289, "value1":0.8099425380336615, "value2":6696254947892698280, "value3":true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099077, "value1":0.4218807485485893, "value2":8151022600373825319, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833739, "value1":0.471092674976307, "value2":4241908968763096086, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512079, "value1":0.6891384899122228, "value2":2796503546861546091, "value3":true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.405406, "value1":0.9060026283921703, "value2":4773855974745798571, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742427, "value1":0.6129480541080172, "value2":3917404693410650609, "value3":true, "__name__":"metric_00000739", "key_5":"k","key_2":"d","key_3":"e", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.613376, "value1":0.3584257069399756, "value2":6068105032813229930, "value3":true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.759402, "value1":0.29210317562134064, "value2":3082388413784210214, "value3":true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090526, "value1":0.4678159387104084, "value2":1605658432958093758, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380672, "value1":0.5086572266239265, "value2":2924449619880684423, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.249203, "value1":0.6593022058411281, "value2":727243123088870599, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.142921, "value1":0.5973096622193752, "value2":2056855353114653381, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041929, "value1":0.03439471294074195, "value2":8729944365498889284, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970035, "value1":0.629094069595545, "value2":954705184774295714, "value3":false, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656521, "value1":0.4935396027418025, "value2":1513112375773530412, "value3":false, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286143, "value1":0.590796213295746, "value2":3085171514913296735, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.611048, "value1":0.3503905353305475, "value2":5805006250107454516, "value3":false, "__name__":"metric_00000748", "key_6":"f","key_3":"e","key_4":"h", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265940, "value1":0.8719050400063297, "value2":2334709836967925044, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.231945, "value1":0.9482208594424564, "value2":7605532380274260726, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099422, "value1":0.6387093062831974, "value2":6416034503475989540, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862301, "value1":0.0844793706976894, "value2":1724212296853776891, "value3":false, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.882958, "value1":0.45080094639121554, "value2":6596820877226155907, "value3":true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851802, "value1":0.784896015737414, "value2":2267862818127250105, "value3":false, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.871761, "value1":0.9219727566018483, "value2":9044274636085577036, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704109, "value1":0.6618033501935818, "value2":8585487696070011938, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.863670, "value1":0.6882846261541673, "value2":5478374167595454238, "value3":true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183651, "value1":0.21762989261712523, "value2":2054870959404109040, "value3":false, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157246, "value1":0.1072057187986972, "value2":2474773268716838947, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263088, "value1":0.6424740570966222, "value2":4311694979316615362, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227492, "value1":0.6060012408935248, "value2":8262744174339677603, "value3":true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919259, "value1":0.26635737492611994, "value2":146832998544852566, "value3":true, "__name__":"metric_00000762", "key_2":"e","key_0":"d", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.810229, "value1":0.6433893313883111, "value2":6020303671131449049, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.021406, "value1":0.5663729785218883, "value2":6847532330100114835, "value3":true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.331141, "value1":0.03123867674281506, "value2":2370510435982901478, "value3":true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313836, "value1":0.7842831796528092, "value2":1464971588517507800, "value3":false, "__name__":"metric_00000711", "key_6":"i","key_0":"j","key_4":"h", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147234, "value1":0.232833406436683, "value2":5302769737190544079, "value3":true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423280, "value1":0.5646235246180139, "value2":3807301011411244203, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.872535, "value1":0.3508318181055984, "value2":8035632162257252295, "value3":true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516267, "value1":0.7376190499237748, "value2":1779371132519351560, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.020480, "value1":0.5129193623789473, "value2":6378565762108475442, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421514, "value1":0.72712434334131, "value2":3908844968753938444, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026297, "value1":0.8255820903801332, "value2":5180017205009891488, "value3":false, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138610, "value1":0.5467990020300735, "value2":8134249053330675938, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754861, "value1":0.5832263769294902, "value2":8216858329154999597, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655676, "value1":0.45804041306603793, "value2":8403704610668639637, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.091295, "value1":0.7313452902241809, "value2":6274232113944917429, "value3":false, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370397, "value1":0.7082796943852117, "value2":16646752364177844, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.072185, "value1":0.8009986319191017, "value2":6123195610574092125, "value3":true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068380, "value1":0.3124676060355689, "value2":2569282749677030904, "value3":true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013897, "value1":0.9321729707819904, "value2":2281258981052686264, "value3":true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.963470, "value1":0.9927608945020866, "value2":6050801587231871156, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481229, "value1":0.3689051456876514, "value2":1924733693360792633, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.295891, "value1":0.44913357839758283, "value2":8069037361024844952, "value3":false, "__name__":"metric_00000779", "key_7":"i","key_1":"h", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409485, "value1":0.18423629567656025, "value2":5066355149165531813, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744404, "value1":0.9737949013567385, "value2":5174295290317701561, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739701, "value1":0.4976173273573324, "value2":5658855721642744182, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.186283, "value1":0.419361633943128, "value2":2053614790636840527, "value3":true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566841, "value1":0.7094596520198607, "value2":2148895338889314160, "value3":false, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.215921, "value1":0.34276587375112927, "value2":1645566772308625552, "value3":false, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.701909, "value1":0.22008909623955855, "value2":2295629303145372991, "value3":true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002697, "value1":0.1765088617877308, "value2":4201797060324467420, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354799, "value1":0.6538892310533279, "value2":4577607391337663337, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345385, "value1":0.3843950816957309, "value2":8800458811056296695, "value3":true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988457, "value1":0.5588868714021769, "value2":8433496452370061074, "value3":false, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415009, "value1":0.1746188760147135, "value2":6912110645446824396, "value3":false, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655363, "value1":0.9949350156565988, "value2":8110347995943103660, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.242069, "value1":0.48932951486152193, "value2":8286237027514157124, "value3":true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085368, "value1":0.0689091306398061, "value2":5859648999386501040, "value3":true, "__name__":"metric_00000801", "key_6":"e","key_3":"j","key_4":"d", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.661208, "value1":0.7864155291083853, "value2":7826276349217880531, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.803222, "value1":0.6015539667825235, "value2":1102772758151131198, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.458180, "value1":0.3809696927779604, "value2":4439664400270353254, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334077, "value1":0.4097523501895673, "value2":7454062761319263955, "value3":true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.885572, "value1":0.548188359160327, "value2":3020054439494862697, "value3":true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489119, "value1":0.6763563343582067, "value2":4242764599158368849, "value3":true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.187849, "value1":0.7091709802238197, "value2":6867639316908364367, "value3":true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663732, "value1":0.42631474518610635, "value2":9001795643794270349, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.585010, "value1":0.6237338911978378, "value2":4365340422086338410, "value3":false, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.379644, "value1":0.15387674044669916, "value2":818900165755081001, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791337, "value1":0.32588514831554516, "value2":1691735796263334884, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.082385, "value1":0.5745615587827129, "value2":772277205802820637, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086473, "value1":0.8230348349086334, "value2":8156169851471463456, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808463, "value1":0.47620124507643474, "value2":2357967788477712632, "value3":true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380686, "value1":0.011323528011005758, "value2":2028993392634762324, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.371579, "value1":0.5028955605922396, "value2":7910467950096619453, "value3":false, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.883127, "value1":0.20067026957438594, "value2":3625887507362062386, "value3":false, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.305224, "value1":0.7123715236402623, "value2":7522375891893442954, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443461, "value1":0.7555458976714571, "value2":655917858760911221, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901204, "value1":0.7387538487064921, "value2":7525092307824794219, "value3":true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.376917, "value1":0.7993175855480273, "value2":2807412127532185201, "value3":true, "__name__":"metric_00000822", "key_6":"h","key_5":"c", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740075, "value1":0.3001865006906782, "value2":7190874681083752315, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209665, "value1":0.89276344594564, "value2":4778320938543530902, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.341730, "value1":0.20200750378616714, "value2":1664606777941171586, "value3":false, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.629099, "value1":0.017583528638071864, "value2":2707650519412422348, "value3":false, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057519, "value1":0.04492925365315839, "value2":2578667952703739482, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.723798, "value1":0.8454916194785576, "value2":1835422305385826626, "value3":true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.034767, "value1":0.201322613886975, "value2":7356183312830760350, "value3":false, "__name__":"metric_00000794", "key_5":"h","key_8":"a","key_1":"b", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520001, "value1":0.5251237247182675, "value2":134900877343320532, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489083, "value1":0.8513994038065534, "value2":2159556869883772494, "value3":false, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344496, "value1":0.16316310680106225, "value2":5518614284082110727, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.724230, "value1":0.6875155435793379, "value2":35737216326475935, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568987, "value1":0.5138501785175705, "value2":4338888431721237426, "value3":false, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598387, "value1":0.4390074014236514, "value2":7041842749721887181, "value3":false, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.626270, "value1":0.2391917526821926, "value2":3091537573368363672, "value3":false, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.297931, "value1":0.11690748126426959, "value2":4039852836857346610, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.175994, "value1":0.6789300486469183, "value2":1246840956922367253, "value3":false, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976992, "value1":0.43367432666147043, "value2":564959136229688173, "value3":false, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703741, "value1":0.47707265609489413, "value2":4068705543834170412, "value3":false, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.235944, "value1":0.8012892224294691, "value2":1988300147401692461, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499512, "value1":0.9990473134151273, "value2":8394845441054007311, "value3":false, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404130, "value1":0.6068400138809874, "value2":1935348003856793271, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542560, "value1":0.12776285353397152, "value2":6795264619125251375, "value3":true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693012, "value1":0.2569652780369043, "value2":1475809083419006811, "value3":false, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068970, "value1":0.3881338979751516, "value2":1218503134871756052, "value3":false, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264926, "value1":0.19791850227382649, "value2":5513222623463327397, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.408769, "value1":0.39494174800117104, "value2":1576673230492191692, "value3":false, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.081892, "value1":0.9402614771700961, "value2":1589947956431908818, "value3":true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.058425, "value1":0.5327215264224379, "value2":445308327885517649, "value3":false, "__name__":"metric_00000850", "key_8":"d","key_3":"b","key_4":"f", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.838442, "value1":0.9519819634366921, "value2":745087205152711718, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064037, "value1":0.23777264651361862, "value2":4675095342249511184, "value3":true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.518857, "value1":0.8856046396694535, "value2":1109051615753148759, "value3":true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077463, "value1":0.8624901705258308, "value2":3793672263940517136, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.921342, "value1":0.9049145350717305, "value2":3570805638374912588, "value3":false, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157723, "value1":0.6223760989748452, "value2":2996869515277202402, "value3":false, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155659, "value1":0.13854963250628805, "value2":6512544911629959850, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977987, "value1":0.036771554508911904, "value2":5994549505435027834, "value3":true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265064, "value1":0.3403489462496419, "value2":2814929460502496771, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404153, "value1":0.3208642154389825, "value2":5256172233350798960, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.312852, "value1":0.011902750850203012, "value2":5526100237606282605, "value3":false, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.491995, "value1":0.46992302986974577, "value2":157557322951942167, "value3":false, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704212, "value1":0.9425835794283676, "value2":915577115687321604, "value3":false, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.576745, "value1":0.7854877804515764, "value2":6382074091209031167, "value3":true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070556, "value1":0.18030779510661127, "value2":6308540602135606131, "value3":true, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542532, "value1":0.3384327861708296, "value2":1613795669513593265, "value3":false, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511484, "value1":0.8996232864271185, "value2":7405365074827637694, "value3":true, "__name__":"metric_00000840", "key_3":"i","key_7":"c","key_2":"b", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.562131, "value1":0.10596222820549657, "value2":3306812514398053557, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.391786, "value1":0.20647295311593183, "value2":5486022828508870343, "value3":false, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466143, "value1":0.6224930113197604, "value2":8535178573473477072, "value3":true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423074, "value1":0.3088852292542975, "value2":3166186037825297943, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.762488, "value1":0.33490363470863166, "value2":2912130613463378266, "value3":true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713117, "value1":0.2993000272696047, "value2":494302122350368825, "value3":false, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028636, "value1":0.6503466890929638, "value2":4759183611197798725, "value3":false, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404461, "value1":0.043678508683084524, "value2":6408629579798949638, "value3":true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077439, "value1":0.2418209601880771, "value2":6820734383602297695, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.624831, "value1":0.4718269105814766, "value2":4622812711240963076, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053083, "value1":0.6875169297990352, "value2":6119677537083039891, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345042, "value1":0.7324917812164005, "value2":5511749504782560024, "value3":true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.623531, "value1":0.9320016848414698, "value2":744369719732623498, "value3":false, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.182404, "value1":0.9674131847138503, "value2":3838480249361897923, "value3":false, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.696213, "value1":0.7881956638078709, "value2":4371203118864420550, "value3":true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.628037, "value1":0.17572736986644139, "value2":3461303046815504013, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273728, "value1":0.6916742591847842, "value2":2382468702478522273, "value3":false, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335163, "value1":0.9758093684084915, "value2":1680436716344950339, "value3":true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782020, "value1":0.1537134689497552, "value2":4414043374214499577, "value3":true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.134363, "value1":0.5083067111414208, "value2":4982306711291417821, "value3":true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.865817, "value1":0.9406393737656967, "value2":5857776137176686892, "value3":true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716601, "value1":0.1916346431757554, "value2":5910562859970195422, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.427366, "value1":0.03546691845658599, "value2":7315776376647279090, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950000, "value1":0.5369396780416313, "value2":7524727304557507728, "value3":true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.325387, "value1":0.06313877213741768, "value2":7980198837673375086, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950927, "value1":0.32484655923390965, "value2":8297079571694626407, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832701, "value1":0.8101399444767259, "value2":6277528961017233439, "value3":true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969582, "value1":0.5893386387714608, "value2":8824793605581596113, "value3":false, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302334, "value1":0.03653755659557745, "value2":2394591637437372554, "value3":true, "__name__":"metric_00000896", "key_5":"i","key_1":"b", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463188, "value1":0.2645292258357236, "value2":7750296583775050063, "value3":false, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.774073, "value1":0.7854839946508317, "value2":4045742754979149401, "value3":false, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.729676, "value1":0.4706720879040935, "value2":4750321291295032700, "value3":false, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618814, "value1":0.2230204325484985, "value2":9003120111243218251, "value3":true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603138, "value1":0.22960374502253963, "value2":6338685193000832935, "value3":false, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.412324, "value1":0.4957126042457222, "value2":8735002444024394358, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657244, "value1":0.08184942404758783, "value2":7470578294751016176, "value3":true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442850, "value1":0.7229538622773333, "value2":4724836800934702763, "value3":false, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156841, "value1":0.7233638365572691, "value2":7858912978560139100, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734329, "value1":0.3404208114361277, "value2":3050306358924697552, "value3":false, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.465656, "value1":0.38391986849101767, "value2":307361261718394449, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.445647, "value1":0.990731352209279, "value2":1207071782335047705, "value3":false, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298620, "value1":0.11180836767709064, "value2":3870338901819904571, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.095445, "value1":0.2870001965302557, "value2":2415945267551238695, "value3":true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.918448, "value1":0.7277791066957175, "value2":2821193899111524763, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541594, "value1":0.09005789780178877, "value2":2925035824598662472, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.631726, "value1":0.9959155652844405, "value2":3833701945431571315, "value3":true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049800, "value1":0.8110556334200572, "value2":2218497283809170432, "value3":false, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070530, "value1":0.65401432282683, "value2":498838541277181702, "value3":false, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736247, "value1":0.7394025858411815, "value2":7148599556072452144, "value3":false, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736608, "value1":0.08218047603491815, "value2":3776524585186258439, "value3":true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003840, "value1":0.6757338653122574, "value2":379207741924258587, "value3":true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362539, "value1":0.11651117191245357, "value2":3906526956584011608, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.539295, "value1":0.44782322310256395, "value2":2595894391836664475, "value3":false, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392973, "value1":0.8520294340743481, "value2":7384112654505361476, "value3":false, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.078027, "value1":0.3547792528217613, "value2":8155526662947982478, "value3":false, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.776543, "value1":0.6604220360650918, "value2":4011373177447838470, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.691748, "value1":0.4862664301051333, "value2":7696312329542980810, "value3":true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964567, "value1":0.11169294328511947, "value2":4727405940623359973, "value3":true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722155, "value1":0.3780569227165093, "value2":3316392130699376882, "value3":true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457388, "value1":0.9243183084724984, "value2":2432925818286345186, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.040558, "value1":0.7743604477646876, "value2":627834147679968207, "value3":true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272066, "value1":0.5917633264655242, "value2":5918933193525430349, "value3":false, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200729, "value1":0.7327561219912169, "value2":1393602674371677220, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.905147, "value1":0.8568183299448732, "value2":9034660617242414820, "value3":false, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310139, "value1":0.802088771431449, "value2":3471155770712582770, "value3":true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.821462, "value1":0.5273591260032361, "value2":5140674777677119228, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.384953, "value1":0.4025027387592999, "value2":5097429126997347384, "value3":true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.829568, "value1":0.8045993326905381, "value2":2809186620050229002, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294923, "value1":0.5644401567133956, "value2":3718732987675906525, "value3":true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.556732, "value1":0.48820899714237304, "value2":4078290062854715892, "value3":true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.176213, "value1":0.2510405463103239, "value2":635371208287508106, "value3":false, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.267004, "value1":0.22999772540025595, "value2":1871342923142731692, "value3":false, "__name__":"metric_00000919", "key_9":"g","key_6":"i", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.506694, "value1":0.4581021751822573, "value2":8794558811813763792, "value3":false, "__name__":"metric_00000924", "key_3":"k","key_4":"a","key_1":"c", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354208, "value1":0.6088901271960526, "value2":132806552362853695, "value3":false, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516024, "value1":0.7664701528892452, "value2":107006237498756274, "value3":false, "__name__":"metric_00000923", "key_6":"e","key_3":"h","key_5":"j", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.750598, "value1":0.3217472443950159, "value2":8378919480372645762, "value3":false, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419841, "value1":0.6663125192971521, "value2":2423610833256438065, "value3":true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164890, "value1":0.2006090031725153, "value2":1844725035830445264, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264867, "value1":0.42671643117928926, "value2":3438912836085300466, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.430421, "value1":0.07257562390395779, "value2":6088518193551139346, "value3":false, "__name__":"metric_00000939", "key_8":"c","key_1":"k","key_6":"g", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.673454, "value1":0.6691702132206785, "value2":8625848517577296022, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995936, "value1":0.702118704788376, "value2":4086471613439852656, "value3":true, "__name__":"metric_00000941", "key_5":"i","key_9":"g","key_4":"i", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964584, "value1":0.7747761523528802, "value2":1036051996047955870, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.948289, "value1":0.25140012327548283, "value2":788619002191666922, "value3":false, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372555, "value1":0.9999769483157629, "value2":7184076213289982875, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.920761, "value1":0.868179497430529, "value2":1697965828697918238, "value3":true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372531, "value1":0.5396817815353727, "value2":7506498166097940651, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.278206, "value1":0.17061135686969672, "value2":713580384695452010, "value3":false, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600654, "value1":0.6616469498902554, "value2":774519242775197957, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188862, "value1":0.5531871959368098, "value2":7422109554403592667, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457643, "value1":0.7840548969774126, "value2":7827846462514659006, "value3":false, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655234, "value1":0.76526166835666, "value2":2375166258510616618, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370401, "value1":0.021438154414782817, "value2":2771089523399378961, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137411, "value1":0.5646878830454839, "value2":5968018682136017050, "value3":false, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423235, "value1":0.36529823070220574, "value2":7162022842889925076, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272788, "value1":0.219359460529438, "value2":181241653539382719, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.707284, "value1":0.3923075865949308, "value2":2782827649523891195, "value3":true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858145, "value1":0.688162185571026, "value2":5086226934464427172, "value3":true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867650, "value1":0.6841047492247676, "value2":8098586901309655486, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.902390, "value1":0.706932118893774, "value2":9086709640294623772, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131998, "value1":0.8578147943558058, "value2":41651666626493230, "value3":false, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932017, "value1":0.24496475938485004, "value2":7307212968529901562, "value3":false, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.469902, "value1":0.3291978179419431, "value2":6674355534843744608, "value3":false, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.715852, "value1":0.5949781040197312, "value2":6585582445453110248, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.439874, "value1":0.25098256408304576, "value2":9009893614730547675, "value3":false, "__name__":"metric_00000972", "key_4":"d","key_6":"k","key_2":"f", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.775831, "value1":0.888090262607404, "value2":7565201070136116477, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480681, "value1":0.47062171981528383, "value2":4853013107516729574, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.654578, "value1":0.4096044979675714, "value2":1606198625715352901, "value3":false, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.881884, "value1":0.31666628836121824, "value2":5391861566395998267, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028180, "value1":0.2934267405568024, "value2":1598172822680744133, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245149, "value1":0.09374074237151239, "value2":6863637452711576117, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105508, "value1":0.7783021910779061, "value2":2669034341302440615, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474623, "value1":0.5494532131881231, "value2":6109855504680847451, "value3":true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625728, "value1":0.471409564376847, "value2":5293699949847690348, "value3":true, "__name__":"metric_00000957", "key_5":"g","key_0":"c","key_1":"i", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382751, "value1":0.7835774962833667, "value2":5926458098948297057, "value3":false, "__name__":"metric_00000960", "key_3":"e","key_2":"c", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.527286, "value1":0.6322552498331545, "value2":437257708157331646, "value3":false, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.166237, "value1":0.6386670258285629, "value2":2053216930342060527, "value3":false, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.942234, "value1":0.7212756328576085, "value2":8527112750674030055, "value3":true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.880290, "value1":0.19477642153523786, "value2":8136028599409004140, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.477164, "value1":0.7162729706892131, "value2":3933534960467417574, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089992, "value1":0.47761006384043037, "value2":4843140910052787407, "value3":false, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.204970, "value1":0.4917659716382411, "value2":8639172605312608175, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.428786, "value1":0.5707960425927678, "value2":757157967003444236, "value3":false, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131753, "value1":0.32321264423119905, "value2":1903644540800881525, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.540623, "value1":0.528958932538034, "value2":8999603816652982043, "value3":true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.991648, "value1":0.7098945581193735, "value2":8146175572944228409, "value3":false, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.523699, "value1":0.499825459180699, "value2":1724724639775860908, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116240, "value1":0.6561999665129753, "value2":5656111967421346554, "value3":false, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618968, "value1":0.2848979660832576, "value2":2049018652807530472, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116765, "value1":0.8252075257277172, "value2":4775243296364010632, "value3":false, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.410327, "value1":0.06668100857135961, "value2":2183721339943212726, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.688645, "value1":0.41653256081146506, "value2":5969120448550176776, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657571, "value1":0.9568582636724965, "value2":730216901987394123, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043883, "value1":0.40826500145612177, "value2":6407138186851676502, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474102, "value1":0.6914723313082367, "value2":469628036375832328, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596154, "value1":0.050104900538538846, "value2":1354999585856659679, "value3":true, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096028, "value1":0.6282483124845963, "value2":116602313358973711, "value3":true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568083, "value1":0.07619222667132636, "value2":6984709326920626754, "value3":false, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.984466, "value1":0.3323790308005344, "value2":675504136493878123, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553260, "value1":0.45304427452622936, "value2":8153841508229333988, "value3":true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781038, "value1":0.2512350719527022, "value2":1697664403294613401, "value3":false, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230875, "value1":0.8575694561008984, "value2":1340681872774498770, "value3":true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180011, "value1":0.7159258322890252, "value2":7263307563948982502, "value3":false, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318368, "value1":0.7865256606198752, "value2":7176709659967207104, "value3":false, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.594668, "value1":0.9818689490949409, "value2":4816922436179228865, "value3":false, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.178515, "value1":0.9620215494327841, "value2":7544435408144975228, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596427, "value1":0.7405089300246521, "value2":6613451887211746808, "value3":true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317317, "value1":0.8954816734384349, "value2":4023989770168105300, "value3":true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.624103, "value1":0.1565943994591959, "value2":5829676556068380878, "value3":false, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.259139, "value1":0.08662232425939413, "value2":2137874058817765285, "value3":true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.495345, "value1":0.7444165320555235, "value2":1512339128303545319, "value3":true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.873798, "value1":0.10364343021646215, "value2":497864294044982023, "value3":false, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077492, "value1":0.7825627514339433, "value2":497480348120915725, "value3":false, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.887058, "value1":0.6684575895211121, "value2":8461019606631895989, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374572, "value1":0.30885911828548585, "value2":6810974248136318973, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.381097, "value1":0.062395135046400096, "value2":1505084432772472894, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796962, "value1":0.4796896579785329, "value2":5095295117314039911, "value3":true, "__name__":"metric_00000015", "key_6":"k","key_8":"k","key_0":"a", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.625725, "value1":0.5385278812414747, "value2":7715803409892624904, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430762, "value1":0.42060637147418184, "value2":7474778452563228226, "value3":true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886443, "value1":0.6738972816294144, "value2":582629852137142515, "value3":false, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467512, "value1":0.5597247611878908, "value2":8349056677712047690, "value3":false, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301942, "value1":0.5324107668792922, "value2":2601407711369777027, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.959260, "value1":0.3409466272260335, "value2":495480232494613355, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824942, "value1":0.690980654161434, "value2":73040149987568706, "value3":false, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647608, "value1":0.9936863999178829, "value2":2595521887523034584, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206529, "value1":0.7042253871626942, "value2":6786989253298041563, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654900, "value1":0.04874287974341856, "value2":6080516591012621533, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.192783, "value1":0.636728117953383, "value2":7620028626322794241, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.757400, "value1":0.3932217938599448, "value2":4695382142084123513, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578308, "value1":0.7154241327471093, "value2":8875337832548314386, "value3":false, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025767, "value1":0.4896117284329862, "value2":7827833960384552450, "value3":false, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695043, "value1":0.4735636518198123, "value2":5776645863491585306, "value3":false, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067392, "value1":0.2568532613491557, "value2":4347058259737260198, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145851, "value1":0.867279626694445, "value2":7767138234344286903, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.491824, "value1":0.43935171533716344, "value2":2117420540525504913, "value3":true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413343, "value1":0.7815374349474034, "value2":7674175325619501370, "value3":false, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408821, "value1":0.6468318207569199, "value2":3382854605353241895, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864749, "value1":0.5018653748831062, "value2":3163263403613237522, "value3":true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811002, "value1":0.1264068238414509, "value2":8051527093894333669, "value3":true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922550, "value1":0.797889997967527, "value2":4244570533853409024, "value3":true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045039, "value1":0.6136304375492249, "value2":7079529098273717788, "value3":true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914685, "value1":0.6884426743061036, "value2":3608727831529868858, "value3":true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.429556, "value1":0.37532682108769366, "value2":1177251454911069616, "value3":false, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779878, "value1":0.5585578516583662, "value2":5769602877819666427, "value3":false, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.711503, "value1":0.3057843361151979, "value2":2044462261279390237, "value3":true, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142192, "value1":0.0045630377024844515, "value2":4953964644129769622, "value3":true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.542755, "value1":0.236125279127913, "value2":5065525208816588827, "value3":false, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578196, "value1":0.8156211595242715, "value2":1914083289002989297, "value3":false, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110649, "value1":0.26711648319193754, "value2":2012127151944954874, "value3":true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041387, "value1":0.3776450082432283, "value2":6801275370589593726, "value3":false, "__name__":"metric_00000047", "key_7":"h","key_2":"g","key_5":"i", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808716, "value1":0.04169212753399746, "value2":9054979711039988827, "value3":false, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062171, "value1":0.7218616273097948, "value2":740018633881444642, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146630, "value1":0.5003843971030013, "value2":1655397867016219461, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.309989, "value1":0.11293107623655986, "value2":3732853355411608797, "value3":false, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.333206, "value1":0.17591880468368, "value2":8123233873446258834, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145427, "value1":0.43545559910718556, "value2":4862315899801656213, "value3":false, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.644901, "value1":0.9950957928656415, "value2":4691155465042626084, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568884, "value1":0.6163869833398097, "value2":9217755948904745777, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930812, "value1":0.3180893474971966, "value2":8238800974641951308, "value3":false, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.148564, "value1":0.7643083780125881, "value2":8977913426229482254, "value3":true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369460, "value1":0.37462703037399614, "value2":2532444456287017290, "value3":true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134410, "value1":0.7407330348790686, "value2":2290478450455253969, "value3":false, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.623624, "value1":0.23159512407081487, "value2":39715061290046297, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497315, "value1":0.9059175925838082, "value2":9042312906596816530, "value3":true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.591581, "value1":0.9196881162354394, "value2":3577426841534493512, "value3":true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.229214, "value1":0.677998841646216, "value2":4155628140480575006, "value3":true, "__name__":"metric_00000063", "key_8":"f","key_1":"c","key_6":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419533, "value1":0.14115394562090194, "value2":4482838294646117015, "value3":true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352495, "value1":0.614805207196906, "value2":6313920398652894644, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158009, "value1":0.13136879968484017, "value2":1444276749086955965, "value3":true, "__name__":"metric_00000067", "key_4":"d","key_9":"a","key_0":"c", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394523, "value1":0.390863430258994, "value2":1479817062186057883, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084220, "value1":0.8798339682629527, "value2":5374208931949760111, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.604039, "value1":0.6734519068343852, "value2":6936119518718549917, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.407184, "value1":0.883448918218706, "value2":2783824499389725952, "value3":false, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.987554, "value1":0.04159873743578623, "value2":6347778463017159524, "value3":true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015322, "value1":0.9633747370317121, "value2":886249751055268990, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903730, "value1":0.2538648955540415, "value2":5257877330086190982, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109241, "value1":0.3895615184638186, "value2":2398924749380096990, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946726, "value1":0.8408201754901692, "value2":5956645672957756887, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.126734, "value1":0.8247534822394186, "value2":8919709316239049787, "value3":true, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419044, "value1":0.20899742823905534, "value2":9208355419350776806, "value3":true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354221, "value1":0.38324817137377604, "value2":7243408993217101630, "value3":true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.620584, "value1":0.25505110416085736, "value2":2000013463174956675, "value3":true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306280, "value1":0.24053747539237172, "value2":104074303789974504, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.694994, "value1":0.14347374003976163, "value2":2424092953659426107, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399682, "value1":0.5974894049526064, "value2":6226843809112311375, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.244602, "value1":0.4472550716634916, "value2":6661706815157471152, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154170, "value1":0.15894047878423925, "value2":1910278084951118791, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.338581, "value1":0.8000650933715514, "value2":932928130299819996, "value3":false, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385782, "value1":0.2346060605687393, "value2":1779773261165382974, "value3":true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.925102, "value1":0.31969179251919677, "value2":2628705870105871940, "value3":true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783196, "value1":0.47412332370344556, "value2":770542511728920904, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779626, "value1":0.6092963005254489, "value2":916209608160555146, "value3":false, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194805, "value1":0.5004447430389768, "value2":4252088119108151132, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.758680, "value1":0.486921957595336, "value2":989531111026827083, "value3":true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477327, "value1":0.1584089004594205, "value2":8696347531856094472, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162006, "value1":0.537289345346911, "value2":2077753746787895762, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613845, "value1":0.3033370157767779, "value2":7793623951886359033, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.373236, "value1":0.3796012074348545, "value2":6115758866481998084, "value3":false, "__name__":"metric_00000082", "key_9":"b","key_4":"h","key_5":"e", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080535, "value1":0.47198866208030016, "value2":6615627174770526167, "value3":true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.401469, "value1":0.5986326083333331, "value2":2667038551831095428, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605284, "value1":0.7390739400172244, "value2":361713221830612946, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.543913, "value1":0.568574574470533, "value2":8152708069238595670, "value3":false, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262409, "value1":0.058861415000961634, "value2":5954875590777239919, "value3":false, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056599, "value1":0.20694947175492623, "value2":3664736464652868067, "value3":false, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.557868, "value1":0.04995052697654973, "value2":2073326631524402431, "value3":true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613501, "value1":0.31045156583158495, "value2":8926511362033688194, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.047167, "value1":0.7509456855802353, "value2":846763672875326934, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362486, "value1":0.30051878680197913, "value2":8317722150065418470, "value3":true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.669592, "value1":0.6148907714481391, "value2":3070072928838789844, "value3":true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061749, "value1":0.4026807862923137, "value2":2054834671958607537, "value3":false, "__name__":"metric_00000110", "key_7":"c","key_8":"e","key_1":"h", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077647, "value1":0.02456732222252374, "value2":4163774252593358275, "value3":false, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439333, "value1":0.49417199138729123, "value2":8947661323810810369, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.592816, "value1":0.25731799708888187, "value2":5220216189538554324, "value3":true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.344893, "value1":0.0022390888763635323, "value2":6178831795736016910, "value3":false, "__name__":"metric_00000112", "key_5":"i","key_8":"h","key_4":"e", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350091, "value1":0.39159828262162255, "value2":3395049172404740462, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417133, "value1":0.19779968591074149, "value2":3982459725160448681, "value3":false, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313282, "value1":0.5954778061697951, "value2":2609201495243589975, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416054, "value1":0.6479875646988947, "value2":6379744126737729174, "value3":false, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416195, "value1":0.9737347842892922, "value2":3437414765355850053, "value3":true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.351132, "value1":0.5274165691272555, "value2":5569886633354264071, "value3":true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273952, "value1":0.4616579953521536, "value2":4410692243343844144, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948309, "value1":0.21495634970202532, "value2":1337568920394438603, "value3":false, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.120015, "value1":0.6827231094645871, "value2":4910087454372528211, "value3":true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.220281, "value1":0.5451953602332654, "value2":5295885271016141901, "value3":false, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222991, "value1":0.38652141757520153, "value2":611291833871236608, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432311, "value1":0.4952506069055437, "value2":6235358954173265246, "value3":true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.305907, "value1":0.7399971630668104, "value2":7853724941674753946, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.701437, "value1":0.07448132935373361, "value2":1676692481709109964, "value3":true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233939, "value1":0.9922833997352541, "value2":8485630674384935881, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.196307, "value1":0.3811609484351306, "value2":6762437704329339708, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.598701, "value1":0.7218662020742757, "value2":5746405219362751126, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535319, "value1":0.470769974287709, "value2":5160211896753607127, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455357, "value1":0.41061237500976755, "value2":3731616592511902245, "value3":true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682037, "value1":0.02412728786773881, "value2":8461839080730070447, "value3":false, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618426, "value1":0.6208202835196076, "value2":3667062185468357188, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094952, "value1":0.9317381860791135, "value2":8865888160428380876, "value3":false, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295576, "value1":0.034878221959591026, "value2":4553331116235737600, "value3":true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605252, "value1":0.5687954362360221, "value2":4628895813773640059, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.167973, "value1":0.8144348491443895, "value2":3884298273747196831, "value3":true, "__name__":"metric_00000139", "key_4":"a","key_1":"k","key_2":"h", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803115, "value1":0.14234455011711838, "value2":7333273581911274613, "value3":false, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809534, "value1":0.31251923098668316, "value2":5000940696241059800, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.504196, "value1":0.37041029395255115, "value2":7518926478801994928, "value3":true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886399, "value1":0.2510698129904212, "value2":1686947485208902024, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322861, "value1":0.6398715658295313, "value2":5260107554586637670, "value3":true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.575248, "value1":0.8821568194363127, "value2":2676120924049029771, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.924938, "value1":0.6642600755588765, "value2":9154692137110805453, "value3":false, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534144, "value1":0.2220263150896839, "value2":7296067315122241408, "value3":true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680757, "value1":0.36579467004129973, "value2":5336171210655722369, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.361476, "value1":0.42049076121425283, "value2":3694543289414649508, "value3":true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.423620, "value1":0.9226817255397154, "value2":5312418306728579961, "value3":true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.398746, "value1":0.4448382205324241, "value2":5199222544329766041, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.492975, "value1":0.6490408315820692, "value2":3620354939805403786, "value3":false, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.732787, "value1":0.26280435119702705, "value2":7186318610805553949, "value3":true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385151, "value1":0.7991171293820531, "value2":1271579952216532684, "value3":false, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067778, "value1":0.2653370821341721, "value2":8977665444967590904, "value3":false, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941069, "value1":0.7995772580327314, "value2":7272267767125923064, "value3":false, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.547209, "value1":0.7653567065104109, "value2":6328078604409399184, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649757, "value1":0.49061605503159605, "value2":5991812000082844933, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.017745, "value1":0.8857176370602151, "value2":8890483656354164816, "value3":false, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478915, "value1":0.49344955581785127, "value2":6898479319738793663, "value3":false, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.600610, "value1":0.1145004603393322, "value2":7934326719175266154, "value3":false, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752045, "value1":0.7917363851798865, "value2":6655960268552880542, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.198446, "value1":0.8915785759457425, "value2":6424163196395537238, "value3":false, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415881, "value1":0.6309493378355412, "value2":623919229201632317, "value3":false, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.652344, "value1":0.6344286572245156, "value2":9189348363675107233, "value3":false, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774566, "value1":0.042498471702267726, "value2":6073051427209338110, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.918626, "value1":0.4530903546971259, "value2":5667043007795158116, "value3":false, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.340057, "value1":0.13032389853235377, "value2":2621616876191250318, "value3":false, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363370, "value1":0.25548611447808117, "value2":7869413584380375849, "value3":false, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316114, "value1":0.7913570973054609, "value2":714341173131110534, "value3":false, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.726210, "value1":0.6340822577426605, "value2":7078070670524096081, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809728, "value1":0.31189335942918994, "value2":2593764393145326151, "value3":false, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828156, "value1":0.2513167249865656, "value2":4199156936630630581, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074063, "value1":0.6299866633999338, "value2":7375082593010468605, "value3":true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551341, "value1":0.5634049844701874, "value2":553811382543796103, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395240, "value1":0.9163098471877911, "value2":2102676899677978041, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812285, "value1":0.5658853008127968, "value2":860920050673825541, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.699745, "value1":0.6686081106918065, "value2":4484995258456828647, "value3":true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862811, "value1":0.006750603964471356, "value2":8966268587906792618, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649656, "value1":0.33455087075476775, "value2":3638358249838833627, "value3":true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978094, "value1":0.9765011127569251, "value2":5363905042672157366, "value3":false, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824326, "value1":0.5852292268881885, "value2":8525521016560267029, "value3":true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972834, "value1":0.2713220476194647, "value2":1730194033904802699, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.513347, "value1":0.20240183224033417, "value2":3586323241046235606, "value3":false, "__name__":"metric_00000182", "key_7":"f","key_9":"b","key_3":"a", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165801, "value1":0.25754092027876374, "value2":3897618512711030520, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245034, "value1":0.276430555426299, "value2":7235576154009687664, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209920, "value1":0.33218423022426385, "value2":6956800146565179953, "value3":false, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802797, "value1":0.9498551679797749, "value2":4204819498986608868, "value3":true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.002994, "value1":0.33304452518131644, "value2":299998551436331074, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.136076, "value1":0.58569890261748, "value2":8821512503402703105, "value3":false, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605724, "value1":0.3779582962124763, "value2":4381826070482821493, "value3":false, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.664393, "value1":0.7508643438704989, "value2":9133405472026903918, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236332, "value1":0.4456796524317141, "value2":3982698031313790805, "value3":true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320127, "value1":0.5073225371595068, "value2":2378617268397591284, "value3":false, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865511, "value1":0.36097655561074876, "value2":2650611720712784810, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777541, "value1":0.27613383935735863, "value2":3133546266043557517, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647086, "value1":0.09971158325173911, "value2":7192486223907690883, "value3":false, "__name__":"metric_00000199", "key_9":"j","key_2":"f","key_5":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.172420, "value1":0.6050664492556429, "value2":3816871624777837698, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639560, "value1":0.40402358999541305, "value2":7105433557941999346, "value3":true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.945205, "value1":0.38392410498955126, "value2":1178256579293465746, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277675, "value1":0.3660884472168037, "value2":6046887408166693464, "value3":false, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219346, "value1":0.5871741254566942, "value2":2616399527910270141, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116739, "value1":0.7713481330592996, "value2":7718152730102741547, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.951685, "value1":0.6692969608681443, "value2":6198709686728043657, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.880602, "value1":0.3938915734742536, "value2":5600053552458054507, "value3":false, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.831337, "value1":0.47065388482990184, "value2":5918647530397871720, "value3":false, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865810, "value1":0.9557189505427555, "value2":1533596579704888254, "value3":false, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578483, "value1":0.3164962822621824, "value2":4784731554506893709, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158191, "value1":0.36054084773482775, "value2":6343389237742355198, "value3":true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761167, "value1":0.9149570761492709, "value2":6140964050711145447, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.955044, "value1":0.34237203961558577, "value2":7321019711278117639, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137258, "value1":0.2734770926499478, "value2":5312006546786938240, "value3":false, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876289, "value1":0.584796985987568, "value2":2146521016660183394, "value3":true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.345388, "value1":0.7891673161299368, "value2":9211813724293662180, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.291358, "value1":0.5443723430932246, "value2":1305196521853642686, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107992, "value1":0.33317636728859645, "value2":4303995107486054548, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.065010, "value1":0.792101797297302, "value2":5624493167148885444, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.512622, "value1":0.026747806334027496, "value2":31336873860845918, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796113, "value1":0.3713464074931213, "value2":4041884710949108537, "value3":true, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.794333, "value1":0.5998279188033826, "value2":2263989663686243457, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930716, "value1":0.7150630462147562, "value2":5925307983835648806, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.631145, "value1":0.12175047903656237, "value2":2424100894887965716, "value3":true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889832, "value1":0.33399829473514187, "value2":3091557798470852457, "value3":false, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.036666, "value1":0.8679594907099448, "value2":6691229273816200596, "value3":true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.853608, "value1":0.5774160801209688, "value2":980449146620020072, "value3":false, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.356874, "value1":0.6962074391730027, "value2":2178872746047471215, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162362, "value1":0.6573318971529195, "value2":8364158949095374627, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110201, "value1":0.41287549892040376, "value2":7444688864434046520, "value3":true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.441462, "value1":0.960708146788197, "value2":7272057802651917640, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.020362, "value1":0.14970758198915565, "value2":5476649529178658344, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434430, "value1":0.7114147585195696, "value2":6784565480587871276, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795723, "value1":0.9222061229277446, "value2":5415396163282444341, "value3":false, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.092199, "value1":0.0594949540077623, "value2":733783646287392653, "value3":false, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277448, "value1":0.36348546035264123, "value2":2527022975440439707, "value3":true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607130, "value1":0.21203009019347513, "value2":3167160934083348057, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424117, "value1":0.42561548619630274, "value2":439532732667459958, "value3":false, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365073, "value1":0.8492201961187499, "value2":1130712017318335183, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248310, "value1":0.6476029521632828, "value2":5228653953193583715, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744619, "value1":0.5713572042259071, "value2":1575714640192742698, "value3":false, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.026559, "value1":0.543673726494076, "value2":3134877395946459112, "value3":true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972610, "value1":0.09611280545479016, "value2":4984229756632609213, "value3":false, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763906, "value1":0.8517215040951974, "value2":6804317567810698432, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105838, "value1":0.8805288612544102, "value2":7653928454276555905, "value3":true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383435, "value1":0.3683543847634455, "value2":2147384927542025505, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027377, "value1":0.4869984514514779, "value2":6620303412327477269, "value3":true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417188, "value1":0.24989866283446016, "value2":7011787964949226310, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084529, "value1":0.9844535394576189, "value2":6876673068454117053, "value3":true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032097, "value1":0.6712678103298776, "value2":3320740930339209104, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394244, "value1":0.7796841432655915, "value2":4446257418849643255, "value3":false, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828297, "value1":0.2648442239286722, "value2":25756618593200917, "value3":true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731687, "value1":0.07632667820503707, "value2":5775744844960229233, "value3":false, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.348738, "value1":0.4462284223326741, "value2":7761375931867531240, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817431, "value1":0.9846258074003919, "value2":5377182091927671364, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603703, "value1":0.1579124346010748, "value2":7829825462713051842, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743604, "value1":0.012350717590309025, "value2":3299133246704731924, "value3":true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941914, "value1":0.37406772316207776, "value2":3581449411399106092, "value3":false, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.323291, "value1":0.2629524147159168, "value2":2560363092546342475, "value3":true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206164, "value1":0.1605470780394023, "value2":1476326866755963525, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376246, "value1":0.8410819090377242, "value2":6697199584394109087, "value3":false, "__name__":"metric_00000249", "key_6":"c","key_0":"d", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.438458, "value1":0.05246539756995679, "value2":2689405601803288451, "value3":true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.807203, "value1":0.730536792123947, "value2":7996980666084265973, "value3":false, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.842861, "value1":0.2085319390948787, "value2":750997239048285659, "value3":true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.097539, "value1":0.943951867959508, "value2":8362598506758468383, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363032, "value1":0.5870242519277287, "value2":6264967362494680685, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212907, "value1":0.4603451527715893, "value2":4171196630516497508, "value3":false, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177573, "value1":0.8948006671577682, "value2":5720852449593066754, "value3":false, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286908, "value1":0.7373750433446898, "value2":2889582543857330611, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.780354, "value1":0.11954278924179859, "value2":206386290438050068, "value3":false, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744582, "value1":0.5414546611545269, "value2":1573274748881736123, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262127, "value1":0.8114751808169933, "value2":1156734310814584450, "value3":false, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.627688, "value1":0.05870166177597653, "value2":8802667963431750791, "value3":false, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432936, "value1":0.5733149806402271, "value2":8971566642401975801, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252007, "value1":0.6150552921263627, "value2":6524226679068869341, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230350, "value1":0.8727510444388746, "value2":5913218581367894399, "value3":true, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.989663, "value1":0.8376111097555229, "value2":2874987014333615633, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.435356, "value1":0.7022017770410863, "value2":6778124246365180945, "value3":false, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673069, "value1":0.2749380629969126, "value2":1514980079784151790, "value3":false, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328789, "value1":0.865778242569301, "value2":384106145681984426, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.986292, "value1":0.9273192971072537, "value2":6526516407617182264, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.240749, "value1":0.7234109649728769, "value2":6790075567204476237, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774780, "value1":0.5871689076780541, "value2":5434961575452625626, "value3":false, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.406076, "value1":0.7420142270194664, "value2":4969819996381131142, "value3":false, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146123, "value1":0.2540849065850774, "value2":4942775185551508816, "value3":true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.928078, "value1":0.7980654446229606, "value2":2117433885577041648, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933008, "value1":0.05348899498806063, "value2":7864335047299894108, "value3":false, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655622, "value1":0.7668514308351416, "value2":5108933691323452489, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799422, "value1":0.8658511508098131, "value2":1584220631636831527, "value3":false, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153780, "value1":0.5100425920946189, "value2":6860613724470124155, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.832729, "value1":0.6352815444925497, "value2":6914920278709539881, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862121, "value1":0.40978666741053005, "value2":8297372411728279444, "value3":false, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.046814, "value1":0.6860769857954689, "value2":7207194751565200226, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.106175, "value1":0.9141202615431735, "value2":8093659818506693875, "value3":true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568924, "value1":0.2818708416167068, "value2":4682897128638978687, "value3":false, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743387, "value1":0.19055511730870822, "value2":6005851706828394010, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.131634, "value1":0.786038669470468, "value2":2052265664401672629, "value3":false, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864417, "value1":0.6509022777920431, "value2":6405639625996257390, "value3":false, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769553, "value1":0.3306779516460208, "value2":63644576329067046, "value3":true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705994, "value1":0.39211177275150355, "value2":6019747875631249901, "value3":false, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.085415, "value1":0.989053918126363, "value2":8753375624734757247, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.966642, "value1":0.146948164341443, "value2":7640844254824954510, "value3":true, "__name__":"metric_00000285", "key_9":"k","key_5":"b","key_8":"b", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.548525, "value1":0.7240074879750936, "value2":8956195213201860326, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073937, "value1":0.7314322966939647, "value2":254693443211564548, "value3":false, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347726, "value1":0.8209138159763902, "value2":6620059103742183058, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.378046, "value1":0.5808416882480918, "value2":6139629334504131344, "value3":true, "__name__":"metric_00000302", "key_6":"a","key_1":"d","key_4":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797601, "value1":0.7784388478906132, "value2":354987022131004184, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411013, "value1":0.45158233474379805, "value2":6507365193047028596, "value3":false, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.152084, "value1":0.34791281429560394, "value2":2152822969747742943, "value3":false, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808545, "value1":0.6451074006535574, "value2":4535693696526329424, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735815, "value1":0.5617099301887051, "value2":4545722791278282166, "value3":true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170237, "value1":0.7594425432841077, "value2":2121502996639945525, "value3":true, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839462, "value1":0.42088416942799856, "value2":4668152333347445855, "value3":true, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977985, "value1":0.9874529650666419, "value2":7279887982544559299, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777993, "value1":0.32638782808208466, "value2":7945394269696237457, "value3":true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931446, "value1":0.5421003602914947, "value2":7796969745729042644, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746112, "value1":0.6059100687669938, "value2":8599287190529748569, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.950726, "value1":0.8607618900249634, "value2":2432626342683975270, "value3":false, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647237, "value1":0.7468406151868747, "value2":7200054131729412480, "value3":false, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.460709, "value1":0.4549015015070196, "value2":1628296475817521290, "value3":false, "__name__":"metric_00000311", "key_5":"d","key_0":"b","key_2":"f", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007652, "value1":0.3132830085813196, "value2":1943034905431272670, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.103043, "value1":0.029190956951399328, "value2":5218539584338803865, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506730, "value1":0.4547431011729936, "value2":3644260335220928016, "value3":true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564064, "value1":0.2217922532972611, "value2":3866244575842378014, "value3":false, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.308096, "value1":0.4735939830858083, "value2":6374663223385153398, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415080, "value1":0.9760882060049622, "value2":1970837347852222146, "value3":true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909084, "value1":0.6294752667120725, "value2":9199477369979291397, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.969935, "value1":0.10788341374154452, "value2":7002111391558217482, "value3":false, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.622182, "value1":0.21938176298467404, "value2":4706131254461339381, "value3":false, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.722928, "value1":0.21266609523973992, "value2":5957309419507426661, "value3":true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062556, "value1":0.1304962392242022, "value2":4535950458577175850, "value3":false, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056693, "value1":0.6559280448597679, "value2":6798153078557023165, "value3":false, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.275009, "value1":0.8708722343930831, "value2":2992233127991751043, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027324, "value1":0.8089539697327325, "value2":6582915861289307476, "value3":false, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769519, "value1":0.6866136235615642, "value2":3188730142010270506, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471216, "value1":0.036922660768241786, "value2":69160799499838231, "value3":false, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.289160, "value1":0.9707556984503585, "value2":3226749743394249266, "value3":true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719377, "value1":0.6150053305666618, "value2":8057451250881285379, "value3":false, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884188, "value1":0.32970677783871427, "value2":6299179245210453660, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.128583, "value1":0.46621614630474323, "value2":3810272679777334373, "value3":false, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.016304, "value1":0.07070759725845735, "value2":3860287779798372481, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556253, "value1":0.3162226930220112, "value2":811964996210240316, "value3":false, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.662220, "value1":0.04332778395279835, "value2":1557538607744070859, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713709, "value1":0.6625325269051697, "value2":4541028658322459718, "value3":true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297119, "value1":0.15013885945438676, "value2":2149756203418913335, "value3":false, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685110, "value1":0.9036182355788681, "value2":4377728436719035164, "value3":false, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846262, "value1":0.34678741602681534, "value2":2162478118726349508, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410862, "value1":0.11378608528656459, "value2":614293064254453793, "value3":false, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.060980, "value1":0.22871224178742908, "value2":4099107042293178536, "value3":false, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960058, "value1":0.5820948808821428, "value2":1201783870513585432, "value3":true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.214885, "value1":0.03550184669484978, "value2":2051901639161172531, "value3":false, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696549, "value1":0.44298333983599963, "value2":8432193022117819833, "value3":true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368813, "value1":0.6629289022292644, "value2":6617784174875177478, "value3":false, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313857, "value1":0.15526487291119703, "value2":95708758682513207, "value3":true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744683, "value1":0.6423949710674116, "value2":3611912809247643043, "value3":false, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.786335, "value1":0.589184376420666, "value2":5802753188533704826, "value3":true, "__name__":"metric_00000353", "key_9":"a","key_0":"a","key_5":"b", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930246, "value1":0.20872893267238513, "value2":2224049910905425698, "value3":true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515204, "value1":0.20960234541456155, "value2":7647445186122035759, "value3":false, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.800072, "value1":0.20249832362898804, "value2":1073544363493262212, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.420652, "value1":0.651081573013809, "value2":355187443325178523, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760554, "value1":0.28915917130829083, "value2":5965361551796616223, "value3":true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.852984, "value1":0.45938574348035566, "value2":2528322122553170113, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352738, "value1":0.900978009547613, "value2":3975764535045467018, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066294, "value1":0.08305245567845902, "value2":2394249365413597097, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654208, "value1":0.08608535610142205, "value2":2036163695122831466, "value3":true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.988436, "value1":0.04790022290628863, "value2":6418792057544120186, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884316, "value1":0.5495267159618948, "value2":4438533595454630352, "value3":true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154168, "value1":0.4348957861418331, "value2":8838135202914157189, "value3":true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874389, "value1":0.42584041610286777, "value2":2273157634255144393, "value3":true, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431882, "value1":0.06486477270973905, "value2":3345519389285150964, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685508, "value1":0.5499323474722744, "value2":5054596449098633988, "value3":true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.660995, "value1":0.17858690508054326, "value2":8567568075077444285, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271767, "value1":0.8266242576408339, "value2":8856688759394626086, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755581, "value1":0.7494066282765741, "value2":4893716534306029427, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171573, "value1":0.6765014228252123, "value2":5524113092747902135, "value3":false, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269557, "value1":0.24446627014298006, "value2":2739808324008179094, "value3":true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.967939, "value1":0.5539434802495194, "value2":3700007071044991493, "value3":false, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.656891, "value1":0.9116275072171186, "value2":7153356187047849001, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248386, "value1":0.09058562459659583, "value2":7526211020899452764, "value3":false, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948164, "value1":0.6887494090196015, "value2":4600657675678245799, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228675, "value1":0.8494478798473957, "value2":317720425357796205, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101265, "value1":0.8190567291465818, "value2":4663927178372352248, "value3":false, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820988, "value1":0.775820207166787, "value2":6038722642101305434, "value3":false, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922087, "value1":0.22553959289224842, "value2":779713635671076382, "value3":false, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.888330, "value1":0.020416312330820526, "value2":7204412129555533981, "value3":true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.290446, "value1":0.9852816320447644, "value2":4592385145364652482, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.679824, "value1":0.17818186899277277, "value2":252209780074899647, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686778, "value1":0.8339304122124636, "value2":3658007614464097080, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257632, "value1":0.614764442927346, "value2":4642146906724062657, "value3":false, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482850, "value1":0.18190770107834264, "value2":8954437725061891309, "value3":false, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998993, "value1":0.7044002667070637, "value2":431084763192513731, "value3":false, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.875898, "value1":0.7812743007979214, "value2":2705148521595588507, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084211, "value1":0.8180424469178956, "value2":7499444983471620842, "value3":false, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053317, "value1":0.3379873516583276, "value2":5874640652872975301, "value3":false, "__name__":"metric_00000390", "key_8":"i","key_1":"k","key_2":"b", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746275, "value1":0.11387537552973875, "value2":4778407492588259899, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744142, "value1":0.945713466310749, "value2":3761848793498692794, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.274112, "value1":0.14967892390238677, "value2":3777191830426872225, "value3":false, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680316, "value1":0.5765506588965501, "value2":3913164426928614622, "value3":false, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.582474, "value1":0.13820731268553305, "value2":1359306396719437577, "value3":true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.572068, "value1":0.9236046727786195, "value2":731327351641596838, "value3":false, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.838138, "value1":0.5915185036534663, "value2":4354316161210605047, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755088, "value1":0.18889070665371863, "value2":4180101489297988878, "value3":false, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362663, "value1":0.4708250834755644, "value2":8678758888644206492, "value3":false, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.464079, "value1":0.13171451384673036, "value2":8187780706682520691, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839674, "value1":0.16354088623961133, "value2":8461213608449952033, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777782, "value1":0.005150900343416017, "value2":9058250336565790520, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.520149, "value1":0.6583211185838453, "value2":179671992454725443, "value3":true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354725, "value1":0.26618356833133044, "value2":4502556297298851018, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.588275, "value1":0.40556503705171876, "value2":1831243556070013492, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.991768, "value1":0.5964588343272011, "value2":7216754835822865383, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.829272, "value1":0.8185283045838991, "value2":7054142095046490626, "value3":true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028671, "value1":0.9570557166393036, "value2":5667948933240210255, "value3":true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171455, "value1":0.3984068015975904, "value2":6709012624962450094, "value3":true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112456, "value1":0.8897199020551853, "value2":619757189037583586, "value3":false, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.366328, "value1":0.9348857036272434, "value2":4207327317390530443, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.787818, "value1":0.06046424642679486, "value2":3800391541062933624, "value3":false, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431436, "value1":0.5742437522880309, "value2":3338797017937456724, "value3":false, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.130697, "value1":0.8564226707318178, "value2":2563408398646331614, "value3":false, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531522, "value1":0.21916844982697795, "value2":6377428321672110838, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799330, "value1":0.2117471404567077, "value2":6841854064863394582, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415853, "value1":0.7458907419184496, "value2":3858225758706811596, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769460, "value1":0.9292446985209156, "value2":4294237114969210404, "value3":true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053422, "value1":0.19156586881852822, "value2":8401149164501095807, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.634910, "value1":0.13000039932274055, "value2":4131562742009700028, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703579, "value1":0.6862332441253757, "value2":7454437023717068913, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327549, "value1":0.40704854485658054, "value2":4932202555006075846, "value3":false, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.314755, "value1":0.5681370276361754, "value2":3506285901156371358, "value3":true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.466686, "value1":0.42167473519309434, "value2":3558723778128133952, "value3":true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160258, "value1":0.4575847781643395, "value2":6023651621402119686, "value3":false, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228508, "value1":0.5111004373311687, "value2":8895687587282710909, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.111511, "value1":0.09506770522467269, "value2":2655655566098996088, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.784900, "value1":0.5435091084418109, "value2":5760921829736802814, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210512, "value1":0.25861533775417506, "value2":6914577108338158111, "value3":false, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.663100, "value1":0.8688427076237359, "value2":9173486087944898117, "value3":false, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066304, "value1":0.7818969735770589, "value2":4913317894269750379, "value3":false, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824798, "value1":0.9535971366603737, "value2":8366422359718269509, "value3":true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.090470, "value1":0.7644445079507416, "value2":6086882240277966349, "value3":false, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994994, "value1":0.046461773505700306, "value2":1146509415122743778, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409080, "value1":0.028382316546416403, "value2":2068020782874554558, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.815596, "value1":0.08796047569888332, "value2":8378736063122737266, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683447, "value1":0.24029660357741323, "value2":8797505888002523812, "value3":false, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.611937, "value1":0.2489739097238809, "value2":7210056341279458271, "value3":true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347662, "value1":0.7553826852514858, "value2":8429002700088766063, "value3":true, "__name__":"metric_00000447", "key_8":"i","key_0":"e","key_5":"g", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318289, "value1":0.5406931125323163, "value2":1745967906139345488, "value3":true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.900516, "value1":0.7025144762707813, "value2":456260840471493385, "value3":true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153699, "value1":0.20283897976623247, "value2":7209402260011396171, "value3":true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.813264, "value1":0.23005364054850463, "value2":4244429874559011543, "value3":false, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.174420, "value1":0.6527990870752745, "value2":5344843801392876512, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142219, "value1":0.1627182050289727, "value2":114175501496781641, "value3":false, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.481454, "value1":0.37374721367966557, "value2":1638215468267960931, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.523380, "value1":0.48736316265440816, "value2":4581537275255860768, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317190, "value1":0.0039195876566355085, "value2":5341281001441055802, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619251, "value1":0.5258022392764804, "value2":4434264026251453750, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.641615, "value1":0.33039615993236077, "value2":6928858155661176736, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961004, "value1":0.5399790911878092, "value2":7046629202789685841, "value3":true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025798, "value1":0.5984426914038641, "value2":6528145482673170360, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.901084, "value1":0.06259887743500082, "value2":3236298745887395724, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.489095, "value1":0.21376661515997028, "value2":7928954356959433352, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655274, "value1":0.5385351363841719, "value2":7272624008760795422, "value3":true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.082369, "value1":0.4014980361714972, "value2":7005111860489403789, "value3":true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519418, "value1":0.19683342336444604, "value2":1435942937723913885, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226945, "value1":0.9922423305002035, "value2":867760431487143669, "value3":false, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752765, "value1":0.9916422768378336, "value2":3430696359309587884, "value3":true, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.078412, "value1":0.6964630260101603, "value2":2102364069319122633, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328609, "value1":0.19123451863699675, "value2":8170554763374290069, "value3":false, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.470499, "value1":0.03919799070813115, "value2":3366292364134765186, "value3":true, "__name__":"metric_00000462", "key_8":"g","key_9":"d","key_6":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740670, "value1":0.08849655325518448, "value2":1691579140130982717, "value3":true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.393865, "value1":0.061408201218711954, "value2":174448511358385380, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.200654, "value1":0.2836024427807901, "value2":4326318289877991961, "value3":false, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970400, "value1":0.7730199010023195, "value2":8574178501642024588, "value3":false, "__name__":"metric_00000465", "key_6":"f","key_1":"f","key_3":"h", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.465169, "value1":0.542725080263732, "value2":9196833379545093375, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.563410, "value1":0.7958042531287333, "value2":7393645101828557391, "value3":false, "__name__":"metric_00000468", "key_5":"i","key_8":"k","key_1":"j", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213281, "value1":0.7928629505338592, "value2":1752466947098105566, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954474, "value1":0.3032546869220648, "value2":1748556721646671927, "value3":true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.847068, "value1":0.7688745080204612, "value2":5187070635183292150, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251152, "value1":0.5682862581346041, "value2":2363792865069284658, "value3":true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360666, "value1":0.08726468762579924, "value2":643126609584411238, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619231, "value1":0.7434795567208692, "value2":4822313794436821661, "value3":false, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603123, "value1":0.10532993201555661, "value2":2544972440464847934, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607459, "value1":0.20036031311934685, "value2":1168760905394403690, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447800, "value1":0.7195092335832372, "value2":5642489879930119928, "value3":true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.850819, "value1":0.6761002637886042, "value2":833307298139301862, "value3":true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539724, "value1":0.5294892272320822, "value2":2667359803805326763, "value3":true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.208291, "value1":0.8393697010611292, "value2":683397536019101988, "value3":false, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550311, "value1":0.9963724696480404, "value2":5468577449184757808, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.023114, "value1":0.32353069050613953, "value2":9187169003922087415, "value3":true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286063, "value1":0.9137524095402076, "value2":6445901279543298486, "value3":true, "__name__":"metric_00000492", "key_9":"k","key_0":"k","key_3":"j", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.204901, "value1":0.4076696093660646, "value2":6677345639081163017, "value3":false, "__name__":"metric_00000477", "key_6":"k","key_0":"b","key_3":"f", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107007, "value1":0.5776778447129313, "value2":8463235586423448216, "value3":false, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271841, "value1":0.6930651904253268, "value2":6787444131360802180, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.069717, "value1":0.7690475248125641, "value2":1837461702894857026, "value3":false, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368696, "value1":0.5454486705451115, "value2":3041446083114186683, "value3":true, "__name__":"metric_00000488", "key_5":"c","key_3":"g", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395274, "value1":0.24368851515268314, "value2":4392956019271269002, "value3":true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352289, "value1":0.22988542503215345, "value2":6550852329237611667, "value3":false, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703725, "value1":0.026480349904897907, "value2":1885153320482283414, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.256528, "value1":0.26111767999722507, "value2":6808025827523544315, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252909, "value1":0.20711336158884097, "value2":3024039988582180695, "value3":true, "__name__":"metric_00000493", "key_9":"h","key_1":"d","key_8":"d", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.633970, "value1":0.07845906332760666, "value2":6511716740509653564, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605465, "value1":0.5950957840062573, "value2":2806262404335314036, "value3":false, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.081867, "value1":0.08398189819376621, "value2":5728413114044088113, "value3":false, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074111, "value1":0.6218602698989528, "value2":1011556855791832544, "value3":true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675019, "value1":0.11095025598530854, "value2":5207952003333828816, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.738057, "value1":0.5410628771237361, "value2":2540645323761255412, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317014, "value1":0.5259626957217648, "value2":3505614768801064603, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776152, "value1":0.302147585405568, "value2":7518755412283013617, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094787, "value1":0.5814132866660779, "value2":4822671923080781538, "value3":false, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.517582, "value1":0.10766355589362117, "value2":8453589798026915731, "value3":false, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882088, "value1":0.8040313368583433, "value2":8495228225039478737, "value3":false, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803773, "value1":0.22925549140141077, "value2":8938270200397652857, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403205, "value1":0.07946210415899989, "value2":7443253750975595381, "value3":true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211183, "value1":0.27303249192461276, "value2":3758490705819764753, "value3":false, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684989, "value1":0.19635657331096867, "value2":3768344870136296803, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596986, "value1":0.9569957010370201, "value2":5843413270558977752, "value3":false, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796083, "value1":0.4870882675033077, "value2":8040326882339558395, "value3":true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.674664, "value1":0.05017490820454691, "value2":1512965384940722080, "value3":false, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682218, "value1":0.05038298218752238, "value2":3666135732590503511, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871616, "value1":0.007362621326077148, "value2":3382155619299232500, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.479139, "value1":0.48744354938142825, "value2":2536302313422526956, "value3":false, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408860, "value1":0.5030243670352939, "value2":3709214658604138842, "value3":false, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.079849, "value1":0.4036712273225728, "value2":2906136993668691733, "value3":true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735107, "value1":0.11572752191146224, "value2":9135232014249483896, "value3":false, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.579000, "value1":0.832018905121829, "value2":4291485121105439680, "value3":false, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297326, "value1":0.3053519839954546, "value2":7762295205550727215, "value3":true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077473, "value1":0.5244918634463549, "value2":3651586678818959294, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211495, "value1":0.3162790532742153, "value2":8307153509301118839, "value3":false, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.018730, "value1":0.4578847558537525, "value2":3121380712594178301, "value3":false, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.254109, "value1":0.0022335788974661955, "value2":1113680598327977122, "value3":false, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327840, "value1":0.38572838732788006, "value2":2539441632284031099, "value3":false, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802473, "value1":0.6047951359580281, "value2":1001558191118170380, "value3":true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299508, "value1":0.05862512584912177, "value2":5514618695776223541, "value3":true, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.011661, "value1":0.31240922806678983, "value2":6137561733651595636, "value3":true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728655, "value1":0.1278483939203575, "value2":9035381570106713199, "value3":true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.667349, "value1":0.5613783650549519, "value2":596862476082819858, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.253799, "value1":0.021910250550882106, "value2":6533597878636060832, "value3":false, "__name__":"metric_00000527", "key_9":"k","key_3":"a","key_5":"b", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.288444, "value1":0.30057474196214407, "value2":3169010106041213703, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.798940, "value1":0.9681683301198919, "value2":3021363800209965251, "value3":true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.981962, "value1":0.9612929543752854, "value2":9171352851679704895, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149939, "value1":0.09981385175599812, "value2":2281824057904455016, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943557, "value1":0.6955975747592801, "value2":2634327756448196264, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459593, "value1":0.6446822058280443, "value2":6342926358496707008, "value3":false, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.734204, "value1":0.6639025089026711, "value2":2476897145958348918, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506327, "value1":0.10053599414522371, "value2":2221394931414407943, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410931, "value1":0.8059701840141592, "value2":3838253097677401347, "value3":false, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.964398, "value1":0.603279381255061, "value2":5782827613318848456, "value3":false, "__name__":"metric_00000549", "key_3":"f","key_0":"f","key_2":"d", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.278202, "value1":0.7115230775467353, "value2":8371900549612489852, "value3":true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408547, "value1":0.21860759782537603, "value2":8039563012338877282, "value3":true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.261349, "value1":0.11227782989658946, "value2":7783215654484926349, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059288, "value1":0.07895743947800146, "value2":5667050368393356509, "value3":true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.601079, "value1":0.6482240475140798, "value2":4251195673123293243, "value3":true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.225112, "value1":0.7216063470142905, "value2":148937314023613788, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977412, "value1":0.6598247780794319, "value2":5965401930753401354, "value3":false, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194225, "value1":0.6141698639690517, "value2":8817766861356540801, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.597081, "value1":0.07693962991598453, "value2":5029342104400232103, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.335617, "value1":0.5583831809146028, "value2":1406686773640908295, "value3":true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916403, "value1":0.7018505752435668, "value2":664450605422502237, "value3":true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497447, "value1":0.8277122605282176, "value2":1219079943087879547, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716326, "value1":0.7509816698301687, "value2":3921665603218422131, "value3":true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716585, "value1":0.2920975208652007, "value2":3756232698934542100, "value3":false, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301567, "value1":0.8942632920542656, "value2":2291395595568091863, "value3":true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.599725, "value1":0.5659814539312558, "value2":7320359369690410226, "value3":true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.614192, "value1":0.7911920709201787, "value2":8781166815844312813, "value3":false, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.195843, "value1":0.3963219582946417, "value2":4716889546078718843, "value3":true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555996, "value1":0.602486402959913, "value2":4687524095847830496, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.917870, "value1":0.15392369008366974, "value2":7045583703847585198, "value3":true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.616039, "value1":0.6919323182152138, "value2":511455339796298853, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930107, "value1":0.12610440100742193, "value2":6997765950733061577, "value3":false, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194345, "value1":0.7209468362854391, "value2":556025580881809360, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797831, "value1":0.2266151348199214, "value2":2004107184327193155, "value3":false, "__name__":"metric_00000572", "key_7":"i","key_1":"d","key_6":"b", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949681, "value1":0.8884059641939672, "value2":7011478951165849176, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213425, "value1":0.9929181204525178, "value2":8631476778841936239, "value3":true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821834, "value1":0.3090901919909019, "value2":5522454281472043559, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.181027, "value1":0.06720359916226379, "value2":4456725170730825432, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.762490, "value1":0.474581541255306, "value2":7005342387987070026, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.759790, "value1":0.5398913013405661, "value2":2663889184009973941, "value3":false, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112196, "value1":0.8025326007392437, "value2":883355344681130779, "value3":true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374886, "value1":0.9718695344446878, "value2":1646239120227544644, "value3":true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.729457, "value1":0.9682037233981026, "value2":6151398746046248850, "value3":true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048424, "value1":0.7273914683080029, "value2":8000884246159354431, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.937043, "value1":0.1526646293780592, "value2":266299265780041426, "value3":true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752328, "value1":0.9655606802627454, "value2":5596664612933771240, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666200, "value1":0.10048257972248233, "value2":4289061240642605106, "value3":true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922103, "value1":0.8963130727289086, "value2":8836663020903305631, "value3":true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105202, "value1":0.8314839649252564, "value2":3551543011617183084, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886891, "value1":0.66836051888972, "value2":72366475317978094, "value3":true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432056, "value1":0.745064543848614, "value2":2189880642993257536, "value3":false, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042716, "value1":0.3956017159748215, "value2":3605211356176936944, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.984359, "value1":0.2911243881226883, "value2":7856628067522273320, "value3":false, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295731, "value1":0.4520766897124352, "value2":610878216891620607, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.650753, "value1":0.03928359938969399, "value2":2072987167295070336, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211564, "value1":0.7625006233645502, "value2":4401190374987004409, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792725, "value1":0.47390601969195995, "value2":4329714295913254034, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.444277, "value1":0.5857721636217157, "value2":3580156964556605157, "value3":true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535990, "value1":0.9433719155892893, "value2":5643233427116703402, "value3":true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.688127, "value1":0.7720856573472716, "value2":5631377472070167195, "value3":true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.697966, "value1":0.9263837993798216, "value2":4093734777340860370, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.996439, "value1":0.3910900735196092, "value2":8375564545525062485, "value3":false, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872727, "value1":0.8324218421388521, "value2":7152928233109153205, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.246110, "value1":0.2551941160691682, "value2":5602869232123697435, "value3":false, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931727, "value1":0.5795265646922866, "value2":7403691202626500132, "value3":true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673141, "value1":0.06896155961831285, "value2":4415828287435091542, "value3":true, "__name__":"metric_00000605", "key_8":"a","key_9":"h","key_7":"d", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684610, "value1":0.16872221976263158, "value2":505317720854796636, "value3":false, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559417, "value1":0.12924994486885147, "value2":4787886921303451969, "value3":false, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094359, "value1":0.5164211297919008, "value2":1315713014945297109, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007976, "value1":0.7735634457421625, "value2":8579409036573615608, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286892, "value1":0.43624877878480095, "value2":6924175474906299981, "value3":true, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.902926, "value1":0.6780220008663965, "value2":4822375571733927097, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.486540, "value1":0.23333524037004374, "value2":6314520721238807162, "value3":true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007170, "value1":0.9857337079596743, "value2":4448133028853613455, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.030552, "value1":0.24355749578071725, "value2":8327413108187969392, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.379606, "value1":0.4729923914146341, "value2":914941226083135930, "value3":false, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835232, "value1":0.9602105867588695, "value2":3810448989361248064, "value3":false, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433684, "value1":0.9595084872985944, "value2":455874222103117607, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411653, "value1":0.037665634780994134, "value2":6833101830522402640, "value3":true, "__name__":"metric_00000587", "key_4":"a","key_5":"j","key_2":"e", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713632, "value1":0.5664925972399699, "value2":7794307235376671149, "value3":true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391266, "value1":0.6608939962095463, "value2":5860330325168511608, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212303, "value1":0.2089060448996, "value2":5651881484326487830, "value3":true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180078, "value1":0.7876093968011537, "value2":2862206598597477486, "value3":true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.175199, "value1":0.14395244434800714, "value2":2376804168343245120, "value3":true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809965, "value1":0.12003398504952464, "value2":5187708593154594566, "value3":false, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.851381, "value1":0.607956049195976, "value2":7288813225458184427, "value3":false, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933118, "value1":0.7246239590339857, "value2":1234716149527350625, "value3":true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.939251, "value1":0.34949617152726564, "value2":1847804623713516854, "value3":true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665651, "value1":0.7331985679295692, "value2":1370393562180863722, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.827386, "value1":0.6896205618409067, "value2":1887650763331073033, "value3":true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.637520, "value1":0.13783942741851515, "value2":7482561145284406008, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657534, "value1":0.26275939644072077, "value2":6263897972267140208, "value3":false, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740684, "value1":0.9117133919549965, "value2":6149129956357538684, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032295, "value1":0.9865289627211871, "value2":5555291464918052387, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545212, "value1":0.5648289415381519, "value2":6789070914120574869, "value3":true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.321143, "value1":0.6917810996904441, "value2":4171532082462131046, "value3":true, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.693006, "value1":0.9813132387635279, "value2":6633038595296515364, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.895453, "value1":0.7852256444188942, "value2":143518625251882031, "value3":true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165192, "value1":0.9962691534924663, "value2":1006093794801043666, "value3":false, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.353290, "value1":0.5341309097782416, "value2":1258165174425707067, "value3":true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257167, "value1":0.7039937676297155, "value2":1145181626509784790, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413647, "value1":0.8270957619940229, "value2":2929580620701623843, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817697, "value1":0.5617132216229666, "value2":9160744958657565723, "value3":false, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.117851, "value1":0.20816644440142626, "value2":5357495560720255737, "value3":false, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545126, "value1":0.5346084869679877, "value2":7401215357009964293, "value3":true, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824766, "value1":0.44946104541199977, "value2":4136655445147421363, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.292920, "value1":0.8837887841085235, "value2":7386066379061583540, "value3":true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949095, "value1":0.3445432872175305, "value2":7354448880530731294, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705243, "value1":0.4595023646368128, "value2":1229720604223933971, "value3":false, "__name__":"metric_00000641", "key_5":"c","key_6":"a","key_1":"j", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610093, "value1":0.4977300016550588, "value2":6055289293785959372, "value3":true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.710138, "value1":0.8494303412007083, "value2":3284734735098008783, "value3":false, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.503055, "value1":0.7306081415151926, "value2":7335475436979640441, "value3":false, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194198, "value1":0.4080577435044893, "value2":1817328694183447312, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409059, "value1":0.8174198346097175, "value2":1028006350387911539, "value3":true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696870, "value1":0.3114167302109578, "value2":5761149591452007692, "value3":true, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483258, "value1":0.6191704897662682, "value2":6062885989254649767, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534924, "value1":0.6117230725465559, "value2":193279975656637243, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639609, "value1":0.47495483196675986, "value2":5738372947955229915, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532964, "value1":0.01280767557324011, "value2":3610414855244726673, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.741210, "value1":0.5736887790720044, "value2":9118221200383527859, "value3":true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459701, "value1":0.09043619123087489, "value2":4263857141830808661, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.064891, "value1":0.8303048678510601, "value2":1417745368253260336, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.193982, "value1":0.5954818624712542, "value2":1866813575466479927, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728941, "value1":0.42773413448912906, "value2":5011276511494346486, "value3":true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978643, "value1":0.9720431088123873, "value2":5835438838603942392, "value3":true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376007, "value1":0.5894070491287171, "value2":1536141728264317947, "value3":false, "__name__":"metric_00000665", "key_8":"g","key_3":"k","key_6":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226693, "value1":0.26901914665508003, "value2":1625748998624229948, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.963696, "value1":0.5769249508695262, "value2":9054438071997375931, "value3":false, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.418644, "value1":0.766497146060167, "value2":5911142909752098984, "value3":true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564856, "value1":0.11883244261524029, "value2":1743904390192603929, "value3":true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055140, "value1":0.52087787431266, "value2":402388317326273396, "value3":false, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165833, "value1":0.8230062512281381, "value2":3737986001509812244, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.537380, "value1":0.3719641116241002, "value2":7620785809655703628, "value3":false, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059138, "value1":0.3359364898323379, "value2":2258599254367397283, "value3":false, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532103, "value1":0.07168825170242699, "value2":8419082906568750462, "value3":false, "__name__":"metric_00000674", "key_7":"f","key_1":"a","key_6":"c", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286202, "value1":0.8607616531331588, "value2":5301488987302943339, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477954, "value1":0.7462715469038064, "value2":8695685405375237694, "value3":false, "__name__":"metric_00000648", "key_2":"e","key_9":"h","key_1":"e", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.377516, "value1":0.1251480000451169, "value2":5693123483571141635, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799701, "value1":0.377835354984664, "value2":7635656423964308029, "value3":true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947251, "value1":0.7193521006934637, "value2":3495998112139060094, "value3":false, "__name__":"metric_00000654", "key_9":"c","key_3":"g","key_8":"h", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.630158, "value1":0.27928250023508844, "value2":8779396817967517012, "value3":true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886093, "value1":0.6334013505397469, "value2":5098375435747464833, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.942245, "value1":0.3390011070632797, "value2":1737381722392040741, "value3":false, "__name__":"metric_00000656", "key_7":"k","key_4":"f","key_6":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.009762, "value1":0.2515798026660567, "value2":2465310044177783178, "value3":false, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.896699, "value1":0.584030830613032, "value2":5977614913522628306, "value3":false, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820378, "value1":0.7811580321417821, "value2":1709536028386531684, "value3":true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653683, "value1":0.7005991085562125, "value2":7295655368561133786, "value3":false, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522361, "value1":0.7296163959268779, "value2":5916302487026661508, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.127496, "value1":0.24876472584091772, "value2":8637191483262110011, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.091531, "value1":0.9069922237355105, "value2":3364698751790785415, "value3":false, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137633, "value1":0.9475238029065713, "value2":6717128691639834645, "value3":false, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550582, "value1":0.1420769771765079, "value2":1644264737205720610, "value3":true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059092, "value1":0.38378121050522995, "value2":4775737880122110580, "value3":true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236099, "value1":0.8215350110894001, "value2":1784075564711617189, "value3":false, "__name__":"metric_00000693", "key_8":"b","key_0":"b","key_5":"e", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.636343, "value1":0.4453072911799391, "value2":3315454218709810887, "value3":true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799575, "value1":0.17197278191202653, "value2":5903905346180673218, "value3":true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.584133, "value1":0.4235206584790561, "value2":1691406943949519298, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015622, "value1":0.34252290670938007, "value2":5505899012400247300, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.021661, "value1":0.11232931832514273, "value2":6468715321661687664, "value3":true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.593330, "value1":0.8735006589361365, "value2":831831146444791684, "value3":false, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273986, "value1":0.25953904800502825, "value2":5320773233674225264, "value3":true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357026, "value1":0.11876193765893824, "value2":8703386441096799470, "value3":true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903715, "value1":0.0317054368098292, "value2":2551886074115870652, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.905359, "value1":0.9780528237638186, "value2":3462176768443711504, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.272907, "value1":0.5077414966128044, "value2":5351891928699763264, "value3":false, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610476, "value1":0.5253243304839273, "value2":7118905075072261164, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684508, "value1":0.3558646139061106, "value2":5157018975048237545, "value3":true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748383, "value1":0.7510255655970447, "value2":4180265211771950319, "value3":true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.801038, "value1":0.2649030050080282, "value2":509475464752529120, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657950, "value1":0.2544231826829063, "value2":4768343360725290987, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149149, "value1":0.24278734421658493, "value2":2310223808628977449, "value3":true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.558331, "value1":0.3866811766896264, "value2":3120828999446479419, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.076776, "value1":0.8191629913767823, "value2":531265332159947696, "value3":false, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881067, "value1":0.16630603149873838, "value2":8244214645789653431, "value3":true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385842, "value1":0.7521248304618617, "value2":4350867045350111094, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914596, "value1":0.12306660567858013, "value2":5899167982504356408, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.159683, "value1":0.24808395371130693, "value2":3321017992196518857, "value3":false, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.058378, "value1":0.4115012836332614, "value2":5551372136464658518, "value3":false, "__name__":"metric_00000708", "key_6":"g","key_0":"a","key_2":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819794, "value1":0.9548946315713374, "value2":2661379806705881657, "value3":true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.293628, "value1":0.30579616916280866, "value2":2771189581765698489, "value3":true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.904159, "value1":0.33983467932998995, "value2":3076540259059103682, "value3":true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946959, "value1":0.14422457278803882, "value2":619548712042120909, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383295, "value1":0.23130372874617672, "value2":6179693520548172628, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482434, "value1":0.8720643520387502, "value2":6593209094509926896, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613596, "value1":0.9574764532675973, "value2":8706916487148773835, "value3":false, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551130, "value1":0.9901941419895447, "value2":5559435899121181473, "value3":true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954673, "value1":0.7022122624865335, "value2":6926608561209244320, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781834, "value1":0.1129731019718482, "value2":3356276176028256450, "value3":true, "__name__":"metric_00000717", "key_4":"f","key_9":"e","key_0":"b", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746610, "value1":0.5148787087784333, "value2":2343142454537691934, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409647, "value1":0.3828843970677094, "value2":4346661084777988652, "value3":true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.427543, "value1":0.6220594320308941, "value2":653819026690948218, "value3":false, "__name__":"metric_00000721", "key_9":"e","key_1":"d", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467056, "value1":0.23591956688182053, "value2":5924417417665977435, "value3":false, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245070, "value1":0.32465613049106207, "value2":8745282484552741587, "value3":true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719344, "value1":0.6149579128373653, "value2":7721284234183306230, "value3":false, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095318, "value1":0.7783448314627689, "value2":7496490910707147022, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696761, "value1":0.23959350451136963, "value2":7667698021180616606, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170976, "value1":0.2107634553446009, "value2":1067019224393043111, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177955, "value1":0.15559883601716398, "value2":570151235008619291, "value3":true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.334876, "value1":0.05250568880425774, "value2":313709473500400875, "value3":true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.645622, "value1":0.5874495193049836, "value2":4998753194711581798, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403716, "value1":0.681377437500311, "value2":6083653541111639303, "value3":true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.511374, "value1":0.3173656336054262, "value2":3280525464397494846, "value3":false, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.218153, "value1":0.6649425163171951, "value2":984695368325234886, "value3":false, "__name__":"metric_00000742", "key_5":"g","key_6":"j","key_4":"a", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424332, "value1":0.13915015029096536, "value2":177058255857848790, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882802, "value1":0.9604830357087591, "value2":3462432728417263080, "value3":false, "__name__":"metric_00000724", "key_9":"f","key_0":"a","key_5":"d", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933587, "value1":0.05218287981155221, "value2":9126870231879692354, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426076, "value1":0.7165203610998703, "value2":4392854854028036091, "value3":false, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.302645, "value1":0.34553849779237905, "value2":2907279110591723700, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269877, "value1":0.8711177705575849, "value2":8366161311313542515, "value3":false, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.995800, "value1":0.6906708270307872, "value2":2179499360020827214, "value3":false, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433217, "value1":0.612549221250184, "value2":3062667501076675896, "value3":true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550193, "value1":0.8112010714019721, "value2":3511775054758615153, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947917, "value1":0.4309103807271251, "value2":5927887337207634099, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874180, "value1":0.8528260534422161, "value2":6693253358548032140, "value3":false, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.714464, "value1":0.7660112509524851, "value2":4667923922359952270, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.088061, "value1":0.2662708805339307, "value2":2956847029720911185, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960166, "value1":0.6603171866146907, "value2":4062360360972662646, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.771991, "value1":0.06535928533875911, "value2":3275575695469134507, "value3":false, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.907738, "value1":0.4729789504578976, "value2":6665898542309531788, "value3":false, "__name__":"metric_00000749", "key_6":"c","key_9":"e","key_5":"f", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.446527, "value1":0.054360638009933175, "value2":8447197342747029062, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871013, "value1":0.9325815658063547, "value2":3295816637987069156, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322468, "value1":0.7896334184172146, "value2":2062716273940484317, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145073, "value1":0.15908044288327283, "value2":31202777072956548, "value3":true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.450280, "value1":0.20707514852594208, "value2":3916360901630366923, "value3":true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.834638, "value1":0.6993008706531769, "value2":5937180712489291188, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116654, "value1":0.23637601643034672, "value2":4626933419743360623, "value3":true, "__name__":"metric_00000765", "key_5":"h","key_9":"b","key_3":"a", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.265735, "value1":0.37825344249371357, "value2":1259576346841819002, "value3":true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683748, "value1":0.8969615470937745, "value2":3878280364099631128, "value3":true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219470, "value1":0.2735521439606725, "value2":2004029025316087157, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476407, "value1":0.3060298444596037, "value2":2752192683000499763, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973916, "value1":0.6236548692215601, "value2":1218902262886276264, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777866, "value1":0.4605467848114755, "value2":1483419366258255115, "value3":false, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045114, "value1":0.6713933571025698, "value2":7389349750224761078, "value3":true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.570321, "value1":0.6477885368549638, "value2":952734344221236, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821043, "value1":0.6987507461356852, "value2":7961454653125032378, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.586904, "value1":0.6114733442241732, "value2":3313794499633786815, "value3":false, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705481, "value1":0.05021439091843144, "value2":1692908847970065889, "value3":false, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497544, "value1":0.7911397038857368, "value2":7192760038857907146, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.932189, "value1":0.9148929522533059, "value2":9058585624863979355, "value3":true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769986, "value1":0.08834899410804685, "value2":1734900992084257549, "value3":false, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695397, "value1":0.9679316547671841, "value2":7641677535001923086, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084900, "value1":0.7534718430023655, "value2":5595569809726413737, "value3":true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.562250, "value1":0.19691330948497934, "value2":5079334807276275801, "value3":false, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.843384, "value1":0.13037385236459426, "value2":1781212766913480310, "value3":false, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.102533, "value1":0.8945673902737089, "value2":4413715911002881, "value3":false, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350432, "value1":0.9789410844857529, "value2":8061203484333522616, "value3":false, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857648, "value1":0.17405749140176002, "value2":9069075917121269047, "value3":false, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559262, "value1":0.2097671258596821, "value2":613585525667388168, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.712886, "value1":0.7017574550139256, "value2":807895944720896464, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.936984, "value1":0.8849159113766228, "value2":6360061394425352910, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545107, "value1":0.7492436378397768, "value2":1930556756703562417, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.681593, "value1":0.044080654493611975, "value2":4672445330328029132, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388330, "value1":0.4856178458787539, "value2":6153421365310799825, "value3":true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.879197, "value1":0.16996161009540478, "value2":260583885651182711, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.770981, "value1":0.7283458398178451, "value2":6187029331538785539, "value3":true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101721, "value1":0.5814924092013966, "value2":3246021123474652609, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665903, "value1":0.2698824852319885, "value2":1593984595070302417, "value3":true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.014945, "value1":0.41108393803888965, "value2":849125531814643716, "value3":false, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455276, "value1":0.875114673636132, "value2":5155846223745306265, "value3":false, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.044985, "value1":0.9997830302808556, "value2":2517744318699317082, "value3":false, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236657, "value1":0.10301472150509888, "value2":3916175947622962913, "value3":false, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.595845, "value1":0.681705831836613, "value2":4919968063926505445, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909129, "value1":0.10658700130911442, "value2":9017518189462648176, "value3":false, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994913, "value1":0.5742718119176038, "value2":2926652004213539521, "value3":false, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350277, "value1":0.15279738754509417, "value2":5229335911692726266, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.022399, "value1":0.3330863087425972, "value2":4953311412687090451, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268930, "value1":0.7193122390898454, "value2":4256538916269794836, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.983431, "value1":0.0510995090351011, "value2":1768238660724776680, "value3":false, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672570, "value1":0.255109482234144, "value2":5911656502600207006, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478340, "value1":0.7629467314333804, "value2":2248461028173940598, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954691, "value1":0.780819131964203, "value2":4988575109325316188, "value3":false, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.854956, "value1":0.888056149862958, "value2":8089385147769653445, "value3":true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028845, "value1":0.6170371731466705, "value2":4427914831964658823, "value3":true, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639652, "value1":0.8230090544258113, "value2":6386454055832759048, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.810891, "value1":0.8133492204735011, "value2":6725307713283048525, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761452, "value1":0.9287463566978789, "value2":1984332858107282238, "value3":false, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318552, "value1":0.12867977830402738, "value2":4258557511687812956, "value3":true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316280, "value1":0.5022437385431537, "value2":995773241737628619, "value3":false, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954122, "value1":0.5240772572203263, "value2":6037304727626529988, "value3":true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177671, "value1":0.76376619698564, "value2":2077146903364533171, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731650, "value1":0.7327573768170443, "value2":3115299633570203304, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.050578, "value1":0.7589539042984136, "value2":8472839195693038957, "value3":true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.606514, "value1":0.9364993124842877, "value2":5612839525396767856, "value3":false, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369368, "value1":0.048054238834199556, "value2":5282267936079713586, "value3":false, "__name__":"metric_00000823", "key_9":"i","key_8":"d", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619618, "value1":0.3228482622950357, "value2":8295983975137448078, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783482, "value1":0.20802671279789772, "value2":3029896725457460111, "value3":true, "__name__":"metric_00000825", "key_7":"j","key_8":"d","key_5":"k", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332994, "value1":0.9302305981560293, "value2":1560834164036895150, "value3":false, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320192, "value1":0.22351800921367507, "value2":367222968940142358, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881266, "value1":0.08414682717481595, "value2":4156359981061160049, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.708929, "value1":0.5859616740437592, "value2":4218658472873761504, "value3":true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.974600, "value1":0.18236565991254491, "value2":7158153525091699583, "value3":false, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760005, "value1":0.2499852109211892, "value2":3103407843415565026, "value3":false, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994395, "value1":0.6346134433025367, "value2":8895656809884955572, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482016, "value1":0.09096777794371665, "value2":6086112719790576068, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442570, "value1":0.8266959930861106, "value2":2275327626493666884, "value3":false, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416700, "value1":0.36518175488705273, "value2":5793835688591222111, "value3":true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.488107, "value1":0.8111992750792233, "value2":3432067066553462639, "value3":true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.296157, "value1":0.748182736526456, "value2":5908492840734386304, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655322, "value1":0.77012733378921, "value2":2328983933411827460, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299380, "value1":0.8617587030482643, "value2":4249851149905507488, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.569550, "value1":0.8799767655587674, "value2":114484019272599760, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.737420, "value1":0.660133828675834, "value2":47092260178100634, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961538, "value1":0.5805975021285331, "value2":3098519932913684476, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197731, "value1":0.7355297940768138, "value2":5347385484676578771, "value3":true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653672, "value1":0.9639932888139914, "value2":4154150822821006863, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209735, "value1":0.25867188901232296, "value2":7573003393945357593, "value3":false, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.243923, "value1":0.34698853560785725, "value2":4046809595104442791, "value3":false, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055125, "value1":0.9118136199691569, "value2":6132859986145115351, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.129146, "value1":0.737958001899574, "value2":3639000020359385593, "value3":true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439824, "value1":0.7612995291197047, "value2":9120669622233440416, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222157, "value1":0.2453738047252645, "value2":5040757825498916672, "value3":false, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916336, "value1":0.7380620101187605, "value2":4223451018219910420, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365543, "value1":0.035239810434389264, "value2":7558213648465294836, "value3":false, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.860762, "value1":0.4102850563256095, "value2":1429493420542448489, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388872, "value1":0.5076851349389522, "value2":5984142931688899418, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.324767, "value1":0.32798077886519317, "value2":7010471358073816774, "value3":true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605843, "value1":0.9054516940429083, "value2":5196636487460887513, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653438, "value1":0.9546349180436571, "value2":1964265110886490351, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576178, "value1":0.9072096436167739, "value2":4650061523788539671, "value3":true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.304613, "value1":0.07126080366571305, "value2":2739761425628315393, "value3":false, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399413, "value1":0.9339282528280148, "value2":2738842204765000811, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943959, "value1":0.08784682744168541, "value2":7785212390244201422, "value3":false, "__name__":"metric_00000813", "key_7":"g","key_8":"h","key_1":"j", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.155440, "value1":0.06596281114416563, "value2":5614935012679054029, "value3":true, "__name__":"metric_00000814", "key_4":"d","key_2":"f", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.585232, "value1":0.38735104883356297, "value2":5357846878220935711, "value3":true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.231739, "value1":0.7284804092280743, "value2":1167665308911410443, "value3":true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555892, "value1":0.5915854017921878, "value2":1567147347614329809, "value3":true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.855303, "value1":0.09176207020704319, "value2":3808984845296778232, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.938710, "value1":0.4819531123010938, "value2":4697805519732493178, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476902, "value1":0.22082616880251751, "value2":2129509252592428060, "value3":false, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744743, "value1":0.33257150382979367, "value2":7648865469799769588, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306532, "value1":0.04481922646422544, "value2":5772762433855832524, "value3":false, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364825, "value1":0.3587026951251621, "value2":5278057080002614026, "value3":true, "__name__":"metric_00000863", "key_4":"c","key_3":"j", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.484995, "value1":0.39478883780380347, "value2":3136752353967205980, "value3":true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.359118, "value1":0.08414989767840077, "value2":1942845542819450866, "value3":true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095178, "value1":0.16106032134150491, "value2":2585752892335863488, "value3":false, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268896, "value1":0.613902523132248, "value2":687319106737333189, "value3":true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716029, "value1":0.4960997765937663, "value2":2263474829486803965, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748556, "value1":0.30380613001187684, "value2":569759032424044386, "value3":false, "__name__":"metric_00000869", "key_3":"c","key_5":"b","key_1":"c", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819011, "value1":0.9766133829878352, "value2":4041486474077430360, "value3":false, "__name__":"metric_00000868", "key_4":"i","key_8":"j","key_3":"i", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109307, "value1":0.7831795468260019, "value2":6028647505036494731, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.179692, "value1":0.36834450950873376, "value2":5143099232657374567, "value3":false, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408811, "value1":0.37776607558956526, "value2":6856480231263656565, "value3":true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.006253, "value1":0.5270930124398272, "value2":1411653061914296136, "value3":true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776912, "value1":0.4551811310760191, "value2":1536148212309064628, "value3":false, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447453, "value1":0.8174960750892899, "value2":4331073111578365688, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696722, "value1":0.2434463944121451, "value2":3006348927793308525, "value3":true, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.747552, "value1":0.46652870663413926, "value2":8487775958608976630, "value3":false, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618043, "value1":0.5667967701695303, "value2":445864738400581228, "value3":false, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162568, "value1":0.6628301293329222, "value2":8197909955604355013, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.861425, "value1":0.3485000903134516, "value2":8816338849324478138, "value3":false, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.806516, "value1":0.4953223448201175, "value2":7711648028791565560, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970539, "value1":0.2083396416590356, "value2":5815736098137722214, "value3":false, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108983, "value1":0.42614208367663026, "value2":5403929578330914973, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095108, "value1":0.4675226715836234, "value2":4103322823365161263, "value3":true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.279008, "value1":0.5189838609534998, "value2":5583248834742899795, "value3":true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080151, "value1":0.04025953736590447, "value2":1709967085975216083, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960603, "value1":0.48558007294067956, "value2":147087377780565755, "value3":true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522699, "value1":0.8996419465199031, "value2":8383086671138869204, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812906, "value1":0.42145746065428685, "value2":3807545219581503501, "value3":false, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973610, "value1":0.9578107981118065, "value2":4941505183486571577, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972491, "value1":0.7644375474791567, "value2":5826757011862504756, "value3":true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048527, "value1":0.2281920875592009, "value2":4718173727183162630, "value3":false, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515627, "value1":0.3746899214010938, "value2":5127946555951269169, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.346004, "value1":0.5308363801498025, "value2":1141799226947758562, "value3":true, "__name__":"metric_00000894", "key_7":"d","key_3":"k","key_5":"c", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391384, "value1":0.5665502681985652, "value2":5138410786658734048, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857410, "value1":0.1451852744317182, "value2":6603434642077187944, "value3":false, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.414061, "value1":0.265028330167258, "value2":8861513360159934632, "value3":false, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653333, "value1":0.28437623266560746, "value2":5007130114350069495, "value3":false, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.123846, "value1":0.29659698191393213, "value2":8540905938700351347, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477802, "value1":0.6141130483574686, "value2":4145514544028354410, "value3":false, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531351, "value1":0.3974291747311908, "value2":7891249160701823099, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197949, "value1":0.9490013407410278, "value2":8218574898456721251, "value3":false, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411857, "value1":0.8458181580037749, "value2":3063835093954499296, "value3":false, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.877510, "value1":0.828501312919876, "value2":5054318282608028643, "value3":true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665063, "value1":0.5885292293078199, "value2":7952781418541268636, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041521, "value1":0.9878113457449454, "value2":7450867973872472368, "value3":true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073061, "value1":0.939649958460976, "value2":7875175737198226822, "value3":false, "__name__":"metric_00000898", "key_5":"k","key_9":"d","key_4":"i", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781888, "value1":0.5031750456506159, "value2":7116386269513023773, "value3":false, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160017, "value1":0.23130907810117704, "value2":2438724480357189127, "value3":false, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835359, "value1":0.4415459106308202, "value2":8077656610946018544, "value3":false, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251312, "value1":0.10148089002006272, "value2":1896400908090076616, "value3":false, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.632483, "value1":0.7757884759608441, "value2":4168124167987668028, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.772638, "value1":0.14296917471420942, "value2":5892426599914094550, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426554, "value1":0.4152913794309927, "value2":3491478566670263783, "value3":true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672520, "value1":0.5730477814369939, "value2":7734177692857714726, "value3":false, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025475, "value1":0.8422765595744146, "value2":7729206509934976065, "value3":true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384117, "value1":0.370521152586741, "value2":5964592513649221172, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666556, "value1":0.5944594406776452, "value2":6664609919274167291, "value3":true, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424484, "value1":0.6898557357450742, "value2":7620342586495006501, "value3":true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313298, "value1":0.7769705418438511, "value2":5904168920374633694, "value3":false, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556524, "value1":0.7197157815883978, "value2":1785992040018522529, "value3":true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360907, "value1":0.5368763819006307, "value2":1221035619431068900, "value3":false, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576297, "value1":0.2976581980063287, "value2":6069726578064867551, "value3":true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.425007, "value1":0.05760812531545847, "value2":3218040486200104849, "value3":false, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442385, "value1":0.7140138290502642, "value2":5982710133241839409, "value3":false, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364950, "value1":0.7133219094796532, "value2":2621884575274392083, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211537, "value1":0.9662389413186122, "value2":6837048628615994370, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.161776, "value1":0.9162528731861198, "value2":98225928051258831, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811493, "value1":0.5668121931771126, "value2":4899385784647025957, "value3":false, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822383, "value1":0.016988952685027904, "value2":7343988970036816255, "value3":false, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865454, "value1":0.48779466867997384, "value2":3996429126028951858, "value3":false, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751902, "value1":0.517057218253837, "value2":4621244619940358234, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.525579, "value1":0.7270638229553396, "value2":4556294068287353316, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822049, "value1":0.8487037862810487, "value2":8759327482669918657, "value3":false, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483767, "value1":0.795243973695518, "value2":5137269638112922032, "value3":false, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042738, "value1":0.7385935337493647, "value2":1091563145939490509, "value3":true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.521929, "value1":0.10598411546030329, "value2":1018022188446431653, "value3":true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792757, "value1":0.4495950673323956, "value2":1856520078086460155, "value3":true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695775, "value1":0.8122734246000456, "value2":2520061166650080735, "value3":false, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105722, "value1":0.9820835252391706, "value2":7455782937968387088, "value3":true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820568, "value1":0.22530315350938107, "value2":3276960573086645000, "value3":false, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.140320, "value1":0.5664165038340241, "value2":7444501896936670459, "value3":false, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.501828, "value1":0.6731262889242471, "value2":1425113709817927232, "value3":true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930172, "value1":0.48625040372483846, "value2":5327468289350467201, "value3":false, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.706858, "value1":0.07914368053314678, "value2":1800402440657604521, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134725, "value1":0.4678852928796053, "value2":2325913710025246340, "value3":true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876578, "value1":0.9424035369996965, "value2":8055626132574707503, "value3":true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108826, "value1":0.29002692913973105, "value2":774202804060462158, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.031892, "value1":0.9251224968475468, "value2":6954662396929356191, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539689, "value1":0.24245379006948578, "value2":460788187782305876, "value3":false, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.858085, "value1":0.02056945040393724, "value2":4590009984548317252, "value3":false, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.202540, "value1":0.37546125893661825, "value2":6399435061817101457, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384494, "value1":0.010928897877400335, "value2":2783102565309398205, "value3":true, "__name__":"metric_00000953", "key_1":"f","key_9":"g","key_0":"k", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061578, "value1":0.259570974455371, "value2":8569411528487848312, "value3":false, "__name__":"metric_00000954", "key_8":"h","key_9":"e","key_2":"h", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027239, "value1":0.09519589010929196, "value2":6611488629400003141, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210858, "value1":0.6912816460992459, "value2":7725980859343417825, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.825726, "value1":0.48050045655332785, "value2":6843673580007675724, "value3":false, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430241, "value1":0.8940246106870715, "value2":7204593499636831454, "value3":false, "__name__":"metric_00000967", "key_5":"d","key_9":"f","key_1":"h", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332710, "value1":0.05615399939144247, "value2":4573382128185893311, "value3":true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374418, "value1":0.5152317418659397, "value2":7908425193534726276, "value3":false, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519252, "value1":0.2299372870124918, "value2":6913359063794068428, "value3":false, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.405529, "value1":0.49571648923268996, "value2":1675726536905469427, "value3":true, "__name__":"metric_00000971", "key_8":"i","key_2":"j", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605031, "value1":0.09917638686306547, "value2":3890367351649717445, "value3":true, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194280, "value1":0.5756913899097403, "value2":6546196099308660172, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763643, "value1":0.049961431289849915, "value2":2156338695320389229, "value3":true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755741, "value1":0.4853393193247385, "value2":5380709479032006182, "value3":true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434015, "value1":0.6730243170500183, "value2":4072172223941247851, "value3":true, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.188499, "value1":0.8036991963254426, "value2":2042780350133201142, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.788778, "value1":0.6590319721348468, "value2":6545712378323399560, "value3":false, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431899, "value1":0.10111044541197162, "value2":3033625496942345343, "value3":false, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846633, "value1":0.4624813135357754, "value2":5961232902625455903, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649911, "value1":0.42335939078880974, "value2":9206873087714105578, "value3":true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532557, "value1":0.8374513257646443, "value2":4063679098902368460, "value3":false, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.993435, "value1":0.026158890681344544, "value2":8561365511354827010, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808220, "value1":0.6771449672369052, "value2":2392568978099066809, "value3":false, "__name__":"metric_00000959", "key_2":"h","key_9":"g","key_0":"b", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357037, "value1":0.7840791560680648, "value2":1293280054604430799, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686467, "value1":0.009361168431443715, "value2":1158105653011230048, "value3":false, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.971166, "value1":0.5749373357687312, "value2":5056280036307903399, "value3":true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.544825, "value1":0.6259231852657728, "value2":1572064409805832355, "value3":false, "__name__":"metric_00000965", "key_9":"c","key_5":"i", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889816, "value1":0.6691815472587526, "value2":5159786074932669495, "value3":false, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751169, "value1":0.5117344773320296, "value2":2709445168494389044, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.358316, "value1":0.5060937970199147, "value2":4487218678538317276, "value3":false, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.577717, "value1":0.8145066136721544, "value2":3597604992290980548, "value3":true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.990084, "value1":0.031203549542102884, "value2":6033709675731250939, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613856, "value1":0.5443798055071071, "value2":1023967738912955990, "value3":false, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795434, "value1":0.2935457393386216, "value2":1392092134182260244, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233100, "value1":0.7087138744541079, "value2":6901046651435105011, "value3":false, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.215643, "value1":0.301381153698089, "value2":5724617386824483093, "value3":true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.536387, "value1":0.050992650300612716, "value2":1926309233688447762, "value3":true, "__name__":"metric_00000998", "key_9":"c","key_3":"e", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.661031, "value1":0.5259350399353443, "value2":3757516621940142781, "value3":false, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.883217, "value1":0.9897587697648693, "value2":166453122026887271, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482631, "value1":0.7903296358928328, "value2":6917975765021057279, "value3":true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301142, "value1":0.03214802664084894, "value2":1209824808667363268, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618630, "value1":0.34174203368906997, "value2":687273686120110296, "value3":true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872422, "value1":0.7312694201106051, "value2":1954246364933112681, "value3":false, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675094, "value1":0.02386805295420225, "value2":2582799143372120508, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.397537, "value1":0.7375439234748727, "value2":3761195316124557282, "value3":false, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998621, "value1":0.44001824362826686, "value2":2298817484668315029, "value3":false, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471460, "value1":0.5040752524973563, "value2":7440124910918057262, "value3":true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303559, "value1":0.34760281561681783, "value2":3856193801537489239, "value3":true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477238, "value1":0.10358452410298057, "value2":3433455763442399890, "value3":true, "__name__":"metric_00000001", "key_5":"g","key_6":"a","key_4":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974082, "value1":0.9936842430072798, "value2":7647779573135010712, "value3":false, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.679212, "value1":0.33597414933520936, "value2":7871330604163867562, "value3":true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.420730, "value1":0.4119033435452023, "value2":7398827678901559124, "value3":false, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.546764, "value1":0.0476713858704466, "value2":112659874965942961, "value3":false, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730784, "value1":0.2868023177018098, "value2":7913444002207179200, "value3":true, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862440, "value1":0.5312714100525405, "value2":6175515648240114931, "value3":true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769163, "value1":0.9602531490116414, "value2":5345461780217485343, "value3":true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855124, "value1":0.553411156209368, "value2":4062088865979243439, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.210418, "value1":0.8711908622408605, "value2":8107060141858943082, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436733, "value1":0.7746479416891845, "value2":3021240871572817126, "value3":false, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.692065, "value1":0.594498398968098, "value2":988266694346712016, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.441926, "value1":0.8776750344471822, "value2":6190746113032965957, "value3":true, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013512, "value1":0.14091811490255743, "value2":4960449076483043020, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548236, "value1":0.32376195419487824, "value2":4889372139161861463, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.828595, "value1":0.6520731819283484, "value2":508152344585097791, "value3":false, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759392, "value1":0.11332264235829892, "value2":8149117049560006848, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901588, "value1":0.1665732927247099, "value2":1166517989622232910, "value3":true, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035913, "value1":0.07988414283718974, "value2":7894355898795371390, "value3":false, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.583982, "value1":0.25757663417932825, "value2":6007061793954380382, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636209, "value1":0.38383005316111324, "value2":4618851528086053316, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672395, "value1":0.002191687957779578, "value2":1207882083575373188, "value3":true, "__name__":"metric_00000031", "key_5":"c","key_7":"f","key_0":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462535, "value1":0.3192805953827191, "value2":365124979437384544, "value3":false, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508318, "value1":0.6798889100739353, "value2":255815407107212152, "value3":true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905933, "value1":0.882876839434679, "value2":1335222091221270902, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.721902, "value1":0.3457876641727297, "value2":7448197074268866329, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.837057, "value1":0.03599562374649498, "value2":1792244431983896309, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056476, "value1":0.22903239483804314, "value2":5638633766102759583, "value3":true, "__name__":"metric_00000012", "key_7":"j","key_2":"e", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.457531, "value1":0.05016525338775747, "value2":2976936137139451754, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617621, "value1":0.16600638886403743, "value2":3903308221116403976, "value3":false, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415798, "value1":0.5206207545088288, "value2":7426460172854544634, "value3":true, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.470076, "value1":0.02758169975135837, "value2":3279193513719457677, "value3":false, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.847049, "value1":0.818759672974311, "value2":5946222659493106047, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367937, "value1":0.4074258090017152, "value2":6858775210252472573, "value3":false, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973712, "value1":0.13887537149474657, "value2":2800343207358775074, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403585, "value1":0.2978164609006992, "value2":6234312472150253679, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904490, "value1":0.7280336451114688, "value2":2026079147622689137, "value3":false, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.541237, "value1":0.1368259441713387, "value2":6424894922861577254, "value3":false, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.783065, "value1":0.5704544299311838, "value2":4180282487041693175, "value3":false, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.890800, "value1":0.7757728483786852, "value2":8132981856885227577, "value3":true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901025, "value1":0.22908718180210688, "value2":4747104448031391939, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180212, "value1":0.5476402281995001, "value2":8223100966387528196, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626159, "value1":0.7996650121856338, "value2":64003761107808118, "value3":false, "__name__":"metric_00000041", "key_9":"g","key_3":"k","key_4":"f", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545550, "value1":0.1740742638651991, "value2":5824563041193046282, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026911, "value1":0.9915609719154322, "value2":3839917984332114189, "value3":false, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970605, "value1":0.049651840615534626, "value2":3663818871674092196, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.795861, "value1":0.6046374060545386, "value2":6467693625848837489, "value3":false, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741291, "value1":0.4689122935380642, "value2":8701125505618349182, "value3":true, "__name__":"metric_00000046", "key_6":"h","key_2":"k","key_4":"e", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.877913, "value1":0.596765963129067, "value2":8094571743010403516, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.790394, "value1":0.06093826578205693, "value2":733010313976904838, "value3":true, "__name__":"metric_00000048", "key_9":"b","key_1":"e", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603208, "value1":0.8364659181351286, "value2":1088146971265159206, "value3":true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823621, "value1":0.47775163306986124, "value2":7344279199603749996, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171666, "value1":0.02782471041039711, "value2":1659046247294927834, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.899246, "value1":0.23664326650032677, "value2":764233009249486091, "value3":true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658885, "value1":0.13203154011115373, "value2":5571262153472749697, "value3":false, "__name__":"metric_00000064", "key_4":"a","key_7":"k","key_1":"j", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.711786, "value1":0.6314799048378197, "value2":8955079539774261393, "value3":true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.358690, "value1":0.4626190815088963, "value2":5349880367436474865, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436669, "value1":0.7604788504004301, "value2":2830736413446294733, "value3":true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283630, "value1":0.11435547311453685, "value2":5764151651661369748, "value3":false, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550602, "value1":0.9394300496620045, "value2":3385128630739961936, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004479, "value1":0.08103496493647538, "value2":1096104398248692524, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272179, "value1":0.007258870887977933, "value2":2565647835692463774, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.207218, "value1":0.8139650845703748, "value2":905077995346686402, "value3":true, "__name__":"metric_00000072", "key_3":"g","key_9":"k","key_0":"g", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217842, "value1":0.48125835395143424, "value2":6789722720604854599, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.155093, "value1":0.7189778774843847, "value2":6292044296017930854, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723501, "value1":0.3703438091023322, "value2":3955334986917106318, "value3":true, "__name__":"metric_00000075", "key_9":"f","key_3":"j","key_5":"i", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216869, "value1":0.12053628422917362, "value2":636444633907082482, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.568591, "value1":0.23735656022841886, "value2":4679225743620568386, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.792289, "value1":0.8769118352474913, "value2":8705903369099981004, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.377367, "value1":0.9028525772471804, "value2":8394233771809162273, "value3":true, "__name__":"metric_00000079", "key_3":"d","key_6":"c","key_0":"d", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511370, "value1":0.06603888812647277, "value2":1587071294143636433, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060860, "value1":0.3291125358805135, "value2":7312301032222805401, "value3":true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816308, "value1":0.6388914782369252, "value2":1128303197769336870, "value3":true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.068805, "value1":0.319277617656504, "value2":6681946261878539245, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.805301, "value1":0.009280749666995351, "value2":544578594920738627, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493659, "value1":0.6717590147189912, "value2":2097325929268646339, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.253857, "value1":0.2354585485783823, "value2":5182832084934966477, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454134, "value1":0.7059786460183258, "value2":8734869026681606253, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769480, "value1":0.16571806998598893, "value2":1451771174019239273, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.043302, "value1":0.0805122859716286, "value2":8615098369512434691, "value3":true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493432, "value1":0.37562446579716846, "value2":447590635610718249, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.340933, "value1":0.476702778910786, "value2":2962733058848207480, "value3":false, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.351270, "value1":0.04893100925940487, "value2":3216639061913163460, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819062, "value1":0.43774247002290106, "value2":6810500585509381104, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.789988, "value1":0.13044652854726224, "value2":6447819044361282673, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.137973, "value1":0.9280703615023543, "value2":2854967575973809701, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359816, "value1":0.6661196608352099, "value2":7585154254761447569, "value3":false, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.374877, "value1":0.2719104405916207, "value2":1097487272974828210, "value3":true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766465, "value1":0.8656426220974218, "value2":2886501368125017740, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267362, "value1":0.594401465949902, "value2":1715419869280476267, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.689698, "value1":0.09904556283985433, "value2":361858677456228074, "value3":true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244000, "value1":0.4739214533638735, "value2":6722075717885814318, "value3":true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957522, "value1":0.4793380026682715, "value2":8094832594089347058, "value3":true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578816, "value1":0.32497702056237604, "value2":8938729328568281395, "value3":false, "__name__":"metric_00000103", "key_6":"h","key_1":"f","key_5":"e", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999599, "value1":0.703313156867648, "value2":5755903281901537381, "value3":false, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829223, "value1":0.9601134377422823, "value2":4062518676536818057, "value3":true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.145200, "value1":0.7216961923243395, "value2":9217282407920967916, "value3":false, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820803, "value1":0.3093137747462916, "value2":7468155061834676810, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682889, "value1":0.04485861766843415, "value2":4270651219964958841, "value3":true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549256, "value1":0.7924629761166294, "value2":5462651594092502924, "value3":true, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515774, "value1":0.1753099089980214, "value2":4245273970497284960, "value3":true, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280472, "value1":0.19807313433421816, "value2":6932983129449398217, "value3":true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.183908, "value1":0.8803355871960044, "value2":3795185008893112660, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.250576, "value1":0.8766744958241532, "value2":8316217539045644702, "value3":true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811589, "value1":0.7673136679205708, "value2":1656000207332411296, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617752, "value1":0.33750626542384204, "value2":2818737959409200195, "value3":true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.402465, "value1":0.5975891290632979, "value2":8117852232376404604, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.779395, "value1":0.6126614135133973, "value2":1972761209604651335, "value3":false, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171014, "value1":0.3147679970132471, "value2":3872394099215939892, "value3":false, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.602514, "value1":0.9979964274456362, "value2":8280861801559743509, "value3":true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.870286, "value1":0.1346288239954595, "value2":2167488486464723184, "value3":true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.902696, "value1":0.53464205049288, "value2":8019370943564861440, "value3":true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050828, "value1":0.7038559591596831, "value2":6864265904757939564, "value3":true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.465826, "value1":0.7686595552419602, "value2":1927886873036321447, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149509, "value1":0.9153544198103084, "value2":7814774915075726621, "value3":true, "__name__":"metric_00000116", "key_9":"f","key_0":"k","key_1":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.485222, "value1":0.9188620757546552, "value2":5192187368195488529, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520165, "value1":0.1037499454739713, "value2":2861179515410191072, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894795, "value1":0.6670064141722539, "value2":8570373623610344103, "value3":true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858375, "value1":0.9690125335533832, "value2":6598305524856472910, "value3":false, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639161, "value1":0.07029962179108588, "value2":4198012464202851461, "value3":true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.422856, "value1":0.243592288273798, "value2":7060563137853890653, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356376, "value1":0.12506395613411664, "value2":3055799774165215720, "value3":false, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296059, "value1":0.7972570100142576, "value2":2611996617533897408, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.461343, "value1":0.024502184861611945, "value2":1123093231220631621, "value3":false, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529133, "value1":0.12083989402126777, "value2":7219868115171352433, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110681, "value1":0.8049351220640796, "value2":4305443631669757518, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.981566, "value1":0.19931635115078358, "value2":4376781646759017505, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631835, "value1":0.4428488032269089, "value2":2962840158667509272, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596817, "value1":0.01388179851872705, "value2":5526059113621970146, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988044, "value1":0.08966434896908937, "value2":845094337456207269, "value3":false, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032366, "value1":0.4632294657773622, "value2":2896178627081634846, "value3":true, "__name__":"metric_00000132", "key_9":"i","key_0":"i","key_3":"a", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.195349, "value1":0.3544490738754114, "value2":5485050023862879453, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.488827, "value1":0.05815655955779658, "value2":5446494658153199948, "value3":false, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723246, "value1":0.6834006388190103, "value2":4274252704044712919, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141449, "value1":0.3967110267075465, "value2":2849522299224594762, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.712863, "value1":0.2643526520952433, "value2":6055035031034459665, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974542, "value1":0.07826746989883603, "value2":8013938940173093770, "value3":false, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476496, "value1":0.08282820679282239, "value2":8740534065670450787, "value3":true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.265568, "value1":0.7141009753996177, "value2":8015024495118709084, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163000, "value1":0.9787912408769939, "value2":1407010919457389562, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560997, "value1":0.22142486556739396, "value2":2439658579639630159, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325355, "value1":0.2775049821475937, "value2":2346830518660166745, "value3":false, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491491, "value1":0.3135314864967855, "value2":1129771089143019455, "value3":true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.459317, "value1":0.2812010467625179, "value2":7830625013744386941, "value3":false, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.684694, "value1":0.4935589110227349, "value2":2540823529765104488, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.237886, "value1":0.07014579422912577, "value2":7022680572815993337, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.414046, "value1":0.06496797245021041, "value2":2711277679569543532, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799639, "value1":0.8106870375099371, "value2":1007449970411192794, "value3":false, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.264752, "value1":0.0960343886163872, "value2":3312684645465619564, "value3":true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035469, "value1":0.0008360965935936516, "value2":5539611781321403864, "value3":true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.306352, "value1":0.07245144250837937, "value2":3686038376310116060, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.048629, "value1":0.3482697350949442, "value2":2011496763457379089, "value3":true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978673, "value1":0.9982728323868918, "value2":702860377347588565, "value3":false, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987564, "value1":0.21515438859453273, "value2":2583082073184523816, "value3":true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.842602, "value1":0.3444834642501587, "value2":4908475207072135082, "value3":false, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966769, "value1":0.7858094998520652, "value2":6077992589658011123, "value3":true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.165450, "value1":0.6383111602434741, "value2":7226409376141032655, "value3":true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.575923, "value1":0.8562665414193692, "value2":2173652645149705334, "value3":true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010707, "value1":0.5955696347147411, "value2":5814682382951778842, "value3":true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271135, "value1":0.1229263700863702, "value2":6350918748538162755, "value3":false, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143782, "value1":0.975371469047403, "value2":8279280670157614673, "value3":false, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536001, "value1":0.2818838543932868, "value2":5548964524114515649, "value3":false, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.274203, "value1":0.525661345793623, "value2":7632566212944122955, "value3":true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394070, "value1":0.7913688238824117, "value2":2991067692464176886, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243187, "value1":0.9989819654882474, "value2":7814314371764134982, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490869, "value1":0.30039262696070124, "value2":7927229014613646504, "value3":false, "__name__":"metric_00000166", "key_6":"i","key_0":"f","key_5":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.850916, "value1":0.8481086988896461, "value2":3502125731621593449, "value3":false, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113680, "value1":0.32930706694057965, "value2":6445034393695608409, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192297, "value1":0.3973423811708676, "value2":8167271156640822516, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646981, "value1":0.6421917320177064, "value2":2441775949474583008, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.482724, "value1":0.3152543309412001, "value2":8846147135578854122, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777161, "value1":0.26611205622992573, "value2":4707069095348547526, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529669, "value1":0.25143936257925226, "value2":9113227763618672958, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708875, "value1":0.10515643642729285, "value2":7582828897771165151, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041864, "value1":0.7204022074713844, "value2":4505351249313209540, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.517118, "value1":0.2778141202519239, "value2":7041089660646534661, "value3":true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255695, "value1":0.5252045968012302, "value2":1609597628350217466, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.001450, "value1":0.15419422555422022, "value2":6306687575854728640, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.230403, "value1":0.7229181740004053, "value2":1306679799234664681, "value3":false, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035841, "value1":0.6895973487797646, "value2":2559143873675894370, "value3":false, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.449904, "value1":0.7368166767257398, "value2":7052097343090100521, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139346, "value1":0.807172748525886, "value2":4308240895194146306, "value3":false, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977592, "value1":0.7560595694496225, "value2":3367349773561100464, "value3":false, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.831129, "value1":0.43661553653957313, "value2":1514573148689105923, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804508, "value1":0.7956852333376209, "value2":8820887809742814806, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858280, "value1":0.8493727617212654, "value2":8684654170930913793, "value3":false, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640988, "value1":0.2223414127619492, "value2":7250214226981174458, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153838, "value1":0.9864893077863718, "value2":1366599113373589549, "value3":false, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144317, "value1":0.034179582463187676, "value2":4811674883286124820, "value3":true, "__name__":"metric_00000186", "key_4":"g","key_7":"h","key_1":"k", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.670072, "value1":0.3811265659697653, "value2":4090252794464650298, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486186, "value1":0.9084897972797951, "value2":2292875585752622652, "value3":false, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724703, "value1":0.04582892017411178, "value2":5556549536478579022, "value3":true, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012322, "value1":0.6305358760062275, "value2":1288975268472701530, "value3":false, "__name__":"metric_00000192", "key_6":"a","key_8":"f","key_0":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.218849, "value1":0.8926861107470235, "value2":7051558642534889749, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.577242, "value1":0.96150058991849, "value2":5205402300881087071, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525938, "value1":0.010643579624627687, "value2":9179800346032814438, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393624, "value1":0.5282994953423363, "value2":6629726373755372175, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697100, "value1":0.328819156896584, "value2":7890032700245067940, "value3":false, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.911016, "value1":0.23451404605523937, "value2":1559532623894065416, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128593, "value1":0.958219222491235, "value2":5238496228333042095, "value3":false, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704394, "value1":0.5513948920447201, "value2":9062299692444161349, "value3":true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255075, "value1":0.35898456496590414, "value2":5087989108776558277, "value3":true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788201, "value1":0.7466905262312481, "value2":479894845243154201, "value3":true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565236, "value1":0.15707319443827472, "value2":93010754190809493, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462851, "value1":0.1393959893683005, "value2":1383438862980713878, "value3":false, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.587928, "value1":0.9257613742750218, "value2":461452136435423636, "value3":false, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.204910, "value1":0.45361150342145223, "value2":8114896084179399798, "value3":true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970393, "value1":0.8262379198959926, "value2":1410864297783479414, "value3":false, "__name__":"metric_00000207", "key_1":"e","key_3":"g","key_0":"k", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571697, "value1":0.05523474900350812, "value2":6912470816433257487, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900965, "value1":0.3994455375010903, "value2":5910800813383852824, "value3":false, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533983, "value1":0.8439578978089138, "value2":6520871065252687028, "value3":false, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732970, "value1":0.4096847894313233, "value2":4961560961953820016, "value3":false, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.512990, "value1":0.9925070463343235, "value2":4031040144184448394, "value3":false, "__name__":"metric_00000212", "key_9":"j","key_3":"b","key_5":"a", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.875107, "value1":0.8529856732344743, "value2":8171410486624791720, "value3":true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.662542, "value1":0.7969197657043129, "value2":4707105172837538302, "value3":false, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.132562, "value1":0.5095481874741737, "value2":8469586257322312509, "value3":false, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171675, "value1":0.7140266335782444, "value2":8998618655550288053, "value3":false, "__name__":"metric_00000217", "key_9":"h","key_1":"h","key_4":"a", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829529, "value1":0.8556707112048024, "value2":84360751498479327, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.383509, "value1":0.6267421890569228, "value2":3548620263321255935, "value3":true, "__name__":"metric_00000219", "key_5":"k","key_7":"h","key_1":"a", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725412, "value1":0.6792289410958199, "value2":8743030074795737682, "value3":true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217960, "value1":0.06523873388277363, "value2":4237433473740985929, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.052828, "value1":0.39596522726411565, "value2":5268673916985408378, "value3":false, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243583, "value1":0.24855400987123744, "value2":7145331513033105803, "value3":true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.643757, "value1":0.4621121441928861, "value2":4607779943623171423, "value3":false, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777085, "value1":0.9713754435345697, "value2":3872286492266737542, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741256, "value1":0.33772530129863026, "value2":2720260557973633476, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040901, "value1":0.5366678442503715, "value2":5043767454211598671, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.566517, "value1":0.3199753043754913, "value2":1787736737657465265, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871703, "value1":0.40150570806322944, "value2":7426113550614212915, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.063390, "value1":0.8093881999536079, "value2":6062591517984503571, "value3":false, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630695, "value1":0.5987756911109672, "value2":4564890959914284940, "value3":false, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089316, "value1":0.9756718556017356, "value2":4422554680156925050, "value3":true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433679, "value1":0.6504367128725771, "value2":7179863028410040853, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418340, "value1":0.5167928405286337, "value2":796797443554380087, "value3":false, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148063, "value1":0.2982285613686749, "value2":3455828331199929793, "value3":false, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.562273, "value1":0.1744043124258891, "value2":7056453619952557866, "value3":false, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554680, "value1":0.9985428971035419, "value2":3134607000357482375, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.498692, "value1":0.34351042666983833, "value2":9219919937586266788, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983651, "value1":0.20667520837153053, "value2":8261113453188452260, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900233, "value1":0.49430245377625404, "value2":1887699130768202184, "value3":false, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.269041, "value1":0.24848195612656065, "value2":27297122896884729, "value3":true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.716521, "value1":0.703324321123364, "value2":5138993531610984864, "value3":true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.331287, "value1":0.6585119514499647, "value2":7805239120031598554, "value3":true, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.898740, "value1":0.7897434254535518, "value2":8577413153173117857, "value3":false, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366667, "value1":0.4615000106211479, "value2":4478388423093675456, "value3":false, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060808, "value1":0.3611416577570406, "value2":1763611736817673526, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987608, "value1":0.6511729153448047, "value2":7895118250604250511, "value3":true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.920802, "value1":0.8966340709602193, "value2":1619385988446426949, "value3":false, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759468, "value1":0.9772199770736059, "value2":486666067155730811, "value3":false, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697569, "value1":0.2215623962484913, "value2":998498599369301173, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550984, "value1":0.23501917809435333, "value2":203018852406902455, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865253, "value1":0.24390812385740993, "value2":2606627451901201346, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050134, "value1":0.6480270130439445, "value2":6247279314207653751, "value3":false, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614221, "value1":0.7050465104295126, "value2":3377246595104905199, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166533, "value1":0.3877660847452995, "value2":7536153185490054501, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644807, "value1":0.7190092729952982, "value2":5813482232408839023, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739661, "value1":0.7257627180887302, "value2":3214351184373245408, "value3":false, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552516, "value1":0.4939513402753451, "value2":465350226895561432, "value3":false, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503358, "value1":0.14929982411048387, "value2":2610344885697195594, "value3":false, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.765487, "value1":0.45537690586006585, "value2":4288868163982148480, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.838240, "value1":0.7605183341936481, "value2":7774940282079152902, "value3":true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646978, "value1":0.988755613370451, "value2":4713140451733570872, "value3":true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746410, "value1":0.8174040579458727, "value2":7395122770602669759, "value3":true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823505, "value1":0.758453176804426, "value2":7518087779323040688, "value3":false, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020233, "value1":0.29899993635819666, "value2":5085917800846866623, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561926, "value1":0.8862149028157266, "value2":3614075635724672575, "value3":true, "__name__":"metric_00000275", "key_2":"i","key_6":"f","key_0":"j", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.437110, "value1":0.32845862992503655, "value2":2977427326936422575, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706546, "value1":0.15047181179887134, "value2":2307845458700050829, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.619350, "value1":0.1117344044076149, "value2":4988357694886705006, "value3":true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.808461, "value1":0.715214325361221, "value2":7172177171113874947, "value3":false, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154987, "value1":0.3436009437038216, "value2":35294647035701645, "value3":true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.293185, "value1":0.1982820741163276, "value2":4721034174581841393, "value3":true, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.177501, "value1":0.24102187665043326, "value2":5183952097737049563, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434009, "value1":0.4074440055222358, "value2":2740366314909969021, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.962158, "value1":0.6945642879060417, "value2":8488893912024069218, "value3":false, "__name__":"metric_00000284", "key_3":"b","key_7":"e","key_0":"b", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037976, "value1":0.05977827983958813, "value2":8714018046983615411, "value3":true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491879, "value1":0.5875339799333666, "value2":3507531774420107282, "value3":true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708852, "value1":0.941564068848663, "value2":7441568589430319196, "value3":true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665427, "value1":0.15650940720497003, "value2":2725018303601038315, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728790, "value1":0.16909603056430217, "value2":7913664106090409634, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748451, "value1":0.9309585511638759, "value2":6423846277129286464, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701551, "value1":0.2886585284950581, "value2":1577558950874330311, "value3":false, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356768, "value1":0.42594704925771354, "value2":3150589955894640969, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056847, "value1":0.8050681337140542, "value2":3504760629437797890, "value3":false, "__name__":"metric_00000293", "key_9":"k","key_4":"b","key_7":"e", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.797591, "value1":0.9381497767913094, "value2":8069233160965701213, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.653188, "value1":0.21115616320062328, "value2":2091006864800018643, "value3":false, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489289, "value1":0.6249865113250967, "value2":3636072468848176197, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.426149, "value1":0.7525465264386705, "value2":2554715790387615481, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629922, "value1":0.244780100518832, "value2":2525567513505106874, "value3":true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.652167, "value1":0.44363110594266053, "value2":2074305704102402281, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111575, "value1":0.7142448949298847, "value2":2599619943612027614, "value3":true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.720934, "value1":0.46314177190410183, "value2":5473312180675819943, "value3":false, "__name__":"metric_00000240", "key_8":"d","key_9":"c","key_4":"d", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.542023, "value1":0.17074635857117051, "value2":576994586783200158, "value3":false, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862606, "value1":0.849600128173387, "value2":4044011490168844428, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.581818, "value1":0.4717710471390932, "value2":2021524622009984217, "value3":false, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.634138, "value1":0.3045108371861026, "value2":769588972141174181, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554282, "value1":0.9061849342045651, "value2":5246859609213477778, "value3":false, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.275353, "value1":0.12209379665466558, "value2":3903264969659125855, "value3":false, "__name__":"metric_00000306", "key_8":"f","key_9":"i","key_5":"d", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931382, "value1":0.38822648110972485, "value2":4321311553224557998, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.703867, "value1":0.6637597201672076, "value2":5683430536850722079, "value3":false, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958497, "value1":0.011335239730124937, "value2":2491874770397540377, "value3":true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569325, "value1":0.8472934144463635, "value2":6850134665353105473, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564173, "value1":0.28056996036761495, "value2":1661710999114510072, "value3":false, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.046264, "value1":0.7988719698656381, "value2":80936298017494610, "value3":false, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.086050, "value1":0.7269357269059649, "value2":3703081511929708827, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.691310, "value1":0.968367674569065, "value2":8685309070712113009, "value3":true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.734580, "value1":0.16065685872806734, "value2":8341255006788851479, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110471, "value1":0.2857648075220212, "value2":7895547866112316696, "value3":false, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.594791, "value1":0.6011580461407761, "value2":8667655502460513802, "value3":false, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.074613, "value1":0.7518175733891237, "value2":6808912328433269494, "value3":false, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827510, "value1":0.4635989764886008, "value2":8535571864109924880, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020243, "value1":0.9625575743602495, "value2":3255772979573003294, "value3":true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508001, "value1":0.9246513785407203, "value2":6789019308045406067, "value3":false, "__name__":"metric_00000315", "key_3":"e","key_2":"b", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.175770, "value1":0.6231844141584031, "value2":5604032128383909386, "value3":false, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338187, "value1":0.7085089535016765, "value2":7296894563489025180, "value3":true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924177, "value1":0.9072759724228442, "value2":2056681116966727911, "value3":false, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474372, "value1":0.09374734312421011, "value2":8184327150731414451, "value3":true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970300, "value1":0.9441105421259884, "value2":6689514820107172641, "value3":false, "__name__":"metric_00000317", "key_4":"f","key_9":"f","key_0":"e", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035348, "value1":0.14347791413195674, "value2":2423771496925044659, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.807051, "value1":0.1533684521364213, "value2":8351770123785179919, "value3":true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814282, "value1":0.48165153274600025, "value2":2348575893675627866, "value3":false, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438370, "value1":0.4065426353146516, "value2":5566206066472585427, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.321909, "value1":0.057863583124824335, "value2":2888660799506273189, "value3":false, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.191383, "value1":0.5857228209038977, "value2":343682983842502232, "value3":false, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.030645, "value1":0.054007147421613756, "value2":7408821068820948342, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.022320, "value1":0.5082171149433462, "value2":708896583672613900, "value3":true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359147, "value1":0.10928795884420436, "value2":5509040341747881439, "value3":true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245107, "value1":0.08990020039433881, "value2":4415685661085482000, "value3":false, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.685276, "value1":0.9327230470765273, "value2":7932626821430255186, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160518, "value1":0.9697823127450904, "value2":9176665002489756811, "value3":true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851145, "value1":0.3835892105704814, "value2":5122340871731266104, "value3":false, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390684, "value1":0.8638095835380812, "value2":2196309567306303445, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197750, "value1":0.5321904528249127, "value2":7059427737086849843, "value3":true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.606374, "value1":0.1712913717696049, "value2":6671507460576496016, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814189, "value1":0.21162810005419255, "value2":6089699189069603789, "value3":true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565029, "value1":0.651039299686122, "value2":2488735588911638142, "value3":false, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310043, "value1":0.3341800696393593, "value2":1886591498798226069, "value3":true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646773, "value1":0.587523277238801, "value2":3984875805092838666, "value3":true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487672, "value1":0.4085632878496485, "value2":4006763171436697564, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180539, "value1":0.614724172390471, "value2":3474361726777387017, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413423, "value1":0.8015313942257714, "value2":3699130733479555941, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375412, "value1":0.6194400337512462, "value2":89560215401334488, "value3":true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739736, "value1":0.01565923183582112, "value2":2040823465506133480, "value3":true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164236, "value1":0.05609752284300993, "value2":459144572970603788, "value3":true, "__name__":"metric_00000343", "key_5":"a","key_8":"g","key_3":"b", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857605, "value1":0.3918518578057805, "value2":2660076743726482223, "value3":false, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182324, "value1":0.19737269797519194, "value2":4090312209234045823, "value3":true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413638, "value1":0.23128603835680528, "value2":2432247666638343859, "value3":true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029664, "value1":0.7948810691175306, "value2":1090049786919167701, "value3":true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953471, "value1":0.7020043042253941, "value2":4064796540960008955, "value3":false, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925306, "value1":0.8273248639036314, "value2":7383533353614933778, "value3":true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.353560, "value1":0.7154582581173291, "value2":7672433526090470392, "value3":false, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.440133, "value1":0.8443212771497766, "value2":4871633339564000823, "value3":true, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079309, "value1":0.9841787259191465, "value2":5158435517537975833, "value3":false, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687146, "value1":0.547850782348478, "value2":107890618456621161, "value3":true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013887, "value1":0.9829126596506367, "value2":2398801344797864345, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142865, "value1":0.17685569327146947, "value2":2003576222991068603, "value3":true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503089, "value1":0.2089707954081848, "value2":5329263674364077685, "value3":false, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055628, "value1":0.9427569700789281, "value2":8376349363379609336, "value3":false, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663522, "value1":0.5447970553906242, "value2":7034169322760923664, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569193, "value1":0.4748210869235274, "value2":5009934188959179459, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.964851, "value1":0.1480433642451733, "value2":5780759776802899813, "value3":false, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.674266, "value1":0.01926887357642196, "value2":5539462413096535, "value3":false, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397018, "value1":0.3343175727423989, "value2":8363367362679630761, "value3":true, "__name__":"metric_00000364", "key_8":"c","key_2":"e","key_7":"k", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665761, "value1":0.9909509158985629, "value2":598831903509406706, "value3":true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225685, "value1":0.9692539977538356, "value2":6753560478082272468, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857888, "value1":0.3089073345023022, "value2":469238271712070571, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296144, "value1":0.8953425439665634, "value2":5808544938153527415, "value3":false, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641028, "value1":0.26770927578250153, "value2":4804718717106479393, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865055, "value1":0.21277992187222125, "value2":7652322888419294762, "value3":true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.147216, "value1":0.4361812114988848, "value2":5453489060842625813, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931652, "value1":0.9338772117034952, "value2":7926554101044226499, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451647, "value1":0.49187863740688065, "value2":626417169671954650, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552836, "value1":0.08117641553234622, "value2":6479037192065714806, "value3":true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644667, "value1":0.8782871355247011, "value2":7618066153253340565, "value3":true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486533, "value1":0.34858818937194813, "value2":8450118240549444684, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080371, "value1":0.1720187423008763, "value2":7683320479544120606, "value3":true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527916, "value1":0.09000309934383972, "value2":5784629514237788639, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682327, "value1":0.9194439212395367, "value2":4269189023261883692, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.961106, "value1":0.4405062734385544, "value2":1799260749356167755, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640959, "value1":0.04934157301568955, "value2":4101326969656042416, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827544, "value1":0.6093692966542871, "value2":5408051744545184279, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.916148, "value1":0.37486434605250213, "value2":8005097965237384093, "value3":true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477104, "value1":0.7823899457952176, "value2":2425125801220437344, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631094, "value1":0.38182215652170554, "value2":6885679048992226816, "value3":true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242126, "value1":0.897478447775271, "value2":5819274494025981203, "value3":false, "__name__":"metric_00000386", "key_4":"g","key_8":"b","key_2":"c", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134175, "value1":0.8364024700095963, "value2":2669419474556636937, "value3":true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.785456, "value1":0.47900969825766393, "value2":427990087950056586, "value3":false, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.702759, "value1":0.4509768057551493, "value2":6899018348994248896, "value3":true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.844804, "value1":0.4506687041971474, "value2":2704465019491437568, "value3":true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841097, "value1":0.08259262088200636, "value2":3254311166592288697, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.579894, "value1":0.634370403808335, "value2":8239103463468673989, "value3":true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.671756, "value1":0.3978266397618443, "value2":2011463990612690211, "value3":true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639712, "value1":0.5843126119887666, "value2":8514657269434627024, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.090535, "value1":0.47166906237349404, "value2":2986627777164864344, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079733, "value1":0.49607189078610653, "value2":7935174201590062011, "value3":false, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484442, "value1":0.10087228739324149, "value2":5758223217302730640, "value3":true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.881297, "value1":0.743802063178793, "value2":7666746241262123009, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189244, "value1":0.8471771982112483, "value2":8717600770763450338, "value3":false, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958729, "value1":0.24046081337833752, "value2":2429080107883352722, "value3":true, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904544, "value1":0.5967867401359257, "value2":5915623307556355066, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.016356, "value1":0.2664341494092446, "value2":6082384923182522956, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906737, "value1":0.8646356549293164, "value2":6093331013624350146, "value3":true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.650026, "value1":0.17820070872685148, "value2":2506978860590163255, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.380897, "value1":0.559051486368018, "value2":959051280528283072, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.645954, "value1":0.44113348983757733, "value2":3640682441395107897, "value3":false, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.705006, "value1":0.7603563234560595, "value2":185556188647790850, "value3":false, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.946294, "value1":0.6649942159160177, "value2":8061815176820463265, "value3":true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.007871, "value1":0.472486274082402, "value2":8110170220979486982, "value3":false, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.817193, "value1":0.01062083501470273, "value2":8652411971395108438, "value3":false, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.203058, "value1":0.2545105965864827, "value2":3692372485982923495, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724358, "value1":0.49614729928160556, "value2":794673815448737267, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029363, "value1":0.8128873282396352, "value2":5575910506626889012, "value3":false, "__name__":"metric_00000412", "key_3":"b","key_6":"b","key_1":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.388594, "value1":0.37922666609989497, "value2":5857319265944344227, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111761, "value1":0.07280095456744823, "value2":7190870932028618354, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556559, "value1":0.011144718316818122, "value2":5637958779343170502, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900852, "value1":0.2436048429102491, "value2":3178328441413468775, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746269, "value1":0.44159443384096786, "value2":1064726188435771697, "value3":false, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089137, "value1":0.5802300524767475, "value2":1179475447484216893, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045149, "value1":0.8412834417368987, "value2":2843566571645769718, "value3":true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032148, "value1":0.9106893921082526, "value2":2375027858901255873, "value3":false, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180832, "value1":0.8856494080629095, "value2":2259505020949178568, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571194, "value1":0.2087472777529694, "value2":3787711915274839208, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113569, "value1":0.6253289147788459, "value2":2290822587909135561, "value3":false, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708878, "value1":0.3051231954056551, "value2":536614295748659218, "value3":false, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397834, "value1":0.004032765819560939, "value2":2143090210108317054, "value3":true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.279982, "value1":0.6698352738967139, "value2":8265876412027179825, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.117595, "value1":0.36898116171165335, "value2":7917937119372643007, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490128, "value1":0.7119026496334794, "value2":4217226709077888871, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.337373, "value1":0.24700927278867024, "value2":8633933010561287594, "value3":false, "__name__":"metric_00000430", "key_4":"c","key_9":"j","key_1":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393403, "value1":0.3187858750570018, "value2":7155530342572901954, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787897, "value1":0.731243787879042, "value2":3162978630167698275, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394636, "value1":0.15744716640466827, "value2":6804804340000012886, "value3":false, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589125, "value1":0.9206991440936233, "value2":3240974047558242833, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122199, "value1":0.9027140609185327, "value2":4793574504420000662, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739368, "value1":0.0296490298815606, "value2":7450276243203379932, "value3":true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839046, "value1":0.885358696687765, "value2":2580404634255699486, "value3":true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432366, "value1":0.6379661863831838, "value2":4996786498907777190, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129751, "value1":0.5622916100360914, "value2":8341777138936467511, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202469, "value1":0.24574613351572672, "value2":7146206121672605367, "value3":true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.832142, "value1":0.5178470928944944, "value2":4133801221430187756, "value3":true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080311, "value1":0.486175008187209, "value2":5039371713095786937, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723875, "value1":0.552001350718302, "value2":7101322585254858336, "value3":true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434143, "value1":0.6525700542145968, "value2":2081065923746015157, "value3":true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.410329, "value1":0.15418313782350743, "value2":2121964379701341165, "value3":false, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423463, "value1":0.984744888870281, "value2":7148485292923472325, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658931, "value1":0.9086990564903296, "value2":4792038160363904869, "value3":true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.891060, "value1":0.6990632896148243, "value2":8537384774472959390, "value3":true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154122, "value1":0.10983855840529697, "value2":3710113544658117053, "value3":true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.543143, "value1":0.3045282199429521, "value2":6184622652371484263, "value3":true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174103, "value1":0.1622180223685842, "value2":1046120514820502820, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393987, "value1":0.12110279784132964, "value2":5830532384578071588, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907289, "value1":0.9509521630571898, "value2":4933206466498891083, "value3":true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987070, "value1":0.5242278192300958, "value2":7374919878503376521, "value3":true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641701, "value1":0.3760479049716257, "value2":4296425671360614419, "value3":false, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.460082, "value1":0.7715932085121491, "value2":4216486260610422297, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.308698, "value1":0.6733032497636853, "value2":8314310095556752952, "value3":false, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356337, "value1":0.9180371571261539, "value2":7442497884658250592, "value3":true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407046, "value1":0.8826200541831951, "value2":6272206334208867752, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263547, "value1":0.5493018603625203, "value2":5332141292811821371, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.690903, "value1":0.6510082736025081, "value2":2282336418703477702, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599129, "value1":0.21529129751322154, "value2":5758745763053757888, "value3":true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.294727, "value1":0.17537508292575937, "value2":1562987372702839556, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905165, "value1":0.5372817599107224, "value2":5788710138851546437, "value3":true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714618, "value1":0.5147887871360798, "value2":2734992135513585397, "value3":false, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.845879, "value1":0.03659856815217442, "value2":717088561981717268, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.172178, "value1":0.5465356547159791, "value2":6019732615424773946, "value3":true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263201, "value1":0.7472793901983326, "value2":6735352007287694901, "value3":false, "__name__":"metric_00000467", "key_4":"i","key_8":"a","key_2":"d", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418982, "value1":0.029853021112705932, "value2":7962917578850181628, "value3":true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.866946, "value1":0.10874512532228968, "value2":4523576734362004451, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556178, "value1":0.3725090999387309, "value2":8591394434164967395, "value3":false, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827178, "value1":0.564605338181153, "value2":1565791376780985293, "value3":true, "__name__":"metric_00000472", "key_9":"d","key_5":"b","key_8":"j", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.427240, "value1":0.5937818761029834, "value2":725738161862814755, "value3":true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153851, "value1":0.6812022447305227, "value2":2627803719401441193, "value3":false, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982516, "value1":0.5197201488713766, "value2":5245967664424169262, "value3":true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.922654, "value1":0.8797142069534722, "value2":2538441830233727435, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036732, "value1":0.8882312994752156, "value2":2860925793915692434, "value3":false, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.848915, "value1":0.5650391916990911, "value2":8021099293155564160, "value3":false, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189564, "value1":0.5152547802767392, "value2":1569067174314315770, "value3":false, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.802112, "value1":0.9536154419324024, "value2":6565733445920939612, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983922, "value1":0.4632188030903474, "value2":3429972812413807249, "value3":true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477177, "value1":0.333324845660299, "value2":4203450320031165809, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097938, "value1":0.9171714255895188, "value2":2925569760522108797, "value3":true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.551984, "value1":0.23531625096597034, "value2":8336676757418711666, "value3":true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.774461, "value1":0.26483886652245114, "value2":1317358716847210463, "value3":true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354793, "value1":0.850445247905183, "value2":3123908181356206943, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.409594, "value1":0.7075083567078372, "value2":7315484295756660732, "value3":false, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407267, "value1":0.6525751933510557, "value2":5480727731388217494, "value3":false, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.572642, "value1":0.5899578957097297, "value2":5245828360681206097, "value3":false, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.731119, "value1":0.6986058457616005, "value2":4059549341372221062, "value3":true, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.627725, "value1":0.5600488168022775, "value2":2564410170230629806, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050520, "value1":0.8920464631997315, "value2":9043850999622300098, "value3":false, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.500683, "value1":0.004883654125166552, "value2":8338418914599161440, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970722, "value1":0.01152412783847713, "value2":4977793912981679064, "value3":true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.860222, "value1":0.796875300130385, "value2":126152372728632215, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629467, "value1":0.658754492873807, "value2":2189248151613602825, "value3":false, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.098126, "value1":0.3838755732008427, "value2":8710340059607681652, "value3":true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812187, "value1":0.7527055571509035, "value2":4522503161830090202, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303237, "value1":0.4170647409134182, "value2":1359570910802075414, "value3":true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794250, "value1":0.581379594095589, "value2":6842995986442641402, "value3":true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247703, "value1":0.7617287060418184, "value2":8955434883327208405, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.815768, "value1":0.5618395871174356, "value2":6646583475463132751, "value3":true, "__name__":"metric_00000501", "key_2":"i","key_5":"b","key_0":"d", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.248823, "value1":0.1954800995898102, "value2":1363474452253194433, "value3":false, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.531845, "value1":0.018792519031917622, "value2":2287281788624224406, "value3":false, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966028, "value1":0.08281088225013297, "value2":1000202598795297845, "value3":false, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.935610, "value1":0.3880941622449169, "value2":3231927844022227730, "value3":true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.268000, "value1":0.37669283923759284, "value2":1820767283408305591, "value3":true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.229141, "value1":0.6757864654636687, "value2":850049194951914043, "value3":true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.889660, "value1":0.14317979698283573, "value2":4562037574783866851, "value3":false, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836795, "value1":0.7422552273700601, "value2":1164639099163690154, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.311980, "value1":0.6163196714525997, "value2":5427592556050607568, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584578, "value1":0.38212773421861185, "value2":1414397003551807223, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065172, "value1":0.5125177091208547, "value2":2833074963007555228, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564271, "value1":0.5135791893267198, "value2":3074560175179976918, "value3":false, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894567, "value1":0.8429539964906779, "value2":3097821505618288757, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.733587, "value1":0.9459948105676399, "value2":3812465080486684733, "value3":true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.053881, "value1":0.6018467371683773, "value2":5589588667277003729, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599521, "value1":0.5951991237753406, "value2":4485462521384926953, "value3":true, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354099, "value1":0.15573795731250434, "value2":8080731261812523696, "value3":true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969724, "value1":0.21276604577267808, "value2":8483165896229109105, "value3":true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263088, "value1":0.8705529546951637, "value2":1868660470516930885, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.212304, "value1":0.28383507730046376, "value2":3678145555670791199, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965823, "value1":0.23336598309503237, "value2":1387918772122086546, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536800, "value1":0.6576906678976834, "value2":6988018932226580717, "value3":false, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.884180, "value1":0.9109732193854059, "value2":2950864149550098491, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448663, "value1":0.37035460819883087, "value2":6933572641209900634, "value3":false, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965580, "value1":0.9403308475710539, "value2":5776171572361995550, "value3":false, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.349421, "value1":0.19847751056585392, "value2":3418305989652553355, "value3":true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.469447, "value1":0.13572479625794354, "value2":4687020001075041801, "value3":true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393705, "value1":0.5764820211787193, "value2":296609812894901072, "value3":false, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862549, "value1":0.3023964248018198, "value2":7566431551834336763, "value3":false, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415447, "value1":0.07086082925570908, "value2":6961625726898421777, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495354, "value1":0.5778296524778606, "value2":2860533724845138756, "value3":true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.510139, "value1":0.9905369504866598, "value2":7400245650080116394, "value3":false, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129326, "value1":0.3322893702950888, "value2":273820673011485924, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.793922, "value1":0.5795215776373468, "value2":7565118610448753571, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114305, "value1":0.4965968551084071, "value2":3076343444745895229, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539515, "value1":0.7424425979048557, "value2":3570552565123324839, "value3":false, "__name__":"metric_00000546", "key_8":"b","key_9":"b","key_0":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714983, "value1":0.4711244304607692, "value2":5134628196697329789, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182520, "value1":0.11684538125954054, "value2":4642866567682796714, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701117, "value1":0.46816724142010296, "value2":8359014347252866849, "value3":true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135967, "value1":0.6405455196141, "value2":6186468831622665956, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.424454, "value1":0.5144728145894553, "value2":1160618526301999790, "value3":false, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476344, "value1":0.8109846569048221, "value2":1049565304185759462, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571700, "value1":0.5504531727903846, "value2":133387170146495310, "value3":false, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324604, "value1":0.16411596790106633, "value2":4350527762007764055, "value3":false, "__name__":"metric_00000554", "key_9":"g","key_1":"e","key_5":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272811, "value1":0.06793234743561845, "value2":3491914082935836765, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496778, "value1":0.2139795243463974, "value2":9108500124467919738, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855922, "value1":0.8034347088304469, "value2":1547578413909373466, "value3":true, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.516953, "value1":0.46095603855138756, "value2":5334901216429463877, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.941610, "value1":0.9546476007247299, "value2":1561805534534904078, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157907, "value1":0.9790803369585773, "value2":7490355100154440772, "value3":false, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.667071, "value1":0.23582538943103912, "value2":6462516448359960159, "value3":true, "__name__":"metric_00000524", "key_3":"d","key_9":"f","key_2":"b", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.447846, "value1":0.08703603490291748, "value2":6137779903742056640, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.201754, "value1":0.703586761350304, "value2":5895955356127986300, "value3":false, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.214135, "value1":0.28977641931297293, "value2":2670270804816119209, "value3":true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171493, "value1":0.09189861336902101, "value2":3835094797774579940, "value3":false, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.507098, "value1":0.8469316808229229, "value2":4394084036001197157, "value3":true, "__name__":"metric_00000559", "key_2":"j","key_4":"d","key_1":"f", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.567706, "value1":0.4315878020065951, "value2":3026063068553888025, "value3":false, "__name__":"metric_00000530", "key_9":"f","key_3":"i","key_5":"c", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.369862, "value1":0.25873173058422444, "value2":2694491932093728275, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393535, "value1":0.2941316364368656, "value2":3384752395363156914, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354927, "value1":0.13513861393265603, "value2":8481431502383347556, "value3":true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.468119, "value1":0.5679212228606271, "value2":2713405380262941377, "value3":false, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163006, "value1":0.4227063637930681, "value2":6321042013913967311, "value3":true, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.656131, "value1":0.5736248371417144, "value2":2472629476600653129, "value3":true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511635, "value1":0.7728428788355186, "value2":7276523252477639153, "value3":false, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080037, "value1":0.9949587251210614, "value2":1148201472548354758, "value3":true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725064, "value1":0.05030431554866517, "value2":4579601693593819193, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.342713, "value1":0.6797941637549237, "value2":2282393859292368923, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012227, "value1":0.25217885012179175, "value2":2820047039844714200, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.121409, "value1":0.08353369182495488, "value2":6648793005050735517, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.944843, "value1":0.7935143781883287, "value2":1141746305838051122, "value3":false, "__name__":"metric_00000571", "key_7":"b","key_9":"b","key_1":"k", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216784, "value1":0.687358900553268, "value2":5413083464946109711, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.744704, "value1":0.26762670212162987, "value2":4557617436876999047, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.105367, "value1":0.7098778066529884, "value2":2505389065681570054, "value3":false, "__name__":"metric_00000574", "key_3":"k","key_2":"f", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164496, "value1":0.375746421410976, "value2":874151975168015904, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648692, "value1":0.26898807878557085, "value2":8174372151408578377, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160502, "value1":0.8995428170685682, "value2":3027568952484727683, "value3":true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149122, "value1":0.0907853096527507, "value2":4677811628653537246, "value3":true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296780, "value1":0.3995862207774202, "value2":3598801053628990992, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.464060, "value1":0.9713793366221031, "value2":4865123999270835737, "value3":false, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288334, "value1":0.6512177032146146, "value2":1792245924965210603, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.869907, "value1":0.007513793036623767, "value2":7995353402339020473, "value3":false, "__name__":"metric_00000582", "key_8":"i","key_1":"i","key_5":"e", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527381, "value1":0.5920004855495694, "value2":1386169961861528891, "value3":false, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.928125, "value1":0.7418903709677874, "value2":5060751330275548305, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131800, "value1":0.9351703924333601, "value2":827562310794270736, "value3":false, "__name__":"metric_00000585", "key_8":"g","key_0":"b","key_2":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367161, "value1":0.9314616031794654, "value2":8545717112597189310, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704807, "value1":0.7062626357522712, "value2":1882734585108600742, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270485, "value1":0.942646876486616, "value2":4665280302852454779, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.521647, "value1":0.7127769947534454, "value2":7002036018179718342, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.011614, "value1":0.2150115012001749, "value2":2368106908969391947, "value3":true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160161, "value1":0.5544338078106065, "value2":6068552071959300958, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545350, "value1":0.5865458699587862, "value2":1122097202466314910, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.948757, "value1":0.9353310253197936, "value2":9083948862293457994, "value3":false, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.003970, "value1":0.13057333820161732, "value2":4887307186667923394, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811016, "value1":0.0597992472783255, "value2":7198325554706277943, "value3":false, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687715, "value1":0.029450254550187717, "value2":7347375517848816816, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892901, "value1":0.5081883055870532, "value2":3289845879859033841, "value3":false, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006320, "value1":0.35956731353734483, "value2":6174658822769262086, "value3":true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.088057, "value1":0.5893474112602632, "value2":2264301146979603153, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366406, "value1":0.09531376481437555, "value2":8460504532640850958, "value3":true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.742879, "value1":0.21104814940031102, "value2":8239563761654323619, "value3":false, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014364, "value1":0.27433866046434513, "value2":1123249940897490575, "value3":true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442350, "value1":0.5779165272183289, "value2":7076967122005445282, "value3":false, "__name__":"metric_00000603", "key_6":"e","key_2":"c","key_3":"k", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.363242, "value1":0.7223710776768617, "value2":4947532477450255339, "value3":false, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794231, "value1":0.9595692700611697, "value2":7086029131344930712, "value3":false, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.696096, "value1":0.2902282025796308, "value2":2826023125040762860, "value3":false, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366795, "value1":0.8338186933189231, "value2":7469909471986494672, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.642061, "value1":0.025310231945638022, "value2":5571346287819468860, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673449, "value1":0.9385869454324781, "value2":1722013371242991173, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375120, "value1":0.09609963357456859, "value2":5591579449982830960, "value3":false, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391308, "value1":0.5308398826498623, "value2":2744980541531859716, "value3":true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658264, "value1":0.08305585159828374, "value2":3868051792086152227, "value3":false, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.540598, "value1":0.9729141879466757, "value2":2886891615332464723, "value3":true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.077420, "value1":0.756799108978319, "value2":3709648144634270824, "value3":true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.762914, "value1":0.7697479099606849, "value2":3475167187400734782, "value3":true, "__name__":"metric_00000614", "key_5":"f","key_6":"g","key_0":"c", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907718, "value1":0.44276263620911915, "value2":8007229889762742191, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.677735, "value1":0.7114610811603074, "value2":420499484804274745, "value3":false, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097065, "value1":0.9614980550005483, "value2":401687894704730769, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107273, "value1":0.7588043489806351, "value2":5258772379410637950, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907641, "value1":0.16046492238486323, "value2":5755077471601284208, "value3":true, "__name__":"metric_00000620", "key_7":"f","key_9":"b","key_2":"j", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407605, "value1":0.5733175508809538, "value2":2858689191661453591, "value3":false, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412001, "value1":0.6163004444656547, "value2":3938161718625625501, "value3":false, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174438, "value1":0.7741718765670191, "value2":8151281186076102637, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433895, "value1":0.6190720762613371, "value2":4753613664907841729, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.754894, "value1":0.6861936508507025, "value2":4982660479879807579, "value3":true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135440, "value1":0.8484346139801648, "value2":4916410761830427120, "value3":false, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901526, "value1":0.7029579471688302, "value2":7091187814115860335, "value3":true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969298, "value1":0.9971487141151381, "value2":4678178370126159004, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.302462, "value1":0.5451450337065363, "value2":4647862155949132966, "value3":false, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174504, "value1":0.8364678736703143, "value2":4662336943284880044, "value3":false, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724429, "value1":0.7605319213856574, "value2":5775369825299529065, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196992, "value1":0.485221307178674, "value2":8161727305469743810, "value3":false, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673298, "value1":0.12884478288346873, "value2":3433606731713363675, "value3":true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.008175, "value1":0.6854875474834051, "value2":7177608068823530117, "value3":true, "__name__":"metric_00000634", "key_8":"b","key_1":"j","key_5":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.152906, "value1":0.937989165728203, "value2":2388297772896467042, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354518, "value1":0.5251867146486633, "value2":2890885067302311396, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.872640, "value1":0.7546063425064781, "value2":8461706694506205274, "value3":false, "__name__":"metric_00000636", "key_8":"f","key_3":"h","key_4":"j", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403113, "value1":0.7711761108505557, "value2":8776285961045082566, "value3":false, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.713712, "value1":0.3686652392717231, "value2":3561820735837135599, "value3":false, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.115073, "value1":0.21496854825964173, "value2":1666306538045744009, "value3":true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.127362, "value1":0.0348593634697097, "value2":3277092601539134615, "value3":false, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.042887, "value1":0.9391085748107487, "value2":3037900421018347064, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957170, "value1":0.2742109117364031, "value2":7538881467268645993, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.284083, "value1":0.6552095129564431, "value2":8324596282478983070, "value3":false, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139776, "value1":0.32060963733799414, "value2":4387891090637798442, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.446847, "value1":0.7514268853453937, "value2":1042496829752491158, "value3":false, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.429327, "value1":0.7027895753887083, "value2":4302894157835600757, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496272, "value1":0.2459555820489922, "value2":4743527397302686896, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599745, "value1":0.3085143386003063, "value2":8510884550594616394, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233545, "value1":0.4081732430516885, "value2":4780887317944713264, "value3":true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.396869, "value1":0.3834820243913789, "value2":2867737400343386587, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.000965, "value1":0.7645272489414043, "value2":8292218247137907442, "value3":false, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390310, "value1":0.9922190988066435, "value2":6520801004207332848, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.408746, "value1":0.11967416242419873, "value2":1289404280671317844, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.513154, "value1":0.43853228030860203, "value2":3816263914269322442, "value3":true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.425564, "value1":0.9721541652281699, "value2":1199225910826614784, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.896976, "value1":0.5841175850919846, "value2":6380661791453585128, "value3":false, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.385017, "value1":0.3642021068005677, "value2":7460310647019127617, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.861063, "value1":0.18830600982566692, "value2":6167598179189147828, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.942353, "value1":0.4661402168565476, "value2":8018642185562026949, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.750347, "value1":0.3217686952651615, "value2":830941682724984711, "value3":true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.926420, "value1":0.0451768298368594, "value2":3858422103614938711, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451255, "value1":0.012250475428480134, "value2":468924453611892408, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708013, "value1":0.5360411826543746, "value2":7399971881816481719, "value3":false, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.661279, "value1":0.5535815660872848, "value2":4401269288089200510, "value3":false, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.710860, "value1":0.6074402471823324, "value2":7073880640711856002, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245087, "value1":0.7738521153135302, "value2":5302937747460405991, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724822, "value1":0.06719994230964553, "value2":1984993846520154478, "value3":true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841255, "value1":0.7199212526961583, "value2":3580548125031735289, "value3":false, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130289, "value1":0.14169983508615547, "value2":7980800511434824689, "value3":false, "__name__":"metric_00000670", "key_6":"i","key_8":"d","key_5":"k", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.215185, "value1":0.05456621441388223, "value2":3313875019669857059, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730585, "value1":0.1938148933748891, "value2":2133259010192691032, "value3":true, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603048, "value1":0.27374938601691284, "value2":6401354620294279338, "value3":true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148949, "value1":0.5947828484485513, "value2":8147515290794947255, "value3":false, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.610152, "value1":0.4085609201069522, "value2":9002523177156729302, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.607808, "value1":0.9266705539974761, "value2":4953852228607795114, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197811, "value1":0.7637421466978226, "value2":7727001922124797631, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672455, "value1":0.2140320622344972, "value2":8504587633123200194, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781600, "value1":0.18861407062595717, "value2":2041192855707568703, "value3":true, "__name__":"metric_00000678", "key_4":"i","key_5":"e","key_0":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648320, "value1":0.9980339865005591, "value2":6740770170971470746, "value3":true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481193, "value1":0.5203406695297639, "value2":6425306083284764527, "value3":true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954444, "value1":0.5949050635321682, "value2":2690790583476351571, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971678, "value1":0.3007304152123297, "value2":4562717443975915242, "value3":true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026021, "value1":0.27575882598297025, "value2":978861683515042668, "value3":true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.840073, "value1":0.15154871188435406, "value2":7561212279807822666, "value3":true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393063, "value1":0.5006737130316841, "value2":835200053720218776, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589637, "value1":0.29002140408035565, "value2":5208166554872726624, "value3":true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525315, "value1":0.7084919237025886, "value2":4888712288784480920, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026489, "value1":0.9459638977891762, "value2":6799511668473867834, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356460, "value1":0.8482487483215856, "value2":4902836930964609800, "value3":false, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871117, "value1":0.35037006883754834, "value2":6550768382747712274, "value3":true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.062992, "value1":0.3329675649141359, "value2":8563631384946292163, "value3":false, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.751101, "value1":0.9811341531679872, "value2":621345745999214417, "value3":false, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.535295, "value1":0.922840944798324, "value2":586063777073635225, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.258523, "value1":0.7865906481221376, "value2":1141638202189461929, "value3":true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.10833711533264617, "value2":5739011556983431812, "value3":false, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407251, "value1":0.4266309513517482, "value2":4708724256538796193, "value3":false, "__name__":"metric_00000697", "key_6":"h","key_2":"e","key_4":"d", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931023, "value1":0.6949078276162761, "value2":6914579995867144592, "value3":false, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.882968, "value1":0.27938573922269233, "value2":2134628658651324357, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599828, "value1":0.31593636777024753, "value2":2026576607314660982, "value3":false, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166118, "value1":0.30221394298164866, "value2":7042763501275182896, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114195, "value1":0.16993347985800208, "value2":356239511173356550, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.676940, "value1":0.8142744662164688, "value2":1615963097558644501, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525949, "value1":0.9287689609270591, "value2":8144174091016595622, "value3":true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852923, "value1":0.6682421979821775, "value2":1660916359233146706, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714742, "value1":0.6940562364820247, "value2":8279633113622088778, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.069728, "value1":0.023054188899827854, "value2":7432273276574908949, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397060, "value1":0.6029753734289958, "value2":5859678381429923175, "value3":false, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788868, "value1":0.4080940744104717, "value2":2775616148615876742, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.497778, "value1":0.0933297131976096, "value2":2710674316640074263, "value3":false, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.251386, "value1":0.5835060543515784, "value2":1694819851220806994, "value3":false, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144280, "value1":0.7762012742074322, "value2":6551654579563921110, "value3":true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778177, "value1":0.8529872059895475, "value2":4843965544345174856, "value3":true, "__name__":"metric_00000712", "key_7":"c","key_6":"k", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495383, "value1":0.8821308188431486, "value2":7181474101563747015, "value3":true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421122, "value1":0.04021723515347381, "value2":8758814764030728391, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055252, "value1":0.5894200751471586, "value2":3745371455267104958, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343067, "value1":0.2157486959489704, "value2":3077439566978720910, "value3":false, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489306, "value1":0.4542119549735926, "value2":5846254702440735178, "value3":false, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233627, "value1":0.2946889464915457, "value2":2465181057584653557, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.523076, "value1":0.579630183513931, "value2":4633358909899504280, "value3":false, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.317577, "value1":0.49633251300853154, "value2":7243885459219565500, "value3":false, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142016, "value1":0.41458124596836643, "value2":87263970443989795, "value3":true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014459, "value1":0.3889314073670658, "value2":6642542110052864260, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.249899, "value1":0.5981564529561014, "value2":4725189280029367657, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.087078, "value1":0.48311123248467286, "value2":7915582091365577059, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.681336, "value1":0.3569069050336655, "value2":7513851204641167627, "value3":true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266570, "value1":0.6724875939535222, "value2":4615308903415712327, "value3":false, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474729, "value1":0.3876423090885996, "value2":319900338975917805, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.328471, "value1":0.061053039369872907, "value2":9085903408798527191, "value3":false, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010016, "value1":0.7472091713600674, "value2":8976037153524824381, "value3":false, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412206, "value1":0.7796477061197584, "value2":6842085213034340664, "value3":false, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288276, "value1":0.26992787569132787, "value2":1806846857113271486, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.501201, "value1":0.781398815935773, "value2":4251812051340782659, "value3":true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270221, "value1":0.051986823543989535, "value2":4993776637838984787, "value3":true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.986643, "value1":0.7021309080168643, "value2":703842817955022555, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.453898, "value1":0.4599169268098947, "value2":7963554429950594113, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202118, "value1":0.94451324780601, "value2":7780362140982384262, "value3":true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.801389, "value1":0.7867483731478175, "value2":7192213512007631986, "value3":false, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.228749, "value1":0.4182819773434862, "value2":3027632485243175966, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.184080, "value1":0.9567901938339193, "value2":5313717148514876661, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.707330, "value1":0.826196713820751, "value2":8078395604816162581, "value3":false, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130983, "value1":0.31932740597678144, "value2":8325619009408095997, "value3":false, "__name__":"metric_00000745", "key_6":"j","key_7":"j","key_5":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477577, "value1":0.8037112800341328, "value2":4641052188020943712, "value3":false, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.835784, "value1":0.981119547136869, "value2":5830864768532244942, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.951325, "value1":0.9017984159519533, "value2":7172321278356171509, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949416, "value1":0.9735936936041008, "value2":2397210877296618753, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.227491, "value1":0.056278467666299646, "value2":3619861078542842990, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614325, "value1":0.9338179733958348, "value2":1856588427373456649, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.167780, "value1":0.6264999203061654, "value2":1887390160426943194, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421682, "value1":0.3034767704367279, "value2":1117719742198046331, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131869, "value1":0.7226359930389739, "value2":6009067442340606148, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.576451, "value1":0.4077319058968275, "value2":5262129379842658951, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487400, "value1":0.49878540971656044, "value2":378267425259112843, "value3":true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417401, "value1":0.7695532802557139, "value2":6353721086899765325, "value3":false, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.597343, "value1":0.9334570369057625, "value2":8106460523980946423, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947755, "value1":0.599677921450711, "value2":1518293404194401856, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533432, "value1":0.4134611723516883, "value2":4789921078575497056, "value3":true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271038, "value1":0.7693055378733823, "value2":2557684070989427251, "value3":false, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.209928, "value1":0.5501502486470365, "value2":6523836811607831530, "value3":false, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.514479, "value1":0.742552198141705, "value2":5448311945921729044, "value3":false, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097323, "value1":0.342716151747015, "value2":5333351730397550620, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036400, "value1":0.9364465322438232, "value2":3515155010826544091, "value3":false, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.161709, "value1":0.4801914817306215, "value2":4167056653387877843, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128367, "value1":0.42861309279829746, "value2":6367124429829008192, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.104233, "value1":0.5604661779053219, "value2":8908744031459681115, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814224, "value1":0.8505352643130756, "value2":3611537515395296790, "value3":true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.257817, "value1":0.9534356262275627, "value2":6358735114691060805, "value3":false, "__name__":"metric_00000769", "key_3":"i","key_5":"j","key_2":"e", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590971, "value1":0.8532839122834444, "value2":4683754442672481085, "value3":false, "__name__":"metric_00000767", "key_9":"f","key_7":"b", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110492, "value1":0.563891691575366, "value2":4631985245890497320, "value3":true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953652, "value1":0.20700443841383587, "value2":1925175802270628257, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012273, "value1":0.5960529408710756, "value2":6611884946690949474, "value3":false, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107781, "value1":0.5528403788424425, "value2":2332669698469083563, "value3":true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289974, "value1":0.4420135407607212, "value2":3690892585669795676, "value3":true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.193764, "value1":0.5159643736419933, "value2":6218735166584063092, "value3":false, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.386095, "value1":0.777297131904327, "value2":636830504959980122, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977928, "value1":0.01676315692753733, "value2":4060533084263704194, "value3":true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.066993, "value1":0.001865435996499948, "value2":4739969679441502931, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.910616, "value1":0.057836079222670464, "value2":5276870038639917478, "value3":true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971825, "value1":0.18153964718314727, "value2":2807508624852997267, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.782626, "value1":0.41404427259634824, "value2":6643422113997639050, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892943, "value1":0.5830967595104455, "value2":499806168567753593, "value3":true, "__name__":"metric_00000783", "key_9":"b","key_7":"b","key_8":"e", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423402, "value1":0.9103545774481498, "value2":911868702803131336, "value3":true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006069, "value1":0.7748758341609289, "value2":6539224747903805609, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143311, "value1":0.3741630815676959, "value2":375500557355639144, "value3":false, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267012, "value1":0.06234671061027635, "value2":5595306219658860658, "value3":false, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548719, "value1":0.7992113362076428, "value2":7214856807865350002, "value3":false, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.919970, "value1":0.973745629473067, "value2":4169123266267227326, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.071680, "value1":0.08778281877161515, "value2":7668684104220358875, "value3":false, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.222513, "value1":0.4425030874471083, "value2":5614432156299415760, "value3":false, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.031556, "value1":0.6971180191067721, "value2":6128893201277772027, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839957, "value1":0.03151274581462018, "value2":8360563655204493448, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924623, "value1":0.5790628849747803, "value2":6190166511034220622, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.647381, "value1":0.18478113887851288, "value2":2971279323948578904, "value3":true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.818067, "value1":0.36190113937740354, "value2":1714448127957842853, "value3":false, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.715974, "value1":0.23843892303460118, "value2":2439937704889068215, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192672, "value1":0.6080907905788115, "value2":976012584384444918, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157561, "value1":0.6894676876528992, "value2":7135259052981755980, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134773, "value1":0.09678670230466922, "value2":8180183833096777286, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.616057, "value1":0.8520049469696344, "value2":2950033021270268791, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255317, "value1":0.7477685612049465, "value2":179441442795111864, "value3":false, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906502, "value1":0.8256982980631076, "value2":4367074024600682775, "value3":true, "__name__":"metric_00000801", "key_4":"d","key_6":"e","key_3":"j", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.116793, "value1":0.8891425230837835, "value2":1719946264236570624, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324866, "value1":0.5386590615308351, "value2":8011171897264590756, "value3":true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.352403, "value1":0.7153316706866707, "value2":4379546221964794474, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836874, "value1":0.14443484962198383, "value2":1482104610634689373, "value3":false, "__name__":"metric_00000805", "key_9":"b","key_5":"c","key_8":"a", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403735, "value1":0.8924124150178864, "value2":9156970764441343023, "value3":false, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141142, "value1":0.434266569307909, "value2":4978801873133265222, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.234292, "value1":0.7164352385365066, "value2":8038096300931135389, "value3":false, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973287, "value1":0.6600392290118359, "value2":2638933878817864633, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728882, "value1":0.476991229582368, "value2":6510855349407866452, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787371, "value1":0.7330600795491425, "value2":6570294519251889382, "value3":true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903661, "value1":0.5944117402324142, "value2":6524268081747181442, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.659983, "value1":0.26592399871964695, "value2":1212972365053449050, "value3":false, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723204, "value1":0.31589384939280846, "value2":2041435063818693781, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701487, "value1":0.5974401671491981, "value2":625551200860992484, "value3":true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.893866, "value1":0.38643143057076507, "value2":6815713871810191176, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561678, "value1":0.4057036812724311, "value2":4919580315129580518, "value3":true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004391, "value1":0.4801368737089002, "value2":7547608457928538573, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.923757, "value1":0.7739736243035474, "value2":6832671633674661606, "value3":true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819284, "value1":0.7572793995611461, "value2":68290908865890434, "value3":false, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.524308, "value1":0.44777317605800043, "value2":4272485517844220240, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804762, "value1":0.20222965946470728, "value2":2164767398150388054, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060732, "value1":0.060291087703427844, "value2":8373529279962935628, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325849, "value1":0.2785168531780935, "value2":8895250356037639155, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.378439, "value1":0.43808690997464833, "value2":7750734163790876517, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.341809, "value1":0.21201598129686366, "value2":5439024219774015639, "value3":true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.027383, "value1":0.519384455440142, "value2":2830105304694072087, "value3":true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578370, "value1":0.4103102602537048, "value2":3491530677431875174, "value3":true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.588914, "value1":0.6688147984910581, "value2":7129525479125432993, "value3":false, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781407, "value1":0.7254015801455519, "value2":2135519716242173777, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852755, "value1":0.6592390034228756, "value2":5378675889994915559, "value3":true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.996821, "value1":0.5491613669334785, "value2":3858296464786926451, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.364027, "value1":0.6056949341817615, "value2":2956076236179631886, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.605233, "value1":0.2433469771258824, "value2":8502972839119365612, "value3":false, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.092828, "value1":0.29453399849858286, "value2":5192582279143186491, "value3":false, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.179799, "value1":0.19121582692459024, "value2":9103159579935481171, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.054748, "value1":0.7255577017707996, "value2":8476394128272498701, "value3":true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.959235, "value1":0.5345272240165994, "value2":8621793756396494146, "value3":true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.735109, "value1":0.7362204691534435, "value2":942332422397834734, "value3":false, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778088, "value1":0.8485308761635748, "value2":8355286652172671018, "value3":true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026529, "value1":0.058403258262061046, "value2":3881865567315099972, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.405396, "value1":0.78515016074467, "value2":3124806945169635921, "value3":false, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325780, "value1":0.8484383369933163, "value2":5639577297430380059, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.125526, "value1":0.9437452957692816, "value2":1600583134494091930, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040638, "value1":0.8412837985326597, "value2":1891897645252751874, "value3":true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225539, "value1":0.7607755326390468, "value2":4494740517836964298, "value3":false, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417989, "value1":0.8351179906534547, "value2":6640751019261660918, "value3":true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.930391, "value1":0.7797566384384467, "value2":5344540873017186839, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130784, "value1":0.9081520679550602, "value2":2673655563019876645, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065288, "value1":0.37878147094860193, "value2":40939122026803562, "value3":true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.023954, "value1":0.7421706325870447, "value2":3936838989809781841, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639378, "value1":0.04809332390342072, "value2":138564553177528595, "value3":false, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.968036, "value1":0.4229941922357839, "value2":3252691734036720501, "value3":true, "__name__":"metric_00000853", "key_6":"k","key_9":"j","key_4":"e", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925601, "value1":0.5862382194484029, "value2":8809171247244862983, "value3":false, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.615163, "value1":0.41538444706998723, "value2":7806072407872661917, "value3":false, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432657, "value1":0.2054174608330251, "value2":523721991362366859, "value3":false, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584157, "value1":0.38115353295991117, "value2":5695252506949414616, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969537, "value1":0.31115941644555445, "value2":3813252754578465682, "value3":false, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.755713, "value1":0.6078558755538082, "value2":4745658402471707078, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.299770, "value1":0.39494561763395397, "value2":7454472376009363943, "value3":false, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549446, "value1":0.9276674467108077, "value2":7450051253648573468, "value3":true, "__name__":"metric_00000870", "key_4":"c","key_6":"c","key_3":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.339276, "value1":0.16417640235809516, "value2":376449946500069421, "value3":true, "__name__":"metric_00000871", "key_7":"j","key_4":"j","key_6":"c", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.044679, "value1":0.022401336029968563, "value2":7584563374980649526, "value3":false, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978082, "value1":0.634805138120068, "value2":2509593268797839298, "value3":false, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020832, "value1":0.2523976489068679, "value2":5330781529709694724, "value3":true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.874340, "value1":0.5447943369952332, "value2":753638192956179932, "value3":false, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.532731, "value1":0.3415364500896806, "value2":4617908501249248943, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.888667, "value1":0.03025871907918381, "value2":389467035709739331, "value3":true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630653, "value1":0.3815158272162314, "value2":1365697000438851836, "value3":false, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947344, "value1":0.018503516339994433, "value2":6964047457370572921, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.635335, "value1":0.7735503313212675, "value2":7863214250826862536, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.323808, "value1":0.9277969228346735, "value2":7104799245414939756, "value3":true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527571, "value1":0.19552679967510578, "value2":2150832405165980506, "value3":false, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.076891, "value1":0.45024886897327476, "value2":7461894110864047862, "value3":true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433797, "value1":0.03730319472343055, "value2":8041320080812623904, "value3":true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745634, "value1":0.6889162665973937, "value2":7455824590286680395, "value3":true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310508, "value1":0.4992657909321023, "value2":5571385578668134720, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343053, "value1":0.21251046051646363, "value2":6470523853548900348, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903927, "value1":0.9720310949900105, "value2":7380705280316406577, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.200927, "value1":0.42664784751514406, "value2":5730113255528003277, "value3":false, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.563845, "value1":0.8055339411145567, "value2":7403476714487686904, "value3":true, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.329254, "value1":0.8021973639133464, "value2":3080745955617166362, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.879352, "value1":0.4277546544111605, "value2":1478811657161573091, "value3":true, "__name__":"metric_00000884", "key_3":"a","key_9":"h","key_0":"i", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442846, "value1":0.006346529763383217, "value2":5543697338259725446, "value3":false, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.094781, "value1":0.40728088001797547, "value2":6613387129832293245, "value3":false, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.975150, "value1":0.3661351435962595, "value2":830819320558121686, "value3":false, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245141, "value1":0.02569157141735858, "value2":2603499185616374516, "value3":false, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949515, "value1":0.3643114110362523, "value2":832829152625075749, "value3":true, "__name__":"metric_00000888", "key_6":"k","key_2":"f","key_4":"i", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741852, "value1":0.28794172682449687, "value2":453442634675086185, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.738181, "value1":0.6087693219046798, "value2":7047508567290984350, "value3":true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.492510, "value1":0.5382269802655012, "value2":2689541564459522343, "value3":false, "__name__":"metric_00000891", "key_3":"d","key_7":"i","key_2":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988702, "value1":0.0728816330863758, "value2":5556532985784164765, "value3":true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.655057, "value1":0.7734197867503856, "value2":5244006855203800595, "value3":true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520001, "value1":0.4376795320695461, "value2":1470774311764721388, "value3":false, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164473, "value1":0.6572062696131487, "value2":1949710197196336093, "value3":false, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.190159, "value1":0.9525604598487357, "value2":5478966602860717950, "value3":true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829372, "value1":0.6923157559412111, "value2":5023489111088251295, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391391, "value1":0.6340217327197775, "value2":8000248111604879168, "value3":false, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883302, "value1":0.8070620002939071, "value2":653446263691796732, "value3":true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745426, "value1":0.45672801954182946, "value2":6893619651580083925, "value3":false, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982512, "value1":0.5060051247468136, "value2":7419982333827237040, "value3":true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827729, "value1":0.15692262245568137, "value2":3804375123078218665, "value3":false, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244618, "value1":0.6655062510702947, "value2":1607640639377335930, "value3":false, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.945643, "value1":0.3793044572114196, "value2":8429298239346478761, "value3":true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626649, "value1":0.22068614209697351, "value2":8899816776188911022, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045981, "value1":0.34313373362766186, "value2":1556434949234629007, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.381111, "value1":0.4930245395823471, "value2":3486502897378084847, "value3":true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596053, "value1":0.23068160326995202, "value2":1094706285002463668, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454071, "value1":0.4016206555788349, "value2":7766949472696777624, "value3":false, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122082, "value1":0.12814082114630934, "value2":6860595913103836152, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267563, "value1":0.915043661363235, "value2":7117774485449523197, "value3":false, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883324, "value1":0.48609131055046695, "value2":168158041910482971, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.798512, "value1":0.9160093333280128, "value2":5957101416277255647, "value3":false, "__name__":"metric_00000912", "key_6":"c","key_9":"a","key_0":"k", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.956973, "value1":0.08410692933360829, "value2":3670111299929079308, "value3":true, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799611, "value1":0.006983554187354543, "value2":560914434528784616, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539372, "value1":0.1527069613506525, "value2":5396674816694339651, "value3":true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.538750, "value1":0.21348159925068225, "value2":4409731296615206554, "value3":false, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.854320, "value1":0.921941847389472, "value2":4199458567691684987, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451148, "value1":0.5527973855585238, "value2":5090625521103681352, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.585722, "value1":0.08217397453744285, "value2":5891174882356923897, "value3":true, "__name__":"metric_00000920", "key_8":"k","key_9":"g","key_2":"c", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280675, "value1":0.17159417706244065, "value2":1155748852941674950, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673546, "value1":0.9064641866629127, "value2":5826431744512876414, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748734, "value1":0.7184282675490825, "value2":465504921239629350, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448890, "value1":0.21947167931430242, "value2":97944850538911159, "value3":false, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.034158, "value1":0.6471204512621925, "value2":6508843210808028097, "value3":true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495253, "value1":0.7273452290068572, "value2":2698451345131648958, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.362458, "value1":0.6109728168729301, "value2":8601029350565666815, "value3":true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263467, "value1":0.5089131780165215, "value2":7125458167541328714, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373946, "value1":0.4318385725650386, "value2":2714678475704530055, "value3":true, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714267, "value1":0.9575832102733836, "value2":6392945512202004757, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448342, "value1":0.22459353529927165, "value2":3260297440141888471, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636333, "value1":0.5831279192445404, "value2":4643961286576988019, "value3":false, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.544083, "value1":0.5292455018046274, "value2":2947716990317042035, "value3":false, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.834093, "value1":0.03492084160563978, "value2":317252294366234894, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266819, "value1":0.3363934052516993, "value2":7398703592096323308, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006504, "value1":0.7543858008870775, "value2":1775651744055633759, "value3":true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114763, "value1":0.5343311798406293, "value2":3440557676505536535, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338231, "value1":0.46269924741830853, "value2":2414529329564577894, "value3":false, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.613570, "value1":0.5208853551038555, "value2":1476820686457140000, "value3":true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954830, "value1":0.41362302765893333, "value2":5510842372132552038, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.009033, "value1":0.39228566619392624, "value2":4427054644662596059, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182689, "value1":0.6549868089884497, "value2":4035004692027824929, "value3":false, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037729, "value1":0.7486872272646202, "value2":4847504507990849152, "value3":true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.025112, "value1":0.34907292952301494, "value2":6027470898165077200, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438904, "value1":0.6318755547819239, "value2":7111135668532014691, "value3":true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812145, "value1":0.3675412312936663, "value2":5692745323903444622, "value3":true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.445020, "value1":0.3643115537301237, "value2":1216854539448528229, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149138, "value1":0.5909969143971484, "value2":7044155605397820069, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841137, "value1":0.5031802947436413, "value2":4756712241095891395, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766360, "value1":0.42396402951055434, "value2":5375665261877407507, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663263, "value1":0.21109789310310256, "value2":7791325581861318665, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.623457, "value1":0.10997466706442727, "value2":4022405196702857232, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687642, "value1":0.5890580006230834, "value2":4798397386267850627, "value3":true, "__name__":"metric_00000952", "key_2":"b","key_0":"i", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041534, "value1":0.7259891148801754, "value2":7530272514921895889, "value3":true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.8243889839817137, "value2":2465746906164654846, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.282429, "value1":0.6519583818560468, "value2":6092402468810251742, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816484, "value1":0.18010854645368016, "value2":8191892113724148654, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851506, "value1":0.5899554257608193, "value2":4617440579117228711, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.382630, "value1":0.5412707148385829, "value2":3548532358946210519, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.473179, "value1":0.6228095400098033, "value2":5572845251799335628, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266666, "value1":0.5868701622069099, "value2":6991916315373855609, "value3":true, "__name__":"metric_00000962", "key_9":"d","key_3":"d","key_7":"f", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.952001, "value1":0.6189910604042005, "value2":6928874617046934238, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107532, "value1":0.28068061790544596, "value2":7672880301382817972, "value3":true, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.688840, "value1":0.6035662636797529, "value2":5891970996329552554, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283889, "value1":0.9151251791608558, "value2":2952612587882160016, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629939, "value1":0.5930273861647344, "value2":7387132779647308781, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418325, "value1":0.9507448442764749, "value2":167010402424657095, "value3":false, "__name__":"metric_00000967", "key_9":"f","key_1":"h","key_5":"d", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373327, "value1":0.6218852126400313, "value2":8371256546452205404, "value3":true, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515481, "value1":0.0854866845303678, "value2":6987828829468502321, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820817, "value1":0.698882043745769, "value2":8235587889784697376, "value3":false, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.365626, "value1":0.4531560215251981, "value2":4722270617283077560, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484276, "value1":0.6659326817404172, "value2":8721021931100680532, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.528708, "value1":0.03487880924213162, "value2":6041728758379789938, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590377, "value1":0.34239911489608105, "value2":7295038165138773264, "value3":true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481162, "value1":0.8346535177821227, "value2":4835378003160835691, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.664721, "value1":0.9670197543347829, "value2":7077761062907993448, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242914, "value1":0.7732131934363016, "value2":6013582492758534765, "value3":true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.185221, "value1":0.10448347281111905, "value2":4626218721873500711, "value3":true, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.126356, "value1":0.6396318028070207, "value2":7845764314900334021, "value3":true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673777, "value1":0.7918565155988029, "value2":7808431007480445571, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706238, "value1":0.49404656958089044, "value2":8867399074074176905, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216638, "value1":0.2610211982854062, "value2":5851177857276860153, "value3":false, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247760, "value1":0.720377370743505, "value2":284915355108242656, "value3":false, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484098, "value1":0.20488798005491424, "value2":8924950640215344491, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.345583, "value1":0.26657891861913446, "value2":1138178968371346598, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.315052, "value1":0.7472751608991773, "value2":5324337837076333029, "value3":true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247789, "value1":0.35415032152422526, "value2":3640906326280302592, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787688, "value1":0.6162646188121896, "value2":8883014095767101323, "value3":true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477237, "value1":0.04016123924628, "value2":9093318721099578903, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560140, "value1":0.30318020971980014, "value2":2686556711005804654, "value3":false, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.002924, "value1":0.5022585667052755, "value2":3145515076422963133, "value3":true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732614, "value1":0.3545238139152862, "value2":1059232874175794196, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.332613, "value1":0.32230993437491456, "value2":2724091454501972237, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012206, "value1":0.931677320101252, "value2":5370261932149351692, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196655, "value1":0.8224901135235602, "value2":9208484728253262649, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.326865, "value1":0.5922171068495821, "value2":7224402625385076257, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289542, "value1":0.9512116686992638, "value2":7523857296190024264, "value3":true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.170535, "value1":0.5872728280225836, "value2":6413447856713598093, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434673, "value1":0.18109149352889556, "value2":1218373397895276791, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} diff --git a/docker/test_prom_metric.hjson b/docker/test_prom_metric.hjson new file mode 100644 index 00000000..291247ff --- /dev/null +++ b/docker/test_prom_metric.hjson @@ -0,0 +1,34 @@ +{ + clickhouse: { + hosts: [ + [ + 127.0.0.1 + ] + ] + port: 9000 + db: gauge + username: "" + password: "" + retryTimes: 0 + } + kafka: { + brokers: 127.0.0.1:9092 + } + task: { + name: test_prom_metric + topic: test_metric_topic + consumerGroup: test_prom_metric + earliest: true + parser: json + autoSchema: true + tableName: default.test_prom_metric + seriesTableName: test_prom_series + excludeColumns: [] + bufferSize: 50000 + PrometheusSchema: true + DynamicSchema: { + Enable: true + } + } + logLevel: info +} \ No newline at end of file diff --git a/go.metrictest.sh b/go.metrictest.sh new file mode 100755 index 00000000..dcd3a94c --- /dev/null +++ b/go.metrictest.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash + +echo "create database" +curl "localhost:8123" -d "CREATE DATABASE IF NOT EXISTS gauge ON CLUSTER abc" + +echo "create metric test tables" +curl "localhost:8123" -d "DROP TABLE IF EXISTS test_prom_metric ON CLUSTER abc SYNC" +curl "localhost:8123" -d "CREATE TABLE test_prom_metric ON CLUSTER abc +( + __series_id Int64, + timestamp DateTime CODEC(DoubleDelta, LZ4), + value Float32 CODEC(ZSTD(15)) +) ENGINE=ReplicatedReplacingMergeTree() +PARTITION BY toYYYYMMDD(timestamp) +ORDER BY (__series_id, timestamp);" + +curl "localhost:8123" -d "DROP TABLE IF EXISTS dist_test_prom_metric ON CLUSTER abc SYNC" +curl "localhost:8123" -d "CREATE TABLE dist_test_prom_metric ON CLUSTER abc AS test_prom_metric ENGINE = Distributed(abc, default, test_prom_metric);" + +curl "localhost:8123" -d "DROP TABLE IF EXISTS test_prom_series ON CLUSTER abc SYNC" +curl "localhost:8123" -d "CREATE TABLE test_prom_series ON CLUSTER abc +( + __series_id Int64, + __mgmt_id Int64, + labels String, + __name__ String +) ENGINE=ReplicatedReplacingMergeTree() +ORDER BY (__name__, __series_id);" + +curl "localhost:8123" -d "DROP TABLE IF EXISTS dist_test_prom_series ON CLUSTER abc SYNC" +curl "localhost:8123" -d "CREATE TABLE dist_test_prom_series ON CLUSTER abc AS test_prom_series ENGINE = Distributed(abc, default, test_prom_series);" + +echo "send messages to kafka" +echo "cat /tmp/test_prom_metric.data | kafka-console-producer --topic test_metric_topic --broker-list localhost:9092" > send.sh +sudo docker cp ./docker/test_prom_metric.data kafka:/tmp/ +sudo docker cp send.sh kafka:/tmp/ +sudo docker exec kafka kafka-topics --bootstrap-server localhost:9093 --topic test_metric_topic --delete +sudo docker exec kafka sh /tmp/send.sh + +echo "start clickhouse_sinker to consume" +timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_prom_metric.hjson + +schema=`curl "localhost:8123" -d 'DESC test_prom_metric' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` +echo "Got test_prom_metric schema => $schema" +[ $schema = "__series_idInt64,timestampDateTimeDoubleDelta,LZ4,value1Nullable(Float64),value2Nullable(Int64),value3Nullable(Bool),valueFloat32ZSTD(15)," ] || exit 1 + + +schema=`curl "localhost:8123" -d 'DESC test_prom_series' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` +echo "Got test_prom_series schema => $schema" +[ $schema = "key_0Nullable(String),key_1Nullable(String),key_2Nullable(String),key_3Nullable(String),key_4Nullable(String),key_5Nullable(String),key_6Nullable(String),key_7Nullable(String),key_8Nullable(String),key_9Nullable(String),labelsString,__mgmt_idInt64,__name__String,__series_idInt64," ] || exit 1 + +echo "check result 1" +count=`curl "localhost:8123" -d 'select count() from dist_test_prom_metric'` +echo "Got test_prom_metric count => $count" +[ $count -le 10000 ] || exit 1 + +count=`curl "localhost:8123" -d 'select count() from dist_test_prom_series'` +echo "Got test_prom_series count => $count" +[ $count -eq 1000 ] || exit 1 \ No newline at end of file diff --git a/go.mod b/go.mod index d495b5b9..dff2919d 100644 --- a/go.mod +++ b/go.mod @@ -3,41 +3,41 @@ module github.com/housepower/clickhouse_sinker go 1.19 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.4.3 - github.com/RoaringBitmap/roaring v1.2.1 - github.com/bytedance/sonic v1.5.0 - github.com/cespare/xxhash/v2 v2.1.2 - github.com/google/gops v0.3.25 - github.com/hjson/hjson-go/v4 v4.2.0 + github.com/ClickHouse/clickhouse-go/v2 v2.6.2 + github.com/RoaringBitmap/roaring v1.2.3 + github.com/bytedance/sonic v1.8.1 + github.com/cespare/xxhash/v2 v2.2.0 + github.com/google/gops v0.3.26 + github.com/hjson/hjson-go/v4 v4.3.0 github.com/jcmturner/gokrb5/v8 v8.4.3 github.com/jinzhu/copier v0.3.5 - github.com/nacos-group/nacos-sdk-go v1.1.2 - github.com/prometheus/client_golang v1.13.0 - github.com/prometheus/common v0.37.0 + github.com/nacos-group/nacos-sdk-go v1.1.4 + github.com/prometheus/client_golang v1.14.0 + github.com/prometheus/common v0.40.0 github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.8.1 - github.com/thanos-io/thanos v0.27.0 - github.com/tidwall/gjson v1.14.2 + github.com/thanos-io/thanos v0.30.2 + github.com/tidwall/gjson v1.14.4 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.10.4 + github.com/twmb/franz-go v1.12.1 github.com/twmb/franz-go/pkg/kadm v1.6.0 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.1 - github.com/valyala/fastjson v1.6.3 - go.uber.org/zap v1.23.0 - golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e - golang.org/x/time v0.0.0-20220722155302-e5dcc9cfc0b9 - gopkg.in/natefinch/lumberjack.v2 v2.0.0 + github.com/valyala/fastjson v1.6.4 + go.uber.org/zap v1.24.0 + golang.org/x/exp v0.0.0-20230213192124-5e25df0256eb + golang.org/x/time v0.3.0 + gopkg.in/natefinch/lumberjack.v2 v2.2.1 ) require ( - github.com/ClickHouse/ch-go v0.50.0 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723 // indirect - github.com/andybalholm/brotli v1.0.4 // indirect + github.com/ClickHouse/ch-go v0.52.1 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.174 // indirect + github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/bits-and-blooms/bitset v1.3.0 // indirect + github.com/bits-and-blooms/bitset v1.5.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect - github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a // indirect + github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/go-errors/errors v1.4.2 // indirect github.com/go-faster/city v1.0.1 // indirect @@ -51,36 +51,37 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.15.12 // indirect - github.com/klauspost/cpuid/v2 v2.1.1 // indirect - github.com/kr/pretty v0.3.0 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect + github.com/klauspost/compress v1.15.15 // indirect + github.com/klauspost/cpuid/v2 v2.2.3 // indirect + github.com/kr/text v0.2.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect - github.com/paulmach/orb v0.7.1 // indirect + github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b // indirect + github.com/paulmach/orb v0.9.0 // indirect github.com/pierrec/lz4/v4 v4.1.17 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_model v0.2.0 // indirect - github.com/prometheus/procfs v0.8.0 // indirect + github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect + github.com/prometheus/client_model v0.3.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect + github.com/rogpeppe/go-internal v1.6.1 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect - github.com/tidwall/pretty v1.2.0 // indirect + github.com/tidwall/pretty v1.2.1 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect - github.com/twmb/franz-go/pkg/kmsg v1.2.0 // indirect - go.opentelemetry.io/otel v1.11.1 // indirect - go.opentelemetry.io/otel/trace v1.11.1 // indirect + github.com/twmb/franz-go/pkg/kmsg v1.4.0 // indirect + go.opentelemetry.io/otel v1.13.0 // indirect + go.opentelemetry.io/otel/trace v1.13.0 // indirect go.uber.org/atomic v1.10.0 // indirect - go.uber.org/multierr v1.8.0 // indirect - golang.org/x/arch v0.0.0-20220919183040-2926576b28c0 // indirect - golang.org/x/crypto v0.3.0 // indirect - golang.org/x/net v0.2.0 // indirect - golang.org/x/sync v0.0.0-20220923202941-7f9b1623fab7 // indirect - golang.org/x/sys v0.2.0 // indirect + go.uber.org/multierr v1.9.0 // indirect + golang.org/x/arch v0.2.0 // indirect + golang.org/x/crypto v0.6.0 // indirect + golang.org/x/net v0.7.0 // indirect + golang.org/x/sync v0.1.0 // indirect + golang.org/x/sys v0.5.0 // indirect google.golang.org/protobuf v1.28.1 // indirect - gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index bb9a300c..94362291 100644 --- a/go.sum +++ b/go.sum @@ -1,57 +1,21 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= -cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= -cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= -cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= -cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= -cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= -cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= -cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= -cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= -cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= -cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= -cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= -cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= -cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= -cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= -cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= -cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= -cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= -cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= -cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= -cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= -cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= -cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= -cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= -cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= -cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= -cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= -cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= -cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= -dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= -github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/ClickHouse/ch-go v0.50.0 h1:f5Bv6WEVIcvhT9ez5+uqGZiggVTxAJ2+tgk1mYQCgAs= -github.com/ClickHouse/ch-go v0.50.0/go.mod h1:lCZ+zUH/OCr16xF4PIg8Q5SfaUVDQLU3b21FMXU+qlQ= -github.com/ClickHouse/clickhouse-go/v2 v2.4.3 h1:DROLjHmUPygvyelxOswE5+Yyul84gRKjIaOLq3B5l1w= -github.com/ClickHouse/clickhouse-go/v2 v2.4.3/go.mod h1:Q95k4+cA11bh5eqQvoxlNG6NCGi0rIAEDbroVIzqng8= +github.com/ClickHouse/ch-go v0.52.1 h1:nucdgfD1BDSHjbNaG3VNebonxJzD8fX8jbuBpfo5VY0= +github.com/ClickHouse/ch-go v0.52.1/go.mod h1:B9htMJ0hii/zrC2hljUKdnagRBuLqtRG/GrU3jqCwRk= +github.com/ClickHouse/clickhouse-go/v2 v2.6.2 h1:Gw6hUHDgwtDqLOjRZFFxqkwdWdYp/Vw1F+F4eAiaMtI= +github.com/ClickHouse/clickhouse-go/v2 v2.6.2/go.mod h1:GcNAg9SniIu+BqzOxRsTmXAGvhlSaUm/Y9GFdWUCbX8= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/RoaringBitmap/roaring v1.2.1 h1:58/LJlg/81wfEHd5L9qsHduznOIhyv4qb1yWcSvVq9A= -github.com/RoaringBitmap/roaring v1.2.1/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= +github.com/RoaringBitmap/roaring v1.2.3 h1:yqreLINqIrX22ErkKI0vY47/ivtJr6n+kMhVOVmhWBY= +github.com/RoaringBitmap/roaring v1.2.3/go.mod h1:plvDsJQpxOC5bw8LRteu/MLWHsHez/3y6cubLI4/1yE= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723 h1:oBwUer3XCooqTDMBJ5vR2tTBv/4Te125UzMoh+/na5c= -github.com/aliyun/alibaba-cloud-sdk-go v1.61.1723/go.mod h1:RcDobYh8k5VP6TNybz9m++gL3ijVI5wueVr0EM10VsU= -github.com/andybalholm/brotli v1.0.4 h1:V7DdXeJtZscaqfNuAdSRuRFzuiKlHSC/Zh3zl9qY3JY= -github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.174 h1:K5561f5qv64GmtQuY0OtkY0H2x1bTXBeYwwWFPzc4D8= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.174/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= +github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -59,33 +23,25 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= -github.com/bits-and-blooms/bitset v1.3.0 h1:h7mv5q31cthBTd7V4kLAZaIThj1e8vPGcSqpPue9KVI= -github.com/bits-and-blooms/bitset v1.3.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= +github.com/bits-and-blooms/bitset v1.5.0 h1:NpE8frKRLGHIcEzkR+gZhiioW1+WbYV6fKwD6ZIpQT8= +github.com/bits-and-blooms/bitset v1.5.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= -github.com/bytedance/sonic v1.5.0 h1:XWdTi8bwPgxIML+eNV1IwNuTROK6EUrQ65ey8yd6fRQ= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/bytedance/sonic v1.8.1 h1:NqAHCaGaTzro0xMmnTCLUyRlbEP6r8MCA1cJUrH3Pu4= +github.com/bytedance/sonic v1.8.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= -github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= -github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a h1:lmGPzuocwDxoPAMr9h16zoJY/USZR9jIh99nrmKk1uI= -github.com/chenzhuoyu/base64x v0.0.0-20220526154910-8bf9453eb81a/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= -github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= -github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= -github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 h1:qSGYFH7+jGhDF8vLC+iwCD4WpbV1EBDSzWkJODFLams= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= +github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dennwc/varint v1.0.0 h1:kGNFFSSw8ToIy3obO/kKr8U9GZYUAxQEVuix4zfDWzE= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/efficientgo/core v1.0.0-rc.0.0.20221201130417-ba593f67d2a4 h1:rydBwnBoywKQMjWF0z8SriYtQ+uUcaFsxuijMjJr5PI= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= @@ -93,95 +49,46 @@ github.com/go-faster/city v1.0.1 h1:4WAxSZ3V2Ws4QRDrscLEDcibJY8uf41H6AhXDrNDcGw= github.com/go-faster/city v1.0.1/go.mod h1:jKcUJId49qdW3L1qKHH/3wPeUstCVpVSXTM6vO3VcTw= github.com/go-faster/errors v0.6.1 h1:nNIPOBkprlKzkThvS/0YaX8Zs9KewLCOSFQS5BU06FI= github.com/go-faster/errors v0.6.1/go.mod h1:5MGV2/2T9yvlrbhe9pD9LO5Z/2zCSq2T8j+Jpi2LAyY= -github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-kit/log v0.2.0 h1:7i2K3eKTos3Vc0enKCfnVcgHh2olr/MyfboYq7cAcFw= -github.com/go-kit/log v0.2.0/go.mod h1:NwTd00d/i8cPZ3xOwwiv2PO5MOcx78fFErGNcVmBjv0= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-logfmt/logfmt v0.5.1 h1:otpy5pqBCBZ1ng9RQ0dPu4PN7ba75Y/aA+UpowDyNVA= -github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= -github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.4 h1:l75CXGRSwbaYNpl/Z2X1XIIAMSCquvXgpVZDhwEIJsc= -github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= -github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= -github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= -github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= -github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= -github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= -github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= -github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= -github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= +github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/gops v0.3.25 h1:Pf6uw+cO6pDhc7HJ71NiG0x8dyQTeQcmg3HQFF39qVw= -github.com/google/gops v0.3.25/go.mod h1:8A7ebAm0id9K3H0uOggeRVGxszSvnlURun9mg3GdYDw= -github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= -github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= -github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/gops v0.3.26 h1:Ziyfd8sEhWVbrCIy59c1WOKodI63Jzojwm0JSZbBPS4= +github.com/google/gops v0.3.26/go.mod h1:vZ68aOXu2zJoybPyGpaHMmrCyd51DCxJoex4cO3ht/o= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= -github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hjson/hjson-go/v4 v4.2.0 h1:GBa/BfCg/68J0dB/ztAYJtVecXpalG4nZkY4UusGZXQ= -github.com/hjson/hjson-go/v4 v4.2.0/go.mod h1:KaYt3bTw3zhBjYqnXkYywcYctk0A2nxeEFTse3rH13E= -github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/hjson/hjson-go/v4 v4.3.0 h1:dyrzJdqqFGhHt+FSrs5n9s6b0fPM8oSJdWo+oS3YnJw= +github.com/hjson/hjson-go/v4 v4.3.0/go.mod h1:KaYt3bTw3zhBjYqnXkYywcYctk0A2nxeEFTse3rH13E= +github.com/inconshreveable/mousetrap v1.0.1/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= @@ -201,45 +108,36 @@ github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9Y github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= -github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= -github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= -github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= -github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.4/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.15.12 h1:YClS/PImqYbn+UILDnqxQCZ3RehC9N318SU3kElDUEM= -github.com/klauspost/compress v1.15.12/go.mod h1:QPwzmACJjUTFsnSHH934V6woptycfrDDJnH7hvFVbGM= +github.com/klauspost/compress v1.15.15 h1:EF27CXIuDsYJ6mmvtBRlEuB2UVOqHG1tAXgZ7yIO+lw= +github.com/klauspost/compress v1.15.15/go.mod h1:ZcK2JAFqKOpnBlxcLsJzYfrS9X1akm9fHZNnD9+Vo/4= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= -github.com/klauspost/cpuid/v2 v2.1.1 h1:t0wUqjowdm8ezddV5k0tLWVklVuvLJpoHeb4WBdydm0= -github.com/klauspost/cpuid/v2 v2.1.1/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= +github.com/klauspost/cpuid/v2 v2.2.3 h1:sxCkb+qR91z4vsqw4vGGZlDgPz3G7gjaLyK3V8y70BU= +github.com/klauspost/cpuid/v2 v2.2.3/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= -github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI= -github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= +github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= +github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -247,14 +145,16 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/mschoch/smat v0.2.0 h1:8imxQsjDm8yFEAVBe7azKmKSgzSkZXDuKkSq9374khM= github.com/mschoch/smat v0.2.0/go.mod h1:kc9mz7DoBKqDyiRL7VZN8KvXQMWeTaVnttLRXOlotKw= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nacos-group/nacos-sdk-go v1.1.2 h1:lWTpf5SXLetQetS7p31eGic/ncqsnn0Zbau1i3eC25Y= -github.com/nacos-group/nacos-sdk-go v1.1.2/go.mod h1:I8Vj4M8ZLpBk7EY2A8RXQE1SbfCA7b56TJBPIFTrUYE= -github.com/paulmach/orb v0.7.1 h1:Zha++Z5OX/l168sqHK3k4z18LDvr+YAO/VjK0ReQ9rU= -github.com/paulmach/orb v0.7.1/go.mod h1:FWRlTgl88VI1RBx/MkrwWDRhQ96ctqMCh8boXhmqB/A= +github.com/nacos-group/nacos-sdk-go v1.1.4 h1:qyrZ7HTWM4aeymFfqnbgNRERh7TWuER10pCB7ddRcTY= +github.com/nacos-group/nacos-sdk-go v1.1.4/go.mod h1:cBv9wy5iObs7khOqov1ERFQrCuTR4ILpgaiaVMxEmGI= +github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b h1:FfH+VrHHk6Lxt9HdVS0PXzSXFyS2NbZKXv33FYPol0A= +github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b/go.mod h1:AC62GU6hc0BrNm+9RK9VSiwa/EUe1bkIeFORAMcHvJU= +github.com/paulmach/orb v0.9.0 h1:MwA1DqOKtvCgm7u9RZ/pnYejTeDJPnr0+0oFajBbJqk= +github.com/paulmach/orb v0.9.0/go.mod h1:SudmOk85SXtmXAB3sLGyJ6tZy/8pdfrV0o6ef98Xc30= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= github.com/pierrec/lz4/v4 v4.1.14/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -264,54 +164,44 @@ github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= -github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= -github.com/prometheus/client_golang v1.12.1/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= -github.com/prometheus/client_golang v1.13.0 h1:b71QUfeo5M8gq2+evJdTPfZhYMAU0uKPkyPJ7TPsloU= -github.com/prometheus/client_golang v1.13.0/go.mod h1:vTeo+zgvILHsnnj/39Ou/1fPN5nJFOEMgftOUOmlvYQ= +github.com/prometheus/client_golang v1.14.0 h1:nJdhIvne2eSX/XRAFV9PcvFFRbrjbcTUj0VP62TMhnw= +github.com/prometheus/client_golang v1.14.0/go.mod h1:8vpkKitgIVNcqrRBWh1C4TIUQgYNtG/XQE4E/Zae36Y= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= -github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.3.0 h1:UBgGFHqYdG/TPFD1B1ogZywDqEkwp3fBMvqdiQ7Xew4= +github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= -github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= -github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= -github.com/prometheus/common v0.37.0 h1:ccBbHCgIiT9uSoFY0vX8H3zsNR5eLt17/RQLUvn8pXE= -github.com/prometheus/common v0.37.0/go.mod h1:phzohg0JFMnBEFGxTDbfu3QyL5GI8gTQJFhYO5B3mfA= +github.com/prometheus/common v0.40.0 h1:Afz7EVRqGg2Mqqf4JuF9vdvp1pi220m55Pi9T2JnO4Q= +github.com/prometheus/common v0.40.0/go.mod h1:L65ZJPSmfn/UBWLQIHV7dBrKFidB/wPlF1y5TlSt9OE= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.8.0 h1:ODq8ZFEaYeCaZOJlZZdJA2AbQR98dSHSM1KW/You5mo= -github.com/prometheus/procfs v0.8.0/go.mod h1:z7EfXMXOkbkqb9IINtpCn86r/to3BnA0uaxHdg830/4= -github.com/prometheus/prometheus v1.8.2-0.20220308163432-03831554a519 h1:MSR9EaMnwV1nsmhQ2eB1Txwebyk/5ulUGcgcYi3ofL4= +github.com/prometheus/procfs v0.9.0 h1:wzCHvIvM5SxWqYvwgVL7yJY8Lz3PKn49KQtpgMYJfhI= +github.com/prometheus/procfs v0.9.0/go.mod h1:+pB4zwohETzFnmlpe6yd2lSc+0/46IYZRB/chUwxUZY= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= +github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= -github.com/shirou/gopsutil/v3 v3.22.4/go.mod h1:D01hZJ4pVHPpCTZ3m3T2+wDF2YAGfd+H4ifUguaQzHM= +github.com/shirou/gopsutil/v3 v3.22.10/go.mod h1:QNza6r4YQoydyCfo6rH0blGfKahgibh4dQmV5xdFkQk= github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= -github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= +github.com/spf13/cobra v1.6.1/go.mod h1:IOw/AERYS7UzyrGinqmz6HLUo219MORXGxhbaJUqzrY= +github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/amqp v1.0.0 h1:kuuDrUJFZL1QYL9hUNuCxNObNzB0bV/ZG5jV3RWAQgo= github.com/streadway/amqp v1.0.0/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= @@ -323,19 +213,22 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/thanos-io/thanos v0.27.0 h1:TWOPMkWX5Q1LHdROI6dg9lKU3x6Q95vO4o/bm8GaCfE= -github.com/thanos-io/thanos v0.27.0/go.mod h1:i0KCpF5fWAWBNZXBkuzGh0603YlMrbhLJdwOO5W21YI= -github.com/tidwall/gjson v1.14.2 h1:6BBkirS0rAHjumnjHF6qgy5d2YAJ1TLIaFE2lzfOLqo= -github.com/tidwall/gjson v1.14.2/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= +github.com/thanos-io/thanos v0.30.2 h1:JcazfBlpX/i747uRxBWHJMopX0g+eSaZYGjqMeQIkPo= +github.com/thanos-io/thanos v0.30.2/go.mod h1:k23ZcbQSK0Jd/siuEti6J6/3DeezIk+YLRBcYikzXl8= +github.com/tidwall/gjson v1.14.4 h1:uo0p8EbA09J7RQaflQ1aBRffTR7xedD2bcIVSYxLnkM= +github.com/tidwall/gjson v1.14.4/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= -github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= +github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= +github.com/tidwall/pretty v1.2.1 h1:qjsOFOWWQl+N3RsoF5/ssm1pHmJJwhjlSbZ51I6wMl4= +github.com/tidwall/pretty v1.2.1/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= github.com/tklauser/go-sysconf v0.3.10/go.mod h1:C8XykCvCb+Gn0oNCWPIlcb0RuglQTYaQ2hGm7jmxEFk= github.com/tklauser/numcpus v0.4.0/go.mod h1:1+UI3pD8NW14VMwdgJNJ1ESk2UnwhAnz5hMwiKKqXCQ= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= @@ -344,346 +237,143 @@ github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.5.3/go.mod h1:eqHYpAuvlTArOdZ1XtPYyOQ1uUb40CSZwbpL3ccjibI= github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go v1.10.4 h1:1PGpRG0uGTSSZCBV6lAMYcuVsyReMqdNBQRd8QCzw9U= -github.com/twmb/franz-go v1.10.4/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= +github.com/twmb/franz-go v1.12.1 h1:8lWT8q0spL40Nfw6eonJ8OoPGLvF9arvadRRmcSiu9Y= +github.com/twmb/franz-go v1.12.1/go.mod h1:Ofc5tSSUJKLmpRNUYSejUsAZKYAHDHywTS322KWdChQ= github.com/twmb/franz-go/pkg/kadm v1.6.0 h1:jfbpdneFgwO8wcvkMnu670+qYmOI4A9USHR/VTsAqrA= github.com/twmb/franz-go/pkg/kadm v1.6.0/go.mod h1:1FifItwSffE++249YqRooeEfDnKRgvki6GqTJezTVAM= github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/kmsg v1.2.0 h1:jYWh2qFw5lDbNv5Gvu/sMKagzICxuA5L6m1W2Oe7XUo= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go/pkg/kmsg v1.4.0 h1:tbp9hxU6m8qZhQTlpGiaIJOm4BXix5lsuEZ7K00dF0s= +github.com/twmb/franz-go/pkg/kmsg v1.4.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 h1:alKdbddkPw3rDh+AwmUEwh6HNYgTvDSFIe/GWYRR9RM= github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0/go.mod h1:k8BoBjyUbFj34f0rRbn+Ky12sZFAPbmShrg0karAIMo= github.com/twmb/franz-go/plugin/kzap v1.1.1 h1:ae8Z2JXn8y9ceZ2AFnwPm5U1A8d6cBvvZlygF46F2N4= github.com/twmb/franz-go/plugin/kzap v1.1.1/go.mod h1:TUlWYqucIyz6U7xLo++gkHBDiJmed5FpYe42rCL2YG0= -github.com/valyala/fastjson v1.6.3 h1:tAKFnnwmeMGPbwJ7IwxcTPCNr3uIzoIj3/Fh90ra4xc= -github.com/valyala/fastjson v1.6.3/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= +github.com/uber/jaeger-client-go v2.30.0+incompatible h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o= +github.com/uber/jaeger-client-go v2.30.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= +github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg= +github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= +github.com/valyala/fastjson v1.6.4 h1:uAUNq9Z6ymTgGhcm0UynUAB6tlbakBrz6CQFax3BXVQ= +github.com/valyala/fastjson v1.6.4/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= +github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.1.1/go.mod h1:RaEWvsqvNKKvBPvcKeFjrG2cJqOkHTiyTpzz23ni57g= +github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgklLGvcBnW8= github.com/xlab/treeprint v1.1.0/go.mod h1:gj5Gd3gPdKtR1ikdDK6fnFLdmIS0X30kTTuNd/WEJu0= -github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d/go.mod h1:rHwXgn7JulP+udvsHwJoVG1YGAP6VLg4y9I5dyZdqmA= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= -go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= -go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= -go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opentelemetry.io/otel v1.11.1 h1:4WLLAmcfkmDk2ukNXJyq3/kiz/3UzCaYq6PskJsaou4= -go.opentelemetry.io/otel v1.11.1/go.mod h1:1nNhXBbWSD0nsL38H6btgnFN2k4i0sNLHNNMZMSbUGE= -go.opentelemetry.io/otel/trace v1.11.1 h1:ofxdnzsNrGBYXbP7t7zpUK281+go5rF7dvdIZXF8gdQ= -go.opentelemetry.io/otel/trace v1.11.1/go.mod h1:f/Q9G7vzk5u91PhbmKbg1Qn0rzH1LJ4vbPHFGkTPtOk= +go.mongodb.org/mongo-driver v1.11.1/go.mod h1:s7p5vEtfbeR1gYi6pnj3c3/urpbLv2T5Sfd6Rp2HBB8= +go.opentelemetry.io/otel v1.13.0 h1:1ZAKnNQKwBBxFtww/GwxNUyTf0AxkZzrukO8MeXqe4Y= +go.opentelemetry.io/otel v1.13.0/go.mod h1:FH3RtdZCzRkJYFTCsAKDy9l/XYjMdNv6QrkFFB8DvVg= +go.opentelemetry.io/otel/trace v1.13.0 h1:CBgRZ6ntv+Amuj1jDsMhZtlAPT6gbyIRdaIzFhfBSdY= +go.opentelemetry.io/otel/trace v1.13.0/go.mod h1:muCvmmO9KKpvuXSf3KKAXXB2ygNYHQ+ZfI5X08d3tds= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.10.0 h1:9qC72Qh0+3MqyJbAn8YU5xVq1frD8bn3JtD2oXtafVQ= go.uber.org/atomic v1.10.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= +go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/multierr v1.8.0 h1:dg6GjLku4EH+249NNmoIciG9N/jURbDG+pFlTkhzIC8= go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= +go.uber.org/multierr v1.9.0 h1:7fIwc/ZtS0q++VgcfqFDxSBZVv/Xo49/SYnDFupUwlI= +go.uber.org/multierr v1.9.0/go.mod h1:X2jQV1h+kxSjClGpnseKVIxpmcjrj7MNnI0bnlfKTVQ= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= -go.uber.org/zap v1.23.0 h1:OjGQ5KQDEUawVHxNwQgPpiypGHOxo2mNZsOqTak4fFY= -go.uber.org/zap v1.23.0/go.mod h1:D+nX8jyLsMHMYrln8A0rJjFt/T/9/bGgIhAqxv5URuY= +go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= +go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.0.0-20220919183040-2926576b28c0 h1:Ibo6J679xStgTWXLeZWlCA0vTvCEFZkUWFWAmBX2Zpk= -golang.org/x/arch v0.0.0-20220919183040-2926576b28c0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.2.0 h1:W1sUEHXiJTfjaFJ5SLo0N6lZn+0eO5gWD1MFeTGqQEY= +golang.org/x/arch v0.2.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.3.0 h1:a06MkbcxBrEFc0w0QIZWXrH/9cCX6KJyWbBOIwAn+7A= -golang.org/x/crypto v0.3.0/go.mod h1:hebNnKkNXi2UzZN1eVRvBB7co0a+JxK6XbPiWVs/3J4= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= -golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= -golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= -golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= -golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e h1:+WEEuIdZHnUeJJmEUjyYC2gfUMj69yZXw17EnHg/otA= -golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e/go.mod h1:Kr81I6Kryrl9sr8s2FK3vxD90NdsKWRuOIl2O4CvYbA= -golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= -golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/crypto v0.6.0 h1:qfktjS5LUO+fFKeJXZ+ikTRijMmljikvG68fpMMruSc= +golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= +golang.org/x/exp v0.0.0-20230213192124-5e25df0256eb h1:PaBZQdo+iSDyHT053FjUCgZQ/9uqVwPOcl7KSWhKn6w= +golang.org/x/exp v0.0.0-20230213192124-5e25df0256eb/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= -golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= -golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= -golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= -golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= -golang.org/x/net v0.2.0 h1:sZfSu1wtKLGlWI4ZZayP0ck9Y73K1ynO6gqzTdBVdPU= -golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/net v0.7.0 h1:rJrUqqhjsgNp7KqAIc25s9pZnjU7TUcSY7HcVZjdn1g= +golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220923202941-7f9b1623fab7 h1:ZrnxWX62AgTKOSagEqxvb3ffipvEDX2pl7E1TdqLqIc= -golang.org/x/sync v0.0.0-20220923202941-7f9b1623fab7/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sync v0.1.0 h1:wsuoTGHzEhffawBOhz5CYhcrV4IdKZbEyZjBMuTp12o= +golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200501052902-10377860bb8e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220128215802-99c3d69c2c27/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.2.0 h1:ljd4t30dBnAvMZaQCevtY0xLLD0A+bRZXbgLMLU1F/A= -golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.5.0 h1:MUK/U/4lj1t1oPg0HfuXDN/Z1wv31ZJ/YcPiGccS4DU= +golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20220722155302-e5dcc9cfc0b9 h1:ftMN5LMiBFjbzleLqtoBZk7KdJwhuybIU+FckUHgoyQ= -golang.org/x/time v0.0.0-20220722155302-e5dcc9cfc0b9/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= +golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= -golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= -google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= -google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= -google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= -google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= -google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= -google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= -google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= -google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= -google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= -google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= -google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= -google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= -google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= -google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= @@ -692,38 +382,24 @@ google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqw gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= -gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.66.2/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= +gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= +gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYskCTPBJVb9jqSc= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= -gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/goversion v1.2.0/go.mod h1:Eih9y/uIBS3ulggl7KNJ09xGSLcuNaLgmvvqa07sgfo= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= -rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= -rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/task/task.go b/task/task.go index ad2fb761..2e1e8449 100644 --- a/task/task.go +++ b/task/task.go @@ -85,7 +85,10 @@ func cloneTask(s *Service, newGroup *Consumer) (service *Service) { // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig, c *Consumer) (service *Service) { ck := output.NewClickHouse(cfg, taskCfg) - pp, _ := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone, taskCfg.TimeUnit) + pp, err := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone, taskCfg.TimeUnit) + if err != nil { + util.Logger.Fatal("failed to create task", zap.String("group", c.grpConfig.Name), zap.String("task", taskCfg.Name), zap.Error(err)) + } service = &Service{ clickhouse: ck, pp: pp, From 07ec921ce5885fe931da31469ae70aede613843c Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 23 Feb 2023 14:37:49 +0800 Subject: [PATCH 315/404] fix - Metrics from GoCollector and ProcessCollector are now being pushed to metric-push-gateway-addrs --- CHANGELOG.md | 2 ++ cmd/clickhouse_sinker/main.go | 1 + statistics/statistics.go | 3 +++ task/sinker.go | 1 + 4 files changed, 7 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 54ec506a..183d2965 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -8,6 +8,8 @@ Bug Fixes: - Fix illegal "TimeZone" value result in sinker crash - Fix wrong parsing result of Decimal type [909](https://github.com/ClickHouse/clickhouse-go/pull/909) +Improvements: +- Metrics from GoCollector and ProcessCollector are now being pushed to metric-push-gateway-addrs #### Version 3.0.0 (2023-02-07) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index df75fa48..481f302a 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -126,6 +126,7 @@ func init() { log.Fatal("unable to determine self ip", err) } selfIP = ip.String() + util.Logger.Info("parsed command options:", zap.Reflect("opts", cmdOps)) } func main() { diff --git a/statistics/statistics.go b/statistics/statistics.go index 9fabf120..57e17e36 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -190,6 +190,7 @@ func (p *Pusher) Init() error { func (p *Pusher) Run() { ticker := time.NewTicker(time.Second * time.Duration(p.pushInterval)) + util.Logger.Info("start pushing metrics to the specified push gateway address ") defer ticker.Stop() FOR: for { @@ -243,6 +244,8 @@ func (p *Pusher) reconnect() { Collector(WriteSeriesDropQuota). Collector(WriteSeriesDropUnchanged). Collector(WriteSeriesSucceed). + Collector(collectors.NewGoCollector()). + Collector(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{})). Grouping("instance", p.instance).Format(expfmt.FmtText) p.inUseAddr = nextAddr } diff --git a/task/sinker.go b/task/sinker.go index 69c29c98..faca1cae 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -92,6 +92,7 @@ func (s *Sinker) Run() { addrs := strings.Split(s.cmdOps.PushGatewayAddrs, ",") s.pusher = statistics.NewPusher(addrs, s.cmdOps.PushInterval, s.httpAddr) if err = s.pusher.Init(); err != nil { + util.Logger.Error("failed to initialize connection to the specified push gateway address", zap.Error(err)) return } go s.pusher.Run() From 2af3853b83ebc60d9f6a405803b6d565980b8f83 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Tue, 28 Feb 2023 11:07:04 +0800 Subject: [PATCH 316/404] Refine the error msg --- output/clickhouse.go | 8 ++++---- task/consumer.go | 1 + 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 0cd5ce60..968baf48 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -277,8 +277,8 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { c.seriesQuota = sq } - query := fmt.Sprintf("SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s.%s ORDER BY sid", c.dbName, tbl) - util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) + query := fmt.Sprintf("SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s.%s FINAL ORDER BY sid", c.dbName, tbl) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", c.taskCfg.Name)) var rs driver.Rows var seriesID, mgmtID int64 if rs, err = conn.Query(context.Background(), query); err != nil { @@ -385,7 +385,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { return } if c.distSeriesTbls == nil { - err = errors.Newf("Please create distributed table for %s.", c.seriesTbl) + err = errors.Newf("Please create distributed table for %s in cluster '%s'.", c.seriesTbl, c.cfg.Clickhouse.Cluster) return } } @@ -451,7 +451,7 @@ func (c *ClickHouse) initSchema() (err error) { return } if c.distMetricTbls == nil { - err = errors.Newf("Please create distributed table for %s.", c.TableName) + err = errors.Newf("Please create distributed table for %s in cluster '%s'.", c.seriesTbl, c.cfg.Clickhouse.Cluster) return } } diff --git a/task/consumer.go b/task/consumer.go index 56774666..cd581094 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -261,6 +261,7 @@ func (c *Consumer) processFetch() { if bufLength > bufThreshold { flushFn() + ticker.Reset(time.Duration(c.grpConfig.FlushInterval) * time.Second) } case <-ticker.C: flushFn() From 5837a989a56909e0954da0d853f4b7894ab3fe08 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Tue, 28 Feb 2023 11:19:26 +0800 Subject: [PATCH 317/404] remove the rowpool & rowspool, reduce the GLBWritingPool size to lower memory pressure --- model/message.go | 35 ----------------------------------- output/clickhouse.go | 7 +------ task/sharding.go | 6 ++++-- task/task.go | 3 ++- util/common.go | 2 +- 5 files changed, 8 insertions(+), 45 deletions(-) diff --git a/model/message.go b/model/message.go index 9cb49ac6..e9ede079 100644 --- a/model/message.go +++ b/model/message.go @@ -5,11 +5,6 @@ import ( "time" ) -var ( - rowsPool sync.Pool - FakedRow Row = make([]interface{}, 0) -) - // MsgWithMeta abstract messages // We are not using interface because virtual call. See https://syslog.ravelin.com/go-interfaces-but-at-what-cost-961e0f58a07b?gi=58f6761d1d70 type InputMessage struct { @@ -41,33 +36,3 @@ type Batch struct { func (b *Batch) Size() int { return len(*b.Rows) } - -func GetRows() (rs *Rows) { - v := rowsPool.Get() - if v == nil { - rows := make(Rows, 0) - return &rows - } - return v.(*Rows) -} - -func PutRows(rs *Rows) { - *rs = (*rs)[:0] - rowsPool.Put(rs) -} - -var rowPool sync.Pool - -func GetRow() *Row { - v := rowPool.Get() - if v == nil { - row := make(Row, 0) - return &row - } - return v.(*Row) -} - -func PutRow(r *Row) { - *r = (*r)[:0] - rowPool.Put(r) -} diff --git a/output/clickhouse.go b/output/clickhouse.go index 968baf48..9b462e19 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -235,12 +235,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { var reconnect bool var dbVer int sc := pool.GetShardConn(batch.BatchIdx) - defer func() { - for _, row := range *batch.Rows { - model.PutRow(row) - } - model.PutRows(batch.Rows) - }() + for { if err = c.write(batch, sc, &dbVer); err == nil { return diff --git a/task/sharding.go b/task/sharding.go index c850ab16..77a1d7c9 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -131,7 +131,8 @@ func NewSharder(service *Service) (sh *Sharder, err error) { msgBuf: make([]*model.Rows, shards), } for i := 0; i < shards; i++ { - sh.msgBuf[i] = model.GetRows() + rs := make(model.Rows, 0) + sh.msgBuf[i] = &rs } return } @@ -165,7 +166,8 @@ func (sh *Sharder) Flush(wg *sync.WaitGroup) { } batch.Wg.Add(1) sh.service.clickhouse.Send(batch) - sh.msgBuf[i] = model.GetRows() + rs := make(model.Rows, 0) + sh.msgBuf[i] = &rs } } if msgCnt > 0 { diff --git a/task/task.go b/task/task.go index 2e1e8449..c4eecc1d 100644 --- a/task/task.go +++ b/task/task.go @@ -215,7 +215,8 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { } func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) (row *model.Row) { - row = model.GetRow() + row1 := make(model.Row, 0) + row = &row1 if service.idxSerID >= 0 { var seriesID, mgmtID int64 var labels []string diff --git a/util/common.go b/util/common.go index 47df8b6d..c2ab1d85 100644 --- a/util/common.go +++ b/util/common.go @@ -64,7 +64,7 @@ func InitGlobalWritingPool(maxWorkers int) { if GlobalWritingPool != nil { return } - queueSize := 3 + queueSize := 1 GlobalWritingPool = NewWorkerPool(maxWorkers, queueSize) Logger.Info("initialized writing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", queueSize)) } From 47510066eab93678a573f7cbf70d2f1bbbb2d079 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Wed, 1 Mar 2023 23:27:54 +0800 Subject: [PATCH 318/404] Exit immediately when receiving one more termination signal --- CHANGELOG.md | 1 + util/app.go | 33 +++++++++++++++++++++++++++------ util/signal.go | 28 ---------------------------- 3 files changed, 28 insertions(+), 34 deletions(-) delete mode 100644 util/signal.go diff --git a/CHANGELOG.md b/CHANGELOG.md index 183d2965..8e843b97 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,7 @@ Bug Fixes: Improvements: - Metrics from GoCollector and ProcessCollector are now being pushed to metric-push-gateway-addrs +- Terminate program immediately when receiving one more exit signal #### Version 3.0.0 (2023-02-07) diff --git a/util/app.go b/util/app.go index 9528d454..417f6ddd 100644 --- a/util/app.go +++ b/util/app.go @@ -17,6 +17,9 @@ package util import ( "fmt" + "os" + "os/signal" + "syscall" "go.uber.org/zap" ) @@ -33,11 +36,29 @@ func Run(appName string, initFunc, jobFunc, cleanupFunc func() error) { } }() - s := WaitForExitSign() - Logger.Info(fmt.Sprintf("%s got the exit signal %s, start to clean", appName, s)) - - if err := cleanupFunc(); err != nil { - Logger.Fatal(appName+" clean failed", zap.Error(err)) + sig := make(chan os.Signal, 1) + done := make(chan struct{}) + signal.Notify(sig, os.Interrupt, syscall.SIGTERM) + count := 0 + for { + select { + case s := <-sig: + if count == 0 { + Logger.Info(fmt.Sprintf("Received termination signal %s, start to clean", s)) + count++ + go func() { + if err := cleanupFunc(); err != nil { + Logger.Fatal(appName+" clean failed", zap.Error(err)) + } + done <- struct{}{} + }() + } else { + Logger.Info(fmt.Sprintf("This is the second termination signal %s. Immediately terminate.", s)) + return + } + case <-done: + Logger.Info(appName + " clean completed, exit") + return + } } - Logger.Info(appName + " clean completed, exit") } diff --git a/util/signal.go b/util/signal.go deleted file mode 100644 index 7f146c93..00000000 --- a/util/signal.go +++ /dev/null @@ -1,28 +0,0 @@ -/*Copyright [2019] housepower - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package util - -import ( - "os" - "os/signal" - "syscall" -) - -func WaitForExitSign() os.Signal { - c := make(chan os.Signal, 1) - signal.Notify(c, os.Interrupt, syscall.SIGTERM) - return <-c -} From d41eef1416bc9966187f9027022b1fb46db498e6 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 2 Mar 2023 15:36:08 +0800 Subject: [PATCH 319/404] feat - Limit the fetch size and poll size based on the BufferSize property --- CHANGELOG.md | 3 ++- docs/configuration/config.md | 3 ++- input/kafka_franz.go | 11 ++++++++--- task/sharding.go | 2 +- task/sinker.go | 3 ++- 5 files changed, 15 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8e843b97..5a6637cd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,6 @@ # Changelog -#### Version 3.0.1 (not released yet) +#### Version 3.0.1 (2023-03-02) Bug Fixes: - Fix nacos publish config error "BUG: got different config" @@ -11,6 +11,7 @@ Bug Fixes: Improvements: - Metrics from GoCollector and ProcessCollector are now being pushed to metric-push-gateway-addrs - Terminate program immediately when receiving one more exit signal +- Limit the fetch size and poll size based on the BufferSize config property #### Version 3.0.0 (2023-02-07) diff --git a/docs/configuration/config.md b/docs/configuration/config.md index dea9e23b..268fc310 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -157,7 +157,8 @@ // interval of flushing the batch. Default to 5, max to 600. "flushInterval": 5, - // batch size to insert into clickhouse. sinker will round upward it to the the nearest 2^n. Default to 262114, max to 1048576. + // Approximate batch size to insert into clickhouse per shard, also control the kafka max.partition.fetch.bytes. + // Sinker will round upward it to the the nearest 2^n. Default to 262114, max to 1048576. "bufferSize": 262114, // In the absence of time zone information, interprets the time as in the given location. Default to "Local" (aka /etc/localtime of the machine on which sinker runs) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 6eb72209..4cf90039 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -81,6 +81,13 @@ func (k *KafkaFranz) Init(cfg *config.Config, gCfg *config.GroupConfig, f chan * kgo.ConsumeTopics(k.grpConfig.Topics...), kgo.ConsumerGroup(k.grpConfig.Name), kgo.DisableAutoCommit(), + ) + + maxPartBytes := int32(1 << (util.GetShift(100*k.grpConfig.BufferSize) - 1)) + + opts = append(opts, + kgo.FetchMaxBytes(maxPartBytes), + kgo.FetchMaxPartitionBytes(maxPartBytes), kgo.OnPartitionsRevoked(k.onPartitionRevoked), kgo.RebalanceTimeout(time.Minute*2), kgo.SessionTimeout(time.Minute*2), @@ -102,9 +109,7 @@ func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { kgo.SeedBrokers(strings.Split(kfkCfg.Brokers, ",")...), // kgo.FetchMaxBytes(), // 50 MB -- take the default config // kgo.BrokerMaxReadBytes(), // 100 MB - kgo.FetchMaxPartitionBytes(1 << 24), // 16MB kgo.MaxConcurrentFetches(2), - //kgo.MetadataMaxAge(...) corresponds to sarama.Config.Metadata.RefreshFrequency kgo.WithLogger(kzap.New(util.Logger)), } if kfkCfg.TLS.Enable { @@ -169,7 +174,7 @@ func (k *KafkaFranz) Run() { k.wgRun.Add(1) defer k.wgRun.Done() for { - fetches := k.cl.PollFetches(k.ctx) + fetches := k.cl.PollRecords(k.ctx, k.grpConfig.BufferSize) err := fetches.Err() if fetches == nil || fetches.IsClientClosed() || errors.Is(err, context.Canceled) { break diff --git a/task/sharding.go b/task/sharding.go index 77a1d7c9..bce56d23 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -171,7 +171,7 @@ func (sh *Sharder) Flush(wg *sync.WaitGroup) { } } if msgCnt > 0 { - util.Logger.Info(fmt.Sprintf("created a batch group for task %v, messages %d", sh.service.taskCfg.Name, msgCnt)) + util.Logger.Info(fmt.Sprintf("created a batch group for task %v with %d shards, total messages %d", sh.service.taskCfg.Name, len(sh.msgBuf), msgCnt)) statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) } } diff --git a/task/sinker.go b/task/sinker.go index faca1cae..f9ecb152 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -339,7 +339,8 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { } else { sort.Strings(c.grpConfig.Topics) sort.Strings(group.Topics) - if !reflect.DeepEqual(c.grpConfig.Topics, group.Topics) { + if !reflect.DeepEqual(c.grpConfig.Topics, group.Topics) || + c.grpConfig.BufferSize != group.BufferSize { deleteConsumers = append(deleteConsumers, name) } else { // apply TaskConfig Change From 187fea3f0b3dde56990ecaf2893c55804859057c Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 3 Mar 2023 00:01:50 +0800 Subject: [PATCH 320/404] Create containers with initial size --- CHANGELOG.md | 2 +- cmd/clickhouse_sinker/main.go | 2 +- cmd/nacos_publish_config/main.go | 2 +- input/kafka_franz.go | 1 - output/clickhouse.go | 14 +++++++-- parser/parser.go | 7 +++-- task/sharding.go | 2 +- task/task.go | 49 +++++++++++++++++--------------- 8 files changed, 46 insertions(+), 33 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5a6637cd..93c50607 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,6 @@ # Changelog -#### Version 3.0.1 (2023-03-02) +#### Version 3.0.1 (2023-03-03) Bug Fixes: - Fix nacos publish config error "BUG: got different config" diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 481f302a..48c6c7cd 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -204,7 +204,7 @@ func main() { util.Logger.Info(fmt.Sprintf("get config from nacos serverAddrs %s, namespaceId %s, group %s, dataId %s", cmdOps.NacosAddr, cmdOps.NacosNamespaceID, cmdOps.NacosGroup, cmdOps.NacosDataID)) rcm = &cm.NacosConfManager{} - properties = make(map[string]interface{}) + properties = make(map[string]interface{}, 8) properties["serverAddrs"] = cmdOps.NacosAddr properties["username"] = cmdOps.NacosUsername properties["password"] = cmdOps.NacosPassword diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 8bbe8099..58273663 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -51,7 +51,7 @@ var ( // Empty is not valid namespaceID func getProperties() map[string]interface{} { - properties := make(map[string]interface{}) + properties := make(map[string]interface{}, 6) properties["serverAddrs"] = *nacosAddr properties["username"] = *nacosUsername properties["password"] = *nacosPassword diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 4cf90039..c695b227 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -107,7 +107,6 @@ func (k *KafkaFranz) Init(cfg *config.Config, gCfg *config.GroupConfig, f chan * func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { opts = []kgo.Opt{ kgo.SeedBrokers(strings.Split(kfkCfg.Brokers, ",")...), - // kgo.FetchMaxBytes(), // 50 MB -- take the default config // kgo.BrokerMaxReadBytes(), // 100 MB kgo.MaxConcurrentFetches(2), kgo.WithLogger(kzap.New(util.Logger)), diff --git a/output/clickhouse.go b/output/clickhouse.go index 9b462e19..35ae3e82 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -261,12 +261,21 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { if c.cfg.Clickhouse.Cluster != "" { tbl = c.distSeriesTbls[0] } + + var count uint64 + conn.QueryRow(context.Background(), fmt.Sprintf("SELECT count() FROM %s.%s FINAL", c.dbName, tbl)).Scan(&count) sq := &seriesQuota{} if v, ok := seriesQuotas.LoadOrStore(c.GetSeriesQuotaKey(), sq); ok { c.seriesQuota = v.(*seriesQuota) + c.seriesQuota.Lock() + defer c.seriesQuota.Unlock() + if len(c.seriesQuota.bmSeries) == int(count) { + // only reload the map when there is difference detected + return + } } else { sq.Lock() - sq.bmSeries = make(map[int64]int64) + sq.bmSeries = make(map[int64]int64, count) sq.nextResetQuota = time.Now().Add(10 * time.Second) sq.Unlock() c.seriesQuota = sq @@ -284,6 +293,7 @@ func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { c.seriesQuota.Lock() defer c.seriesQuota.Unlock() + for rs.Next() { if err = rs.Scan(&seriesID, &mgmtID); err != nil { err = errors.Wrapf(err, "") @@ -412,7 +422,7 @@ func (c *ClickHouse) initSchema() (err error) { return } } else { - c.Dims = make([]*model.ColumnWithType, 0) + c.Dims = make([]*model.ColumnWithType, 0, len(c.taskCfg.Dims)) for _, dim := range c.taskCfg.Dims { c.Dims = append(c.Dims, &model.ColumnWithType{ Name: dim.Name, diff --git a/parser/parser.go b/parser/parser.go index 0c96a0b0..ea74ac16 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -1,10 +1,11 @@ -/*Copyright [2019] housepower +/* +Copyright [2019] housepower Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, @@ -95,7 +96,7 @@ func NewParserPool(name string, csvFormat []string, delimiter string, timezone s timeUnit: timeunit, } if csvFormat != nil { - pp.csvFormat = make(map[string]int) + pp.csvFormat = make(map[string]int, len(csvFormat)) for i, title := range csvFormat { pp.csvFormat[title] = i } diff --git a/task/sharding.go b/task/sharding.go index bce56d23..2e78e8f9 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -166,7 +166,7 @@ func (sh *Sharder) Flush(wg *sync.WaitGroup) { } batch.Wg.Add(1) sh.service.clickhouse.Send(batch) - rs := make(model.Rows, 0) + rs := make(model.Rows, 0, realSize) sh.msgBuf[i] = &rs } } diff --git a/task/task.go b/task/task.go index c4eecc1d..fc963a6a 100644 --- a/task/task.go +++ b/task/task.go @@ -214,59 +214,62 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { return nil } -func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) (row *model.Row) { - row1 := make(model.Row, 0) - row = &row1 +func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) (r *model.Row) { if service.idxSerID >= 0 { - var seriesID, mgmtID int64 - var labels []string // If some labels are not Prometheus native, ETL shall calculate and pass "__series_id" and "__mgmt_id". val := metric.GetInt64("__series_id", false) - seriesID = val.(int64) + seriesID := val.(int64) val = metric.GetInt64("__mgmt_id", false) - mgmtID = val.(int64) + mgmtID := val.(int64) + newSeries := service.clickhouse.AllowWriteSeries(seriesID, mgmtID) + rowcount := service.idxSerID + 1 // including __series_id + if newSeries { + rowcount += (service.numDims - service.idxSerID + 3) + } + + row := make(model.Row, 0, rowcount) for i := 0; i < service.idxSerID; i++ { - dim := service.dims[i] - val := model.GetValueByType(metric, dim) - *row = append(*row, val) + row = append(row, model.GetValueByType(metric, service.dims[i])) } - *row = append(*row, seriesID) // __series_id - newSeries := service.clickhouse.AllowWriteSeries(seriesID, mgmtID) + row = append(row, seriesID) // __series_id if newSeries { - *row = append(*row, mgmtID, nil) // __mgmt_id, labels + var labels []string + row = append(row, mgmtID, nil) // __mgmt_id, labels for i := service.idxSerID + 3; i < service.numDims; i++ { dim := service.dims[i] val := model.GetValueByType(metric, dim) - *row = append(*row, val) + row = append(row, val) if val != nil && dim.Type.Type == model.String && dim.Name != service.nameKey && dim.Name != "le" && (service.lblBlkList == nil || !service.lblBlkList.MatchString(dim.Name)) { // "labels" JSON excludes "le", so that "labels" can be used as group key for histogram queries. labelVal := val.(string) labels = append(labels, fmt.Sprintf(`%s: %s`, strconv.Quote(dim.Name), strconv.Quote(labelVal))) } } - (*row)[service.idxSerID+2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) + row[service.idxSerID+2] = fmt.Sprintf("{%s}", strings.Join(labels, ", ")) } + return &row } else { + row := make(model.Row, 0, len(service.dims)) for _, dim := range service.dims { if strings.HasPrefix(dim.Name, "__kafka") { if strings.HasSuffix(dim.Name, "_topic") { - *row = append(*row, msg.Topic) + row = append(row, msg.Topic) } else if strings.HasSuffix(dim.Name, "_partition") { - *row = append(*row, msg.Partition) + row = append(row, msg.Partition) } else if strings.HasSuffix(dim.Name, "_offset") { - *row = append(*row, msg.Offset) + row = append(row, msg.Offset) } else if strings.HasSuffix(dim.Name, "_key") { - *row = append(*row, string(msg.Key)) + row = append(row, string(msg.Key)) } else if strings.HasSuffix(dim.Name, "_timestamp") { - *row = append(*row, *msg.Timestamp) + row = append(row, *msg.Timestamp) } else { - *row = append(*row, nil) + row = append(row, nil) } } else { val := model.GetValueByType(metric, dim) - *row = append(*row, val) + row = append(row, val) } } + return &row } - return } From 55c6db019399ef7f4b930d802f670a96dc827ce2 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Mon, 6 Mar 2023 11:19:51 +0800 Subject: [PATCH 321/404] Combine nacos log into sinker log --- clickhouse_sinker-dashboard.json | 1554 ++++++++++++++++++++++++++++-- cmd/clickhouse_sinker/main.go | 2 +- config/config.go | 2 +- config_manager/nacos.go | 23 +- task/sinker.go | 4 + 5 files changed, 1511 insertions(+), 74 deletions(-) diff --git a/clickhouse_sinker-dashboard.json b/clickhouse_sinker-dashboard.json index 93a079fb..d231efd5 100644 --- a/clickhouse_sinker-dashboard.json +++ b/clickhouse_sinker-dashboard.json @@ -40,7 +40,7 @@ "panels": [], "repeat": "datasource", "repeatDirection": "h", - "title": "Global", + "title": "Clickhouse-sinker", "type": "row" }, { @@ -67,7 +67,10 @@ "tooltip": false, "viz": false }, - "lineInterpolation": "linear", + "lineInterpolation": "smooth", + "lineStyle": { + "fill": "solid" + }, "lineWidth": 1, "pointSize": 5, "scaleDistribution": { @@ -102,7 +105,7 @@ "overrides": [] }, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, "y": 1 @@ -127,7 +130,7 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "sum(sum by(task)(rate(clickhouse_sinker_consume_msgs_total[1m])))", + "expr": "sum(sum by(task)(rate(clickhouse_sinker_consume_msgs_total{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}[1m])))", "interval": "", "legendFormat": "", "range": true, @@ -196,12 +199,12 @@ "overrides": [] }, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, "y": 1 }, - "id": 16, + "id": 12, "options": { "legend": { "calcs": [], @@ -221,14 +224,14 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "sum(sum by(task) (rate(clickhouse_sinker_flush_msgs_total[1m])))", + "expr": "sum(sum by(task) (clickhouse_sinker_writing_pool_backlog{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}))", "interval": "", "legendFormat": "", "range": true, "refId": "A" } ], - "title": "clickhouse_sinker_flush_msgs_total rate", + "title": "clickhouse_sinker_writing_pool_backlog", "type": "timeseries" }, { @@ -290,12 +293,12 @@ "overrides": [] }, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 9 + "y": 8 }, - "id": 12, + "id": 16, "options": { "legend": { "calcs": [], @@ -315,14 +318,14 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "sum(sum by(task) (clickhouse_sinker_writing_pool_backlog))", + "expr": "sum(sum by(task) (rate(clickhouse_sinker_flush_msgs_total{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}[1m])))", "interval": "", "legendFormat": "", "range": true, "refId": "A" } ], - "title": "clickhouse_sinker_writing_pool_backlog", + "title": "clickhouse_sinker_flush_msgs_total rate", "type": "timeseries" }, { @@ -384,10 +387,10 @@ "overrides": [] }, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 9 + "y": 8 }, "id": 10, "options": { @@ -409,7 +412,7 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "sum(sum by(task) (clickhouse_sinker_shard_msgs))", + "expr": "sum(sum by(task) (clickhouse_sinker_shard_msgs{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}))", "interval": "", "legendFormat": "", "range": true, @@ -478,10 +481,10 @@ "overrides": [] }, "gridPos": { - "h": 8, + "h": 5, "w": 12, "x": 0, - "y": 17 + "y": 15 }, "id": 8, "options": { @@ -503,7 +506,7 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "sum by(task)(rate(clickhouse_sinker_parse_msgs_error_total[1m]))", + "expr": "sum by(task)(rate(clickhouse_sinker_parse_msgs_error_total{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}[1m]))", "interval": "", "legendFormat": "", "range": true, @@ -572,10 +575,10 @@ "overrides": [] }, "gridPos": { - "h": 8, + "h": 5, "w": 12, "x": 12, - "y": 17 + "y": 15 }, "id": 14, "options": { @@ -597,7 +600,7 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "sum(sum by(task)(rate(clickhouse_sinker_flush_msgs_error_total[1m])))", + "expr": "sum(sum by(task)(rate(clickhouse_sinker_flush_msgs_error_total{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}[1m])))", "interval": "", "legendFormat": "", "range": true, @@ -614,10 +617,8 @@ }, "fieldConfig": { "defaults": { - "custom": { - "align": "auto", - "displayMode": "auto", - "inspect": false + "color": { + "mode": "thresholds" }, "mappings": [], "thresholds": { @@ -637,22 +638,28 @@ "overrides": [] }, "gridPos": { - "h": 9, - "w": 24, + "h": 4, + "w": 12, "x": 0, - "y": 25 + "y": 20 }, "id": 20, "options": { - "footer": { - "fields": "", - "reducer": [ - "sum" + "colorMode": "value", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" ], - "show": false + "fields": "", + "values": false }, - "frameIndex": 7, - "showHeader": true + "text": { + "titleSize": 4 + }, + "textMode": "value_and_name" }, "pluginVersion": "9.1.2", "targets": [ @@ -662,39 +669,1458 @@ "uid": "${datasource}" }, "editorMode": "code", - "expr": "clickhouse_sinker_consume_offsets", + "expr": " {instance=~\"${instance}\",job=~\"${job}\",topic=~\"${topic}\"}", "legendFormat": "__auto", "range": true, "refId": "A" } ], "title": "committed offset", - "type": "table" - } - ], - "refresh": "5s", - "schemaVersion": 37, - "style": "dark", - "tags": [], - "templating": { - "list": [ - { - "current": { - "selected": false, - "text": "Prometheus-12", - "value": "Prometheus-12" + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } }, - "hide": 0, - "includeAll": false, - "multi": false, - "name": "datasource", - "options": [], - "query": "prometheus", - "queryValue": "", - "refresh": 1, - "regex": "", - "skipUrlSync": false, - "type": "datasource" + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 20 + }, + "id": 22, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "9.1.2", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "clickhouse_sinker_consume_lags{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}", + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "clickhouse_sinker_consume_lags", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "{__name__=\"clickhouse_sinker_write_series_allow_changed\", instance=\"192.168.110.14:8999\", job=\"ck_sinker14\", task=\"sink-metric\"}" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 24, + "x": 0, + "y": 24 + }, + "id": 26, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "clickhouse_sinker_write_series_allow_changed{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}", + "legendFormat": "__auto", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "clickhouse_sinker_write_series_allow_new{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "clickhouse_sinker_write_series_drop_quota{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "clickhouse_sinker_write_series_drop_unchanged{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "D" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "clickhouse_sinker_write_series_succeed{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "E" + } + ], + "title": "clickhouse_sinker_write_series_*", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "custom": { + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "scaleDistribution": { + "type": "linear" + } + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 34 + }, + "id": 24, + "options": { + "calculate": true, + "calculation": { + "xBuckets": { + "mode": "size" + }, + "yBuckets": { + "mode": "count", + "scale": { + "log": 2, + "type": "log" + } + } + }, + "cellGap": 1, + "color": { + "exponent": 0.5, + "fill": "dark-orange", + "mode": "scheme", + "scale": "exponential", + "scheme": "Turbo", + "steps": 128 + }, + "exemplars": { + "color": "rgba(255,0,255,0.7)" + }, + "filterValues": { + "le": 1e-9 + }, + "legend": { + "show": true + }, + "rowsFrame": { + "layout": "auto" + }, + "tooltip": { + "show": true, + "yHistogram": false + }, + "yAxis": { + "axisPlacement": "left", + "decimals": 0, + "reverse": false, + "unit": "ms" + } + }, + "pluginVersion": "9.1.2", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "ceil(increase(clickhouse_sinker_writing_durations_bucket{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\",le!=\"+Inf\"}[1m]))", + "legendFormat": "{{le}}", + "range": true, + "refId": "A" + } + ], + "title": "clickhouse_sinker_writing_durations_bucket", + "type": "heatmap" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ms" + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "{instance=\"192.168.110.14:8999\", job=\"ck_sinker14\", table=\"prom_metric_nopool\", task=\"sink-metric\"}" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 34 + }, + "id": 56, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "rate(clickhouse_sinker_writing_durations_sum{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}[1m]) / rate(clickhouse_sinker_writing_durations_count{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}[1m])", + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "clickhouse_sinker_writing_durations_average", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 41 + }, + "id": 36, + "panels": [], + "title": "Process_*", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "{__name__=\"process_resident_memory_bytes\", instance=\"192.168.110.14:8999\", job=\"ck_sinker14\"}" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 42 + }, + "id": 40, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.1.2", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "process_resident_memory_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "legendFormat": "__auto", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "process_virtual_memory_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "B" + } + ], + "title": "process_resident_memory_bytes", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "percent" + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 42 + }, + "id": 38, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "rate(process_cpu_seconds_total{instance=~\"${instance}\",job=~\"${job}\"}[1m])*100", + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "process_cpu_seconds_total", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 49 + }, + "id": 44, + "panels": [], + "title": "Go_*", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "{__name__=\"go_memstats_alloc_bytes\", instance=\"192.168.110.14:8999\", job=\"ck_sinker14\"}" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 50 + }, + "id": 48, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_alloc_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "legendFormat": "__auto", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_alloc_bytes_total{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "B" + } + ], + "title": "go_memstats_alloc_bytes", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 50 + }, + "id": 46, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_goroutines{instance=~\"${instance}\",job=~\"${job}\"}", + "legendFormat": "__auto", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_threads{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "B" + } + ], + "title": "go_parallelism", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "{__name__=\"go_memstats_heap_inuse_bytes\", instance=\"192.168.110.14:8999\", job=\"ck_sinker14\"}" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 58 + }, + "id": 50, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_heap_alloc_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "legendFormat": "__auto", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_heap_idle_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_heap_inuse_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_heap_released_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "D" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_heap_sys_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "E" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_heap_inuse_bytes{instance=~\"${instance}\",job=~\"${job}\"} + go_memstats_heap_idle_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "F" + } + ], + "title": "go_memstats_heap_*_bytes", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 58 + }, + "id": 52, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_stack_inuse_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "legendFormat": "__auto", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_stack_sys_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "B" + } + ], + "title": "go_memstats_stack_*_bytes", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 66 + }, + "id": 54, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_gc_sys_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "legendFormat": "__auto", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "editorMode": "code", + "expr": "go_memstats_next_gc_bytes{instance=~\"${instance}\",job=~\"${job}\"}", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "B" + } + ], + "title": "go_memstats_gc_*_bytes", + "type": "timeseries" + } + ], + "refresh": "5s", + "schemaVersion": 37, + "style": "dark", + "tags": [], + "templating": { + "list": [ + { + "current": { + "selected": false, + "text": "Prometheus-12", + "value": "Prometheus-12" + }, + "hide": 0, + "includeAll": false, + "multi": false, + "name": "datasource", + "options": [], + "query": "prometheus", + "queryValue": "", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "type": "datasource" + }, + { + "current": { + "selected": false, + "text": "192.168.110.14:8999", + "value": "192.168.110.14:8999" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values(instance)", + "hide": 0, + "includeAll": true, + "label": "instance", + "multi": false, + "name": "instance", + "options": [], + "query": { + "query": "label_values(instance)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 0, + "type": "query" + }, + { + "current": { + "selected": false, + "text": "ck_sinker14", + "value": "ck_sinker14" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values(job)", + "hide": 0, + "includeAll": true, + "label": "job", + "multi": false, + "name": "job", + "options": [], + "query": { + "query": "label_values(job)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 0, + "type": "query" + }, + { + "current": { + "selected": false, + "text": "sink-metric", + "value": "sink-metric" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values(task)", + "hide": 0, + "includeAll": true, + "label": "task", + "multi": false, + "name": "task", + "options": [], + "query": { + "query": "label_values(task)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 0, + "type": "query" + }, + { + "current": { + "selected": false, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values(topic)", + "hide": 0, + "includeAll": true, + "label": "topic", + "multi": false, + "name": "topic", + "options": [], + "query": { + "query": "label_values(topic)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 0, + "type": "query" } ] }, @@ -719,6 +2145,6 @@ "timezone": "", "title": "clickhouse_sinker_new", "uid": "8f73nucGa", - "version": 3, + "version": 29, "weekStart": "" -} \ No newline at end of file +} diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 48c6c7cd..04aefa9b 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -100,7 +100,7 @@ func initCmdOptions() { flag.StringVar(&cmdOps.NacosUsername, "nacos-username", cmdOps.NacosUsername, "nacos username") flag.StringVar(&cmdOps.NacosPassword, "nacos-password", cmdOps.NacosPassword, "nacos password") flag.StringVar(&cmdOps.NacosNamespaceID, "nacos-namespace-id", cmdOps.NacosNamespaceID, - `nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work!`) + `nacos namespace ID. Neither DEFAULT_NAMESPACE_ID("public") nor namespace name work! When namespace is 'public', fill in the blank string here!`) flag.StringVar(&cmdOps.NacosGroup, "nacos-group", cmdOps.NacosGroup, `nacos group name. Empty string doesn't work!`) flag.StringVar(&cmdOps.NacosDataID, "nacos-dataid", cmdOps.NacosDataID, "nacos dataid") flag.StringVar(&cmdOps.NacosServiceName, "nacos-service-name", cmdOps.NacosServiceName, "nacos service name") diff --git a/config/config.go b/config/config.go index 5989c2c2..acb4b960 100644 --- a/config/config.go +++ b/config/config.go @@ -202,7 +202,7 @@ func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { // Normalize and validate configuration func (cfg *Config) Normallize(constructGroup bool, httpAddr string) (err error) { if len(cfg.Clickhouse.Hosts) == 0 || cfg.Kafka.Brokers == "" { - err = errors.Newf("invalid configuration") + err = errors.Newf("invalid configuration, Clickhouse or Kafka section is missing!") return } diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 370eda54..78605145 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -23,7 +23,6 @@ import ( "github.com/nacos-group/nacos-sdk-go/vo" "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" - "gopkg.in/natefinch/lumberjack.v2" ) var _ RemoteConfManager = (*NacosConfManager)(nil) @@ -88,16 +87,10 @@ func (ncm *NacosConfManager) Init(properties map[string]interface{}) (err error) TimeoutMs: 5000, ListenInterval: 10000, NotLoadCacheAtStart: true, - LogDir: filepath.Join(logDir, "nacos_log"), CacheDir: filepath.Join(logDir, "nacos_cache"), - LogLevel: "debug", + CustomLogger: &NacosLogger{util.Logger.Sugar()}, Username: properties["username"].(string), Password: properties["password"].(string), - LogRollingConfig: &lumberjack.Logger{ - MaxSize: 10, // megabytes - MaxBackups: 1, - LocalTime: true, - }, } ncm.configClient, err = clients.CreateConfigClient(map[string]interface{}{ @@ -433,3 +426,17 @@ func (ncm *NacosConfManager) assign() (err error) { return } + +type NacosLogger struct { + *zap.SugaredLogger +} + +// override the loglevel to avoid log blow up +func (nlog *NacosLogger) Info(args ...interface{}) { + nlog.Debug(args...) +} + +// override the loglevel to avoid log blow up +func (nlog *NacosLogger) Infof(fmt string, args ...interface{}) { + nlog.Debugf(fmt, args...) +} diff --git a/task/sinker.go b/task/sinker.go index f9ecb152..8a3337bd 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -253,6 +253,10 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { err = s.applyAnotherConfig(newCfg) } + + if len(s.consumers) == 0 && s.cmdOps.NacosServiceName != "" { + util.Logger.Warn("No task fetched from Nacos, make sure the program is running with correct commandline option!") + } return } From f4eb186043de3bb9e4226d66e71e6d5e8af9cfc7 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Wed, 8 Mar 2023 14:01:25 +0800 Subject: [PATCH 322/404] update dmseries quota map when applying new config, retry query when facing ErrAcquireConnTimeout error --- go.mod | 4 +- model/metric.go | 8 ++ output/clickhouse.go | 155 ++++++++++---------------------------- output/clickhouse_util.go | 51 +++++++++---- task/consumer.go | 3 + task/sinker.go | 155 +++++++++++++++++++++++++++++++++----- 6 files changed, 227 insertions(+), 149 deletions(-) diff --git a/go.mod b/go.mod index dff2919d..06de1167 100644 --- a/go.mod +++ b/go.mod @@ -8,10 +8,12 @@ require ( github.com/bytedance/sonic v1.8.1 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.26 + github.com/google/uuid v1.3.0 github.com/hjson/hjson-go/v4 v4.3.0 github.com/jcmturner/gokrb5/v8 v8.4.3 github.com/jinzhu/copier v0.3.5 github.com/nacos-group/nacos-sdk-go v1.1.4 + github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.14.0 github.com/prometheus/common v0.40.0 github.com/shopspring/decimal v1.3.1 @@ -43,7 +45,6 @@ require ( github.com/go-faster/city v1.0.1 // indirect github.com/go-faster/errors v0.6.1 // indirect github.com/golang/protobuf v1.5.2 // indirect - github.com/google/uuid v1.3.0 // indirect github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/jcmturner/aescts/v2 v2.0.0 // indirect github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect @@ -61,7 +62,6 @@ require ( github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b // indirect github.com/paulmach/orb v0.9.0 // indirect github.com/pierrec/lz4/v4 v4.1.17 // indirect - github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_model v0.3.0 // indirect github.com/prometheus/procfs v0.9.0 // indirect diff --git a/model/metric.go b/model/metric.go index c364e1c4..aeb5a67a 100644 --- a/model/metric.go +++ b/model/metric.go @@ -18,6 +18,7 @@ package model import ( "regexp" "sync" + "time" ) // Metric interface for metric collection @@ -98,3 +99,10 @@ func NewOrderedMap() *OrderedMap { om.values = map[interface{}]interface{}{} return &om } + +type SeriesQuota struct { + sync.Mutex + NextResetQuota time.Time + BmSeries map[int64]int64 + WrSeries int +} diff --git a/output/clickhouse.go b/output/clickhouse.go index 35ae3e82..30ecb21a 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -19,7 +19,6 @@ import ( "context" "fmt" "math" - "sort" "strings" "sync" "time" @@ -49,16 +48,8 @@ var ( // zooKeeper Session expired issue: https://cwiki.apache.org/confluence/display/ZOOKEEPER/FAQ#:~:text=How%20should%20I%20handle%20SESSION_EXPIRED%3F replicaSpecificErrorCodes = []int32{225, 242, 252, 319, 999, 1000} //NO_ZOOKEEPER, TABLE_IS_READ_ONLY, TOO_MANY_PARTS, UNKNOWN_STATUS_OF_INSERT, KEEPER_EXCEPTION, POCO_EXCEPTION wrSeriesQuota = 16384 - seriesQuotas sync.Map ) -type seriesQuota struct { - sync.Mutex - nextResetQuota time.Time - bmSeries map[int64]int64 - wrSeries int -} - // ClickHouse is an output service consumers from kafka messages type ClickHouse struct { Dims []*model.ColumnWithType @@ -77,7 +68,7 @@ type ClickHouse struct { distMetricTbls []string distSeriesTbls []string - seriesQuota *seriesQuota + seriesQuota *model.SeriesQuota numFlying int32 mux sync.Mutex @@ -133,19 +124,19 @@ func (c *ClickHouse) Send(batch *model.Batch) { func (c *ClickHouse) AllowWriteSeries(sid, mid int64) (allowed bool) { c.seriesQuota.Lock() defer c.seriesQuota.Unlock() - mid2, loaded := c.seriesQuota.bmSeries[sid] + mid2, loaded := c.seriesQuota.BmSeries[sid] if !loaded { allowed = true statistics.WriteSeriesAllowNew.WithLabelValues(c.taskCfg.Name).Inc() } else if mid != mid2 { - if c.seriesQuota.wrSeries < wrSeriesQuota { - c.seriesQuota.wrSeries++ + if c.seriesQuota.WrSeries < wrSeriesQuota { + c.seriesQuota.WrSeries++ allowed = true } else { now := time.Now() - if now.After(c.seriesQuota.nextResetQuota) { - c.seriesQuota.nextResetQuota = now.Add(10 * time.Second) - c.seriesQuota.wrSeries = 1 + if now.After(c.seriesQuota.NextResetQuota) { + c.seriesQuota.NextResetQuota = now.Add(10 * time.Second) + c.seriesQuota.WrSeries = 1 allowed = true } } @@ -179,10 +170,10 @@ func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err err for _, row := range seriesRows { sid := (*row)[c.IdxSerID].(int64) mid := (*row)[c.IdxSerID+1].(int64) - if _, loaded := c.seriesQuota.bmSeries[sid]; loaded { - c.seriesQuota.wrSeries-- + if _, loaded := c.seriesQuota.BmSeries[sid]; loaded { + c.seriesQuota.WrSeries-- } - c.seriesQuota.bmSeries[sid] = mid + c.seriesQuota.BmSeries[sid] = mid } c.seriesQuota.Unlock() util.Logger.Info("ClickHouse.writeSeries succeeded", zap.Int("series", len(seriesRows)), zap.String("task", c.taskCfg.Name)) @@ -256,55 +247,6 @@ func (c *ClickHouse) loopWrite(batch *model.Batch) { } } -func (c *ClickHouse) initBmSeries(conn clickhouse.Conn) (err error) { - tbl := c.seriesTbl - if c.cfg.Clickhouse.Cluster != "" { - tbl = c.distSeriesTbls[0] - } - - var count uint64 - conn.QueryRow(context.Background(), fmt.Sprintf("SELECT count() FROM %s.%s FINAL", c.dbName, tbl)).Scan(&count) - sq := &seriesQuota{} - if v, ok := seriesQuotas.LoadOrStore(c.GetSeriesQuotaKey(), sq); ok { - c.seriesQuota = v.(*seriesQuota) - c.seriesQuota.Lock() - defer c.seriesQuota.Unlock() - if len(c.seriesQuota.bmSeries) == int(count) { - // only reload the map when there is difference detected - return - } - } else { - sq.Lock() - sq.bmSeries = make(map[int64]int64, count) - sq.nextResetQuota = time.Now().Add(10 * time.Second) - sq.Unlock() - c.seriesQuota = sq - } - - query := fmt.Sprintf("SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s.%s FINAL ORDER BY sid", c.dbName, tbl) - util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", c.taskCfg.Name)) - var rs driver.Rows - var seriesID, mgmtID int64 - if rs, err = conn.Query(context.Background(), query); err != nil { - err = errors.Wrapf(err, "") - return err - } - defer rs.Close() - - c.seriesQuota.Lock() - defer c.seriesQuota.Unlock() - - for rs.Next() { - if err = rs.Scan(&seriesID, &mgmtID); err != nil { - err = errors.Wrapf(err, "") - return err - } - c.seriesQuota.bmSeries[seriesID] = mgmtID - } - util.Logger.Info(fmt.Sprintf("loaded %d series from %v", len(c.seriesQuota.bmSeries), tbl), zap.String("task", c.taskCfg.Name)) - return -} - func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { if !c.taskCfg.PrometheusSchema { c.IdxSerID = -1 @@ -395,10 +337,6 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { } } - // Initialize bmSeries - if err = c.initBmSeries(conn); err != nil { - return - } return } @@ -464,7 +402,6 @@ func (c *ClickHouse) initSchema() (err error) { } func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { - var queries []string var onCluster string taskCfg := c.taskCfg chCfg := &c.cfg.Clickhouse @@ -480,8 +417,9 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { util.Logger.Warn("number of columns reaches upper limit", zap.Int("limit", maxDims), zap.Int("current", len(c.Dims))) return } + var i int - var affectDistMetric, affectDistSeries bool + var alterSeries, alterMetric string newKeys.Range(func(key, value interface{}) bool { i++ if i > newKeysQuota { @@ -509,47 +447,39 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return false } if c.taskCfg.PrometheusSchema && intVal == model.String { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.dbName, c.seriesTbl, onCluster, strKey, strVal) - queries = append(queries, query) - affectDistSeries = true + alterSeries += fmt.Sprintf("ADD COLUMN IF NOT EXISTS `%s` %s, ", strKey, strVal) } else { if c.taskCfg.PrometheusSchema && intVal > model.String { util.Logger.Fatal("unsupported metric value type", zap.String("type", strVal), zap.String("name", strKey), zap.String("task", c.taskCfg.Name)) } - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s ADD COLUMN IF NOT EXISTS `%s` %s", c.dbName, c.TableName, onCluster, strKey, strVal) - queries = append(queries, query) - affectDistMetric = true + alterMetric += fmt.Sprintf("ADD COLUMN IF NOT EXISTS `%s` %s, ", strKey, strVal) } return true }) if err != nil { return } - sort.Strings(queries) + sc := pool.GetShardConn(0) var conn clickhouse.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } - for _, query := range queries { - util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) - if err = conn.Exec(context.Background(), query); err != nil { - err = errors.Wrapf(err, query) - return - } + + fields := []zap.Field{zap.String("task", taskCfg.Name)} + if alterSeries != "" { + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, c.seriesTbl, onCluster, alterSeries[:len(alterSeries)-2]) + Execute(conn, context.Background(), EXEC, query, fields) + query = fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, c.distSeriesTbls[0], onCluster, alterSeries[:len(alterSeries)-2]) + Execute(conn, context.Background(), EXEC, query, fields) } - if chCfg.Cluster != "" { - if affectDistMetric { - if err = recreateDistTbls(chCfg.Cluster, c.dbName, c.TableName, c.distMetricTbls, conn); err != nil { - return - } - } - if affectDistSeries { - if err = recreateDistTbls(chCfg.Cluster, c.dbName, c.seriesTbl, c.distSeriesTbls, conn); err != nil { - return - } - } + if alterMetric != "" { + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, c.TableName, onCluster, alterSeries[:len(alterSeries)-2]) + Execute(conn, context.Background(), EXEC, query, fields) + query = fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, c.distMetricTbls[0], onCluster, alterSeries[:len(alterSeries)-2]) + Execute(conn, context.Background(), EXEC, query, fields) } + return } @@ -563,13 +493,7 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { } query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed\(\'%s\', \'%s\', \'%s\'.*\)')`, c.dbName, chCfg.Cluster, c.dbName, table) - util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) - - var rows driver.Rows - if rows, err = conn.Query(context.Background(), query); err != nil { - err = errors.Wrapf(err, "") - return - } + rows := Execute(conn, context.Background(), QUERY, query, []zap.Field{zap.String("task", taskCfg.Name)}).(driver.Rows) defer rows.Close() for rows.Next() { var name string @@ -582,19 +506,20 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { return } -func UpdateSeriesQuotas(c map[string]struct{}) { - seriesQuotas.Range(func(key, value any) bool { - k := key.(string) - if _, ok := c[k]; !ok { - seriesQuotas.Delete(k) - } - return true - }) +func (c *ClickHouse) GetSeriesQuotaKey() string { + if c.taskCfg.PrometheusSchema { + return c.dbName + "." + c.distSeriesTbls[0] + } + return "" } -func (c *ClickHouse) GetSeriesQuotaKey() string { +func (c *ClickHouse) GetDistMetricTable() string { if c.taskCfg.PrometheusSchema { - return c.dbName + "." + c.seriesTbl + return c.distMetricTbls[0] } return "" } + +func (c *ClickHouse) SetSeriesQuota(sq *model.SeriesQuota) { + c.seriesQuota = sq +} diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index d213609a..a2da7e93 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -3,6 +3,7 @@ package output import ( "context" "fmt" + "time" "github.com/ClickHouse/clickhouse-go/v2" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" @@ -104,20 +105,44 @@ func getDims(database, table string, excludedColumns []string, parser string, co return } -func recreateDistTbls(cluster, database, table string, distTbls []string, conn clickhouse.Conn) (err error) { - var queries []string - for _, distTbl := range distTbls { - queries = append(queries, fmt.Sprintf("DROP TABLE IF EXISTS `%s`.`%s` ON CLUSTER `%s` SYNC", database, distTbl, cluster)) - queries = append(queries, fmt.Sprintf("CREATE TABLE IF NOT EXISTS `%s`.`%s` ON CLUSTER `%s` AS `%s`.`%s` ENGINE = Distributed(`%s`, `%s`, `%s`);", - database, distTbl, cluster, database, table, - cluster, database, table)) - } - for _, query := range queries { - util.Logger.Info(fmt.Sprintf("executing sql=> %s", query)) - if err = conn.Exec(context.Background(), query); err != nil { - err = errors.Wrapf(err, "") +const ( + EXEC = iota + QUERYROW + QUERY +) + +// LoopWrite will dead loop to write the records +func Execute(conn driver.Conn, ctx context.Context, querykind int, query string, fields []zap.Field, args ...interface{}) (result interface{}) { + var times int + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), fields...) + var err error + + for { + zapFields := fields + switch querykind { + case EXEC: + err = conn.Exec(ctx, query, args...) + case QUERYROW: + result = conn.QueryRow(ctx, query, args...) + case QUERY: + result, err = conn.Query(ctx, query, args...) + default: + util.Logger.Fatal("unknown query kind specified") + } + + if err == nil { return + } else if errors.Is(err, context.Canceled) { + util.Logger.Info("Execute failed due to the context has been cancelled", fields...) + return + } else if errors.Is(err, clickhouse.ErrAcquireConnTimeout) { + time.Sleep(10 * time.Second) + } else { + zapFields = append(zapFields, zap.Error(err)) + util.Logger.Fatal("Query execution failed", zapFields...) } + zapFields = append(zapFields, zap.Int("try", times), zap.Error(err)) + util.Logger.Error("Query execution failed", zapFields...) + times++ } - return } diff --git a/task/consumer.go b/task/consumer.go index cd581094..58d85334 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -77,6 +77,9 @@ func (c *Consumer) addTask(tsk *Service) { } func (c *Consumer) start() { + if c.state.Load() == util.StateRunning { + return + } c.inputer = input.NewKafkaFranz() c.state.Store(util.StateRunning) if err := c.inputer.Init(c.sinker.curCfg, c.grpConfig, c.fetchesCh, c.cleanupFn); err == nil { diff --git a/task/sinker.go b/task/sinker.go index 8a3337bd..27f49a41 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -17,6 +17,7 @@ package task import ( "context" + "fmt" "os" "reflect" "sort" @@ -25,6 +26,10 @@ import ( "sync" "time" + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + + "github.com/google/uuid" "github.com/housepower/clickhouse_sinker/config" cm "github.com/housepower/clickhouse_sinker/config_manager" "github.com/housepower/clickhouse_sinker/model" @@ -32,9 +37,8 @@ import ( "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" + "github.com/pkg/errors" "go.uber.org/zap" - - _ "github.com/ClickHouse/clickhouse-go/v2" ) // Sinker object maintains number of task for each partition @@ -53,6 +57,8 @@ type Sinker struct { exitCh chan struct{} stopCommitCh chan struct{} consumerRestartCh chan *Consumer + + seriesQuotas sync.Map } // NewSinker get an instance of sinker with the task list @@ -283,10 +289,16 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { return } } - c.start() s.consumers[group] = c } + if err = s.loadBmSeries(); err != nil { + return + } + for _, c := range s.consumers { + c.start() + } + util.Logger.Info("applied the first config") return } @@ -296,8 +308,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) { // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). s.stopAllTasks() - // 1.1 Cleanup the SeriesQuotas when stopping tasks - output.UpdateSeriesQuotas(make(map[string]struct{})) + // 2. Initialize clickhouse connections. chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, @@ -324,10 +335,16 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { } tasksToStart = append(tasksToStart, tsk.Name) } - c.start() s.consumers[group] = c } + if err = s.loadBmSeries(); err != nil { + return + } + for _, c := range s.consumers { + c.start() + } + util.Logger.Info("started tasks", zap.Reflect("tasks", tasksToStart)) } else if !reflect.DeepEqual(newCfg.Tasks, s.curCfg.Tasks) || !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { // Find the consumer difference @@ -351,6 +368,10 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { for _, tCfg := range group.Configs { if !reflect.DeepEqual(tCfg, s.consumers[name].grpConfig.Configs[tCfg.Name]) { task := NewTaskService(newCfg, tCfg, s.consumers[name]) + if task.taskCfg.PrometheusSchema { + // stop the task for reloading the bmseries + s.consumers[name].stop() + } if err = task.Init(); err != nil { return } @@ -375,18 +396,6 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { delete(s.consumers, v) } wg.Wait() - // 1.1) Update the SeriesQuotas when stopping tasks - tables := make(map[string]struct{}) - for _, c := range s.consumers { - c.tasks.Range(func(key, value any) bool { - k := value.(*Service).clickhouse.GetSeriesQuotaKey() - if k != "" { - tables[k] = struct{}{} - } - return true - }) - } - output.UpdateSeriesQuotas(tables) // 2) fire up new consumers // Record the new config @@ -400,10 +409,16 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { return } } - c.start() s.consumers[v.Name] = c } } + + if err = s.loadBmSeries(); err != nil { + return + } + for _, c := range s.consumers { + c.start() + } } util.Logger.Info("applied another config", zap.Int("number", s.numCfg)) s.numCfg++ @@ -449,3 +464,105 @@ func (s *Sinker) commitFn() { } } } + +func (s *Sinker) loadBmSeries() (err error) { + // series table could be shared between multiple tasks + tables := make(map[string][]*Service) + for _, c := range s.consumers { + c.tasks.Range(func(key, value any) bool { + k := value.(*Service).clickhouse.GetSeriesQuotaKey() + if k != "" { + tables[k] = append(tables[k], value.(*Service)) + } + return true + }) + } + + // delete the seriesQuota which no longer being used + s.seriesQuotas.Range(func(key, value any) bool { + k := key.(string) + if _, ok := tables[k]; !ok { + s.seriesQuotas.Delete(k) + util.Logger.Info("dropping seriesQuota", zap.String("series table", k)) + } + return true + }) + + // initialize seriesQuota for new series tables + for k, v := range tables { + var query, dbname, mergetable string + var conn clickhouse.Conn + sq := &model.SeriesQuota{} + mysq := sq + if s, ok := s.seriesQuotas.LoadOrStore(k, sq); ok { + q := s.(*model.SeriesQuota) + mysq = q + for _, svc := range v { + svc.clickhouse.SetSeriesQuota(q) + } + mysq.Lock() + defer mysq.Unlock() + if len(mysq.BmSeries) > 0 { + // only reload the map when the map is empty + return + } + } else { + var count uint64 + var reg string + for _, svc := range v { + r := svc.clickhouse.GetDistMetricTable() + if r != "" { + reg += ("^" + r + "$|") + } + } + reg = reg[:len(reg)-1] + mergetable = strings.ReplaceAll(k+uuid.New().String(), "-", "_") + dbname = strings.Split(k, ".")[0] + + query = fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %s as %s ENGINE=Merge('%s', '%s')`, + mergetable, v[0].clickhouse.GetDistMetricTable(), dbname, reg) + + sc := pool.GetShardConn(0) + if conn, _, err = sc.NextGoodReplica(0); err != nil { + return + } + output.Execute(conn, context.Background(), output.EXEC, query, []zap.Field{zap.String("task", v[0].taskCfg.Name)}) + + query = fmt.Sprintf(`WITH (SELECT max(timestamp) FROM %s) AS m + SELECT count() FROM %s FINAL WHERE __series_id GLOBAL IN ( + SELECT DISTINCT __series_id FROM %s WHERE timestamp >= addDays(m, -1));`, mergetable, k, mergetable) + output.Execute(conn, context.Background(), output.QUERYROW, query, []zap.Field{zap.String("task", v[0].taskCfg.Name)}).(driver.Row).Scan(&count) + + mysq.Lock() + mysq.BmSeries = make(map[int64]int64, count) + mysq.NextResetQuota = time.Now().Add(10 * time.Second) + mysq.Unlock() + for _, svc := range v { + svc.clickhouse.SetSeriesQuota(mysq) + } + } + + query = fmt.Sprintf(`WITH (SELECT max(timestamp) FROM %s) AS m + SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s FINAL WHERE sid GLOBAL IN ( + SELECT DISTINCT toInt64(__series_id) FROM %s WHERE timestamp >= addDays(m, -1) + ) ORDER BY sid;`, mergetable, k, mergetable) + rs := output.Execute(conn, context.Background(), output.QUERY, query, []zap.Field{zap.String("task", v[0].taskCfg.Name)}).(driver.Rows) + defer rs.Close() + + mysq.Lock() + defer mysq.Unlock() + var seriesID, mgmtID int64 + for rs.Next() { + if err = rs.Scan(&seriesID, &mgmtID); err != nil { + return errors.Wrapf(err, "") + } + mysq.BmSeries[seriesID] = mgmtID + } + util.Logger.Info(fmt.Sprintf("loaded %d series from %v", len(mysq.BmSeries), k)) + + query = fmt.Sprintf(`DROP TABLE IF EXISTS %s `, mergetable) + output.Execute(conn, context.Background(), output.EXEC, query, []zap.Field{zap.String("task", v[0].taskCfg.Name)}) + } + + return +} From 57d39c42267899ac0d441167fb545f3ae3466da3 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Wed, 8 Mar 2023 18:21:09 +0800 Subject: [PATCH 323/404] reload bmseries every day to offload out-of-date record --- CHANGELOG.md | 10 ++ docker/test_prom_metric.hjson | 1 + go.mod | 2 +- model/metric.go | 1 + output/clickhouse.go | 79 +++++++++--- output/clickhouse_util.go | 37 ------ task/sinker.go | 222 ++++++++++++++++++++++------------ util/common.go | 22 ++++ 8 files changed, 237 insertions(+), 137 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 93c50607..ecf4139a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,14 @@ # Changelog +#### Version 3.0.2 (2023-03-10) + +Improvements: +- update sinker grafana dashboard +- combine nacos log into sinker log +- update dmseries map when applying new config, reload the records from series table every single day +- avoid recreating dist tables, alter the table schema instead + + #### Version 3.0.1 (2023-03-03) Bug Fixes: @@ -13,6 +22,7 @@ Improvements: - Terminate program immediately when receiving one more exit signal - Limit the fetch size and poll size based on the BufferSize config property + #### Version 3.0.0 (2023-02-07) New Features: diff --git a/docker/test_prom_metric.hjson b/docker/test_prom_metric.hjson index 291247ff..0aa7b45c 100644 --- a/docker/test_prom_metric.hjson +++ b/docker/test_prom_metric.hjson @@ -1,5 +1,6 @@ { clickhouse: { + cluster: abc hosts: [ [ 127.0.0.1 diff --git a/go.mod b/go.mod index 06de1167..ba4049ea 100644 --- a/go.mod +++ b/go.mod @@ -13,7 +13,6 @@ require ( github.com/jcmturner/gokrb5/v8 v8.4.3 github.com/jinzhu/copier v0.3.5 github.com/nacos-group/nacos-sdk-go v1.1.4 - github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.14.0 github.com/prometheus/common v0.40.0 github.com/shopspring/decimal v1.3.1 @@ -62,6 +61,7 @@ require ( github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b // indirect github.com/paulmach/orb v0.9.0 // indirect github.com/pierrec/lz4/v4 v4.1.17 // indirect + github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_model v0.3.0 // indirect github.com/prometheus/procfs v0.9.0 // indirect diff --git a/model/metric.go b/model/metric.go index aeb5a67a..6ef3e20d 100644 --- a/model/metric.go +++ b/model/metric.go @@ -105,4 +105,5 @@ type SeriesQuota struct { NextResetQuota time.Time BmSeries map[int64]int64 WrSeries int + Birth time.Time } diff --git a/output/clickhouse.go b/output/clickhouse.go index 30ecb21a..b8813d71 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -19,13 +19,13 @@ import ( "context" "fmt" "math" + "sort" "strings" "sync" "time" - "github.com/ClickHouse/clickhouse-go/v2/lib/driver" - "github.com/ClickHouse/clickhouse-go/v2" + "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" @@ -48,6 +48,8 @@ var ( // zooKeeper Session expired issue: https://cwiki.apache.org/confluence/display/ZOOKEEPER/FAQ#:~:text=How%20should%20I%20handle%20SESSION_EXPIRED%3F replicaSpecificErrorCodes = []int32{225, 242, 252, 319, 999, 1000} //NO_ZOOKEEPER, TABLE_IS_READ_ONLY, TOO_MANY_PARTS, UNKNOWN_STATUS_OF_INSERT, KEEPER_EXCEPTION, POCO_EXCEPTION wrSeriesQuota = 16384 + + SeriesQuotas sync.Map ) // ClickHouse is an output service consumers from kafka messages @@ -337,6 +339,13 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { } } + sq, _ := SeriesQuotas.LoadOrStore(c.GetSeriesQuotaKey(), + &model.SeriesQuota{ + NextResetQuota: time.Now().Add(10 * time.Second), + Birth: time.Now(), + }) + c.seriesQuota = sq.(*model.SeriesQuota) + return } @@ -419,7 +428,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } var i int - var alterSeries, alterMetric string + var alterSeries, alterMetric []string newKeys.Range(func(key, value interface{}) bool { i++ if i > newKeysQuota { @@ -447,12 +456,12 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return false } if c.taskCfg.PrometheusSchema && intVal == model.String { - alterSeries += fmt.Sprintf("ADD COLUMN IF NOT EXISTS `%s` %s, ", strKey, strVal) + alterSeries = append(alterSeries, fmt.Sprintf("ADD COLUMN IF NOT EXISTS `%s` %s", strKey, strVal)) } else { if c.taskCfg.PrometheusSchema && intVal > model.String { util.Logger.Fatal("unsupported metric value type", zap.String("type", strVal), zap.String("name", strKey), zap.String("task", c.taskCfg.Name)) } - alterMetric += fmt.Sprintf("ADD COLUMN IF NOT EXISTS `%s` %s, ", strKey, strVal) + alterMetric = append(alterMetric, fmt.Sprintf("ADD COLUMN IF NOT EXISTS `%s` %s", strKey, strVal)) } return true }) @@ -466,18 +475,35 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return } - fields := []zap.Field{zap.String("task", taskCfg.Name)} - if alterSeries != "" { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, c.seriesTbl, onCluster, alterSeries[:len(alterSeries)-2]) - Execute(conn, context.Background(), EXEC, query, fields) - query = fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, c.distSeriesTbls[0], onCluster, alterSeries[:len(alterSeries)-2]) - Execute(conn, context.Background(), EXEC, query, fields) + alterTable := func(tbl, col string) error { + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, tbl, onCluster, col) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) + return util.RetryOperation(func() error { return conn.Exec(context.Background(), query) }, -1, []error{clickhouse.ErrAcquireConnTimeout}) } - if alterMetric != "" { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, c.TableName, onCluster, alterSeries[:len(alterSeries)-2]) - Execute(conn, context.Background(), EXEC, query, fields) - query = fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, c.distMetricTbls[0], onCluster, alterSeries[:len(alterSeries)-2]) - Execute(conn, context.Background(), EXEC, query, fields) + + if len(alterSeries) != 0 { + sort.Strings(alterSeries) + columns := strings.Join(alterSeries, ",") + if err = alterTable(c.seriesTbl, columns); err != nil { + return err + } + for _, distTbl := range c.distSeriesTbls { + if err = alterTable(distTbl, columns); err != nil { + return err + } + } + } + if len(alterMetric) != 0 { + sort.Strings(alterMetric) + columns := strings.Join(alterMetric, ",") + if err = alterTable(c.TableName, columns); err != nil { + return err + } + for _, distTbl := range c.distMetricTbls { + if err = alterTable(distTbl, columns); err != nil { + return err + } + } } return @@ -493,7 +519,12 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { } query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed\(\'%s\', \'%s\', \'%s\'.*\)')`, c.dbName, chCfg.Cluster, c.dbName, table) - rows := Execute(conn, context.Background(), QUERY, query, []zap.Field{zap.String("task", taskCfg.Name)}).(driver.Rows) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) + var rows driver.Rows + if rows, err = conn.Query(context.Background(), query); err != nil { + err = errors.Wrapf(err, "") + return + } defer rows.Close() for rows.Next() { var name string @@ -508,14 +539,22 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { func (c *ClickHouse) GetSeriesQuotaKey() string { if c.taskCfg.PrometheusSchema { - return c.dbName + "." + c.distSeriesTbls[0] + if c.cfg.Clickhouse.Cluster != "" { + return c.dbName + "." + c.distSeriesTbls[0] + } else { + return c.dbName + "." + c.seriesTbl + } } return "" } -func (c *ClickHouse) GetDistMetricTable() string { +func (c *ClickHouse) GetMetricTable() string { if c.taskCfg.PrometheusSchema { - return c.distMetricTbls[0] + if c.cfg.Clickhouse.Cluster != "" { + return c.distMetricTbls[0] + } else { + return c.TableName + } } return "" } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index a2da7e93..44899e4f 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -3,7 +3,6 @@ package output import ( "context" "fmt" - "time" "github.com/ClickHouse/clickhouse-go/v2" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" @@ -110,39 +109,3 @@ const ( QUERYROW QUERY ) - -// LoopWrite will dead loop to write the records -func Execute(conn driver.Conn, ctx context.Context, querykind int, query string, fields []zap.Field, args ...interface{}) (result interface{}) { - var times int - util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), fields...) - var err error - - for { - zapFields := fields - switch querykind { - case EXEC: - err = conn.Exec(ctx, query, args...) - case QUERYROW: - result = conn.QueryRow(ctx, query, args...) - case QUERY: - result, err = conn.Query(ctx, query, args...) - default: - util.Logger.Fatal("unknown query kind specified") - } - - if err == nil { - return - } else if errors.Is(err, context.Canceled) { - util.Logger.Info("Execute failed due to the context has been cancelled", fields...) - return - } else if errors.Is(err, clickhouse.ErrAcquireConnTimeout) { - time.Sleep(10 * time.Second) - } else { - zapFields = append(zapFields, zap.Error(err)) - util.Logger.Fatal("Query execution failed", zapFields...) - } - zapFields = append(zapFields, zap.Int("try", times), zap.Error(err)) - util.Logger.Error("Query execution failed", zapFields...) - times++ - } -} diff --git a/task/sinker.go b/task/sinker.go index 27f49a41..ce795a85 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -27,7 +27,6 @@ import ( "time" "github.com/ClickHouse/clickhouse-go/v2" - "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/google/uuid" "github.com/housepower/clickhouse_sinker/config" @@ -37,10 +36,21 @@ import ( "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" - "github.com/pkg/errors" "go.uber.org/zap" ) +var ( + createTableSQL = `CREATE TABLE IF NOT EXISTS %s as %s.%s ENGINE=Merge('%s', '%s')` + dropTableSQL = `DROP TABLE IF EXISTS %s ` + countSeriesSQL = `WITH (SELECT max(timestamp) FROM %s) AS m + SELECT count() FROM %s FINAL WHERE __series_id GLOBAL IN ( + SELECT DISTINCT __series_id FROM %s WHERE timestamp >= addDays(m, -1));` + loadSeriesSQL = `WITH (SELECT max(timestamp) FROM %s) AS m + SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s FINAL WHERE sid GLOBAL IN ( + SELECT DISTINCT toInt64(__series_id) FROM %s WHERE timestamp >= addDays(m, -1) + ) ORDER BY sid;` +) + // Sinker object maintains number of task for each partition type Sinker struct { curCfg *config.Config @@ -57,8 +67,6 @@ type Sinker struct { exitCh chan struct{} stopCommitCh chan struct{} consumerRestartCh chan *Consumer - - seriesQuotas sync.Map } // NewSinker get an instance of sinker with the task list @@ -103,6 +111,9 @@ func (s *Sinker) Run() { } go s.pusher.Run() } + + reloadBmSeriesTicker := time.NewTicker(time.Hour) + defer reloadBmSeriesTicker.Stop() if s.rcm == nil { if _, err = os.Stat(s.cmdOps.LocalCfgFile); err == nil { if newCfg, err = config.ParseLocalCfgFile(s.cmdOps.LocalCfgFile); err != nil { @@ -146,6 +157,11 @@ func (s *Sinker) Run() { util.Logger.Info("consumer restarted when applying another config", zap.String("consumer", c.grpConfig.Name)) } + case <-reloadBmSeriesTicker.C: + util.Logger.Info("offloading out-of-date series record") + if err = s.reloadBmSeries(); err != nil { + util.Logger.Error("reloadBmSeries failed", zap.Error(err)) + } } } } else { @@ -193,6 +209,11 @@ func (s *Sinker) Run() { util.Logger.Info("consumer restarted when applying another config", zap.String("consumer", c.grpConfig.Name)) } + case <-reloadBmSeriesTicker.C: + util.Logger.Info("offloading out-of-date series record") + if err = s.reloadBmSeries(); err != nil { + util.Logger.Error("reloadBmSeries failed", zap.Error(err)) + } } } } @@ -292,7 +313,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { s.consumers[group] = c } - if err = s.loadBmSeries(); err != nil { + if err = s.initBmSeries(); err != nil { return } for _, c := range s.consumers { @@ -338,7 +359,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { s.consumers[group] = c } - if err = s.loadBmSeries(); err != nil { + if err = s.initBmSeries(); err != nil { return } for _, c := range s.consumers { @@ -368,10 +389,6 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { for _, tCfg := range group.Configs { if !reflect.DeepEqual(tCfg, s.consumers[name].grpConfig.Configs[tCfg.Name]) { task := NewTaskService(newCfg, tCfg, s.consumers[name]) - if task.taskCfg.PrometheusSchema { - // stop the task for reloading the bmseries - s.consumers[name].stop() - } if err = task.Init(); err != nil { return } @@ -413,7 +430,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { } } - if err = s.loadBmSeries(); err != nil { + if err = s.initBmSeries(); err != nil { return } for _, c := range s.consumers { @@ -465,7 +482,7 @@ func (s *Sinker) commitFn() { } } -func (s *Sinker) loadBmSeries() (err error) { +func (s *Sinker) initBmSeries() (err error) { // series table could be shared between multiple tasks tables := make(map[string][]*Service) for _, c := range s.consumers { @@ -479,90 +496,137 @@ func (s *Sinker) loadBmSeries() (err error) { } // delete the seriesQuota which no longer being used - s.seriesQuotas.Range(func(key, value any) bool { + output.SeriesQuotas.Range(func(key, value any) bool { k := key.(string) if _, ok := tables[k]; !ok { - s.seriesQuotas.Delete(k) + output.SeriesQuotas.Delete(k) util.Logger.Info("dropping seriesQuota", zap.String("series table", k)) } return true }) - // initialize seriesQuota for new series tables + var conn clickhouse.Conn + if conn, _, err = pool.GetShardConn(0).NextGoodReplica(0); err != nil { + return + } + + // initialize seriesQuota for series tables for k, v := range tables { - var query, dbname, mergetable string - var conn clickhouse.Conn - sq := &model.SeriesQuota{} - mysq := sq - if s, ok := s.seriesQuotas.LoadOrStore(k, sq); ok { - q := s.(*model.SeriesQuota) - mysq = q - for _, svc := range v { - svc.clickhouse.SetSeriesQuota(q) - } - mysq.Lock() - defer mysq.Unlock() - if len(mysq.BmSeries) > 0 { - // only reload the map when the map is empty - return - } - } else { - var count uint64 - var reg string - for _, svc := range v { - r := svc.clickhouse.GetDistMetricTable() - if r != "" { - reg += ("^" + r + "$|") - } + var sq *model.SeriesQuota + if sqAny, ok := output.SeriesQuotas.Load(k); ok { + sq = sqAny.(*model.SeriesQuota) + sq.Lock() + res := sq.BmSeries != nil && len(sq.BmSeries) > 0 + sq.Unlock() + if res { + continue } - reg = reg[:len(reg)-1] - mergetable = strings.ReplaceAll(k+uuid.New().String(), "-", "_") - dbname = strings.Split(k, ".")[0] + } - query = fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %s as %s ENGINE=Merge('%s', '%s')`, - mergetable, v[0].clickhouse.GetDistMetricTable(), dbname, reg) + seriesMap, err := loadBmSeries(conn, k, v) + if err != nil { + return err + } - sc := pool.GetShardConn(0) - if conn, _, err = sc.NextGoodReplica(0); err != nil { - return - } - output.Execute(conn, context.Background(), output.EXEC, query, []zap.Field{zap.String("task", v[0].taskCfg.Name)}) - - query = fmt.Sprintf(`WITH (SELECT max(timestamp) FROM %s) AS m - SELECT count() FROM %s FINAL WHERE __series_id GLOBAL IN ( - SELECT DISTINCT __series_id FROM %s WHERE timestamp >= addDays(m, -1));`, mergetable, k, mergetable) - output.Execute(conn, context.Background(), output.QUERYROW, query, []zap.Field{zap.String("task", v[0].taskCfg.Name)}).(driver.Row).Scan(&count) - - mysq.Lock() - mysq.BmSeries = make(map[int64]int64, count) - mysq.NextResetQuota = time.Now().Add(10 * time.Second) - mysq.Unlock() - for _, svc := range v { - svc.clickhouse.SetSeriesQuota(mysq) - } + sq.Lock() + sq.BmSeries = seriesMap + sq.Birth = time.Now() + sq.Unlock() + } + + return +} + +func (s *Sinker) reloadBmSeries() (err error) { + // find the out-of-date SeriesQuota + sqMap := make(map[string]*model.SeriesQuota) + now := time.Now() + output.SeriesQuotas.Range(func(key, value any) bool { + k := key.(string) + v := value.(*model.SeriesQuota) + v.Lock() + if now.Sub(v.Birth) > time.Hour*24 { + sqMap[k] = v } + v.Unlock() + return true + }) - query = fmt.Sprintf(`WITH (SELECT max(timestamp) FROM %s) AS m - SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s FINAL WHERE sid GLOBAL IN ( - SELECT DISTINCT toInt64(__series_id) FROM %s WHERE timestamp >= addDays(m, -1) - ) ORDER BY sid;`, mergetable, k, mergetable) - rs := output.Execute(conn, context.Background(), output.QUERY, query, []zap.Field{zap.String("task", v[0].taskCfg.Name)}).(driver.Rows) - defer rs.Close() - - mysq.Lock() - defer mysq.Unlock() - var seriesID, mgmtID int64 - for rs.Next() { - if err = rs.Scan(&seriesID, &mgmtID); err != nil { - return errors.Wrapf(err, "") + // series table could be shared between multiple tasks + tables := make(map[string][]*Service) + for _, c := range s.consumers { + c.tasks.Range(func(key, value any) bool { + k := value.(*Service).clickhouse.GetSeriesQuotaKey() + if _, ok := sqMap[k]; ok { + tables[k] = append(tables[k], value.(*Service)) } - mysq.BmSeries[seriesID] = mgmtID + return true + }) + } + + var conn clickhouse.Conn + if conn, _, err = pool.GetShardConn(0).NextGoodReplica(0); err != nil { + return + } + + // reload seriesQuotas which is out-of-date + for k, v := range tables { + seriesMap, err := loadBmSeries(conn, k, v) + if err != nil { + return err } - util.Logger.Info(fmt.Sprintf("loaded %d series from %v", len(mysq.BmSeries), k)) - query = fmt.Sprintf(`DROP TABLE IF EXISTS %s `, mergetable) - output.Execute(conn, context.Background(), output.EXEC, query, []zap.Field{zap.String("task", v[0].taskCfg.Name)}) + sq := sqMap[k] + sq.Lock() + sq.BmSeries = seriesMap + sq.Birth = time.Now() + sq.Unlock() + util.Logger.Info(fmt.Sprintf("reloaded %d series from %v", len(seriesMap), k)) } return } + +func loadBmSeries(conn clickhouse.Conn, sqKey string, tasks []*Service) (result map[int64]int64, err error) { + // merge all metric tables to get the latest timestamp + // old bmseries record won't be loaded into memory to avoid OOM + var reg string + for _, svc := range tasks { + r := svc.clickhouse.GetMetricTable() + if r != "" { + reg += ("^" + r + "$|") + } + } + mergetable := strings.ReplaceAll(sqKey+uuid.New().String(), "-", "_") + dbname := strings.Split(sqKey, ".")[0] + query := fmt.Sprintf(createTableSQL, mergetable, dbname, tasks[0].clickhouse.GetMetricTable(), dbname, reg[:len(reg)-1]) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) + conn.Exec(context.Background(), query) + + var count int + query = fmt.Sprintf(countSeriesSQL, mergetable, sqKey, mergetable) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) + conn.QueryRow(context.Background(), query).Scan(&count) + seriesMap := make(map[int64]int64, count) + + query = fmt.Sprintf(loadSeriesSQL, mergetable, sqKey, mergetable) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) + rs, err := conn.Query(context.Background(), query) + if err != nil { + return nil, err + } + defer rs.Close() + + var seriesID, mgmtID int64 + for rs.Next() { + if err = rs.Scan(&seriesID, &mgmtID); err != nil { + return nil, err + } + seriesMap[seriesID] = mgmtID + } + query = fmt.Sprintf(dropTableSQL, mergetable) + util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) + conn.Exec(context.Background(), query) + + return seriesMap, nil +} diff --git a/util/common.go b/util/common.go index c2ab1d85..ccc2cd7f 100644 --- a/util/common.go +++ b/util/common.go @@ -17,6 +17,7 @@ package util import ( "bytes" + "context" "crypto/tls" "crypto/x509" "fmt" @@ -27,6 +28,7 @@ import ( "reflect" "strconv" "strings" + "time" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -283,3 +285,23 @@ func SetLogLevel(newLogLevel string) { logAtomLevel.SetLevel(lvl) } } + +// dead loop to execute the operation +func RetryOperation(operation func() error, times int, tolerant []error) (err error) { +LOOP: + for times != 0 { + if err = operation(); err == nil || errors.Is(err, context.Canceled) { + return + } + for _, e := range tolerant { + if errors.Is(err, e) { + time.Sleep(10 * time.Second) + times-- + continue LOOP + } + } + return + } + + return +} From 180029fbafe1d6aca6b493b1e33c8a4dc1bb2d03 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Wed, 8 Mar 2023 16:01:35 +0800 Subject: [PATCH 324/404] calculate topic lags every 10 seconds --- CHANGELOG.md | 1 + config_manager/lags.go | 2 +- config_manager/nacos.go | 18 ++++++++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ecf4139a..98f5b1f8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,7 @@ Improvements: - combine nacos log into sinker log - update dmseries map when applying new config, reload the records from series table every single day - avoid recreating dist tables, alter the table schema instead +- update clickhouse_sinker_consume_lags metric every 10 secs #### Version 3.0.1 (2023-03-03) diff --git a/config_manager/lags.go b/config_manager/lags.go index 403c0df6..266b2fd9 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -26,7 +26,7 @@ func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err defer adm.Close() defer cl.Close() - stateLags = make(map[string]StateLag) + stateLags = make(map[string]StateLag, len(cfg.Tasks)) for _, taskCfg := range cfg.Tasks { var state string var totalLags int64 diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 78605145..88087f9a 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -226,6 +226,8 @@ func (ncm *NacosConfManager) Run() { // Assign regularly to handle lag change ticker := time.NewTicker(5 * time.Minute) defer ticker.Stop() + lagticker := time.NewTicker(10 * time.Second) + defer lagticker.Stop() LOOP_FOR: for { select { @@ -236,6 +238,12 @@ LOOP_FOR: util.Logger.Debug("assign triggered by 5 min timer") if err := ncm.assign(); err != nil { util.Logger.Error("assign failed", zap.Error(err)) + } else { + lagticker.Reset(10 * time.Second) + } + case <-lagticker.C: + if err := ncm.calculateGroupLag(); err != nil { + util.Logger.Error("calculate lag failed", zap.Error(err)) } } } @@ -427,6 +435,16 @@ func (ncm *NacosConfManager) assign() (err error) { return } +func (ncm *NacosConfManager) calculateGroupLag() (err error) { + if len(ncm.curInsts) == 0 || ncm.curCfg == nil || ncm.curInsts[0] != ncm.instance { + // Only the first instance is capable to report the lag + return + } + _, err = GetTaskStateAndLags(ncm.curCfg) + + return +} + type NacosLogger struct { *zap.SugaredLogger } From 595df6dbf56fc96b3796dc4b2053aed005d10bbd Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 10 Mar 2023 14:56:29 +0800 Subject: [PATCH 325/404] update the systest script to cover the blacklist&whitelist --- docker/test_prom_metric.data | 20000 ++++++++++++++++---------------- docker/test_prom_metric.hjson | 2 + go.metrictest.sh | 3 +- 3 files changed, 10004 insertions(+), 10001 deletions(-) diff --git a/docker/test_prom_metric.data b/docker/test_prom_metric.data index 9c7f3a1a..eae3e67e 100644 --- a/docker/test_prom_metric.data +++ b/docker/test_prom_metric.data @@ -1,10000 +1,10000 @@ -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.860439, "value1":0.30809143156043617, "value2":8296859231789889769, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697105, "value1":0.07464704753781262, "value2":3626726022956376302, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499281, "value1":0.2598747988733079, "value2":1472024199460334836, "value3":false, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389559, "value1":0.9035230182410444, "value2":5735152112214129185, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259346, "value1":0.6088348511469825, "value2":2118449310673513905, "value3":false, "__name__":"metric_00000013", "key_6":"k","key_7":"e","key_3":"f", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265140, "value1":0.41315907782222017, "value2":6967606456828959585, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273243, "value1":0.3670231186776885, "value2":3094287292036347916, "value3":false, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697145, "value1":0.4712183861515544, "value2":233543654937300193, "value3":true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.570507, "value1":0.11967792858372435, "value2":1467009477255473639, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421572, "value1":0.25352243267763397, "value2":5433840270731074230, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423291, "value1":0.6724498851050211, "value2":652727500980374465, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072653, "value1":0.4078722430828867, "value2":6479204450678692857, "value3":false, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.576237, "value1":0.03506340983128008, "value2":2409142807432889102, "value3":false, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928873, "value1":0.9749671198417696, "value2":806127466997750314, "value3":false, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.121356, "value1":0.3710118368988569, "value2":2961209029330787894, "value3":true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.527082, "value1":0.427919527676362, "value2":4168209888386865559, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971852, "value1":0.4558638608790214, "value2":7271190379082523197, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758405, "value1":0.642984416191772, "value2":635944986021178390, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.191870, "value1":0.4028374291432409, "value2":4925274338231144549, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.177741, "value1":0.112412024356175, "value2":1799812085131373089, "value3":false, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.562889, "value1":0.9463319128677472, "value2":8604221624040619668, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.145145, "value1":0.6344301595600267, "value2":5934019334637844764, "value3":true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512765, "value1":0.12698238608047413, "value2":3179145500529665851, "value3":false, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.947023, "value1":0.9362933587076374, "value2":6552003615608175727, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479188, "value1":0.8368835012768996, "value2":1048667356432629681, "value3":true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.886895, "value1":0.147063588220359, "value2":2554258082886382046, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495557, "value1":0.4897837285320758, "value2":3386687451644953805, "value3":false, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267510, "value1":0.7137029646630675, "value2":4723657657521756601, "value3":true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662037, "value1":0.39308757655823606, "value2":5160142240464547787, "value3":true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.001360, "value1":0.2816582812804884, "value2":6076665375380746353, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.501969, "value1":0.4236689406205709, "value2":725918256085862567, "value3":true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.956392, "value1":0.4674704766085337, "value2":4478503621053198593, "value3":false, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442861, "value1":0.15012494378573155, "value2":6132679974455498265, "value3":true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394657, "value1":0.8655032404449484, "value2":2033181385746149676, "value3":true, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.574670, "value1":0.8578324023084722, "value2":9030512731073145161, "value3":false, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844866, "value1":0.401882448563829, "value2":6461587433040802039, "value3":false, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.677301, "value1":0.17574043431028194, "value2":13200916467489149, "value3":true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643407, "value1":0.09478762453322248, "value2":9040728601657069686, "value3":true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.461584, "value1":0.2313038745196538, "value2":8542291140491717734, "value3":false, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480373, "value1":0.33926139409918454, "value2":1470854458100424231, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.387467, "value1":0.9027179517679272, "value2":2152676801390448497, "value3":false, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.081729, "value1":0.5930629975633843, "value2":3441892662554339501, "value3":true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.523164, "value1":0.27807377593445504, "value2":8726571622906981746, "value3":true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643880, "value1":0.8856942439946653, "value2":6532657219501338712, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.233730, "value1":0.12117132337083063, "value2":5262200042073716921, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113242, "value1":0.15754750121773875, "value2":3512597140842356375, "value3":true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.592540, "value1":0.8501957441234611, "value2":1120617963379990831, "value3":false, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.754300, "value1":0.2516365689787487, "value2":1009529273537079157, "value3":true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.738781, "value1":0.9115797570978498, "value2":654040880693718238, "value3":false, "__name__":"metric_00000057", "key_5":"j","key_2":"a","key_3":"b", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730945, "value1":0.567968215036453, "value2":3597506204077270120, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.940173, "value1":0.9712119929757523, "value2":5983635236065602039, "value3":false, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326670, "value1":0.07295490390000714, "value2":6022184335907772623, "value3":true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.571799, "value1":0.5813154467308365, "value2":9191368645337346592, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.547202, "value1":0.12691857025568046, "value2":3820781159866636536, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097732, "value1":0.6481650315571341, "value2":6773475876286839908, "value3":true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.184127, "value1":0.5736971101036478, "value2":7284609149743260027, "value3":false, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.925677, "value1":0.3771049289873205, "value2":1181794630742224031, "value3":true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.157660, "value1":0.5233716379944181, "value2":4014079834827187874, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.413654, "value1":0.21255841055237895, "value2":4422822169208563976, "value3":true, "__name__":"metric_00000067", "key_9":"a","key_0":"c","key_4":"d", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282429, "value1":0.5134755380331291, "value2":2875914720894216239, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510799, "value1":0.4811695837424015, "value2":2131765002626290228, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855059, "value1":0.1803859944439004, "value2":4143870194342227353, "value3":true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841548, "value1":0.8921162909498762, "value2":2995639098842929032, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904367, "value1":0.6827455696983789, "value2":4486024070965990492, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903308, "value1":0.5275068815139481, "value2":4431489892412543662, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369269, "value1":0.3088848570530923, "value2":2711948390039375929, "value3":true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.246425, "value1":0.22469514297321577, "value2":3311794715258074199, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.627147, "value1":0.5085084020761419, "value2":7449844520407226749, "value3":true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757065, "value1":0.7340080407469496, "value2":1399142812283896331, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404001, "value1":0.6927872548099725, "value2":4611922647039691390, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.548980, "value1":0.9504303368975154, "value2":784663243740669037, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088930, "value1":0.5112126366657939, "value2":1061303860942908201, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808273, "value1":0.07227725639854525, "value2":705291637800185980, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.195134, "value1":0.9915791690400784, "value2":1380206536675804157, "value3":true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588271, "value1":0.9410252962292832, "value2":2976150472594021014, "value3":false, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830440, "value1":0.5990964794781901, "value2":6208539757039787288, "value3":true, "__name__":"metric_00000083", "key_9":"a","key_1":"g","key_6":"b", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.988764, "value1":0.12421784563285446, "value2":600116094525731026, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793643, "value1":0.013993638720735705, "value2":249013181149620349, "value3":false, "__name__":"metric_00000085", "key_5":"j","key_1":"d", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537335, "value1":0.9710360883032237, "value2":3990188937402642001, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.783911, "value1":0.5202417888338783, "value2":9207852713084863631, "value3":true, "__name__":"metric_00000086", "key_4":"b","key_6":"f","key_2":"k", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000369, "value1":0.4757654945131251, "value2":617662898409566699, "value3":true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720049, "value1":0.11616063522827627, "value2":1134111314294224704, "value3":true, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850537, "value1":0.856287241545387, "value2":3915015074142268096, "value3":false, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117449, "value1":0.23938708551093754, "value2":6182563875693526929, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720636, "value1":0.6929840624902686, "value2":7184719611552077354, "value3":false, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890463, "value1":0.39990668829147297, "value2":5236439635142539127, "value3":false, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598510, "value1":0.794840854439141, "value2":7790247245768394828, "value3":false, "__name__":"metric_00000094", "key_2":"e","key_7":"j","key_1":"c", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.103618, "value1":0.837788909134762, "value2":1746463551678400384, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890891, "value1":0.701693361054133, "value2":750546117784279688, "value3":true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243694, "value1":0.04252945224095913, "value2":7022047191936829657, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698720, "value1":0.8318935119459053, "value2":2403728383571794535, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.362462, "value1":0.5425805070018154, "value2":7722798226519636439, "value3":false, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447582, "value1":0.6901117678727204, "value2":1225566659535077888, "value3":false, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884009, "value1":0.9848633762091424, "value2":1508601246687431755, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.895176, "value1":0.1370326687436836, "value2":7673037864112111033, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.521633, "value1":0.14853540424497083, "value2":6875647668629772843, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388849, "value1":0.856681841687182, "value2":289008565390108021, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.747443, "value1":0.735786478790032, "value2":7309545501096360932, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528790, "value1":0.010059663849976508, "value2":3366632872144662566, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.586068, "value1":0.8316103940598538, "value2":6216090017784726937, "value3":true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154422, "value1":0.6449454723419513, "value2":5588370277273470937, "value3":true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.476987, "value1":0.688075774177057, "value2":3232344568636281729, "value3":false, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.176248, "value1":0.8630722552735859, "value2":7054623290814757387, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.553861, "value1":0.6275284095417549, "value2":8413581575795957099, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482699, "value1":0.30152766631318734, "value2":2728742455813076667, "value3":true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228157, "value1":0.09912287337858451, "value2":1368124455579266783, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.363726, "value1":0.2556344439654489, "value2":8083466247113334458, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598631, "value1":0.00654705967457182, "value2":678136325524679028, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188097, "value1":0.33693220746708563, "value2":5349169986765993063, "value3":false, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.591197, "value1":0.3158077067804786, "value2":8408038257495371649, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.723537, "value1":0.8935982855510853, "value2":6781320910854884909, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.714830, "value1":0.8483752317059017, "value2":4114947449469032042, "value3":true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510153, "value1":0.8569271663104114, "value2":2163617609666100779, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.556403, "value1":0.46846499185707535, "value2":4543935722642344774, "value3":false, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115027, "value1":0.13242237154578534, "value2":4001391755557856082, "value3":false, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700402, "value1":0.31919253163578853, "value2":9168039730828764814, "value3":true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473591, "value1":0.914966327309789, "value2":7574079944969934140, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657212, "value1":0.7123671399711553, "value2":5845893001941155972, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389666, "value1":0.46892477810031413, "value2":3773448585226589360, "value3":true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365529, "value1":0.10038955567825718, "value2":7677417085782135970, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.181780, "value1":0.3807188874415628, "value2":213730861941923164, "value3":false, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632990, "value1":0.6352610882615881, "value2":7695215886438259449, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.801703, "value1":0.5435516239407236, "value2":1197124892718412788, "value3":false, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943146, "value1":0.6459645738185148, "value2":257718117926443886, "value3":false, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047331, "value1":0.2599891672558978, "value2":1859555081827581052, "value3":false, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.329003, "value1":0.7536967883211322, "value2":8944365307464980699, "value3":true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750531, "value1":0.8871777984871712, "value2":4584326435451561707, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820378, "value1":0.6275908409813838, "value2":2808244151477882254, "value3":false, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.664852, "value1":0.971622999773127, "value2":3565786429109692783, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502157, "value1":0.19081539446623427, "value2":1479393056820182163, "value3":false, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396316, "value1":0.5123967312660013, "value2":8943627767884961054, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934631, "value1":0.19466544110858217, "value2":3956006751549658946, "value3":true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206549, "value1":0.017226775576345523, "value2":7020366329672792405, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118256, "value1":0.5188472954068934, "value2":8138153858856845378, "value3":true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584343, "value1":0.14448963894972008, "value2":4790553180218846981, "value3":false, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.877457, "value1":0.6880296047171478, "value2":1215785358113912003, "value3":true, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409285, "value1":0.05411761237341633, "value2":4709001633389005280, "value3":true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364854, "value1":0.8998190824029622, "value2":2511578657661549153, "value3":true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.973548, "value1":0.8069354470874932, "value2":2595652771451514899, "value3":true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.073905, "value1":0.2517811560702055, "value2":6140023292659546232, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.262163, "value1":0.5630653619265122, "value2":3018188000144695876, "value3":true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641275, "value1":0.6900145697913901, "value2":5216098013140123128, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392564, "value1":0.018717055152633055, "value2":8077083634713789634, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932220, "value1":0.8388125579510224, "value2":2685186121830778256, "value3":false, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206042, "value1":0.49376606624615665, "value2":5850726108082863170, "value3":false, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215184, "value1":0.356531340492791, "value2":4642776908341780996, "value3":true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.558758, "value1":0.575409651955595, "value2":2207683625879983306, "value3":true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953416, "value1":0.9073618953615663, "value2":5651749773058007750, "value3":true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.648335, "value1":0.46358450476088653, "value2":1290620516019440255, "value3":true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.487910, "value1":0.012547280333283235, "value2":4919259233823782968, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.378926, "value1":0.7085830860476057, "value2":2171649428344764858, "value3":true, "__name__":"metric_00000157", "key_8":"j","key_3":"b","key_4":"k", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802501, "value1":0.31728201505130527, "value2":5419079224591774860, "value3":false, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.656795, "value1":0.24875018847328167, "value2":485829881375178402, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938831, "value1":0.08769582144908596, "value2":6951540802412020838, "value3":false, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231956, "value1":0.7863053106916462, "value2":1030265256144215264, "value3":true, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.631246, "value1":0.46917511334330375, "value2":3774063736313175623, "value3":false, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305077, "value1":0.834556547518894, "value2":636203159633468096, "value3":true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808070, "value1":0.5245784354956413, "value2":5259003801435693215, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641827, "value1":0.7755759714857714, "value2":3467599768461222251, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.306769, "value1":0.47958408429359173, "value2":5365936635633468542, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294040, "value1":0.5158994633208911, "value2":7429034992856357163, "value3":true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.961943, "value1":0.7435043392340425, "value2":8500250963468273773, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935561, "value1":0.5801624578827164, "value2":8754050195411098007, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444846, "value1":0.030970547839451835, "value2":8387858736748608077, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358418, "value1":0.5401368375372775, "value2":7107146838213157067, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.546704, "value1":0.46330699877542014, "value2":8769213962266908815, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.270889, "value1":0.46793759900681503, "value2":6364231395353410490, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549392, "value1":0.3430929972372329, "value2":4966477192488000261, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428697, "value1":0.7243138275027047, "value2":2836857643122176656, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993399, "value1":0.40084653235121936, "value2":6794009333725376121, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.506725, "value1":0.07262914457616802, "value2":3459356114730929347, "value3":false, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202924, "value1":0.07381206265687722, "value2":2649194281139606297, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.411238, "value1":0.9684314554036981, "value2":3705584143921682379, "value3":false, "__name__":"metric_00000178", "key_5":"h","key_6":"k","key_4":"c", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358603, "value1":0.534711919539449, "value2":4138168909279457429, "value3":false, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989681, "value1":0.19207319251633845, "value2":1593591737448226138, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.052714, "value1":0.27756026777792103, "value2":6377557735209128817, "value3":true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614036, "value1":0.018601161874125947, "value2":151552447676967905, "value3":false, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305181, "value1":0.031288435638661224, "value2":3937949086874523036, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715232, "value1":0.6397334976781403, "value2":7614941574913315366, "value3":true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705549, "value1":0.6996034412212425, "value2":6200772172174019130, "value3":true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.302276, "value1":0.5239925847096263, "value2":1057710529698426724, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200111, "value1":0.2567985665426193, "value2":4529602646404884783, "value3":false, "__name__":"metric_00000002", "key_2":"g","key_8":"f","key_1":"a", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.727436, "value1":0.43685005741153693, "value2":3304580792456252079, "value3":false, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.351586, "value1":0.3164739225094333, "value2":3897998541684002875, "value3":false, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.665954, "value1":0.5906227199646251, "value2":6746215572356057637, "value3":true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361921, "value1":0.3364933451115498, "value2":761972198573021391, "value3":false, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.889779, "value1":0.36794570756599093, "value2":2090465055639313788, "value3":false, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.699184, "value1":0.5483254879513706, "value2":7720905658039555449, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388162, "value1":0.5287855210410441, "value2":7195732574645022274, "value3":false, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154496, "value1":0.6641476496489667, "value2":9137939751947784172, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327473, "value1":0.5518373975194654, "value2":3971910670147599886, "value3":true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063454, "value1":0.43820340286329656, "value2":5405166066848368254, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.781841, "value1":0.031101766659472223, "value2":4189381224837071716, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171755, "value1":0.46498479978119844, "value2":4931395791650935239, "value3":false, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.559617, "value1":0.8752912652134259, "value2":8586908304160851241, "value3":true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.323852, "value1":0.35821728605182146, "value2":5163275989952284916, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030550, "value1":0.8225486647874358, "value2":2481647473431927932, "value3":false, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.712533, "value1":0.368964069168292, "value2":8062115249497297834, "value3":true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298843, "value1":0.30262512534550123, "value2":5999722310995378983, "value3":false, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778244, "value1":0.3716152000047593, "value2":6978129792906951032, "value3":true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.144352, "value1":0.42812039567105226, "value2":6591666290600795514, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616112, "value1":0.5501694032291233, "value2":5953070646927834342, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843613, "value1":0.7957093053513518, "value2":779447045934204434, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.021516, "value1":0.3651263217338076, "value2":1101615937937937836, "value3":false, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.707183, "value1":0.9474415697823648, "value2":3163832075607398652, "value3":true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807066, "value1":0.328067579350897, "value2":1404186285892115543, "value3":true, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327059, "value1":0.4531450671817094, "value2":3186214527240993583, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.629874, "value1":0.555595498018525, "value2":7512625623217629070, "value3":false, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203064, "value1":0.24518269440439186, "value2":7974348531674940058, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865398, "value1":0.7125714321236861, "value2":1185618794516530658, "value3":false, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.098668, "value1":0.6050958429061423, "value2":7737776979675298527, "value3":false, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659514, "value1":0.8901996322311656, "value2":7752156196236457080, "value3":false, "__name__":"metric_00000220", "key_6":"h","key_9":"d","key_5":"g", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650309, "value1":0.7869508663362883, "value2":6010716207202506080, "value3":false, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.779127, "value1":0.6366846028665012, "value2":7797452629031711950, "value3":false, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.281652, "value1":0.9118975722819298, "value2":6103834325448620626, "value3":false, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868834, "value1":0.18117722257021981, "value2":6098726761586315616, "value3":true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865255, "value1":0.5168875940701739, "value2":7616170573520868343, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000667, "value1":0.8879098452429443, "value2":2716291319460579102, "value3":false, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938952, "value1":0.27931352835868933, "value2":2351410106131072829, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524371, "value1":0.5375950258397566, "value2":7234763394924546015, "value3":true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718980, "value1":0.7918266548241415, "value2":1156714261082582562, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532951, "value1":0.4063578415664877, "value2":2401980297367889095, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780599, "value1":0.15947463153475608, "value2":6812388220933080605, "value3":true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078621, "value1":0.6070073793267291, "value2":1012316510571499763, "value3":true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404621, "value1":0.9613908610430107, "value2":6888867545087821595, "value3":true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.130043, "value1":0.26321351304547996, "value2":4446851417687808950, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602508, "value1":0.38864606344069585, "value2":2797998328851090673, "value3":false, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244218, "value1":0.5263437247014857, "value2":884821963010111580, "value3":false, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415267, "value1":0.3515494200462383, "value2":3356463166369801537, "value3":false, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303765, "value1":0.5270577861362743, "value2":2138283533246685033, "value3":true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.254002, "value1":0.303619701206094, "value2":7629281847530948891, "value3":false, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588498, "value1":0.5481446854335168, "value2":7641330013258723790, "value3":false, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.620575, "value1":0.6420150180085478, "value2":7965173856733830205, "value3":false, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243174, "value1":0.400523011031962, "value2":4931302022232098402, "value3":false, "__name__":"metric_00000240", "key_9":"c","key_4":"d","key_8":"d", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.104291, "value1":0.4501470045022247, "value2":6419830717700946282, "value3":true, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605242, "value1":0.5489672412234975, "value2":25272414308366080, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705962, "value1":0.2696978321817019, "value2":7266661553560831399, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.204594, "value1":0.050667752810758936, "value2":4939383554296719335, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.694791, "value1":0.8451677349011486, "value2":1862202581082766068, "value3":true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178458, "value1":0.47634419936555994, "value2":3993368962808231698, "value3":true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879184, "value1":0.05969390345398635, "value2":5274917008847651276, "value3":false, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854470, "value1":0.8497662902937875, "value2":3035062077693412013, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.397422, "value1":0.08213093074253511, "value2":5967759139282299209, "value3":true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584032, "value1":0.08273000380656788, "value2":2735658568798834919, "value3":true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822339, "value1":0.581737952515166, "value2":2604615641240906232, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.308039, "value1":0.34242828701171635, "value2":3092061640651599674, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.913723, "value1":0.005597195694116873, "value2":2620704762797681315, "value3":false, "__name__":"metric_00000254", "key_3":"b","key_8":"c","key_1":"f", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618957, "value1":0.6413425640197833, "value2":5287503767189177120, "value3":false, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454987, "value1":0.32512559877918956, "value2":2506424876460932044, "value3":true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447066, "value1":0.20850990987663237, "value2":7977478851800531911, "value3":true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.683345, "value1":0.838649315370365, "value2":4999282712485449007, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935535, "value1":0.4089194818619663, "value2":722917113975878160, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439326, "value1":0.8812716905619982, "value2":6658838831662104981, "value3":false, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069099, "value1":0.15024751394280958, "value2":1328305014122533446, "value3":true, "__name__":"metric_00000258", "key_7":"f","key_3":"a","key_5":"e", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473312, "value1":0.9280459649583414, "value2":3438360323166932806, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215711, "value1":0.8566516621694267, "value2":1663749581830575806, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312134, "value1":0.566681231011358, "value2":6911167899525793851, "value3":true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862090, "value1":0.5364584771183928, "value2":7729755763890728353, "value3":true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456397, "value1":0.7096399907515605, "value2":5531147306124199678, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.587843, "value1":0.8985325214047336, "value2":4030305760543278105, "value3":false, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710467, "value1":0.5099257391896752, "value2":1179017586099298270, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188204, "value1":0.8723809422062396, "value2":5093258983701866401, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.681846, "value1":0.23032248565211208, "value2":3357194399728955391, "value3":true, "__name__":"metric_00000264", "key_2":"j","key_4":"c","key_1":"i", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614220, "value1":0.9249072784237294, "value2":4254302260697988757, "value3":true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.028000, "value1":0.2236758406407766, "value2":5499863050282572666, "value3":true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923027, "value1":0.4209047854675161, "value2":4430077989918325993, "value3":false, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203873, "value1":0.8404470170534011, "value2":532376833912389989, "value3":false, "__name__":"metric_00000269", "key_2":"f","key_6":"b","key_1":"d", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.541847, "value1":0.24084743744285772, "value2":2314457094095648600, "value3":true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932124, "value1":0.7743913731102018, "value2":7842905515694643998, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.986636, "value1":0.18488491173653657, "value2":1233277417792576747, "value3":false, "__name__":"metric_00000275", "key_6":"f","key_0":"j","key_2":"i", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.491189, "value1":0.5404267488261955, "value2":5193038540757554939, "value3":true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.449064, "value1":0.9915502184839048, "value2":3914891481887672833, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203773, "value1":0.11544040430435766, "value2":8720914144754355340, "value3":true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.637865, "value1":0.5397959742632489, "value2":5016935922539098494, "value3":true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.164476, "value1":0.40957356383804516, "value2":8101990321123394900, "value3":true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212208, "value1":0.31170589199654203, "value2":8172096864950686883, "value3":false, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427578, "value1":0.9500075990560398, "value2":4448206698192420471, "value3":false, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375888, "value1":0.08043137384857349, "value2":5428459057669693793, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.253419, "value1":0.30426981771564987, "value2":3701960397375147641, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.064111, "value1":0.6613366419136151, "value2":9214004723321204161, "value3":false, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618634, "value1":0.28919063506417325, "value2":2549339895850480456, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.581311, "value1":0.5011828194867549, "value2":2394702011956086199, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.619872, "value1":0.3579296101205142, "value2":3114339742928723546, "value3":true, "__name__":"metric_00000289", "key_6":"f","key_0":"c", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928952, "value1":0.7843473952520837, "value2":4007100053019753408, "value3":true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.678699460362967, "value2":1449401258062210262, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.791597, "value1":0.5225036343492986, "value2":7759663471842896645, "value3":true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794616, "value1":0.3395042657376233, "value2":1662502503940889595, "value3":true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.466754, "value1":0.9041538990976105, "value2":535273624649157699, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693734, "value1":0.009440601901378784, "value2":2148867264362048197, "value3":true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841713, "value1":0.059732250656449104, "value2":6677903169635737819, "value3":true, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.746032, "value1":0.16006431951166478, "value2":5022444125278904149, "value3":false, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.158292, "value1":0.8619758377742992, "value2":3793514584183151927, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364688, "value1":0.7050682785197309, "value2":8826236482901828470, "value3":true, "__name__":"metric_00000296", "key_9":"f","key_2":"e","key_7":"e", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273970, "value1":0.40265473643498934, "value2":8303926135399734280, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030530, "value1":0.7286415975665108, "value2":160039363895729267, "value3":true, "__name__":"metric_00000244", "key_5":"j","key_3":"g", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266449, "value1":0.033547786099323434, "value2":7620487585623102750, "value3":false, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025977, "value1":0.17793807237193235, "value2":8744525786084307091, "value3":false, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.806031, "value1":0.5052650450730324, "value2":4060064708931174932, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035732, "value1":0.9743453198182447, "value2":4099915557434576704, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.646092, "value1":0.21956311768718226, "value2":8589888788620337480, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078229, "value1":0.4079449893422866, "value2":6173402600986499343, "value3":false, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666881, "value1":0.9863097557928853, "value2":8279527785132450943, "value3":true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.741108, "value1":0.31651275509392585, "value2":1547895388613496809, "value3":true, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866620, "value1":0.6883437972608208, "value2":8802986125706842330, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718942, "value1":0.8847629636653906, "value2":800711854768452001, "value3":true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375188, "value1":0.9532398566953418, "value2":2394341312296044798, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624088, "value1":0.6934451210618476, "value2":8474880456434193538, "value3":true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680602, "value1":0.2209467711169388, "value2":8621094656072865370, "value3":false, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.522533, "value1":0.5439997834907557, "value2":8198708215843245045, "value3":false, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439975, "value1":0.8101860664171935, "value2":8929518033480490042, "value3":true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602144, "value1":0.17084337908018365, "value2":6000683736434217499, "value3":true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594415, "value1":0.3959186882607224, "value2":1454959879954934650, "value3":false, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917433, "value1":0.849867816493911, "value2":8615713829914363729, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439894, "value1":0.7578484758847723, "value2":3887712481108512291, "value3":true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.965697, "value1":0.8251003697840577, "value2":836422853680285494, "value3":true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259251, "value1":0.4596388195214864, "value2":1597474370288095053, "value3":true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512513, "value1":0.783528381108141, "value2":8045517537006666296, "value3":true, "__name__":"metric_00000314", "key_4":"i","key_6":"g","key_3":"c", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.452258, "value1":0.4477592871421156, "value2":1459300159451076858, "value3":true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735408, "value1":0.5067192602284456, "value2":7587131980242978873, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.108601, "value1":0.3076343047473968, "value2":5066182910326911353, "value3":true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419278, "value1":0.12962316194187043, "value2":8864033357034639760, "value3":true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336677, "value1":0.24448588819210298, "value2":4567827434979480913, "value3":true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131541, "value1":0.02538083388681256, "value2":6449819714583284688, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235312, "value1":0.7537710944451972, "value2":3840033782203811299, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713440, "value1":0.6660557019192793, "value2":6317879942694845725, "value3":true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250100, "value1":0.2226451425042414, "value2":6756354166360377412, "value3":false, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.545039, "value1":0.7203310314493744, "value2":6637075765032095898, "value3":false, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846538, "value1":0.5133638544522705, "value2":6146673076285104795, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.099134, "value1":0.26713136173061386, "value2":5047631075889612988, "value3":false, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.999599, "value1":0.19120338269199497, "value2":3009559225203387009, "value3":false, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456369, "value1":0.6496125050532283, "value2":8358687825156497010, "value3":false, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.882684, "value1":0.4767076839685737, "value2":2058131066392704343, "value3":true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326891, "value1":0.0072166654150041835, "value2":3877874561619455924, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.753206, "value1":0.9891987223468547, "value2":2500770621881849387, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.505150, "value1":0.33485751726586077, "value2":3075226382158740800, "value3":true, "__name__":"metric_00000334", "key_6":"k","key_3":"h", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.337847, "value1":0.7587725777413157, "value2":4030061892629350122, "value3":true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.061555, "value1":0.5715183894256888, "value2":1380477982282562120, "value3":false, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102225, "value1":0.18128048387136908, "value2":1107305884228635610, "value3":false, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.663571, "value1":0.2082433172088735, "value2":2331488443653185612, "value3":false, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400352, "value1":0.36344640764280667, "value2":4156784209596494480, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567282, "value1":0.6853752986060114, "value2":1738184775840508143, "value3":true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580625, "value1":0.21556187277712435, "value2":1716362597563830669, "value3":true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.068750, "value1":0.5617303119904301, "value2":215978150703712746, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948465, "value1":0.4015798445122638, "value2":4918723285934260509, "value3":true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818268, "value1":0.4923649301244579, "value2":2640505566674328479, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344093, "value1":0.24900402040054898, "value2":3729697701643357874, "value3":true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096571, "value1":0.84402994415599, "value2":5393515911153006286, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880117, "value1":0.7901487854013907, "value2":2074107282217049907, "value3":false, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923332, "value1":0.6838866899470619, "value2":1947909482655854334, "value3":false, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.432183, "value1":0.35953677139878504, "value2":3191972662401496477, "value3":false, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251030, "value1":0.9314169082560206, "value2":2792551904040732916, "value3":true, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.315849, "value1":0.9521659578315848, "value2":278261838735054052, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.790453, "value1":0.9152891069156645, "value2":1668356628128321724, "value3":true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.260477, "value1":0.2126605025888137, "value2":6877799862560969541, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.213669, "value1":0.023418304769384207, "value2":4056268380917249508, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.647354, "value1":0.1608657659472805, "value2":7997008030753632127, "value3":false, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364473, "value1":0.2924502770902306, "value2":2291771922128424424, "value3":false, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.399561, "value1":0.6211538305973102, "value2":6848308885300872697, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135252, "value1":0.6565821688014496, "value2":4331814359581194434, "value3":true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365542, "value1":0.9118204132722055, "value2":2417655238291803790, "value3":true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758482, "value1":0.17112508145844177, "value2":1991118257939966294, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.032543, "value1":0.08125554957598005, "value2":6803892048654362563, "value3":false, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.520072, "value1":0.4275632442269591, "value2":2250122493336577938, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.900940, "value1":0.14035051591842881, "value2":3790466116523762354, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794030, "value1":0.6000173402567552, "value2":1696544400981072969, "value3":true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.760746, "value1":0.10139199442682838, "value2":2726344565529290448, "value3":false, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687331, "value1":0.770768299751441, "value2":8765361349894188719, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035711, "value1":0.48562696854381604, "value2":95585559412343711, "value3":true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739784, "value1":0.5825982451536238, "value2":2923240563522303723, "value3":true, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454300, "value1":0.4006888000990195, "value2":2468975659398879205, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884626, "value1":0.8026957862944495, "value2":3605855909002945057, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395577, "value1":0.28352828399222396, "value2":7602348181385211161, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250646, "value1":0.2316228525268274, "value2":29961024893090406, "value3":true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421024, "value1":0.7990468794285002, "value2":2264750479744979673, "value3":false, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380413, "value1":0.677589393211687, "value2":1235396941989449609, "value3":true, "__name__":"metric_00000383", "key_4":"f","key_1":"j", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680912, "value1":0.25625503996995874, "value2":2396418552817135385, "value3":false, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624528, "value1":0.5836651399092099, "value2":8089438271359233374, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.578517, "value1":0.7858613429590018, "value2":3691936139324370664, "value3":false, "__name__":"metric_00000385", "key_7":"h","key_0":"b","key_2":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.824035, "value1":0.3471915666356168, "value2":7523600740099466883, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069777, "value1":0.6471590850539041, "value2":545376892289254611, "value3":false, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390523, "value1":0.8681181997676135, "value2":5603080382596898622, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.590994, "value1":0.6390330221088167, "value2":6898933669096943734, "value3":true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222958, "value1":0.7361676398339201, "value2":7798894502817396812, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.313614, "value1":0.9956617360294369, "value2":8114343594051219927, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.471497, "value1":0.2028990027253043, "value2":8672046519486478878, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299387, "value1":0.90821231206369, "value2":7139029616253188226, "value3":true, "__name__":"metric_00000373", "key_7":"c","key_9":"g","key_2":"j", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822834, "value1":0.7462012785016429, "value2":2497140784779218477, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.958689, "value1":0.6493610280581492, "value2":7375518635495424751, "value3":true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755680, "value1":0.3173769236575059, "value2":1038885732982663879, "value3":true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.906259, "value1":0.7803156579997583, "value2":1671762930600330124, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320586, "value1":0.047928946477745184, "value2":2364817790122909266, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.237873, "value1":0.5919698389786022, "value2":2456361237893496498, "value3":false, "__name__":"metric_00000391", "key_9":"c","key_2":"j","key_4":"d", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.183324, "value1":0.25157377599570596, "value2":3302857039571325572, "value3":true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427134, "value1":0.9570675751476113, "value2":241686111761765133, "value3":false, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.353599, "value1":0.19102611162010216, "value2":7635824178108933655, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933576, "value1":0.5497618436271264, "value2":178483151772454986, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.888923, "value1":0.6996148110998511, "value2":7261028738310365771, "value3":true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414681, "value1":0.45596913295291375, "value2":8486467978432271730, "value3":false, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354962, "value1":0.6882059355313528, "value2":4219371503140919650, "value3":true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149786, "value1":0.048507930319514725, "value2":3361043402215780354, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320396, "value1":0.48978604231367256, "value2":8391350623429868941, "value3":false, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.189903, "value1":0.49045511147932325, "value2":4568889448781542889, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.531173, "value1":0.015565420626107091, "value2":1532700275127700107, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.911622, "value1":0.9315116645649003, "value2":845964128004551720, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447558, "value1":0.6852687200103666, "value2":1997580526601582597, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389703, "value1":0.7679907754096043, "value2":8650394236218808766, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221750, "value1":0.8409665471906171, "value2":8068148192803213421, "value3":false, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.679200, "value1":0.5889377996015627, "value2":6603283664639278646, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.658212, "value1":0.20335938525562192, "value2":6814120282733950138, "value3":false, "__name__":"metric_00000411", "key_8":"c","key_4":"d","key_7":"g", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917850, "value1":0.07209516605018491, "value2":3465054040297242627, "value3":true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.463048, "value1":0.8121431937413348, "value2":7711657955494481167, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280683, "value1":0.9792224347770722, "value2":6637142200642308638, "value3":true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938341, "value1":0.6616785812007566, "value2":4430772053877820, "value3":true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778057, "value1":0.21301644528770988, "value2":5550954213897210258, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.435560, "value1":0.25143911919093503, "value2":8602791844656832235, "value3":false, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980296, "value1":0.4483217784998845, "value2":8061950324304064559, "value3":false, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.845189, "value1":0.23223688199088727, "value2":2019205569808444797, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030298, "value1":0.009150371898544056, "value2":2179401507551835981, "value3":true, "__name__":"metric_00000415", "key_9":"e","key_4":"j","key_7":"k", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439657, "value1":0.5569344336018628, "value2":3896413214679997959, "value3":false, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037278, "value1":0.0752646633945245, "value2":2986036073412577705, "value3":false, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005641, "value1":0.470809080359862, "value2":7672259019023724818, "value3":true, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169543, "value1":0.6908284999086921, "value2":6236799599986001574, "value3":false, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.168435, "value1":0.1656190822947285, "value2":2938449310867540367, "value3":true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901700, "value1":0.6243405512256929, "value2":2234990933462726180, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282060, "value1":0.14169460807404005, "value2":820280722029319872, "value3":true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868372, "value1":0.8809198418653182, "value2":7786878634034667094, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700989, "value1":0.028109982788470232, "value2":1201575202891520734, "value3":false, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.304734, "value1":0.8385104418360362, "value2":6703304420616815596, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901530, "value1":0.8011320750502264, "value2":7539936763030777880, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862233, "value1":0.47963788532278256, "value2":3960941336665870754, "value3":false, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.511307, "value1":0.1832580868434161, "value2":7188980001326450399, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700199, "value1":0.6958679254162045, "value2":2878046054907323632, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299563, "value1":0.31054488059495927, "value2":8075467755079023785, "value3":true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383092, "value1":0.290554447303189, "value2":6694033035935161393, "value3":false, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.696053, "value1":0.16066281765668294, "value2":3961201674767505870, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392128, "value1":0.5736001777474582, "value2":168331305812241403, "value3":false, "__name__":"metric_00000435", "key_9":"h","key_3":"j","key_5":"a", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752731, "value1":0.4825725339245043, "value2":7800665573914476062, "value3":false, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293547, "value1":0.12126767569308682, "value2":1612512565132344690, "value3":false, "__name__":"metric_00000437", "key_8":"a","key_0":"b","key_3":"d", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874204, "value1":0.7014202076829791, "value2":476647849984184674, "value3":false, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.132714, "value1":0.5122323816164429, "value2":646780496771729886, "value3":true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358264, "value1":0.8238092496263985, "value2":9061776775816645023, "value3":false, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358876, "value1":0.15679289393208987, "value2":8163846432367416312, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921374, "value1":0.8339787140810333, "value2":3649804286907707134, "value3":true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395687, "value1":0.875997280853855, "value2":9212367459114589124, "value3":false, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.459642, "value1":0.7528040980492385, "value2":4150547104368392230, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782674, "value1":0.7121212215364499, "value2":6531310437181949577, "value3":true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.229939, "value1":0.32814366772056375, "value2":6374161097327904245, "value3":true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594347, "value1":0.6163078141302316, "value2":1538685760517154018, "value3":true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.009874, "value1":0.36102842691855586, "value2":1131193318052688252, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.606094, "value1":0.7074086026345966, "value2":7018588286889757726, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.777708, "value1":0.9795566345359867, "value2":7358687237436509918, "value3":true, "__name__":"metric_00000445", "key_4":"a","key_2":"i","key_3":"k", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.963990, "value1":0.9872552891680104, "value2":6171290753066239119, "value3":true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772158, "value1":0.4830977012506835, "value2":6211566808050254036, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.952768, "value1":0.43798623227092814, "value2":3513514828808633618, "value3":true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566018, "value1":0.7713759201698291, "value2":5320413786417127410, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.563469, "value1":0.6599121888373632, "value2":1404388254364767252, "value3":true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786109, "value1":0.9075953036601083, "value2":8494075902382221765, "value3":true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812207, "value1":0.5042296550006388, "value2":2362200937900291190, "value3":true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093099, "value1":0.3386496883604208, "value2":7071312823587957868, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.381119, "value1":0.031714519630475106, "value2":2849328345417435639, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.815827, "value1":0.14737457839891155, "value2":199255383994378266, "value3":false, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050041, "value1":0.5503190199883978, "value2":7356958752349024534, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.472083, "value1":0.12465887530301682, "value2":6392360821928490144, "value3":false, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713056, "value1":0.12067504962982098, "value2":4673070422231359809, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.034034, "value1":0.25231429703394476, "value2":5489447259456982836, "value3":true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502818, "value1":0.4156967719918407, "value2":4612443309724781532, "value3":false, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427071, "value1":0.24921718574451088, "value2":7646229328406586652, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643652, "value1":0.07902787804812644, "value2":34188484750195348, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.488097, "value1":0.7204444807131233, "value2":4412925367441450672, "value3":true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.180195, "value1":0.3056776312063749, "value2":7687653658691030932, "value3":false, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.873382, "value1":0.6827451433784463, "value2":1751876103177277887, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529999, "value1":0.39651784939201806, "value2":1747188854631655971, "value3":true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971674, "value1":0.8695844176281972, "value2":2878591399245613156, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702173, "value1":0.9786793419693177, "value2":4388723377141224465, "value3":true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844886, "value1":0.690355517305138, "value2":6283981589400857877, "value3":false, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298362, "value1":0.6666814629745373, "value2":6187669811817639042, "value3":true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266804, "value1":0.7091063369563013, "value2":4664946045775147969, "value3":false, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400948, "value1":0.8630878707275965, "value2":7287257182404940937, "value3":true, "__name__":"metric_00000477", "key_3":"f","key_6":"k","key_0":"b", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135133, "value1":0.6122563921627874, "value2":2139209888725866722, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.370672, "value1":0.01650778323939298, "value2":3489547069770540399, "value3":false, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.279256, "value1":0.7619200195662909, "value2":5915401631721082002, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170496, "value1":0.01005373661086586, "value2":1482576688135431000, "value3":false, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499284, "value1":0.36435815211219275, "value2":1326048436273736829, "value3":true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185099, "value1":0.9391170841095738, "value2":8924546898546218998, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772826, "value1":0.7574432732083424, "value2":4488845007322620734, "value3":false, "__name__":"metric_00000484", "key_6":"h","key_5":"f", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169744, "value1":0.5739689347042248, "value2":741420931729893242, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.998044, "value1":0.8965817632976586, "value2":6239062200473429860, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594662, "value1":0.6301979977894112, "value2":2054875786295626784, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755433, "value1":0.06261106040596412, "value2":5455645386351916850, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.682584, "value1":0.03167274357401835, "value2":310883965859994658, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072455, "value1":0.9148162667555992, "value2":382836579698407350, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.725171, "value1":0.3647728943926306, "value2":2421892347307790025, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798022, "value1":0.22510760744817873, "value2":2442091377551743699, "value3":true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688933, "value1":0.07828818881955478, "value2":6735636563745227652, "value3":false, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.649962, "value1":0.9679958182791478, "value2":2607996809419409341, "value3":false, "__name__":"metric_00000493", "key_8":"d","key_9":"h","key_1":"d", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.382199, "value1":0.14064207963814382, "value2":1623159006724541520, "value3":true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293647, "value1":0.919909760697381, "value2":58019425907871235, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715683, "value1":0.40584027014114893, "value2":7884356555342436928, "value3":false, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.496280, "value1":0.6458580424511011, "value2":1385234809290785609, "value3":false, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037411, "value1":0.5356364996267606, "value2":4485181030850732806, "value3":false, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.092167, "value1":0.08471879259284998, "value2":1047090205307301338, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203791, "value1":0.9626197309889007, "value2":458130535579789756, "value3":false, "__name__":"metric_00000501", "key_5":"b","key_0":"d","key_2":"i", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395018, "value1":0.7684680824595476, "value2":787781374740198173, "value3":false, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408536, "value1":0.147741569501137, "value2":3967017244497727033, "value3":false, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693578, "value1":0.011896444619318294, "value2":4942230313540034093, "value3":false, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780418, "value1":0.8750655230631833, "value2":8170450018971591774, "value3":true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812004, "value1":0.011539724469496575, "value2":1314191368195668298, "value3":true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.090381, "value1":0.870578773268634, "value2":5701169500016143920, "value3":false, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512721, "value1":0.9238819426049193, "value2":5759090244464429103, "value3":false, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938235, "value1":0.5991043580028484, "value2":7404773139120927140, "value3":false, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.751490, "value1":0.1795090853269717, "value2":6360470713237738478, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267071, "value1":0.625707646354563, "value2":6040902311152523365, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.002110, "value1":0.08011219998212876, "value2":5850595301611238867, "value3":true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257610, "value1":0.8187562858163207, "value2":8061987864802598079, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.446622, "value1":0.7315172185892279, "value2":2428635533374338014, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730403, "value1":0.7209673234694193, "value2":8416924883569369777, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.814732, "value1":0.8815237283497432, "value2":1852270652578106849, "value3":true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.318594, "value1":0.610086630044871, "value2":4524314099691048466, "value3":true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.852591, "value1":0.9383715352044596, "value2":453270928675336465, "value3":false, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989585, "value1":0.8337913033606761, "value2":6440744488245743508, "value3":false, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666424, "value1":0.9182270518817748, "value2":6618322333494361668, "value3":true, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.010788, "value1":0.6687390259133706, "value2":1979105606239962267, "value3":false, "__name__":"metric_00000522", "key_9":"a","key_0":"c","key_6":"j", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752852, "value1":0.4586423067764851, "value2":3289626005749005579, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005307, "value1":0.4780784157090063, "value2":2142574953538675780, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752795, "value1":0.7908677214518265, "value2":5326267086837385392, "value3":false, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202377, "value1":0.001998326079086547, "value2":2359915256333417411, "value3":true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216786, "value1":0.03377151611828389, "value2":5536807322346137933, "value3":true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243069, "value1":0.48298272047333335, "value2":6984955197084383580, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199895, "value1":0.6668507451944505, "value2":1160751712622009388, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605559, "value1":0.014501073348709297, "value2":2976544013604671560, "value3":true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046312, "value1":0.45283346882557124, "value2":1106313893608419094, "value3":true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841418, "value1":0.9439193659449161, "value2":1585819181629654484, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.816329, "value1":0.8314432076191595, "value2":671388396305183300, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.672039, "value1":0.19428152515853786, "value2":4678503251263728412, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.652936, "value1":0.05999646842980615, "value2":6549432288618958292, "value3":true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212442, "value1":0.6061066332871502, "value2":6044859751282824016, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200351, "value1":0.9858548441265623, "value2":6095603519503403992, "value3":true, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047144, "value1":0.8823748802173365, "value2":2346527148275584633, "value3":false, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272669, "value1":0.49903571079222264, "value2":7606961266627055929, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244016, "value1":0.3505008339279857, "value2":9165454668662797150, "value3":false, "__name__":"metric_00000538", "key_6":"e","key_0":"a","key_4":"f", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595436, "value1":0.1769134432972796, "value2":4183298646463696655, "value3":true, "__name__":"metric_00000540", "key_7":"e","key_6":"f", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.832887, "value1":0.05781334979440461, "value2":2976605700126921242, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.417486, "value1":0.03855593830020744, "value2":6499287743834404350, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850592, "value1":0.8819483685824858, "value2":3365942495366951256, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904339, "value1":0.4096492831256871, "value2":7870603688751891889, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849740, "value1":0.5985549217817763, "value2":5972692186135936344, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080962, "value1":0.6508975263449264, "value2":620195756611450814, "value3":true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.172604, "value1":0.36758717062599394, "value2":3093546059237948200, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.611964, "value1":0.09212647541898711, "value2":600064894791793802, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.136731, "value1":0.3518781010346145, "value2":4172668838781409561, "value3":false, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221150, "value1":0.7696791785774878, "value2":4882972115027784491, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118097, "value1":0.3822037091545315, "value2":5547505099823892976, "value3":false, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.209184, "value1":0.3119528694472212, "value2":8126435007719933970, "value3":true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.708227, "value1":0.8414289260599137, "value2":1887259462865725736, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127263, "value1":0.417919506626244, "value2":3560935891426387222, "value3":false, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.500871, "value1":0.16110435350355187, "value2":2428171488019731436, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.848583, "value1":0.4961159478803282, "value2":5456304085389531044, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.468311, "value1":0.9289954682212651, "value2":4285742227173179225, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.160825, "value1":0.59740490878922, "value2":9140238758107732393, "value3":true, "__name__":"metric_00000559", "key_4":"d","key_1":"f","key_2":"j", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423818, "value1":0.2325925821959117, "value2":7321787707684172397, "value3":true, "__name__":"metric_00000558", "key_7":"k","key_5":"i","key_6":"g", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.689012, "value1":0.5753370946686959, "value2":5125074341961139213, "value3":true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941993, "value1":0.009508850201920292, "value2":2477284877979206282, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344874, "value1":0.012483207365877868, "value2":4898429158465049577, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625728, "value1":0.9281594076368007, "value2":771428325581269474, "value3":false, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701217, "value1":0.8504319614540372, "value2":3030161014659152421, "value3":true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.288648, "value1":0.7225335235603462, "value2":4184194222447924377, "value3":false, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735765, "value1":0.7338027307218647, "value2":8053319255149398119, "value3":false, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853312, "value1":0.31226366748735784, "value2":48160047989763460, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289371, "value1":0.5473984505709404, "value2":2530747852470429293, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825683, "value1":0.2907108545003152, "value2":3061253498575467483, "value3":false, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532231, "value1":0.5859058541023265, "value2":4975638816250812874, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512440, "value1":0.511793469522306, "value2":5419688896766823090, "value3":true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709116, "value1":0.0431590704274129, "value2":565903028384253279, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709410, "value1":0.45910825737421945, "value2":425558835807211279, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.970542, "value1":0.1219905172153899, "value2":24331626097709718, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.043601, "value1":0.49284843085319935, "value2":7672646678928068810, "value3":true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739151, "value1":0.4246916831482226, "value2":4312260845153215448, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489280, "value1":0.8582555742505251, "value2":3110601136234274420, "value3":false, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935791, "value1":0.4664631517475295, "value2":4539773222290649379, "value3":false, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.654384, "value1":0.10389342147634097, "value2":5731036292461383708, "value3":false, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193783, "value1":0.5459140576966053, "value2":6006070322533917783, "value3":false, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409382, "value1":0.472091611755319, "value2":4277330658526282078, "value3":true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423822, "value1":0.4751601960954781, "value2":2652715507916534737, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.660521, "value1":0.38225985421870495, "value2":9011573276516869930, "value3":true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975891, "value1":0.5238616298909866, "value2":472916590695431743, "value3":false, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843227, "value1":0.6679925899402425, "value2":906046342124593465, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829312, "value1":0.16298135421846682, "value2":1838361763539535864, "value3":false, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480674, "value1":0.259388407981778, "value2":6440770726430652573, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690879, "value1":0.0735354737983211, "value2":9181104589648553044, "value3":true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126782, "value1":0.4486397942700849, "value2":7462094819783188022, "value3":false, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948901, "value1":0.9959329498625173, "value2":5479458805805092746, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127088, "value1":0.9575385572151783, "value2":2275563983258531267, "value3":true, "__name__":"metric_00000592", "key_7":"b","key_0":"k","key_1":"e", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259685, "value1":0.3372857504409974, "value2":6098187454917492671, "value3":true, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336884, "value1":0.24553452230487985, "value2":8241989909296706238, "value3":false, "__name__":"metric_00000594", "key_3":"a","key_9":"e","key_1":"g", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169076, "value1":0.17392408086361133, "value2":7622066852267723117, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977238, "value1":0.5433155948072138, "value2":6898148363520327391, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.278455, "value1":0.6005033834895888, "value2":8001662694808432224, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993722, "value1":0.9597867721155696, "value2":1299805439881399559, "value3":false, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.530999, "value1":0.2209888826520892, "value2":4066778144109849949, "value3":true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782378, "value1":0.7523792160016672, "value2":5556510651542494721, "value3":false, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.219804, "value1":0.1717195689538366, "value2":7048403972015561437, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.946856, "value1":0.5985084131744225, "value2":4885828087902675075, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759489, "value1":0.27172565847552677, "value2":7664668067438271399, "value3":false, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687774, "value1":0.42250850460802286, "value2":7088707511295378411, "value3":true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560049, "value1":0.2616853019800064, "value2":2950184265013984605, "value3":false, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097585, "value1":0.2684028422029654, "value2":675122184817942746, "value3":true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205813, "value1":0.44731824636014533, "value2":1428863070750239410, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.543640, "value1":0.544977219392789, "value2":2157919960028175094, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566856, "value1":0.8008181036581771, "value2":7402387212023409312, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.765221, "value1":0.8596387394697663, "value2":1232636922510789742, "value3":false, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.386997, "value1":0.10171781350903462, "value2":2933191729226405023, "value3":true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.674282, "value1":0.05504930364585503, "value2":724010655267136473, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820827, "value1":0.1512145830665697, "value2":6612187124412654634, "value3":false, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537712, "value1":0.6818284933003707, "value2":833546983771380703, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.020134, "value1":0.7293182995525076, "value2":2126763229656176078, "value3":false, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.919742, "value1":0.47707314485124014, "value2":8035884626808128837, "value3":false, "__name__":"metric_00000616", "key_5":"d","key_7":"j","key_1":"k", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941486, "value1":0.13322259126479358, "value2":2182305210138540441, "value3":false, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.089091, "value1":0.6999475934755898, "value2":5942262186593070970, "value3":false, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.922220, "value1":0.01137462679977003, "value2":4337636877649564119, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846010, "value1":0.10326336153396976, "value2":8556623018979218867, "value3":true, "__name__":"metric_00000620", "key_9":"b","key_2":"j","key_7":"f", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361361, "value1":0.9603555432714893, "value2":7276258796427191775, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303375, "value1":0.9198009606494528, "value2":3654960055261327174, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354372, "value1":0.9154736221447883, "value2":6754725607301294498, "value3":false, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836769, "value1":0.17500751807732254, "value2":5225830062891206724, "value3":false, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.514481, "value1":0.712612132062798, "value2":5405795909104498480, "value3":false, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.857662, "value1":0.4771648896771537, "value2":9197955420056459031, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218504, "value1":0.43388279897052595, "value2":4876293374213376800, "value3":true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854318, "value1":0.29673697161398044, "value2":2557767286012732771, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.621120, "value1":0.07462842813102197, "value2":5246869953360521284, "value3":false, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482880, "value1":0.4261787117422624, "value2":7122235599178548659, "value3":false, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880280, "value1":0.38443671400954443, "value2":5167341510277945257, "value3":false, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849192, "value1":0.5161558417921461, "value2":4578699148249247151, "value3":false, "__name__":"metric_00000631", "key_8":"g","key_9":"d","key_1":"i", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826648, "value1":0.873997512950574, "value2":7985090519609883665, "value3":true, "__name__":"metric_00000632", "key_3":"i","key_7":"j","key_2":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199619, "value1":0.8346974218021976, "value2":8292656933621980920, "value3":true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.959402, "value1":0.6444072126847691, "value2":4320840087483118841, "value3":true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117065, "value1":0.9272062952128088, "value2":4910297155014939874, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.551716, "value1":0.13475804603313432, "value2":8222696695222433380, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643768, "value1":0.8311693472811903, "value2":7965400634034447655, "value3":false, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.484767, "value1":0.5064114690168263, "value2":5755735513825830154, "value3":true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377345, "value1":0.1504036301883094, "value2":2988203951245132390, "value3":true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991676, "value1":0.972772556086247, "value2":1470890803841584803, "value3":true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361353, "value1":0.6540839592057779, "value2":985314732060254627, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585584, "value1":0.7709261757498476, "value2":2565995410162492585, "value3":true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.704212, "value1":0.3118041755703027, "value2":6201623885191421475, "value3":true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.518173, "value1":0.9814365163985729, "value2":8388580742116222397, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.187366, "value1":0.6732011290892043, "value2":5408814647988876882, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.296035, "value1":0.5028422803953023, "value2":7236554080452628944, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.768145, "value1":0.8394583660519598, "value2":5325173567069752762, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549064, "value1":0.12911593374108088, "value2":8305911548302819241, "value3":true, "__name__":"metric_00000648", "key_9":"h","key_1":"e","key_2":"e", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758468, "value1":0.0009372001034959489, "value2":5525854549400493996, "value3":false, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.525822, "value1":0.9778487221852099, "value2":2741333418274183362, "value3":true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396148, "value1":0.9074357631670523, "value2":8809499083604789974, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414129, "value1":0.8132493023109502, "value2":4942425104780689529, "value3":false, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.398880, "value1":0.39814811004109957, "value2":6997600198610651439, "value3":false, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.667851, "value1":0.7812827159518755, "value2":111949400321343617, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.065047, "value1":0.5608722991855313, "value2":128957686943303349, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.494481, "value1":0.17816439204883128, "value2":8141342025149658982, "value3":true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.515335, "value1":0.9138017166447161, "value2":4317898835838327168, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713454, "value1":0.43899440111826654, "value2":143487884063527116, "value3":true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185777, "value1":0.5460630190657308, "value2":3180840112779032107, "value3":false, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450264, "value1":0.9313181115462055, "value2":8328870888335802008, "value3":true, "__name__":"metric_00000661", "key_5":"k","key_6":"e","key_1":"d", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567912, "value1":0.6046426149595721, "value2":3206194822374359841, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.896555, "value1":0.6026393505223315, "value2":4823045987798108744, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710017, "value1":0.5880756405721221, "value2":4714155846612246340, "value3":true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265002, "value1":0.824988737224255, "value2":8146626468221994382, "value3":true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029140, "value1":0.9928569418325714, "value2":1115616498765210900, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.994788, "value1":0.6191499887981317, "value2":1491651902318479820, "value3":true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310266, "value1":0.6540537390689438, "value2":8869916203173244502, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786610, "value1":0.5759332726822003, "value2":7232889220252179533, "value3":true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036697, "value1":0.31309276046690226, "value2":4197456548662367361, "value3":true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.684262, "value1":0.39030765212524027, "value2":8590977643860948235, "value3":true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.295632, "value1":0.4747190839423866, "value2":2819114746608115923, "value3":false, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.613481, "value1":0.02792670630263574, "value2":4669805487355644126, "value3":true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.462933, "value1":0.7475159252861773, "value2":2383279322222812533, "value3":true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179395, "value1":0.19102052131729996, "value2":1856517930791738382, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991666, "value1":0.3546374487300389, "value2":6354333338863606534, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.568174, "value1":0.4332276746738033, "value2":8645759887904070759, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.321899, "value1":0.2992200951220634, "value2":3116962887997170979, "value3":true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390308, "value1":0.02747272568305014, "value2":926243844801181597, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.805520, "value1":0.5096841522295752, "value2":7414378226567615667, "value3":true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230674, "value1":0.03960399937089453, "value2":1710851993634315848, "value3":true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.425150, "value1":0.4167707239792382, "value2":5244023674899540768, "value3":true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879335, "value1":0.4867774224489378, "value2":553896137255768087, "value3":false, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.021415180040758975, "value2":6576886716131511876, "value3":true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.875807, "value1":0.22236040526017317, "value2":8247944924957719281, "value3":true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.596786, "value1":0.5382707900600514, "value2":6504957777564642511, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853544, "value1":0.6591705273657046, "value2":6314593273328284879, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.792273, "value1":0.8726534877662263, "value2":8304088035724056780, "value3":true, "__name__":"metric_00000689", "key_8":"a","key_3":"i","key_5":"h", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050549, "value1":0.962790936441039, "value2":4045656520780865827, "value3":false, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493970, "value1":0.39340114596435655, "value2":5294670971014916935, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251929, "value1":0.5462996764305353, "value2":5897296033220091874, "value3":true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115726, "value1":0.8575474248088841, "value2":2146148594913460918, "value3":true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.787443, "value1":0.8185914401069716, "value2":6891346883169646879, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.982163, "value1":0.8404209142239473, "value2":2046890513913645569, "value3":false, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529940, "value1":0.15216580562912718, "value2":235763707139823565, "value3":true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036141, "value1":0.21797894391861414, "value2":2037043718914992589, "value3":false, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489425, "value1":0.578085671307559, "value2":1968373430108550733, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377536, "value1":0.5230650685409198, "value2":1948377520185967051, "value3":true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.644803, "value1":0.09118740593062584, "value2":3209698888069603256, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149640, "value1":0.2149828010685946, "value2":324803347557649497, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.788006, "value1":0.35151805877228876, "value2":5736730574020425869, "value3":true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528326, "value1":0.12892234054630594, "value2":40722644873633005, "value3":true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.517453, "value1":0.12096018419440699, "value2":8591818996293676301, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580879, "value1":0.18567989692889683, "value2":2424108579790648867, "value3":false, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.418474, "value1":0.10109548853357979, "value2":6805501278544557691, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192558, "value1":0.5389855440079544, "value2":2468356112053849635, "value3":true, "__name__":"metric_00000706", "key_6":"h","key_9":"d","key_5":"e", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700652, "value1":0.8426222872981881, "value2":5586434719045643994, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836161, "value1":0.6261463649204317, "value2":7998455739612119, "value3":false, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.410987, "value1":0.19322483253914569, "value2":7602154106800870019, "value3":true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920472, "value1":0.07895320792744571, "value2":3621399116338923589, "value3":false, "__name__":"metric_00000709", "key_8":"a","key_9":"g","key_1":"c", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058885, "value1":0.038506638786952846, "value2":8443519094492233492, "value3":false, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.357953, "value1":0.49873265104684444, "value2":6689514258543734175, "value3":true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444144, "value1":0.7257783571311162, "value2":2132381976912359864, "value3":true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.003034, "value1":0.21025100795379004, "value2":4437597220292182437, "value3":false, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794651, "value1":0.0987214108191527, "value2":4856891238525466286, "value3":true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.101167, "value1":0.5530900889416458, "value2":5869924593453820746, "value3":true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813524, "value1":0.7032391307910569, "value2":2962680547459287448, "value3":true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102641, "value1":0.05280577735508823, "value2":6197989994765662479, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.042807, "value1":0.46787236774210145, "value2":6987091087352317601, "value3":true, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023412, "value1":0.4117202092054757, "value2":5734736277249168119, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.582529, "value1":0.14733901739015626, "value2":2008347713878225513, "value3":false, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843555, "value1":0.45648522904759514, "value2":8748231721848476899, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.609920, "value1":0.5332106841490003, "value2":3789122902542055079, "value3":true, "__name__":"metric_00000722", "key_9":"d","key_0":"h","key_1":"b", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.004873, "value1":0.9250068088247592, "value2":4060601419917043509, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598410, "value1":0.20935422794575423, "value2":3458524447228408749, "value3":true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327673, "value1":0.6622534351078718, "value2":1975526610989314262, "value3":true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701834, "value1":0.018453031281240855, "value2":7208203361477507972, "value3":false, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.544888, "value1":0.022192838541852087, "value2":420807956823837942, "value3":true, "__name__":"metric_00000727", "key_8":"e","key_2":"i", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.333657, "value1":0.6116791297905804, "value2":6780782056382077493, "value3":true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.224370, "value1":0.58595743970275, "value2":2273745727787058306, "value3":false, "__name__":"metric_00000729", "key_5":"h","key_0":"f","key_1":"a", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874813, "value1":0.4160482798834957, "value2":5130896603060858656, "value3":false, "__name__":"metric_00000732", "key_8":"k","key_4":"i", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800557, "value1":0.6485257630686503, "value2":8645194213531066428, "value3":false, "__name__":"metric_00000730", "key_6":"b","key_2":"g","key_3":"e", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798578, "value1":0.3562881109226897, "value2":1936635714706401113, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222829, "value1":0.18800753190841593, "value2":3868467029118261102, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394596, "value1":0.9715849939407708, "value2":2892398559190644533, "value3":true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088141, "value1":0.19677712178571452, "value2":1290002108232651066, "value3":false, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046604, "value1":0.7961651460552367, "value2":6142675575156594148, "value3":false, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755275, "value1":0.26382736011465824, "value2":4790349544446362608, "value3":true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.300482, "value1":0.6458439650096045, "value2":5076464481756841749, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294901, "value1":0.3937755673758644, "value2":4837753175838197566, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717849, "value1":0.8465941273216573, "value2":4538488428913350745, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018027, "value1":0.7099513420503992, "value2":6589927070191288064, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686148, "value1":0.31341443389871104, "value2":4542163293900247497, "value3":true, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554562, "value1":0.6021518819649633, "value2":657935532838913237, "value3":false, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549050, "value1":0.9913051319754902, "value2":9133655827436281017, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.564021, "value1":0.3933144754852601, "value2":8691343872756358848, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.555563, "value1":0.12216755310514293, "value2":8181517965767093469, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991822, "value1":0.9046701031504584, "value2":7169764745600183537, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536610, "value1":0.6245723927615995, "value2":5410550698568344254, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.640481, "value1":0.7953791600529995, "value2":8896491255145975602, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.983650, "value1":0.39162313489970674, "value2":5865250699451794074, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.748358, "value1":0.3770029381067136, "value2":7671767291617145847, "value3":true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632308, "value1":0.9672641688276767, "value2":3043416827366033954, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829101, "value1":0.63693380465596, "value2":3688916592463553468, "value3":false, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.271187, "value1":0.7822523952945989, "value2":7538651950678502209, "value3":false, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757851, "value1":0.4993335289484615, "value2":8433879559771522559, "value3":false, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.022503, "value1":0.5685717542133112, "value2":5232283843486040657, "value3":false, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046586, "value1":0.6991953129630171, "value2":767301512539468704, "value3":true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170088, "value1":0.9099698273235788, "value2":6375281583140134087, "value3":true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.819909, "value1":0.9395198166293325, "value2":9208855013717569494, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078675, "value1":0.35105170816316494, "value2":7441300028994205, "value3":false, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977562, "value1":0.27747207540421476, "value2":2610474267165011454, "value3":false, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.577027, "value1":0.4084658901530552, "value2":7348713799224820014, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705040, "value1":0.13989635719875274, "value2":8770545555481284876, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.821460, "value1":0.5135499070770553, "value2":1707896495831766919, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369139, "value1":0.07309187404043328, "value2":5262455623289086241, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536485, "value1":0.3546193597483191, "value2":5532328840062432345, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632890, "value1":0.87913605477188, "value2":4206324763882989783, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.352893, "value1":0.22467668553540646, "value2":8020391351501404610, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058110, "value1":0.08476564402932071, "value2":2402316898103266567, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312920, "value1":0.14715981410041812, "value2":515215788552749041, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335941, "value1":0.5607746607254104, "value2":4029056354863603408, "value3":false, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493922, "value1":0.5228229815154212, "value2":5917395680452547526, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.721451, "value1":0.8180794492453682, "value2":1670905717947422538, "value3":false, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.347574, "value1":0.7935601761833552, "value2":6084751528698562987, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.722657, "value1":0.4075231573944958, "value2":7748408088453297294, "value3":false, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.451863, "value1":0.3657057400324225, "value2":7302526748126423806, "value3":false, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495468, "value1":0.10049524830706323, "value2":7515132691589300738, "value3":true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193570, "value1":0.5847911145927462, "value2":4656707981929867290, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866643, "value1":0.9003805975682342, "value2":3827225598428319501, "value3":false, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.535740, "value1":0.5365482035917958, "value2":3695801733453249426, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.197799, "value1":0.7228560209917014, "value2":5697296448849523746, "value3":false, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536756, "value1":0.2916665533141402, "value2":8616484406626615233, "value3":true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231668, "value1":0.5491980830875538, "value2":5401247188111922849, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836905, "value1":0.6668911346325062, "value2":5956397501314084543, "value3":true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.060078, "value1":0.41433534941101835, "value2":259739043603601632, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933921, "value1":0.15068665469322526, "value2":4615815953640952732, "value3":false, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.147080, "value1":0.17748824888193676, "value2":838395034976203435, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662077, "value1":0.7882764714348716, "value2":4457961488907810285, "value3":true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023154, "value1":0.24361462379316554, "value2":7839025824541823553, "value3":false, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826915, "value1":0.569504859021508, "value2":6251982817444532308, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585595, "value1":0.8340020457470065, "value2":2938024413765410558, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146190, "value1":0.9581727102338353, "value2":4413141265483838870, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.245440, "value1":0.6417183759653234, "value2":5364021531028647444, "value3":false, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493343, "value1":0.4938700840392069, "value2":3091997225543832896, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807379, "value1":0.4690524091058998, "value2":3061093742738985330, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.208752, "value1":0.8639618600039569, "value2":3187072739982271504, "value3":true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.011216, "value1":0.18367276511432976, "value2":2360627840986057827, "value3":true, "__name__":"metric_00000799", "key_8":"d","key_6":"i", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.440539, "value1":0.6977700817568234, "value2":3581694346818534680, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489990, "value1":0.309872070478555, "value2":7457465639523954616, "value3":false, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113130, "value1":0.6897957784827682, "value2":90019976280058795, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.604031, "value1":0.797243453747431, "value2":1234756804789352079, "value3":true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.881184, "value1":0.28155440991949554, "value2":4070028978929644169, "value3":true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.367426, "value1":0.016167903292426008, "value2":4863636678376461059, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.769132, "value1":0.967909876972054, "value2":9131005500727231455, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.905490, "value1":0.6194024275206055, "value2":210353074026913884, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.915138, "value1":0.6781979759382127, "value2":6372020977221093753, "value3":true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093709, "value1":0.6187727010105798, "value2":312866041452100946, "value3":true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758312, "value1":0.8273322264897901, "value2":7512565884920091651, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.284383, "value1":0.39739645167776044, "value2":8773985992873355471, "value3":true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216803, "value1":0.8653155039130678, "value2":8937229562905505386, "value3":true, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.770263, "value1":0.9293790766191896, "value2":4560937013728510181, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096191, "value1":0.22323546625261562, "value2":8010469458396680860, "value3":true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.984434, "value1":0.9454942818827864, "value2":660593493740706018, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222194, "value1":0.02420800916517378, "value2":5602553742039024765, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752900, "value1":0.3672193480709247, "value2":6716506478528347816, "value3":false, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.967703, "value1":0.2944363320939628, "value2":7783913193752729609, "value3":true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.600560, "value1":0.1440348243160143, "value2":9110640288768831559, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400353, "value1":0.2233756240085349, "value2":2151981935371948568, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.269597, "value1":0.9641297731318997, "value2":8169933506866910786, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.486730, "value1":0.1006988853266946, "value2":3116477845424070312, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953298, "value1":0.4756644907363518, "value2":2464865227545117596, "value3":false, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705992, "value1":0.8173093209310923, "value2":8219739418749111053, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.851844, "value1":0.28729444422491424, "value2":191979366247033806, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616037, "value1":0.08828307120467993, "value2":1006916952645293364, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.007265, "value1":0.43891779173017603, "value2":4642029911324023153, "value3":false, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408095, "value1":0.9633558663387003, "value2":2764041604557827140, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825923, "value1":0.12824733189500837, "value2":7952369571604255477, "value3":true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853643, "value1":0.8488628783599163, "value2":3110877858489702411, "value3":false, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686891, "value1":0.5538740126823626, "value2":7618257543247002890, "value3":true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408459, "value1":0.2427343835864084, "value2":1637171870696137413, "value3":false, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.635245, "value1":0.42768890444095087, "value2":4013428109133869754, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793205, "value1":0.23512439386454564, "value2":5799487535459689842, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920141, "value1":0.9415332414772868, "value2":1520706400052294853, "value3":true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.930133, "value1":0.47818008921632604, "value2":4567447002026438924, "value3":false, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.599550, "value1":0.9293362893760267, "value2":1195481394235809552, "value3":false, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903387, "value1":0.11210859774886604, "value2":3335652489364502188, "value3":true, "__name__":"metric_00000839", "key_4":"h","key_0":"f", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698033, "value1":0.3148016773642106, "value2":4939787017916634516, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813706, "value1":0.3325590668831616, "value2":3252287249574729793, "value3":false, "__name__":"metric_00000841", "key_2":"d","key_6":"b","key_1":"i", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.048837, "value1":0.053013772957975554, "value2":4819883401481045104, "value3":false, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310950, "value1":0.41839244392518726, "value2":8733225969530074129, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.062225, "value1":0.858129811136947, "value2":7961256500615532109, "value3":true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428880, "value1":0.2502872252911392, "value2":7064927828432353118, "value3":false, "__name__":"metric_00000842", "key_6":"c","key_0":"c","key_3":"b", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560002, "value1":0.11724155985130476, "value2":3115965127965032390, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657940, "value1":0.5473444163031204, "value2":8727302484888278411, "value3":true, "__name__":"metric_00000847", "key_9":"d","key_0":"k","key_5":"g", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554826, "value1":0.376684927630788, "value2":2585922874708871992, "value3":false, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079791, "value1":0.9982500603963644, "value2":7391513557121794219, "value3":true, "__name__":"metric_00000846", "key_8":"e","key_1":"h","key_4":"k", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.174380, "value1":0.13350980294424206, "value2":9115933928609206333, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690448, "value1":0.8022085172421284, "value2":5503579136981394270, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178959, "value1":0.5153762777175099, "value2":4750624720468516504, "value3":true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880368, "value1":0.3826318223193023, "value2":3994162284701586149, "value3":true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594211, "value1":0.0032667252772956594, "value2":211951125332337865, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080419, "value1":0.3371270895888467, "value2":970359359029436118, "value3":false, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354035, "value1":0.6830560620508377, "value2":351440888758257733, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686006, "value1":0.15832224781392848, "value2":7973904723234304705, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688086, "value1":0.14780349920952468, "value2":2027463870413046281, "value3":false, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.407959, "value1":0.04437078472668187, "value2":1566333717176214387, "value3":true, "__name__":"metric_00000856", "key_6":"d","key_3":"i","key_5":"a", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943264, "value1":0.6495072511389037, "value2":5853361075489349645, "value3":false, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400773, "value1":0.2676114612948212, "value2":6666522398737719346, "value3":true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228255, "value1":0.29124903807175784, "value2":128611338987653451, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702835, "value1":0.39306652609769904, "value2":4553551121428133233, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346580, "value1":0.9157209666515556, "value2":4177607716262676740, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000990, "value1":0.3155726084083431, "value2":7473596942971857359, "value3":true, "__name__":"metric_00000862", "key_8":"d","key_1":"b","key_4":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380807, "value1":0.5730835803736063, "value2":6788452783424900655, "value3":true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018359, "value1":0.11046900836774676, "value2":61241953976306769, "value3":true, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097897, "value1":0.6529098432421965, "value2":9123131154054179256, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118936, "value1":0.312336727346538, "value2":7294156215339952794, "value3":false, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.256520, "value1":0.817765585396917, "value2":4276314621602341409, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.239028, "value1":0.27988068073239764, "value2":8048493989378214976, "value3":false, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.359463, "value1":0.33204054027814056, "value2":4286372625461580954, "value3":false, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625120, "value1":0.8053374337323631, "value2":1700890355040268131, "value3":false, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.519453, "value1":0.2764225180730322, "value2":3111696426903897415, "value3":false, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.019685, "value1":0.5511793734194003, "value2":3991968620227079942, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.040695, "value1":0.6967938051735891, "value2":8919927137208366483, "value3":true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750443, "value1":0.9319592750217813, "value2":7402732360820495311, "value3":true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.497932, "value1":0.6545667018250457, "value2":7616613351817353333, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.773399, "value1":0.5580363664821956, "value2":314222158348617253, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.869323, "value1":0.7418088662526952, "value2":8657709877874943583, "value3":true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812016, "value1":0.4480192862830272, "value2":8470890763908250393, "value3":true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079326, "value1":0.3872756599905638, "value2":4349661970334556580, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.186686, "value1":0.8815811840060216, "value2":938888750792010987, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235117, "value1":0.8673030373216688, "value2":4951632884410168399, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335964, "value1":0.6073261439232968, "value2":8008517076279782152, "value3":false, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063573, "value1":0.12394184604832796, "value2":1362755555847887698, "value3":true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.685355, "value1":0.2361653248224142, "value2":2210259628223947558, "value3":true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029806, "value1":0.3517864573309168, "value2":6754043647766872898, "value3":true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.593026, "value1":0.38937941917051955, "value2":1945974862147158833, "value3":true, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595711, "value1":0.9836966964700219, "value2":4159340747167974849, "value3":true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.143026, "value1":0.5945202518873367, "value2":3646754986945085167, "value3":true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733493, "value1":0.18903944802082667, "value2":8837255893708962102, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.120646, "value1":0.4261710458329441, "value2":6946544380857137617, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.163665, "value1":0.24245216879890383, "value2":555667735507028882, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934782, "value1":0.31976487170355755, "value2":6301272802564976177, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975795, "value1":0.2007851651150272, "value2":866319078855783895, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630757, "value1":0.2788228743238552, "value2":2540783851122363232, "value3":false, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.661317, "value1":0.5218066875354199, "value2":8869741999431102494, "value3":true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980967, "value1":0.7624092409640837, "value2":628203903335846730, "value3":false, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380622, "value1":0.4837921884626896, "value2":3628530218991586661, "value3":false, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.248953, "value1":0.02097807162306582, "value2":4598542592315882026, "value3":true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.384743, "value1":0.7608954854738191, "value2":1886663813765371215, "value3":false, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.950140, "value1":0.2550158514673341, "value2":4065397223823067237, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.516999, "value1":0.6292445006161491, "value2":4125156478992786356, "value3":true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733841, "value1":0.37206344867268365, "value2":3082112298619357468, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086119, "value1":0.5738504787109479, "value2":5573707094695082413, "value3":true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800475, "value1":0.7767745931253813, "value2":6702226199646136395, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.175876, "value1":0.8346328055447434, "value2":1066680083349750545, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752359, "value1":0.7257723037237084, "value2":7568317247183343047, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.165816, "value1":0.19937439179339156, "value2":2068495750767911677, "value3":false, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.597980, "value1":0.6555890535602861, "value2":2883065227864053972, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.138569, "value1":0.22532513024599726, "value2":8297154458719010882, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.668715, "value1":0.11645984529741488, "value2":5555518408390232617, "value3":true, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063265, "value1":0.001988016117254865, "value2":8483308279719955566, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086644, "value1":0.4644931461203292, "value2":6366519140385299368, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.140571, "value1":0.5416650487590775, "value2":1665979086436164947, "value3":true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.045805, "value1":0.9888897436719353, "value2":7697021265197143502, "value3":true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272912, "value1":0.2088625700639475, "value2":8643238667192348189, "value3":true, "__name__":"metric_00000916", "key_9":"k","key_3":"e","key_7":"b", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289278, "value1":0.005139072821858179, "value2":1003021960883868455, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.242063, "value1":0.32725526486053264, "value2":2911166424297597473, "value3":false, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.393077, "value1":0.6314586962746932, "value2":2631691711252794732, "value3":true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.274319, "value1":0.9496413905130506, "value2":3462918152034782288, "value3":true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659954, "value1":0.9955163986622194, "value2":2213038727032863641, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657842, "value1":0.11877952023160425, "value2":2754577958762836530, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146197, "value1":0.5152391159350134, "value2":4396760573611697803, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759786, "value1":0.051119901574673096, "value2":965796598734594927, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.891935, "value1":0.6695409149263635, "value2":7204593893035944941, "value3":true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096828, "value1":0.7785299263381299, "value2":843269019498255824, "value3":true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822107, "value1":0.06887356745350987, "value2":3890683236062212546, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818726, "value1":0.3529788280342954, "value2":2023962428787621327, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.356164, "value1":0.8626419946204176, "value2":3137373073213134707, "value3":false, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855692, "value1":0.3727202998903748, "value2":1868074746512330224, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.899737, "value1":0.49932818019000214, "value2":5247723900283585273, "value3":false, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205758, "value1":0.752048623202744, "value2":405978535011434243, "value3":false, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.492517, "value1":0.7587591048194695, "value2":570855823822973980, "value3":true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025167, "value1":0.9352906688766658, "value2":409025014085718364, "value3":false, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524374, "value1":0.456131184246001, "value2":6492711389883342442, "value3":true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.173025, "value1":0.7216031002138218, "value2":1900883846963453532, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991241, "value1":0.9435537729789849, "value2":6744125432328938384, "value3":false, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126845, "value1":0.9483471232225058, "value2":2429204263257823531, "value3":true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079973, "value1":0.14998274238182085, "value2":1609358502486987778, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257883, "value1":0.09750371692825593, "value2":5050315729103715843, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.074081, "value1":0.21018141055409362, "value2":9169596995481234183, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218738, "value1":0.9047134659722221, "value2":9140209877004134552, "value3":false, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.745794, "value1":0.39056909591189853, "value2":4170147818303980952, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179295, "value1":0.9580414905588824, "value2":4615545745316982257, "value3":false, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442990, "value1":0.018901929544229163, "value2":2360761388936503714, "value3":true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.422188, "value1":0.5272513199907374, "value2":3200950480296318673, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192831, "value1":0.33577000839194515, "value2":8465993285574620045, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.227691, "value1":0.814891459750041, "value2":5020817184863163049, "value3":false, "__name__":"metric_00000947", "key_7":"f","key_1":"d","key_2":"h", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171624, "value1":0.32281185598307915, "value2":8148752929610663940, "value3":true, "__name__":"metric_00000949", "key_2":"c","key_7":"h","key_1":"b", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.490347, "value1":0.4777481785589013, "value2":2903978983753469512, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217872, "value1":0.6839140142887666, "value2":3130769003103218800, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.992440, "value1":0.1702388495618268, "value2":6704155277469548917, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.834388, "value1":0.855009067710764, "value2":8656769438285137265, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230809, "value1":0.6026285375258597, "value2":1622052925759175826, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344218, "value1":0.6418025631389815, "value2":8245500230948114788, "value3":true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086815, "value1":0.450124296857116, "value2":4866053118697224871, "value3":true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802304, "value1":0.9289196028774398, "value2":6637963067322995548, "value3":true, "__name__":"metric_00000955", "key_6":"j","key_7":"d","key_1":"e", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.840991, "value1":0.6695124623580792, "value2":5437276365728147032, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.833585, "value1":0.22730504686256575, "value2":1520963367111518952, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.744045, "value1":0.8193501328748979, "value2":279541774649644194, "value3":false, "__name__":"metric_00000959", "key_9":"g","key_0":"b","key_2":"h", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.736126, "value1":0.6177732267307673, "value2":738077807298530261, "value3":false, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131891, "value1":0.46856485152824584, "value2":1245296884531108950, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419961, "value1":0.9237029205265445, "value2":1223547652782219884, "value3":false, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512633, "value1":0.3170684617445516, "value2":3777467556351824325, "value3":false, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630355, "value1":0.021704073637768814, "value2":8674907845809369167, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.309670, "value1":0.16245539540033113, "value2":5206956291412580457, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.348146, "value1":0.7658591727792013, "value2":8113096525411729139, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346099, "value1":0.09833103805087655, "value2":3329067421410580481, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086337, "value1":0.8850305283927763, "value2":8626010840258936486, "value3":false, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479373, "value1":0.20404187483062786, "value2":2412557827504873850, "value3":false, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921982, "value1":0.7871473016496364, "value2":1874563226125050237, "value3":false, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415457, "value1":0.9305937676701617, "value2":4246181816757480088, "value3":false, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035701, "value1":0.11812212256088726, "value2":7168381109733048473, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.225156, "value1":0.9869870812720318, "value2":137813480312065053, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836380, "value1":0.860500054517146, "value2":6804060104865284962, "value3":false, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289062, "value1":0.385288303480441, "value2":3917001550422656910, "value3":false, "__name__":"metric_00000975", "key_7":"g","key_1":"k","key_2":"a", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885780, "value1":0.1315685249986314, "value2":2596575576892432777, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948363, "value1":0.36071231455630776, "value2":628986020889092104, "value3":true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369788, "value1":0.40617296244272144, "value2":1817522335798570734, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086188, "value1":0.6439285363835136, "value2":4867721675008841698, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.995659, "value1":0.2954211415738649, "value2":316663728175376955, "value3":true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.129393, "value1":0.050215829131005675, "value2":6231361900716566583, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400907, "value1":0.39112211803964236, "value2":5808809619414218326, "value3":true, "__name__":"metric_00000980", "key_7":"f","key_0":"d","key_4":"d", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717436, "value1":0.943850779238098, "value2":1447465700474277383, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217700, "value1":0.1707128166287673, "value2":4689212155367810810, "value3":true, "__name__":"metric_00000983", "key_9":"e","key_0":"b","key_7":"c", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.908215, "value1":0.13836337191309858, "value2":3286491733476954470, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383068, "value1":0.3171639131993519, "value2":5768120780013343252, "value3":true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758654, "value1":0.8248294031178783, "value2":1259633164417337834, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885742, "value1":0.7054910487633926, "value2":965825539457944695, "value3":false, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450755, "value1":0.5160791051802902, "value2":980291621965651603, "value3":true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280407, "value1":0.20023478153485316, "value2":216774102527727538, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953286, "value1":0.17708932991625895, "value2":7861520056013490110, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428154, "value1":0.3840775072053381, "value2":5128389599856507984, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650144, "value1":0.5475742595996844, "value2":37238921657180760, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880799, "value1":0.27347132520371564, "value2":7086278310918665364, "value3":true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179258, "value1":0.09483479166767275, "value2":3722096865477015794, "value3":false, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025175, "value1":0.24738190446733826, "value2":2326403401160366476, "value3":true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680444, "value1":0.6441894024482266, "value2":2704266700740984761, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830593, "value1":0.6314118537062262, "value2":8997005770669681872, "value3":false, "__name__":"metric_00000999", "key_9":"h","key_3":"c","key_5":"b", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177647, "value1":0.3203303212434556, "value2":8965923512665629393, "value3":false, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923489, "value1":0.29642566667944875, "value2":1203313530737226131, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.811446, "value1":0.5561455146353509, "value2":570880137152142782, "value3":false, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544802, "value1":0.9164364466036977, "value2":3723655635493793654, "value3":false, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.542566, "value1":0.7689437007202478, "value2":3938961297360550598, "value3":false, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819459, "value1":0.4225560622422357, "value2":2765920289473813577, "value3":true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971131, "value1":0.7588578513174423, "value2":554701584189491661, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638701, "value1":0.9383542993382983, "value2":2675775339679034911, "value3":false, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658985, "value1":0.8385957143790167, "value2":9118144992996965702, "value3":true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449591, "value1":0.6354046681746691, "value2":4796404340287643460, "value3":true, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.851526, "value1":0.7827632730352198, "value2":4383742830241177102, "value3":true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929626, "value1":0.7374713580232243, "value2":6076523832966313810, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.289625, "value1":0.6865021261714863, "value2":2978049869639889816, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394738, "value1":0.6344060546968167, "value2":293482016535092319, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.487799, "value1":0.01884442354627017, "value2":6314666407653758310, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.189751, "value1":0.5131352606856101, "value2":5370657937415537626, "value3":false, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.532641, "value1":0.7500243297537671, "value2":5814407549612083184, "value3":false, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413938, "value1":0.35879356972502013, "value2":7060341260629410136, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464627, "value1":0.5028910273480031, "value2":4767839332233680092, "value3":false, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.840321, "value1":0.13633703551235915, "value2":2307803453888302997, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177282, "value1":0.5212414108183161, "value2":7948818597964566740, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394656, "value1":0.22032364884959718, "value2":8010953658147063588, "value3":true, "__name__":"metric_00000023", "key_4":"b","key_2":"j", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.193561, "value1":0.6737972615452449, "value2":7205681986933877897, "value3":true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919748, "value1":0.5508022412545106, "value2":1935155841798114792, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.549841, "value1":0.6412542942797744, "value2":5725068566993480801, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755602, "value1":0.7570856701300407, "value2":2582449631680230244, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569314, "value1":0.9682571063393881, "value2":3213998604408735177, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220995, "value1":0.4682404519517274, "value2":1296712240822053674, "value3":false, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030188, "value1":0.13463362539315563, "value2":6167313691161388680, "value3":false, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955102, "value1":0.12893740940580098, "value2":937550317375879979, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912054, "value1":0.23584661351992364, "value2":8270059203922837294, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248552, "value1":0.873739706342713, "value2":5483897643292875706, "value3":false, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.668856, "value1":0.06480463060086489, "value2":3429839268398826587, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628990, "value1":0.06856495811246685, "value2":2353082280387987376, "value3":true, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.364427, "value1":0.8257731797802594, "value2":8162649044337833619, "value3":false, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488610, "value1":0.47476344691483213, "value2":7563742944450884141, "value3":true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.659600, "value1":0.22983785061318257, "value2":5566259052586123346, "value3":false, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715473, "value1":0.5778415124343756, "value2":379000163266106555, "value3":true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265567, "value1":0.856852436111815, "value2":96718295321822248, "value3":true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647902, "value1":0.9996700403145788, "value2":6837057660421261848, "value3":false, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.866334, "value1":0.8219801167193398, "value2":8280931000915145951, "value3":false, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144409, "value1":0.604589772741639, "value2":4205152374038619548, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192185, "value1":0.27115827101755235, "value2":5770338217899591097, "value3":true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300761, "value1":0.009332804989256857, "value2":4018403627826051427, "value3":false, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.312951, "value1":0.45431720098201106, "value2":2937253947933810175, "value3":false, "__name__":"metric_00000044", "key_4":"d","key_0":"j","key_1":"c", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647625, "value1":0.3268497558683786, "value2":5914220923444612314, "value3":true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878656, "value1":0.9409966965511731, "value2":1289273008938800365, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057134, "value1":0.3301792558402197, "value2":3318812578583185116, "value3":true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.439955, "value1":0.2652202727470005, "value2":6508684150448276421, "value3":false, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827995, "value1":0.5808204202626653, "value2":2858312188918612316, "value3":false, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863057, "value1":0.013094412054074, "value2":8217074167029879656, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936866, "value1":0.17076998229838808, "value2":1405785613574281058, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.188817, "value1":0.4102266284907626, "value2":9200292054736167191, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919291, "value1":0.849823159364745, "value2":8783211669019586620, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.684825, "value1":0.5612249810486886, "value2":2459193643509253683, "value3":false, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.132931, "value1":0.06165133682856612, "value2":3724912889225979746, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939721, "value1":0.7022523416408233, "value2":3787551523413319921, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423514, "value1":0.16397574471331047, "value2":2217544936229011402, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556341, "value1":0.3363631636072514, "value2":8296724122152500212, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.797045, "value1":0.9775102685569385, "value2":8717038963718857217, "value3":true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.037101, "value1":0.5188119627862039, "value2":8253271773680540946, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.472369, "value1":0.723269226446299, "value2":1830427992517853225, "value3":true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350765, "value1":0.45427831592086265, "value2":2459543486390463113, "value3":true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.172879, "value1":0.9405309891223543, "value2":2118657389939971315, "value3":true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902535, "value1":0.42952097429562314, "value2":3142662533255542276, "value3":true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863286, "value1":0.06876792355358291, "value2":1443459731022840202, "value3":false, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.137921, "value1":0.6964987593861792, "value2":7256675183245953924, "value3":true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.477178, "value1":0.3581558179516239, "value2":6351662172251250145, "value3":false, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286204, "value1":0.148299090562339, "value2":1443868893374130520, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286734, "value1":0.45842257280884463, "value2":1988366284391476937, "value3":false, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787180, "value1":0.5028191040745897, "value2":4148707399092925318, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.586935, "value1":0.0518139414168187, "value2":8586911735374624592, "value3":true, "__name__":"metric_00000072", "key_9":"k","key_0":"g","key_3":"g", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021854, "value1":0.8978228143933898, "value2":1413164249310483673, "value3":true, "__name__":"metric_00000073", "key_6":"b","key_4":"k", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318004, "value1":0.5329901050180244, "value2":700502281930980696, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381834, "value1":0.37533803419078043, "value2":1751644850942508358, "value3":false, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781575, "value1":0.7741607009730129, "value2":7455242385880444942, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.901091, "value1":0.9760509636456571, "value2":1834882752759920223, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805005, "value1":0.18640631542045302, "value2":3539140922926641426, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734757, "value1":0.14536274157280238, "value2":6893258320350162281, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.340380, "value1":0.2042782161421506, "value2":2006565586318459391, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.271132, "value1":0.008492488378786304, "value2":6129670920743570493, "value3":false, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947658, "value1":0.35165969319536833, "value2":4951697844001184553, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.735049, "value1":0.4770090728759761, "value2":2452955784370225782, "value3":true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337551, "value1":0.7050663230338188, "value2":4249578695875539573, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.633134, "value1":0.6874967232601639, "value2":8449525705234658507, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412613, "value1":0.8000775398308927, "value2":5399334563984433319, "value3":true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.520666, "value1":0.3288020539774752, "value2":1684836256027900746, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063862, "value1":0.4985615503389622, "value2":5250749533704010894, "value3":false, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.776154, "value1":0.039036691522280384, "value2":8622652029902283303, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.115396, "value1":0.2649092470624877, "value2":678667215803198769, "value3":false, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.728254, "value1":0.3531001293726284, "value2":2004607802946409884, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242732, "value1":0.9820165334533849, "value2":7506111952193927451, "value3":false, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969092, "value1":0.45874224514029954, "value2":6402112089715656350, "value3":false, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.528740, "value1":0.8320354986171156, "value2":8412309334445466976, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.907053, "value1":0.18136550029947487, "value2":6212227004856794281, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630067, "value1":0.8986694902601496, "value2":9213857752862359137, "value3":false, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133736, "value1":0.8699930415468984, "value2":4081508478635244211, "value3":false, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.198697, "value1":0.017191657622434766, "value2":1486415534842367537, "value3":true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.690493, "value1":0.3223491896226482, "value2":7372352323440542564, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.404617, "value1":0.6240663182478166, "value2":4691037631120171189, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120685, "value1":0.9919062682339983, "value2":8460618157376864427, "value3":true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.688005, "value1":0.3953177302348419, "value2":5961626802588462897, "value3":true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093767, "value1":0.23061987147968935, "value2":2892986034831352810, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.543984, "value1":0.16501271900248737, "value2":317259514400114949, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842969, "value1":0.25446808569640833, "value2":6002379582827585880, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269131, "value1":0.8842302265963506, "value2":2140205221312415384, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972033, "value1":0.9857791825041854, "value2":8003941482296671818, "value3":false, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508672, "value1":0.5492264125933453, "value2":4184449934578666153, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.812228, "value1":0.3691868332062656, "value2":940690450246364953, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033340, "value1":0.4347207480857335, "value2":7925423134984867831, "value3":true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921934, "value1":0.7008140011057534, "value2":3950204529429726081, "value3":false, "__name__":"metric_00000109", "key_8":"e","key_3":"i", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.159043, "value1":0.4015464920656329, "value2":6141367601228839763, "value3":false, "__name__":"metric_00000112", "key_8":"h","key_4":"e","key_5":"i", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207886, "value1":0.923323917554864, "value2":2538142530573084913, "value3":false, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560000, "value1":0.988479108310322, "value2":3590995919192964810, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324806, "value1":0.14623600559543842, "value2":7983994507018983485, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.278078, "value1":0.47875025775481017, "value2":9135606324353342316, "value3":false, "__name__":"metric_00000115", "key_9":"e","key_5":"i", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978376, "value1":0.646562029258319, "value2":6969654958747462453, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.038811, "value1":0.27278148584909157, "value2":3766949452289754200, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174153, "value1":0.456204859878279, "value2":5412969439184865040, "value3":false, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152394, "value1":0.6304591834176713, "value2":7508490770671251540, "value3":false, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079217, "value1":0.7093512830025359, "value2":8041107853739821078, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736508, "value1":0.9456361684627089, "value2":2695850503301130596, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.689579, "value1":0.8717416155878271, "value2":914033909063513507, "value3":true, "__name__":"metric_00000120", "key_7":"d","key_2":"d","key_3":"e", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716602, "value1":0.6181082501584888, "value2":7525886338688800823, "value3":false, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485128, "value1":0.805498923411989, "value2":3515190992320449261, "value3":false, "__name__":"metric_00000126", "key_2":"c","key_0":"k","key_1":"a", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761534, "value1":0.3987574394917293, "value2":565378419404884224, "value3":false, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234797, "value1":0.32687033897000584, "value2":4678721697918141366, "value3":false, "__name__":"metric_00000122", "key_9":"k","key_1":"g","key_8":"i", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099258, "value1":0.5165730450377122, "value2":7897877163898991964, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.855254, "value1":0.6113796894495002, "value2":5439835292581168290, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098218, "value1":0.6331475060559588, "value2":7978662530169854255, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.392907, "value1":0.7806228764363937, "value2":5349308434124377933, "value3":false, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174842, "value1":0.7994951238912167, "value2":213268455628772714, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.654871, "value1":0.23130216238112586, "value2":6353589500271587044, "value3":false, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255121, "value1":0.029192800621726373, "value2":2049991280590108935, "value3":false, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.454836, "value1":0.6345686250008671, "value2":8305250329173845884, "value3":false, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.196946, "value1":0.9679432223428265, "value2":4005113690213723694, "value3":false, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.514102, "value1":0.9460847081604973, "value2":2275533825263312731, "value3":true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.267365, "value1":0.5628401308203138, "value2":2718632724301463408, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.702558, "value1":0.3615743138551236, "value2":5265258619209069561, "value3":true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.157371, "value1":0.9914192491349261, "value2":7107333633697743515, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367362, "value1":0.30046228876641146, "value2":8731213027442541680, "value3":true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502940, "value1":0.33029188840622603, "value2":7282340644676767522, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773440, "value1":0.7398342089000881, "value2":6064862964524030137, "value3":false, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370133, "value1":0.5464083248908671, "value2":8595565634194089280, "value3":false, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819624, "value1":0.22563478104006193, "value2":6291072404703346422, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.822444, "value1":0.6431602385348965, "value2":3571751115605471571, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154889, "value1":0.661415650101091, "value2":2897716721956096259, "value3":false, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.706177, "value1":0.9541112843149568, "value2":4382405639585131280, "value3":false, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.892582, "value1":0.273637580698836, "value2":1569537455975017743, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912404, "value1":0.02958676932622315, "value2":864163789414463895, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164999, "value1":0.8749507184007825, "value2":9170076213645968437, "value3":false, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.609248, "value1":0.6230590575325244, "value2":1182671592625390639, "value3":true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402824, "value1":0.8555668782169765, "value2":3430464294959504849, "value3":false, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.905017, "value1":0.6152490194289306, "value2":3258848281050130827, "value3":true, "__name__":"metric_00000153", "key_8":"j","key_9":"d","key_5":"i", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.669648, "value1":0.6859907392101928, "value2":2219730699877135652, "value3":false, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435602, "value1":0.12441938240097333, "value2":3663118450934184743, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736951, "value1":0.9806346124707411, "value2":6495060662787801507, "value3":false, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502755, "value1":0.6962847488848957, "value2":7479724851955507611, "value3":true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.618421, "value1":0.31789340504585667, "value2":8738918799232514086, "value3":true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827476, "value1":0.7869780281846211, "value2":5474883946012414730, "value3":false, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490095, "value1":0.6962736110006601, "value2":6409713241665137545, "value3":false, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985050, "value1":0.2615773286934893, "value2":6009116516814474343, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502430, "value1":0.19305751383330208, "value2":1654152222563323901, "value3":true, "__name__":"metric_00000161", "key_3":"e","key_8":"b","key_2":"g", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349955, "value1":0.7772270603571774, "value2":73068209272270733, "value3":true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781804, "value1":0.6130002187106758, "value2":5094562490162409190, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.158107, "value1":0.12199479281505482, "value2":887695971080523742, "value3":false, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.575029, "value1":0.9090539881797902, "value2":3726306097696429975, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.197382, "value1":0.010351058305108105, "value2":3390177787518625882, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959617, "value1":0.10545857069830916, "value2":8523151888175341811, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789446, "value1":0.7606780493785521, "value2":2782849609854347711, "value3":true, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384471, "value1":0.3297433712153455, "value2":8732496713018613730, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.440642, "value1":0.05991406624275138, "value2":8221843418889420393, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485145, "value1":0.4324948321911064, "value2":1223798989569651011, "value3":false, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930239, "value1":0.797530519094042, "value2":3933281842105360464, "value3":false, "__name__":"metric_00000174", "key_7":"j","key_0":"h","key_4":"c", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921694, "value1":0.3827550255479697, "value2":1163470615431943983, "value3":false, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780691, "value1":0.8316468698353685, "value2":3969005124884722997, "value3":true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563279, "value1":0.5313651121371132, "value2":1275907147042029939, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099445, "value1":0.8066245132140291, "value2":1404833370385176869, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.897823, "value1":0.395354453907472, "value2":4125787827950753058, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562742, "value1":0.0008610845550682808, "value2":2683131454862577523, "value3":true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981050, "value1":0.6811899501971241, "value2":2963479936998599518, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.090076, "value1":0.7872489837661779, "value2":5920042185451264661, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904987, "value1":0.582385162743904, "value2":3052394546288639081, "value3":false, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.807620, "value1":0.8276864738323398, "value2":1770112128995343558, "value3":true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.313153, "value1":0.7000557306246208, "value2":8804754539464742313, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648178, "value1":0.6035738426070452, "value2":274869451073352526, "value3":false, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382503, "value1":0.7570596198853545, "value2":8494223821423922555, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981825, "value1":0.4954131435773338, "value2":5546139769442373925, "value3":false, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.896817, "value1":0.3057668662697838, "value2":2146747890118623175, "value3":false, "__name__":"metric_00000187", "key_5":"k","key_1":"b", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857258, "value1":0.7535991261099572, "value2":7241562330594412223, "value3":true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.954553, "value1":0.6272550393596944, "value2":7059824369010485707, "value3":true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611570, "value1":0.896770550906892, "value2":4157172203390286832, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.200060, "value1":0.6395440920752251, "value2":2313959820318356273, "value3":true, "__name__":"metric_00000189", "key_9":"k","key_2":"g","key_3":"j", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.305817, "value1":0.7944717065357103, "value2":9079049510197634026, "value3":false, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771177, "value1":0.5686457722223018, "value2":5926649416008895009, "value3":true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.194460, "value1":0.2030568091950128, "value2":5211606615226569619, "value3":false, "__name__":"metric_00000194", "key_2":"c","key_6":"b","key_1":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541777, "value1":0.38229879472765593, "value2":8330651996424197311, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566620, "value1":0.24307591832233774, "value2":2839998819511281148, "value3":false, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376323, "value1":0.5739727137071835, "value2":5399793491142182010, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480111, "value1":0.7895749118902282, "value2":7450043826116159674, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453645, "value1":0.4875344798695633, "value2":6926040231429456296, "value3":false, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734559, "value1":0.3865838466523982, "value2":9001676782517223243, "value3":true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.171659, "value1":0.33364493236506704, "value2":5739131266488499620, "value3":false, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.966196, "value1":0.1807801172943221, "value2":7703455497589255747, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087000, "value1":0.14745151659831435, "value2":3780273793092000978, "value3":false, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537557, "value1":0.6679609579098379, "value2":7524235382942600940, "value3":true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624433, "value1":0.43524838478159544, "value2":8707151351953730491, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.764878, "value1":0.9857854697655962, "value2":2628899540411422018, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087516, "value1":0.9235134986127471, "value2":1244667248757759893, "value3":false, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.092707, "value1":0.8123903328089516, "value2":1510741329638496293, "value3":true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212617, "value1":0.7803051371802152, "value2":1782941121051923216, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110139, "value1":0.9520250869731262, "value2":8515109734542519287, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217411, "value1":0.569745410096317, "value2":7158729472403289306, "value3":true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578681, "value1":0.4421036889589163, "value2":5172660091174561754, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.917747, "value1":0.9646587109518411, "value2":4932202474624997807, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490742, "value1":0.18375289991112742, "value2":5134730937919546380, "value3":true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698691, "value1":0.8462874020691947, "value2":6789099466073417633, "value3":false, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987429, "value1":0.47456618667904216, "value2":7333730409059985654, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.552031, "value1":0.8731675557704086, "value2":2383267067188016407, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.024510, "value1":0.7398931410894473, "value2":4688375563960023614, "value3":true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.885940, "value1":0.5267822661948509, "value2":3106942003445818164, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.128634, "value1":0.17304999217293912, "value2":4202058405826758012, "value3":true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919245, "value1":0.5246293149604686, "value2":7962001396490479168, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.784582, "value1":0.9051007718281217, "value2":6345441367106704431, "value3":true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.631365, "value1":0.18200897376922306, "value2":2743187358723124435, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632586, "value1":0.49203338164778787, "value2":310459975834593918, "value3":false, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.986910, "value1":0.21502125960121787, "value2":9213029499233048926, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.903530, "value1":0.20307359325640195, "value2":3425839773997981556, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495175, "value1":0.42906594953086846, "value2":1910613263092063181, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560265, "value1":0.27498766656590357, "value2":8893579102721900369, "value3":false, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.298348, "value1":0.05982261910728163, "value2":8036644582013512610, "value3":false, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624831, "value1":0.8394666399726175, "value2":1345154507982673038, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.123368, "value1":0.13834539586421774, "value2":6342191610731262762, "value3":false, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.086336, "value1":0.17439021756766837, "value2":6128400320706208802, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421248, "value1":0.39843282468447905, "value2":8680280397492385017, "value3":false, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139454, "value1":0.456434331043375, "value2":4836743646420773621, "value3":false, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351219, "value1":0.777135306661456, "value2":3903934605632263710, "value3":true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.414090, "value1":0.8143501039400677, "value2":4413351639615300728, "value3":false, "__name__":"metric_00000237", "key_7":"f","key_0":"d","key_5":"h", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231721, "value1":0.3138412091160347, "value2":9168517044165068794, "value3":true, "__name__":"metric_00000236", "key_1":"h","key_0":"d", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.230702, "value1":0.8804508000643604, "value2":512236854100511456, "value3":true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.708957, "value1":0.47311614120900003, "value2":7136526398161878544, "value3":false, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.475928, "value1":0.32108039730398563, "value2":7292530251105851041, "value3":true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.643383, "value1":0.9689860631255371, "value2":1734391035548707204, "value3":true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.792203, "value1":0.8844015947765959, "value2":6318884852252048224, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.323993, "value1":0.18316994233066625, "value2":3785806626345479252, "value3":true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.055153, "value1":0.9975090536646855, "value2":9101914663697333394, "value3":false, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762055, "value1":0.14538112505308043, "value2":6589674161435979608, "value3":false, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457017, "value1":0.9313588309515796, "value2":2386410717081263989, "value3":false, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118975, "value1":0.2538090940895061, "value2":2359952847484695171, "value3":true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887012, "value1":0.26127955373376677, "value2":7323797183712585078, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465777, "value1":0.39956363244564946, "value2":4290799816610955241, "value3":true, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919817, "value1":0.24654070770474018, "value2":6141631506827241212, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878721, "value1":0.7298060681766352, "value2":7628091662411871299, "value3":false, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308905, "value1":0.8062336262819177, "value2":3004783841504994012, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630086, "value1":0.9740479758667072, "value2":7021198218943014146, "value3":true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661157, "value1":0.0030467411917742306, "value2":8705979164600447258, "value3":true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865574, "value1":0.36919272421680055, "value2":4523649159933689100, "value3":true, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.185605, "value1":0.5443213859941823, "value2":3228781454351122262, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865038, "value1":0.09385639771871399, "value2":7227272424843868171, "value3":false, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798023, "value1":0.8684611657481685, "value2":2233953571760268466, "value3":true, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730804, "value1":0.08710486562835452, "value2":6236391362613657084, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.202824, "value1":0.4792114852099944, "value2":445497349923776863, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173553, "value1":0.039235946095809125, "value2":7869971288440786845, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908688, "value1":0.5052600498486294, "value2":4519674082380595002, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341267, "value1":0.007052586435709472, "value2":3150628489852334883, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881706, "value1":0.2885539517449333, "value2":1797762712648874995, "value3":false, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216914, "value1":0.4200170713130975, "value2":8711469515389655710, "value3":false, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.219622, "value1":0.8598302806140433, "value2":3863311400727592540, "value3":true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.656021, "value1":0.6463537908056233, "value2":1050074674903654190, "value3":true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641122, "value1":0.40560841010225995, "value2":742946648471490108, "value3":false, "__name__":"metric_00000268", "key_9":"d","key_6":"j","key_8":"c", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755478, "value1":0.42687719134929036, "value2":8589479839598188558, "value3":true, "__name__":"metric_00000270", "key_8":"d","key_0":"a","key_1":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.848432, "value1":0.9962729980353733, "value2":780000159187900550, "value3":true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.195275, "value1":0.6773988604333073, "value2":5522000829662160867, "value3":true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787345, "value1":0.6486468864516003, "value2":4138825123686123358, "value3":false, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.553619, "value1":0.7566207604613129, "value2":5341523136140526945, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982202, "value1":0.8174248836774913, "value2":4875324336295746206, "value3":true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317829, "value1":0.25574317763537074, "value2":3850513628767917465, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366612, "value1":0.634692324496315, "value2":8508176811169559100, "value3":false, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.931082, "value1":0.9070818575294899, "value2":2941126198839449331, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419466, "value1":0.6035953996525765, "value2":5226492465996637298, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207446, "value1":0.7148273386681443, "value2":6830161919863899962, "value3":false, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339115, "value1":0.9680080839353958, "value2":3346579129519946511, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329518, "value1":0.5052033589082856, "value2":8326166807060477796, "value3":false, "__name__":"metric_00000282", "key_2":"c","key_0":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314104, "value1":0.23744990369573346, "value2":6679180757317175227, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177954, "value1":0.6916907655144761, "value2":6598624424043999988, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220478, "value1":0.9419151786449748, "value2":6868454728302182507, "value3":true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.962386, "value1":0.29200153388880246, "value2":3531740125944089227, "value3":false, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.522912, "value1":0.4384617611060238, "value2":7012028395173793023, "value3":true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.701530, "value1":0.008494038169472137, "value2":4096870035163582532, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019574, "value1":0.540551382306373, "value2":7575443997806932703, "value3":true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558742, "value1":0.30242558718747614, "value2":1312291174476606343, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.597198, "value1":0.9990604033963557, "value2":5377437181323103863, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321750, "value1":0.8571357709325697, "value2":3223326577646383979, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976585, "value1":0.794736424631542, "value2":7297990135456560387, "value3":false, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053373, "value1":0.9010563829072253, "value2":5702408670366972568, "value3":false, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.500145, "value1":0.16016689363963474, "value2":7655127244946192653, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.347755, "value1":0.19254449629837572, "value2":1438411533080776332, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.047080, "value1":0.9596252631833848, "value2":4943940353426671568, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.737278, "value1":0.6304724450380977, "value2":5075506433971212474, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842472, "value1":0.9682019821931608, "value2":7537248780629599850, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210066, "value1":0.016073835132125416, "value2":1148941512571603438, "value3":false, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019003, "value1":0.15376503679496498, "value2":6744619422647456093, "value3":true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972351, "value1":0.6522824510722263, "value2":3635978187235320184, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464404, "value1":0.1987173098042302, "value2":2325620189736714816, "value3":false, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505822, "value1":0.8969967220271704, "value2":2772428523245878847, "value3":true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.858017, "value1":0.42532104055952885, "value2":7057664582771584406, "value3":false, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827767, "value1":0.11843519151662998, "value2":4642257980991996610, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457807, "value1":0.6680540521232995, "value2":1413093760337990232, "value3":true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307237, "value1":0.2483558233491924, "value2":1657535478313961463, "value3":false, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187495, "value1":0.8787274497214357, "value2":1012440122187350073, "value3":false, "__name__":"metric_00000309", "key_5":"h","key_6":"k","key_2":"b", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.935106, "value1":0.6622345557285527, "value2":873018521033719372, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212483, "value1":0.03664769373990424, "value2":8662944034974337497, "value3":false, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.279577, "value1":0.06940570148672788, "value2":3397490112016095138, "value3":false, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.679928, "value1":0.94467398885254, "value2":8140288354763339321, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348611, "value1":0.09177297933527545, "value2":2243053697955925595, "value3":true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033795, "value1":0.004266699548950342, "value2":8869596053230724066, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716957, "value1":0.952311087977132, "value2":3488323066281405120, "value3":true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275244, "value1":0.9153132837467353, "value2":6017024334903831134, "value3":false, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523199, "value1":0.8894820330529598, "value2":2495751532034294665, "value3":false, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.548536, "value1":0.36452752239967207, "value2":4087473006792877554, "value3":false, "__name__":"metric_00000319", "key_9":"g","key_2":"b", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452175, "value1":0.03794808741352215, "value2":8378297363369384496, "value3":false, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156999, "value1":0.3422567916396813, "value2":3950482866588863620, "value3":false, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508543, "value1":0.5441836551476523, "value2":5324891351187729379, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397072, "value1":0.8991563074648984, "value2":484650303993845367, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074791, "value1":0.08635154390096551, "value2":7636185572736407362, "value3":false, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214033, "value1":0.529318647161814, "value2":3428935700531030330, "value3":false, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023178, "value1":0.9863231971076605, "value2":952801547411635885, "value3":false, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.655562, "value1":0.1342499141610202, "value2":6341506872735720793, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834066, "value1":0.5801423696428913, "value2":2989261336292506730, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.107225, "value1":0.08595942695174727, "value2":7162816956772874074, "value3":true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.303930, "value1":0.5732092807009482, "value2":6606617148160680700, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.915970, "value1":0.10263255284529503, "value2":8480457956761770794, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356355, "value1":0.6141228451134201, "value2":7156945200312274543, "value3":false, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645757, "value1":0.7597072830851564, "value2":2605675246955821391, "value3":false, "__name__":"metric_00000333", "key_9":"e","key_1":"f","key_4":"b", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760044, "value1":0.048059905194606296, "value2":3102725962018871003, "value3":true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949450, "value1":0.803383211503761, "value2":5157179277981661443, "value3":true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124187, "value1":0.16836795463840862, "value2":5627024937490827799, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621202, "value1":0.11873453496853606, "value2":971627245321019634, "value3":true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762026, "value1":0.5957950719970895, "value2":1706809321238862597, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391493, "value1":0.7101377073031195, "value2":6966030075201405586, "value3":false, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.168903, "value1":0.22863352792292077, "value2":1427332489767053365, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698944, "value1":0.5031422927319381, "value2":2536670790706398066, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985917, "value1":0.7010791398723792, "value2":2424373004486759234, "value3":true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611987, "value1":0.023052053167488012, "value2":1572028785346213886, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.084732, "value1":0.8225285464369793, "value2":7247308476502074358, "value3":true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476584, "value1":0.6320619568566006, "value2":3013845267574452926, "value3":false, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.358865, "value1":0.9597368730404013, "value2":1895755548879639735, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559280, "value1":0.7939551495955407, "value2":9185811549617457738, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338153, "value1":0.6433609372521104, "value2":5456961662990895069, "value3":false, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120489, "value1":0.02598857402797232, "value2":2690067127374441413, "value3":false, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.026856, "value1":0.3509430162667547, "value2":5912250260618117609, "value3":true, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.911457, "value1":0.5658091707632997, "value2":3970272443516058585, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563432, "value1":0.24659033075994388, "value2":728046743750740729, "value3":true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.596212, "value1":0.5615056517168183, "value2":7033619460114819908, "value3":false, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051460, "value1":0.5771358250664421, "value2":8203484790496543671, "value3":false, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.824484, "value1":0.05697931696566816, "value2":6635808805223685263, "value3":true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627182, "value1":0.9130566878289693, "value2":8110244852760276756, "value3":false, "__name__":"metric_00000355", "key_3":"e","key_9":"e","key_2":"d", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207192, "value1":0.7274854390257628, "value2":5772335372245409001, "value3":true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799146, "value1":0.5204858317687999, "value2":128757778781441708, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014826, "value1":0.7097519460607616, "value2":9006806179184131719, "value3":true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648719, "value1":0.01042589454449313, "value2":1630317652722526040, "value3":false, "__name__":"metric_00000359", "key_6":"j","key_9":"k","key_3":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.665998, "value1":0.6184281613297867, "value2":3187261205020385384, "value3":false, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.501321, "value1":0.23807711629991066, "value2":1285207877035639838, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184165, "value1":0.7331865941823509, "value2":8439731025981805138, "value3":true, "__name__":"metric_00000360", "key_9":"f","key_0":"a","key_5":"j", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801569, "value1":0.9860558447882487, "value2":6774600722253198808, "value3":false, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384223, "value1":0.22369926816026842, "value2":5826656674460169842, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.980495, "value1":0.6229185444454209, "value2":4872634527752777210, "value3":false, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.380826, "value1":0.7806130098804899, "value2":384791734560367497, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163193, "value1":0.5398716860823138, "value2":4387950486971806786, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.823934, "value1":0.5810656874809516, "value2":450490831163611562, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.608002, "value1":0.7930931659235, "value2":7050949938710758790, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363629, "value1":0.5705690869328316, "value2":4003836970877260330, "value3":true, "__name__":"metric_00000369", "key_6":"e","key_7":"a","key_5":"i", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.997480, "value1":0.2946419071937674, "value2":5707538095602523121, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.835470, "value1":0.023853067484955352, "value2":5038338303194879356, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.720798, "value1":0.48474572095528634, "value2":682322783035637435, "value3":false, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.573983, "value1":0.3514813766672496, "value2":2903766540993385647, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730487, "value1":0.8217554814463921, "value2":2948047805979067123, "value3":true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016747, "value1":0.12213673109770266, "value2":3888549493731213502, "value3":false, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.081414, "value1":0.18665321280658956, "value2":6792269259144278363, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300754, "value1":0.9731729557371976, "value2":7090692680090167231, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337262, "value1":0.4023504772804693, "value2":4873228147801762854, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990413, "value1":0.6497908067795414, "value2":3504328384430870621, "value3":false, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.276664, "value1":0.785362017819063, "value2":1481672280882279109, "value3":true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.357896, "value1":0.03050056454630596, "value2":6649840382760115378, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.095552, "value1":0.31901448906359003, "value2":7956338378631849376, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.518227, "value1":0.922032137621878, "value2":785067472984268171, "value3":false, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094243, "value1":0.6170957228718421, "value2":6792110304537802959, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782822, "value1":0.20221744914264872, "value2":3178187263071942184, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.539464, "value1":0.9703158465709303, "value2":1060145839109092280, "value3":true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773211, "value1":0.9517836987483416, "value2":7463838442968425219, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.695794, "value1":0.34514076166527163, "value2":1111746250317888633, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074230, "value1":0.19136080585905593, "value2":4573587278465589025, "value3":false, "__name__":"metric_00000389", "key_6":"c","key_7":"b","key_0":"j", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982043, "value1":0.2232136951539441, "value2":7688263110504645334, "value3":false, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939724, "value1":0.3301646825680086, "value2":5317348954333303134, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275048, "value1":0.1567976387714283, "value2":8628385701502049400, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.251237, "value1":0.601867452525591, "value2":4258975777323734306, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051509, "value1":0.87447596502846, "value2":390569829027012116, "value3":true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453014, "value1":0.7517900109611292, "value2":5319112069730300764, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.859148, "value1":0.03968223769223721, "value2":6092044766196787249, "value3":false, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564895, "value1":0.8282334480284984, "value2":8177460795118738605, "value3":false, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.213064, "value1":0.37245338326241034, "value2":5313011500830153125, "value3":false, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234759, "value1":0.6391009807318464, "value2":3598159575817066989, "value3":true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922540, "value1":0.8881740732484977, "value2":4573970560902118758, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247842, "value1":0.6867454879676517, "value2":2359279288069431907, "value3":true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350225, "value1":0.6867879732162172, "value2":37306184729818872, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881388, "value1":0.4406197726220826, "value2":724124821973247241, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.950564, "value1":0.1322036312488787, "value2":4198819767892784951, "value3":true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.666395, "value1":0.2251980228042216, "value2":606006971675717615, "value3":false, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.344497, "value1":0.029652052432054064, "value2":5846707393638380751, "value3":false, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.703773, "value1":0.6376539079560296, "value2":6681127841717987695, "value3":false, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295458, "value1":0.5526352586149329, "value2":7043209016069060231, "value3":false, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435502, "value1":0.09713433968048614, "value2":1019638678991819627, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598132, "value1":0.963189125504533, "value2":6458016500274290636, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627640, "value1":0.4641061727729553, "value2":8689246494106761338, "value3":false, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254760, "value1":0.8326486356175677, "value2":4407797931839765907, "value3":false, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805732, "value1":0.6423132619728287, "value2":6833967417402080897, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.326823, "value1":0.9067956612338547, "value2":2841002873709302821, "value3":true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.088104, "value1":0.06965395874499884, "value2":1789284717431249823, "value3":true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.948712, "value1":0.9884448336440479, "value2":1290858986155893057, "value3":true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511709, "value1":0.8105556502461225, "value2":3470144572708895882, "value3":false, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163730, "value1":0.8658645130208062, "value2":6466367002189635673, "value3":false, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260153, "value1":0.5010858345134921, "value2":555883752624156654, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.274796, "value1":0.8267418905611275, "value2":7590347778036203754, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338150, "value1":0.7906313079639156, "value2":7837780129396014797, "value3":true, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938845, "value1":0.03371523590847688, "value2":2642591228031199330, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254822, "value1":0.42059549793106427, "value2":4614016824012787995, "value3":true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003989, "value1":0.05975709544316542, "value2":1382695454770978636, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.577151, "value1":0.8937534168973357, "value2":6521815205949311706, "value3":false, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.253675, "value1":0.40184282199800475, "value2":8286880530272716169, "value3":false, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205638, "value1":0.7633247932732301, "value2":6913337973948968095, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939093, "value1":0.03322965819693288, "value2":2797917596305577577, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248192, "value1":0.5307465178536894, "value2":6134584955455781673, "value3":true, "__name__":"metric_00000430", "key_9":"j","key_1":"j","key_4":"c", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.593487, "value1":0.05419262497736568, "value2":9190929989995856492, "value3":false, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.182515, "value1":0.035610068777040586, "value2":6469776877211677470, "value3":true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160124, "value1":0.662164487805106, "value2":8768604308735587138, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317998, "value1":0.9757873511779303, "value2":1199801679672428467, "value3":true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922339, "value1":0.9554985330074205, "value2":8286943158249961765, "value3":false, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023973, "value1":0.01669695080382125, "value2":4408497776354893523, "value3":true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.399088, "value1":0.37811424022789003, "value2":499643435878747492, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.974576, "value1":0.6392617923906977, "value2":1832499551397243524, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423426, "value1":0.1431493881970496, "value2":4016805275891833987, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220477, "value1":0.006541995474356055, "value2":5608461794122610089, "value3":false, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014455, "value1":0.20051860544075079, "value2":4298831902740539404, "value3":false, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771702, "value1":0.9711913949733121, "value2":4548136488165596312, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837484, "value1":0.3199900890395432, "value2":7619719494433917806, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632963, "value1":0.8179930116909135, "value2":42611344634328983, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739321, "value1":0.8571068088303897, "value2":1929086780074260377, "value3":true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169110, "value1":0.2241172770388991, "value2":3015343857994873164, "value3":true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558214, "value1":0.5245963976733786, "value2":4636974583954671385, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551826, "value1":0.8529460688457188, "value2":7676686801578586657, "value3":false, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317389, "value1":0.34597315939279244, "value2":8526544610592186971, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131556, "value1":0.9023875402465826, "value2":3147425851236763117, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330907, "value1":0.7187683220932978, "value2":6304935909987923644, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419527, "value1":0.3464275048663091, "value2":800017209533647436, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325070, "value1":0.32409707654758807, "value2":3093719017131899877, "value3":false, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094166, "value1":0.6576755239099957, "value2":4375841020494708801, "value3":false, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184361, "value1":0.454784333846635, "value2":8511384704032602153, "value3":false, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424067, "value1":0.7746556148202395, "value2":8409657959538616305, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.282726, "value1":0.19274194173178774, "value2":4484284290811930974, "value3":true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.044679, "value1":0.17881938606509734, "value2":2991918820431220404, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622917, "value1":0.5058188298556251, "value2":2680090310035550156, "value3":true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.686600, "value1":0.019991692028205214, "value2":8990563930001611676, "value3":true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.977493, "value1":0.8045660581528546, "value2":3965120458727174879, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.910280, "value1":0.15532301386230277, "value2":2751777764504449883, "value3":true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981678, "value1":0.6634934084628059, "value2":3338856320416894701, "value3":false, "__name__":"metric_00000463", "key_5":"a","key_0":"g","key_3":"e", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099077, "value1":0.4991967563162609, "value2":7696501412266603819, "value3":true, "__name__":"metric_00000464", "key_3":"c","key_0":"a","key_1":"j", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169242, "value1":0.7573800949559079, "value2":4682651688122452104, "value3":false, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.854077, "value1":0.38315712266426233, "value2":7277684317066282125, "value3":false, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017721, "value1":0.43030519598600153, "value2":8176728734487897933, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449823, "value1":0.0905672633924478, "value2":8038615141368276704, "value3":true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505199, "value1":0.29347749204963386, "value2":5939807103722962422, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.429412, "value1":0.3821527042310373, "value2":7616377560194630586, "value3":false, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645364, "value1":0.31800894756504583, "value2":5324276910842211792, "value3":false, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789951, "value1":0.12346800778619027, "value2":7282935021388766519, "value3":true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.005934, "value1":0.4652775738745803, "value2":5150985627139247681, "value3":true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770208, "value1":0.8318069314007921, "value2":8693296121103057557, "value3":false, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.036795, "value1":0.31504722243353955, "value2":5332439897432100886, "value3":false, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.557676, "value1":0.04547498827953966, "value2":1747115516814035342, "value3":true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295720, "value1":0.25765308010583876, "value2":7172964053161542723, "value3":true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.862280, "value1":0.6690305275979546, "value2":8530225967231964009, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464755, "value1":0.015096554191777058, "value2":2158727717387461667, "value3":true, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902724, "value1":0.3729818170388733, "value2":2937474665692773587, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.150211, "value1":0.9010548939518067, "value2":8689173712211659171, "value3":true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502794, "value1":0.637718327451074, "value2":1058416603563820737, "value3":true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756334, "value1":0.9093486580620052, "value2":3914139843013624351, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109763, "value1":0.3482544181233465, "value2":3984218999735374107, "value3":true, "__name__":"metric_00000482", "key_7":"g","key_2":"f","key_3":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.007357, "value1":0.530193826214922, "value2":7784257234281286192, "value3":false, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133042, "value1":0.14994561696634548, "value2":6913972686763727632, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.499117, "value1":0.9693770819840257, "value2":292923979595115495, "value3":true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.256986, "value1":0.8560085418753359, "value2":1633179891830212156, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348017, "value1":0.4525195870224779, "value2":1498113059165586202, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.389164, "value1":0.4747861877903318, "value2":5663253941505962378, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.712219, "value1":0.23770447169372633, "value2":7127281555547067699, "value3":false, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799547, "value1":0.4771589732318728, "value2":2601851094636806560, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761432, "value1":0.9902875537117704, "value2":2441735233721216793, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.594915, "value1":0.9327820854039164, "value2":7435962671657247487, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.018250, "value1":0.35410376862709275, "value2":6088069648662464618, "value3":true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.640810, "value1":0.6282310495408681, "value2":4206520774588356560, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929377, "value1":0.30194565379846455, "value2":371174649294622523, "value3":false, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.817275, "value1":0.4745483085572834, "value2":2810603774900787473, "value3":true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.046061, "value1":0.9540450542751275, "value2":6567731530813672613, "value3":false, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976385, "value1":0.2747180574854192, "value2":7052984344810070260, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833030, "value1":0.8111270330849011, "value2":2406611793520084033, "value3":true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146154, "value1":0.019440584474240373, "value2":1431804086688415352, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541319, "value1":0.42910271299796143, "value2":5008131544913964869, "value3":true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.478974, "value1":0.3992297014608565, "value2":6638251650936294767, "value3":false, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.591553, "value1":0.9631516171967569, "value2":6328755963396316225, "value3":true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.836675, "value1":0.8925642028256048, "value2":2169296529295363291, "value3":false, "__name__":"metric_00000506", "key_6":"g","key_3":"e","key_5":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240233, "value1":0.5601423681829663, "value2":2372802153601657174, "value3":false, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926558, "value1":0.8669054217402387, "value2":8750941328640682000, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.785340, "value1":0.9294643708923529, "value2":8967392703697598518, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321372, "value1":0.6020812110006013, "value2":1942102877471692500, "value3":false, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578919, "value1":0.17388396608421544, "value2":5054354296681821565, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830121, "value1":0.06504953435269434, "value2":7812868363877570134, "value3":true, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110821, "value1":0.12907416408116543, "value2":1398896999021006979, "value3":false, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318471, "value1":0.6665917244504806, "value2":5816507337427212221, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881196, "value1":0.35509697330525436, "value2":3799008703902351704, "value3":true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503152, "value1":0.686811705668727, "value2":1281918055642507847, "value3":false, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.393488, "value1":0.8980510009425112, "value2":3157487117727981952, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993113, "value1":0.8339917765115039, "value2":1528182230124691520, "value3":true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441452, "value1":0.5723022066521427, "value2":8611586563060872543, "value3":false, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902474, "value1":0.21553093452664723, "value2":1591141288904852990, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.937761, "value1":0.5975557556565823, "value2":1778079885483684894, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.509879, "value1":0.8107703383515009, "value2":7338854303608988296, "value3":true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321562, "value1":0.7839743525651934, "value2":407341572079351661, "value3":false, "__name__":"metric_00000523", "key_5":"k","key_0":"g","key_3":"g", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.083126, "value1":0.8340311472033722, "value2":414058498508156341, "value3":true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.266082, "value1":0.7848792912531005, "value2":1487975780946804498, "value3":false, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.001354, "value1":0.042658133657431034, "value2":2686477010080712855, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984348, "value1":0.12772415816911367, "value2":1744454398245616628, "value3":true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.545130, "value1":0.012749865379431171, "value2":3614758343709591135, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.211075, "value1":0.26931470974880833, "value2":5335597373851226907, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517993, "value1":0.8277823010534222, "value2":441076351209513172, "value3":true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.783905, "value1":0.6446774753163863, "value2":2444284131966329588, "value3":false, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140755, "value1":0.7307559658816861, "value2":1750453360454837174, "value3":false, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.243464, "value1":0.5611645445748624, "value2":5031122288522829688, "value3":false, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247597, "value1":0.26059857605419207, "value2":6822156384989813472, "value3":true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053905, "value1":0.6298269719196294, "value2":6661136217666523640, "value3":true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.058995, "value1":0.7306532383370615, "value2":5311315770055711299, "value3":true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030527, "value1":0.08351380687475202, "value2":7155821886727737087, "value3":true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.498371, "value1":0.2731396400770269, "value2":89197923125259866, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397662, "value1":0.0870595280214176, "value2":4020978501682465140, "value3":false, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173834, "value1":0.7945264755888277, "value2":3882559205765407193, "value3":false, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497055, "value1":0.09364593584875654, "value2":891682366412177732, "value3":false, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065765, "value1":0.835147994300049, "value2":2505979555724543511, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.765901, "value1":0.992206355507561, "value2":5788507149763623428, "value3":true, "__name__":"metric_00000543", "key_6":"a","key_2":"k","key_3":"h", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622912, "value1":0.44608293916585673, "value2":946502417712586156, "value3":true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366924, "value1":0.9938167708003688, "value2":7200121887993962206, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.072114, "value1":0.001073896039030629, "value2":8685581835197772410, "value3":false, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265774, "value1":0.30130760063683054, "value2":8411046196525683207, "value3":false, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.749589, "value1":0.9239863104020335, "value2":4219975890726145199, "value3":true, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190061, "value1":0.9734172533312344, "value2":8137079207203857103, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.519945, "value1":0.07187194342466732, "value2":7825744617647803155, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.649495, "value1":0.6408236940193898, "value2":3048798030528052662, "value3":false, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320456, "value1":0.5081060875649162, "value2":719352320345764599, "value3":true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476698, "value1":0.052307893890373255, "value2":2742427547083463898, "value3":false, "__name__":"metric_00000552", "key_8":"d","key_2":"i", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571531, "value1":0.4132003718642955, "value2":6484766831533191693, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682702, "value1":0.44751287991255284, "value2":4055042383084807325, "value3":true, "__name__":"metric_00000556", "key_4":"e","key_6":"b","key_3":"i", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.890718, "value1":0.33741266801359987, "value2":6934783766842267527, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452747, "value1":0.3663537726308516, "value2":3312655545498739922, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199996, "value1":0.15068464159094366, "value2":5665646131255744951, "value3":true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.983795, "value1":0.8775221552243728, "value2":9069927522091992178, "value3":false, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.583294, "value1":0.838152847228304, "value2":1035319401212201491, "value3":true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212587, "value1":0.8012210929096193, "value2":5585326063344558523, "value3":true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641779, "value1":0.9247211843697717, "value2":2117637859665911236, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.406293, "value1":0.6198872109290668, "value2":5911640262763562610, "value3":false, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.277897, "value1":0.027423430536495416, "value2":7628618445567261117, "value3":false, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.832579, "value1":0.5331161695907243, "value2":8305462638082561375, "value3":false, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205268, "value1":0.018199222166683245, "value2":475925606235710366, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.615075, "value1":0.9915416739226851, "value2":2204646069144435635, "value3":false, "__name__":"metric_00000565", "key_6":"d","key_7":"e","key_1":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424612, "value1":0.9757938651409276, "value2":6266511548869428739, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286078, "value1":0.0846398426026624, "value2":3081995387268873440, "value3":true, "__name__":"metric_00000569", "key_9":"g","key_4":"f", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787171, "value1":0.8365446036859473, "value2":5979460348923075463, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381115, "value1":0.9123835513620147, "value2":3398482109219075054, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.114157, "value1":0.804291931301114, "value2":6012424958858906101, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134911, "value1":0.00736667824808292, "value2":2400000632738793301, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164780, "value1":0.21287762980212552, "value2":8645521771535465924, "value3":true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710538, "value1":0.7411138160092244, "value2":8685307998868002992, "value3":false, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585644, "value1":0.08786909591629084, "value2":7343395510792995060, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125543, "value1":0.7332366768127239, "value2":8370868046930361421, "value3":false, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.932834, "value1":0.9015644945676353, "value2":5236314268804910193, "value3":false, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093732, "value1":0.497896904599941, "value2":6397325408101186539, "value3":false, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486123, "value1":0.5097658314843312, "value2":1337466807316653828, "value3":false, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904889, "value1":0.6556792189077274, "value2":4516335409033295305, "value3":true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.387498, "value1":0.9910872699779568, "value2":1557997601588004148, "value3":true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710931, "value1":0.5132845499506883, "value2":3319795795293053140, "value3":true, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.515215, "value1":0.049432502275940204, "value2":2300709145422274852, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821813, "value1":0.6825139883510115, "value2":8194491426131939536, "value3":true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865373, "value1":0.7044541486613806, "value2":3703555177588303304, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837580, "value1":0.8572176121238669, "value2":2344100903459752932, "value3":true, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314910, "value1":0.4347186053772807, "value2":7676829676819086499, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.946948, "value1":0.7950628854157453, "value2":2267010392465955325, "value3":false, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015107, "value1":0.8624602552899475, "value2":9148330689311945183, "value3":false, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488041, "value1":0.4618418780870866, "value2":8123711349904092581, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710726, "value1":0.4226915062812881, "value2":5491521860827839646, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841455, "value1":0.9825903973183027, "value2":3077499782633025057, "value3":true, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.592689, "value1":0.9193972918242918, "value2":929238328644383650, "value3":false, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993922, "value1":0.14695440679147787, "value2":8882303717901413025, "value3":true, "__name__":"metric_00000596", "key_4":"b","key_8":"b","key_0":"a", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731320, "value1":0.9934902241842611, "value2":1321453009633438522, "value3":true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604377, "value1":0.24174531205655925, "value2":5870091064000157680, "value3":true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628164, "value1":0.8064136026821553, "value2":8786888366338651141, "value3":false, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.484350, "value1":0.5079394885547678, "value2":6920245211342331930, "value3":true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978330, "value1":0.9354707910737625, "value2":7581764658078201917, "value3":true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632366, "value1":0.23595321295755245, "value2":1511878176205605753, "value3":true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142194, "value1":0.6306719216610874, "value2":2431345756658073693, "value3":false, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397195, "value1":0.7323027634538346, "value2":8521159495965572956, "value3":true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.996298, "value1":0.8484427529278864, "value2":7246857551445151219, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.328211, "value1":0.8531255802404514, "value2":4506921974707672140, "value3":false, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308884, "value1":0.5324739320542478, "value2":6505796324920546, "value3":false, "__name__":"metric_00000608", "key_4":"g","key_9":"i","key_1":"e", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.721813, "value1":0.2030510736245931, "value2":493933611483820748, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325138, "value1":0.6318807488600625, "value2":3427677815511198978, "value3":true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.717950, "value1":0.13850468444662895, "value2":3714125763364851426, "value3":false, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984959, "value1":0.2313158857459622, "value2":6179178857252455230, "value3":false, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767352, "value1":0.04737365094297706, "value2":7312736110130720225, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517326, "value1":0.38308887661557994, "value2":4397262624851606256, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537458, "value1":0.8630190277228629, "value2":1405251953788249605, "value3":true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.589024, "value1":0.6853812314292282, "value2":6568591835038822877, "value3":true, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013121, "value1":0.9022971605517441, "value2":5391410131187089227, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585652, "value1":0.027029007730938633, "value2":6270487841856618503, "value3":false, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.069148, "value1":0.5452016240009335, "value2":7230673973301658030, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870318, "value1":0.8893099847439015, "value2":2072311642337421510, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.525589, "value1":0.9489296395713415, "value2":1934395693866121179, "value3":true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.411127, "value1":0.5543622037481519, "value2":869973100557162310, "value3":true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870905, "value1":0.1429227422154167, "value2":1580405259981052384, "value3":true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214057, "value1":0.007127408341584659, "value2":5021161294907689526, "value3":false, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324636, "value1":0.2886203159980936, "value2":3536603792281429135, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566190, "value1":0.2435195285039059, "value2":7314231811000308948, "value3":true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.302446, "value1":0.3969373764014023, "value2":861549335043188440, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265270, "value1":0.8392596283575425, "value2":229012885919848214, "value3":false, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306041, "value1":0.6755326135952276, "value2":687305007623331029, "value3":true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.920842, "value1":0.14122582181388352, "value2":8912506495776009399, "value3":false, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970414, "value1":0.32375225588910955, "value2":3874667148173594242, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098047, "value1":0.5080753238940934, "value2":259459341225408666, "value3":true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814708, "value1":0.5851579994151463, "value2":4279032101744191170, "value3":false, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857725, "value1":0.8750883468056723, "value2":8160658677842295703, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288649, "value1":0.3318827172603761, "value2":8934632128032959148, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.651119, "value1":0.31525481325301247, "value2":8244510685212193822, "value3":true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065672, "value1":0.08637516713839134, "value2":707728667641140855, "value3":true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620357, "value1":0.8983386747310453, "value2":3756826533006983142, "value3":true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707292, "value1":0.4495030238963459, "value2":5996025393915497140, "value3":true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.443961, "value1":0.930457120039426, "value2":6519425791992110608, "value3":false, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109167, "value1":0.09168467762421384, "value2":2057706949013098262, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.481603, "value1":0.3529978146205129, "value2":6544227882618708766, "value3":false, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770746, "value1":0.6378677748113686, "value2":1182963631296444077, "value3":true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967875, "value1":0.8756904550723122, "value2":516292034493240162, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324624, "value1":0.03689035176673176, "value2":6723940798795470811, "value3":true, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556389, "value1":0.7611742354824225, "value2":6343475929372871725, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538576, "value1":0.1188977043392838, "value2":1679649620772949796, "value3":true, "__name__":"metric_00000646", "key_6":"k","key_7":"a","key_1":"j", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.531730, "value1":0.2793539457328025, "value2":8309265375086957584, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488778, "value1":0.8806255322454197, "value2":2637346612964090286, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174493, "value1":0.4058829790367517, "value2":4908420813809791761, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.875676, "value1":0.8084081237917962, "value2":1370185119560523418, "value3":true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214353, "value1":0.22148397653369978, "value2":3829075843579764833, "value3":true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402832, "value1":0.9983967808941802, "value2":373850521427271723, "value3":true, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.574399, "value1":0.1403913922108672, "value2":1025325851242540472, "value3":false, "__name__":"metric_00000651", "key_4":"j","key_2":"i", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538957, "value1":0.5637672549308663, "value2":180146514999582015, "value3":true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043198, "value1":0.6621042931314224, "value2":4672271958823003044, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.455391, "value1":0.37463255563131875, "value2":7050585404363795282, "value3":true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118353, "value1":0.7019235890065948, "value2":6561897677595087821, "value3":true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562842, "value1":0.4216296582302376, "value2":4671864964861835857, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.613979, "value1":0.9101669225413574, "value2":5476069179621211969, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079079, "value1":0.6285830126630634, "value2":1800090753316916399, "value3":true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990168, "value1":0.4228502254109105, "value2":4769618752656230984, "value3":true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630399, "value1":0.0593879368852638, "value2":8425926359679957872, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381514, "value1":0.3170447182462336, "value2":134775839032388004, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306533, "value1":0.09290124329204048, "value2":3878589204829724343, "value3":true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363988, "value1":0.7494592976903763, "value2":588119535129085693, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.726401, "value1":0.2843848594033847, "value2":797824336547840313, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.425192, "value1":0.6747059449207641, "value2":6618945540166900762, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486703, "value1":0.9234732601693532, "value2":7284867992059891857, "value3":false, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.162651, "value1":0.4998767239554866, "value2":3758664262196916666, "value3":true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638502, "value1":0.14222618431131662, "value2":7183266992692325102, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814797, "value1":0.04744514500578937, "value2":8807075724873574935, "value3":true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617563, "value1":0.7113712918981732, "value2":7369990382580999103, "value3":false, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391047, "value1":0.4667345592294561, "value2":2593217151341486604, "value3":false, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423210, "value1":0.2993388901152061, "value2":7736063295280636221, "value3":false, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.233156, "value1":0.2669621811468438, "value2":9163725732269873210, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780343, "value1":0.4836321272043857, "value2":6239122865745239252, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270794, "value1":0.8124904828437979, "value2":5664361933553425625, "value3":true, "__name__":"metric_00000676", "key_2":"k","key_5":"k","key_0":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.567390, "value1":0.1509148834766825, "value2":769811157679024909, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502511, "value1":0.3496329905819851, "value2":8064435486484275503, "value3":false, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.995318, "value1":0.07944294849213016, "value2":7290799161307742730, "value3":true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160027, "value1":0.9323372972664894, "value2":8694402707307208350, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265068, "value1":0.7804375599856387, "value2":2823069570137165166, "value3":true, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852790, "value1":0.04269382129150226, "value2":4247785862022220763, "value3":true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.813337, "value1":0.3041270110008107, "value2":1162341534698170174, "value3":true, "__name__":"metric_00000683", "key_6":"a","key_9":"c","key_1":"k", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821104, "value1":0.05863251524747595, "value2":5517633377182805397, "value3":false, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801029, "value1":0.7039209651517294, "value2":5777192245942090986, "value3":true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.292328, "value1":0.0789099713548181, "value2":4341141001609491414, "value3":true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.650140, "value1":0.6547556555322551, "value2":488100188330068974, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881952, "value1":0.6384170556890851, "value2":7393134503131526080, "value3":false, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019410, "value1":0.4521619097561246, "value2":2730473646049580719, "value3":false, "__name__":"metric_00000688", "key_7":"b","key_8":"c","key_2":"b", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413658, "value1":0.35836104075629194, "value2":5913006016396086189, "value3":false, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.388542, "value1":0.347328329631466, "value2":3415671885617145536, "value3":false, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908514, "value1":0.682932258250745, "value2":1535326911373174401, "value3":true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453424, "value1":0.011430118976046925, "value2":6615057602694952188, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661892, "value1":0.04490375141966574, "value2":3586691570128338490, "value3":true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598897, "value1":0.9840071764206562, "value2":6756637664735447989, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.614741, "value1":0.45336506256132186, "value2":7160462685010833845, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959437, "value1":0.33491802794663117, "value2":4263644093884006866, "value3":false, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394710, "value1":0.09508520954191016, "value2":8086669194950309707, "value3":true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.400389, "value1":0.748914928293926, "value2":7561318401530983897, "value3":true, "__name__":"metric_00000699", "key_2":"a","key_7":"c","key_0":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551520, "value1":0.29033239272123923, "value2":219973352444233301, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.433012, "value1":0.8026759332563937, "value2":1035833541375117050, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.097876, "value1":0.3123782080487283, "value2":97482352559342166, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971923, "value1":0.09350074090215915, "value2":6603209689656348295, "value3":true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385293, "value1":0.025071498526727982, "value2":7821611042801827653, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.582177, "value1":0.17895122462677535, "value2":4851272893829158392, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715577, "value1":0.12145793831580125, "value2":6773868244129033160, "value3":true, "__name__":"metric_00000706", "key_9":"d","key_5":"e","key_6":"h", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.245667, "value1":0.5083106874300092, "value2":5342790051223313569, "value3":true, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.191947, "value1":0.7213188287503117, "value2":8184500849109232134, "value3":false, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.527410, "value1":0.2848785581313827, "value2":8059297953437098619, "value3":true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023841, "value1":0.08963330011439892, "value2":8744048608508050406, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.861975, "value1":0.9569428255590289, "value2":5863440324678791040, "value3":true, "__name__":"metric_00000711", "key_4":"h","key_6":"i","key_0":"j", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.020139, "value1":0.5328792585389814, "value2":6851298079925452135, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383316, "value1":0.6736870667139099, "value2":1034982274545531879, "value3":true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490723, "value1":0.7069141316227696, "value2":1238860616186948989, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774619, "value1":0.15017350186839917, "value2":4145739713631541991, "value3":true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.724625, "value1":0.41136382513830944, "value2":2470375246542099625, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014004, "value1":0.45322459474776666, "value2":3424033124770728498, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370213, "value1":0.7936480778398273, "value2":7651653355869440262, "value3":true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930663, "value1":0.28375382304907193, "value2":4573094024293120951, "value3":true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.108477, "value1":0.2514498975622512, "value2":6565807048098733104, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318641, "value1":0.09067602148053568, "value2":2118726105929309608, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341849, "value1":0.5146408700742566, "value2":5544935195040202056, "value3":true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016279, "value1":0.2887329883061575, "value2":4016937139552222061, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696532, "value1":0.35345697419681527, "value2":7198819764217760607, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356409, "value1":0.6101596793669094, "value2":2338966817998610022, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882128, "value1":0.8757983585066079, "value2":1264113150578508254, "value3":true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214730, "value1":0.04033669277556926, "value2":2311999714610864651, "value3":true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125040, "value1":0.8510776359004306, "value2":9203972476999185975, "value3":false, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617466, "value1":0.4880211576700948, "value2":7876488006268555933, "value3":false, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.580824, "value1":0.738663404243217, "value2":6159559973328325759, "value3":false, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.894051, "value1":0.0394639970488642, "value2":477392450960926806, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112539, "value1":0.11952574252776728, "value2":2511563277039044978, "value3":true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.918404, "value1":0.30833608301592924, "value2":8563798643952041377, "value3":false, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385588, "value1":0.17478857683593713, "value2":6552271330519870932, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139243, "value1":0.42492669927426513, "value2":7691897138631144930, "value3":false, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715207, "value1":0.8647651137464607, "value2":8497862595581854195, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760625, "value1":0.24627316716260067, "value2":1409311771773591601, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.880305, "value1":0.7511958117841795, "value2":8663780143091433962, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.121986, "value1":0.503001087851081, "value2":86404765376501910, "value3":true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.378220, "value1":0.7154879418930685, "value2":4699613906263091859, "value3":false, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367776, "value1":0.3357733171654874, "value2":2109430818036060968, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051853, "value1":0.46132527877007656, "value2":1864815246368296251, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503984, "value1":0.5450705451757059, "value2":2199582755716092199, "value3":false, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270031, "value1":0.15857947720543178, "value2":5111440735129620852, "value3":false, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376563, "value1":0.08077432861716789, "value2":7964637453463198864, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017382, "value1":0.4697079883099492, "value2":6862028548685305626, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938781, "value1":0.3698227408392578, "value2":1642534448554432249, "value3":false, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.869838, "value1":0.4366942978360045, "value2":4007380928114996020, "value3":true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.309900, "value1":0.05509289000926218, "value2":8881026026944444737, "value3":true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321699, "value1":0.5508642725465765, "value2":1065357963175430930, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242134, "value1":0.05194281440569991, "value2":2955861771024227742, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394641, "value1":0.6802778418350491, "value2":7829903676952590571, "value3":false, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.011752, "value1":0.2679251298930048, "value2":46390372055260426, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156821, "value1":0.8759623901682069, "value2":6663991521677358276, "value3":true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621515, "value1":0.8461818248462066, "value2":4793482631115954367, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.906690, "value1":0.4454231810933155, "value2":1968123317376605778, "value3":false, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833508, "value1":0.5580322939662696, "value2":542437654648170086, "value3":false, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.864441, "value1":0.13709821675852493, "value2":3775094265545831122, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349796, "value1":0.5463888647509524, "value2":3843833923457334958, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210241, "value1":0.13053751777334896, "value2":2547919115148938351, "value3":false, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.534237, "value1":0.13104275846768723, "value2":2757438080188257397, "value3":false, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.479346, "value1":0.6936882347927468, "value2":3593937018182399578, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013374, "value1":0.647079383488511, "value2":6027310083597491507, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353768, "value1":0.5713686637635252, "value2":3396180021694291926, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704388, "value1":0.028813882790100154, "value2":6631675215659864441, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350375, "value1":0.09828506398906811, "value2":262432636815220332, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130984, "value1":0.05874581283090899, "value2":5455068607213223389, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269711, "value1":0.16897643510231944, "value2":2188619802248362391, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798909, "value1":0.49628972013165096, "value2":8871333460189078193, "value3":false, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314472, "value1":0.003776179814725882, "value2":5465069878443166467, "value3":true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841977, "value1":0.016019153138836187, "value2":5090055543864076033, "value3":true, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.431741, "value1":0.4503254906693805, "value2":3437452976536184232, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021144, "value1":0.7157770425720729, "value2":1627465028436054242, "value3":true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.492677, "value1":0.170722769795088, "value2":5333195898301269961, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.572483, "value1":0.07518223050679318, "value2":749837753058622914, "value3":false, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.070547, "value1":0.7819934575507779, "value2":1257841836293236196, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834097, "value1":0.7836504674507244, "value2":7822820024554629611, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.725415, "value1":0.6205675290840149, "value2":7546425291277331498, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756846, "value1":0.4577877981753711, "value2":521428696388054794, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551591, "value1":0.4656287234062602, "value2":3666151433771478256, "value3":true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367967, "value1":0.3303667849726779, "value2":1336037347437447384, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124888, "value1":0.9362913257587775, "value2":8565731764409689287, "value3":false, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356964, "value1":0.763103278688178, "value2":7384734667573077593, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.336557, "value1":0.18917586865284508, "value2":3164250330171533065, "value3":false, "__name__":"metric_00000786", "key_4":"e","key_1":"c", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947920, "value1":0.7279862820897369, "value2":5585150661666129475, "value3":false, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320214, "value1":0.02069659588289354, "value2":4096509130796828102, "value3":true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339884, "value1":0.3585015534256289, "value2":1745908303614221947, "value3":false, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736733, "value1":0.6300320540961489, "value2":4976479107859231308, "value3":true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843896, "value1":0.8618710521424175, "value2":409303658417169135, "value3":false, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351779, "value1":0.31536023315738654, "value2":7469097912657923197, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146297, "value1":0.47961745828082863, "value2":120290133475149079, "value3":false, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.299654, "value1":0.953430875530432, "value2":1804061300635059106, "value3":false, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.653880, "value1":0.6130300298494126, "value2":3600820249700194803, "value3":false, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.241886, "value1":0.4261506936601424, "value2":6410123065530378279, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.719862, "value1":0.5557094068088962, "value2":6619604443134241152, "value3":false, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192985, "value1":0.4372446351220488, "value2":5419318851099904514, "value3":false, "__name__":"metric_00000797", "key_5":"i","key_8":"b","key_0":"j", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.062254, "value1":0.35078106654250335, "value2":6337545142680530695, "value3":false, "__name__":"metric_00000795", "key_8":"i","key_9":"d","key_1":"f", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.826832, "value1":0.03911586835937395, "value2":8510739605077299393, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.466577, "value1":0.8386295972027331, "value2":3712956826274262169, "value3":false, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774816, "value1":0.2874652343932132, "value2":1661160295946815788, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497396, "value1":0.37671291600810797, "value2":7809177916397546184, "value3":false, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.061112, "value1":0.6756138122485794, "value2":3821150629117787840, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.711530, "value1":0.9726549946550717, "value2":4361793701491206585, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419723, "value1":0.8422780123510716, "value2":8387473970870580835, "value3":false, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740430, "value1":0.3687691277865257, "value2":49347396348935758, "value3":false, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544648, "value1":0.34026181496926355, "value2":3168616117094841931, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.031693, "value1":0.35157464321590376, "value2":6956034409157435335, "value3":true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682310, "value1":0.4968537230062955, "value2":1715909753935447521, "value3":false, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571570, "value1":0.7444971868018181, "value2":773947913317871760, "value3":true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412817, "value1":0.17790194335965473, "value2":453971816672635527, "value3":true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154029, "value1":0.695415820029077, "value2":4412491024371312379, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187573, "value1":0.025946190955721947, "value2":8268616070900349780, "value3":false, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297634, "value1":0.39036160577230816, "value2":9117936615987168462, "value3":true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882305, "value1":0.6498141826508745, "value2":1021392582780518091, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288985, "value1":0.6094023317572287, "value2":5522861781820654768, "value3":false, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.516532, "value1":0.4913308899910666, "value2":2877526282810678571, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.663856, "value1":0.6052287079408516, "value2":5244892044841001586, "value3":true, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177812, "value1":0.8518899507803316, "value2":4611087097036165557, "value3":false, "__name__":"metric_00000817", "key_5":"f","key_7":"d","key_3":"c", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789804, "value1":0.9253854154062459, "value2":4746855558380871130, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163177, "value1":0.054868844952025875, "value2":4671327502125519320, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753554, "value1":0.0480631824565462, "value2":7078677034582099346, "value3":false, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537888, "value1":0.3970428718849328, "value2":8317481519599523582, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887867, "value1":0.9625491057630078, "value2":851265403943296561, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739174, "value1":0.3876147534715181, "value2":5378120884849587893, "value3":true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263149, "value1":0.6578316627430515, "value2":199930804061246062, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926006, "value1":0.9018811245039401, "value2":8736925807617681839, "value3":false, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559532, "value1":0.017637548071686292, "value2":2677656350646305112, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.940793, "value1":0.36532273109204866, "value2":7505680103024683972, "value3":true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416010, "value1":0.7581313246147069, "value2":7561931289424116286, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130651, "value1":0.31761722951334315, "value2":7760092958155535983, "value3":false, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330390, "value1":0.3514672157667692, "value2":1963737467345840913, "value3":true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731094, "value1":0.5668209441148129, "value2":6834927574629705768, "value3":true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003464, "value1":0.44994944703983886, "value2":1268238147301227399, "value3":true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488880, "value1":0.15568001025824393, "value2":3802105015027414434, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099752, "value1":0.49578700566450296, "value2":6660233803101117286, "value3":true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131762, "value1":0.8279103415624635, "value2":8189053738835694283, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.818080, "value1":0.7683869146409037, "value2":4901733131698699140, "value3":false, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502240, "value1":0.2102196895388282, "value2":8419378719359168551, "value3":false, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130382, "value1":0.00017527290277822845, "value2":7209832064279367432, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697720, "value1":0.977742512542472, "value2":5886772442159349223, "value3":true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152164, "value1":0.5452125862262094, "value2":2301031936204454118, "value3":true, "__name__":"metric_00000843", "key_9":"i","key_0":"f", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307306, "value1":0.27827706723478657, "value2":8114605959354081308, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.687815, "value1":0.407696242982513, "value2":8251434258695514324, "value3":true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.957403, "value1":0.9248822561074296, "value2":5435740408050442827, "value3":false, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297405, "value1":0.735268728902142, "value2":6544542317901773513, "value3":false, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505182, "value1":0.6434041992526011, "value2":6723613820733288827, "value3":false, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953853, "value1":0.08239495901880306, "value2":7797622216144272838, "value3":true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.816223, "value1":0.026034555587018983, "value2":8247070392223609361, "value3":false, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781228, "value1":0.009350455430296063, "value2":6344110503212444401, "value3":false, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993318, "value1":0.7246275363827985, "value2":1117977296074531052, "value3":true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.346481, "value1":0.04745532854484892, "value2":8841440982349072666, "value3":true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240566, "value1":0.6244142217945766, "value2":4416869640928141403, "value3":false, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627039, "value1":0.8121528941309705, "value2":2364077225142023978, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.605014, "value1":0.041614775390040444, "value2":5587679092002855017, "value3":false, "__name__":"metric_00000854", "key_2":"f","key_4":"g","key_1":"d", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.675250, "value1":0.22265983684713628, "value2":852855496341453985, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.626117, "value1":0.13978101794521924, "value2":4898929345737475246, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.272053, "value1":0.8981952631418685, "value2":4155867998384044766, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216853, "value1":0.7930982218994412, "value2":1990843980684870249, "value3":true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231757, "value1":0.11381414833270587, "value2":5629821232198202573, "value3":true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967616, "value1":0.9325867021628451, "value2":6458012454076604601, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.105465, "value1":0.20026280910311015, "value2":1519145667363005058, "value3":true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959715, "value1":0.8692424311015335, "value2":3441946116265084781, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.916752, "value1":0.09650654250745302, "value2":7435925512873628832, "value3":true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314127, "value1":0.5339606052113064, "value2":8918601923652673265, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955659, "value1":0.21237559939985842, "value2":6700000271323408810, "value3":false, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.462571, "value1":0.8031758318689627, "value2":4200089724641662061, "value3":false, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704628, "value1":0.42388353536260726, "value2":1676230088334959718, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781871, "value1":0.7853951443498608, "value2":4677884824803639265, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093020, "value1":0.44592927039430036, "value2":5106772041668223293, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.066421, "value1":0.7167334468150509, "value2":916960599462889530, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753333, "value1":0.8855172765987326, "value2":3498076904045993942, "value3":true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263558, "value1":0.039242126444255614, "value2":1994687158267196250, "value3":false, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.529319, "value1":0.11390721956642542, "value2":6770255083967052135, "value3":true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.642302, "value1":0.25917237536080345, "value2":3561054032858646761, "value3":true, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.800222, "value1":0.5648139239977721, "value2":7859467413386922358, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.714908, "value1":0.3172565417584279, "value2":4018189432904936827, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495891, "value1":0.1699695672081134, "value2":2951696383925672800, "value3":true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.927247, "value1":0.5193998219352067, "value2":7461947996252867047, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465786, "value1":0.04561481590524083, "value2":8852624735734959732, "value3":true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970044, "value1":0.9502884615051707, "value2":5544298343674474830, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.224365, "value1":0.7835642667167426, "value2":7989661444310811565, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970171, "value1":0.044357973684653176, "value2":6240172232063272107, "value3":false, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.022705, "value1":0.2909464675793184, "value2":6791113566169749215, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.791002, "value1":0.30758766636926194, "value2":4087253744537420622, "value3":true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.526835, "value1":0.26218100233396685, "value2":761773481307699191, "value3":true, "__name__":"metric_00000882", "key_9":"c","key_3":"d","key_6":"a", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057757, "value1":0.5941262990661297, "value2":5203993881422592468, "value3":false, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621587, "value1":0.9726557103153585, "value2":4065263619832011048, "value3":true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936496, "value1":0.4220278511438201, "value2":537814119030374569, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476922, "value1":0.7453981222005723, "value2":3086149467770264289, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544208, "value1":0.1648752329298873, "value2":1528219915951406823, "value3":false, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480134, "value1":0.5864292944345278, "value2":1915038657623169872, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.634200, "value1":0.06085546420380519, "value2":1586772996290355012, "value3":true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.290157, "value1":0.4479966729250899, "value2":603667588545059551, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523397, "value1":0.8019975659364824, "value2":2263229113306312715, "value3":false, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260677, "value1":0.49446211956319075, "value2":5384294748496824064, "value3":true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.040570, "value1":0.2634688640512974, "value2":7081899407358309984, "value3":true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.670159, "value1":0.4242108328120063, "value2":1003697559007887859, "value3":true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.181699, "value1":0.7357308047023233, "value2":7674299813279760629, "value3":false, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.437960, "value1":0.34081697732260335, "value2":1366760825001014571, "value3":true, "__name__":"metric_00000899", "key_5":"f","key_0":"b","key_3":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585340, "value1":0.9880202469237767, "value2":5993013748473116382, "value3":true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758920, "value1":0.528974137063132, "value2":5387464413435497334, "value3":true, "__name__":"metric_00000900", "key_4":"c","key_7":"j","key_0":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190405, "value1":0.24309504115260316, "value2":8699961182370479292, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.045104, "value1":0.42560682798506067, "value2":3749517767528292571, "value3":false, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.239270, "value1":0.11837048427737668, "value2":4852849985587057450, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647950, "value1":0.24581023177518593, "value2":7946632180521405772, "value3":false, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969355, "value1":0.36754556888022905, "value2":7897267252653597746, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416692, "value1":0.9421287451192997, "value2":7881806374967586751, "value3":true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.052040, "value1":0.27660174901670026, "value2":5294738319793885165, "value3":false, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112174, "value1":0.8996109424352536, "value2":5125031699148488781, "value3":true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696416, "value1":0.42330986040349045, "value2":7061433961041368821, "value3":false, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127933, "value1":0.6743423459526591, "value2":5703788929519053213, "value3":false, "__name__":"metric_00000911", "key_9":"a","key_3":"k","key_4":"h", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421658, "value1":0.6878542321395752, "value2":3553548366690488868, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697493, "value1":0.169783750302659, "value2":7055001682108205827, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.550811, "value1":0.35231569219578235, "value2":1455572534223159241, "value3":false, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.493018, "value1":0.08084254027780423, "value2":2900266808751124367, "value3":true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949248, "value1":0.8847972628692381, "value2":4051840894997158475, "value3":true, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142802, "value1":0.3270345064861823, "value2":2685380632911712697, "value3":true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.004387, "value1":0.822755239334881, "value2":7165522026622587078, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620895, "value1":0.40285181555826477, "value2":7279952302942542040, "value3":true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.988553, "value1":0.42488091295486163, "value2":7195769969726296289, "value3":true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314916, "value1":0.24528560160904253, "value2":3412797507763244235, "value3":false, "__name__":"metric_00000921", "key_7":"b","key_3":"e","key_4":"e", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740841, "value1":0.7678978195926474, "value2":7353627394146413053, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767489, "value1":0.9849856324971001, "value2":4180092134684532856, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.681082, "value1":0.08407929345646925, "value2":2247545351887675892, "value3":false, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987608, "value1":0.3158564571415852, "value2":3372134575580371247, "value3":false, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953355, "value1":0.6808029749520165, "value2":6416401290847025754, "value3":false, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.934118, "value1":0.3815229371385279, "value2":5398231187188543220, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349522, "value1":0.008036202059570214, "value2":6632834525194225583, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852636, "value1":0.6082938917169154, "value2":2209452475312580755, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.491711, "value1":0.7070391677298287, "value2":5867779644024249420, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329284, "value1":0.23697436114335285, "value2":2747492895656100802, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217160, "value1":0.013558152161070156, "value2":543215008088311315, "value3":true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376202, "value1":0.3967940240541657, "value2":4348396634707940652, "value3":true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134419, "value1":0.2570940106942101, "value2":5495331795379375428, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.693581, "value1":0.6383501980446401, "value2":7332225459965791621, "value3":true, "__name__":"metric_00000934", "key_8":"i","key_7":"h", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843591, "value1":0.374550163160159, "value2":1957979143426115517, "value3":false, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127314, "value1":0.16562410193780275, "value2":5514782476287105847, "value3":false, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564210, "value1":0.47834402715548674, "value2":7864140566183633471, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.673777, "value1":0.656108656648855, "value2":6919590747679310147, "value3":false, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.071957, "value1":0.3923477172589352, "value2":7954657800672945455, "value3":true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497858, "value1":0.21899200976683347, "value2":5922902740218283725, "value3":false, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063776, "value1":0.819166087262725, "value2":497846112853863498, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.039234, "value1":0.06262728093114268, "value2":976242972769343091, "value3":true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658955, "value1":0.48196452599993433, "value2":2417102300700963835, "value3":true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255288, "value1":0.13639741045424703, "value2":7715812700116132347, "value3":true, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.496112, "value1":0.1915457705996175, "value2":7008066197109595718, "value3":false, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144334, "value1":0.011357361010319224, "value2":7600237475494790795, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.129843, "value1":0.45829887966275845, "value2":1540110795133843915, "value3":false, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.828068, "value1":0.8516996075016723, "value2":8405984298577097957, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870240, "value1":0.16467566190225996, "value2":1556056753506941522, "value3":false, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199050, "value1":0.7512917767548035, "value2":3201410658564101147, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830765, "value1":0.6115828253610872, "value2":1648965358200873623, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385351, "value1":0.44894635523974136, "value2":2127367263922899732, "value3":true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.555613, "value1":0.566989554108829, "value2":4275387093192470829, "value3":true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758445, "value1":0.45045350568682, "value2":6540058212119502734, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051410, "value1":0.07872483972360628, "value2":7661444664176600116, "value3":false, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.027756, "value1":0.1866583235803357, "value2":4402492796625360355, "value3":false, "__name__":"metric_00000957", "key_1":"i","key_5":"g","key_0":"c", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043704, "value1":0.3399629891033194, "value2":2843620218251838442, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.218394, "value1":0.6792705695737844, "value2":3717810518285734277, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.489979, "value1":0.2797515798899019, "value2":5603667040320572095, "value3":false, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394844, "value1":0.47076120967369356, "value2":2551470543073926967, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.176040, "value1":0.28649026862589944, "value2":8089352485313652968, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015062, "value1":0.8503661846365486, "value2":9115866206649167217, "value3":true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.867138, "value1":0.18222497145716, "value2":748036164126661849, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560015, "value1":0.16282129139411827, "value2":7758713833273415824, "value3":false, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782776, "value1":0.3488004874020465, "value2":715341283187680350, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.547186, "value1":0.07214699982761937, "value2":8071660169425611932, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383243, "value1":0.8304505195235525, "value2":4079430116382080627, "value3":true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551280, "value1":0.6676351538773236, "value2":4287835647581016602, "value3":true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.853493, "value1":0.3478301116617329, "value2":7815908195604525371, "value3":false, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511951, "value1":0.7049888722888338, "value2":1948135132689353800, "value3":false, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173192, "value1":0.5458720430431239, "value2":5871330428575335057, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413126, "value1":0.46553694703913845, "value2":7586060029018039815, "value3":false, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781439, "value1":0.705616760521343, "value2":1127274508185282406, "value3":false, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.732371, "value1":0.37205742400318453, "value2":5213749986182819618, "value3":false, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569115, "value1":0.03375202087271923, "value2":118632103331380976, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.102614, "value1":0.2822023902913289, "value2":1391807523431966730, "value3":true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140423, "value1":0.44611401204912965, "value2":828680952801675744, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.085991, "value1":0.7893667946565419, "value2":8411361994319110857, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.975471, "value1":0.7162325854468116, "value2":79462509030833666, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.244787, "value1":0.40407729683075916, "value2":2829176965864438571, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.304099, "value1":0.2959757161094293, "value2":5764137406549726855, "value3":false, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837106, "value1":0.4258452743236353, "value2":5341156407592196309, "value3":true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707606, "value1":0.031525280019894485, "value2":868621351492758022, "value3":false, "__name__":"metric_00000984", "key_9":"g","key_0":"k","key_3":"f", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441177, "value1":0.3225860911008584, "value2":3070966375799941978, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.077150, "value1":0.044429243112689185, "value2":8848387799149411370, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604969, "value1":0.8353491972869558, "value2":5277008629054064552, "value3":false, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537157, "value1":0.4103644112452139, "value2":9148697464094805661, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353178, "value1":0.8020014291715372, "value2":611786907029670311, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505264, "value1":0.10898025782239415, "value2":5937127177824419205, "value3":true, "__name__":"metric_00000989", "key_2":"h","key_7":"f","key_0":"c", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.228410, "value1":0.43346447533111176, "value2":7102784791187505051, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321463, "value1":0.9635846235850585, "value2":7091075060854093514, "value3":false, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.999019, "value1":0.15889354502430938, "value2":7074771352556045066, "value3":false, "__name__":"metric_00000992", "key_5":"d","key_6":"a","key_4":"k", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.766619, "value1":0.4943151198279696, "value2":6293058242069710093, "value3":true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382183, "value1":0.47579561094609324, "value2":2482575383850980092, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.054382, "value1":0.7191240626399533, "value2":7914822293313169877, "value3":true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255873, "value1":0.055171339528865125, "value2":1928364223437432733, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.535958, "value1":0.40417404005201557, "value2":8512533852699686666, "value3":true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394085, "value1":0.8317795195864898, "value2":6061937875766891692, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.752931, "value1":0.1811338646540346, "value2":3103290849364510613, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.049507, "value1":0.11124247410560267, "value2":8253232345223333946, "value3":false, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.067068, "value1":0.02792863936331565, "value2":1942738411033963266, "value3":true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746589, "value1":0.43603502537000616, "value2":5153565802559198697, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903064, "value1":0.3556713649253779, "value2":3489353708315351817, "value3":false, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.856458, "value1":0.28933348763855665, "value2":5247367677027973150, "value3":false, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397179, "value1":0.7130148119357395, "value2":5728198037062323847, "value3":true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.927222, "value1":0.7639936733901646, "value2":101990787192301191, "value3":false, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005808, "value1":0.03609120161396754, "value2":9216428818715898202, "value3":true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812165, "value1":0.4866254594918793, "value2":3026112033227979237, "value3":false, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420851, "value1":0.7402262167603453, "value2":6177484091405440598, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836705, "value1":0.0336701243488239, "value2":4778937227074309153, "value3":true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097260, "value1":0.6626429057561898, "value2":7257306969130399155, "value3":true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496088, "value1":0.36910009322506554, "value2":3657880390271983286, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870639, "value1":0.7067162570632016, "value2":1102103625009499222, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.020747, "value1":0.46587247219742656, "value2":5967339025562916268, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.724139, "value1":0.8163768770313428, "value2":2067500216739340556, "value3":false, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.369554, "value1":0.7671983408631192, "value2":4937837819229078201, "value3":false, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382589, "value1":0.94426076786917, "value2":575895696952869704, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115327, "value1":0.47575759482061636, "value2":2541949700573197783, "value3":false, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260734, "value1":0.8252666210219215, "value2":5316874961818228049, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.872358, "value1":0.8794099168533623, "value2":5800876034859982246, "value3":false, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.054141, "value1":0.05686166977091221, "value2":5301631175885078161, "value3":true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759774, "value1":0.8076817854451119, "value2":2675452361460215935, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921855, "value1":0.0467410376485931, "value2":7851926061831999904, "value3":false, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578515, "value1":0.9029496688391901, "value2":3737429676030030627, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812804, "value1":0.47887479306222563, "value2":1687173944192158201, "value3":false, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472625, "value1":0.8441160499425835, "value2":1515286134892734672, "value3":false, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760814, "value1":0.33274037254007843, "value2":1188458814270546919, "value3":true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788501, "value1":0.15942957594654944, "value2":2916736695606434573, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098854, "value1":0.8030098284257553, "value2":2336601872331852302, "value3":true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894386, "value1":0.4139981184160349, "value2":7897864995270774175, "value3":true, "__name__":"metric_00000032", "key_9":"k","key_5":"b","key_8":"g", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117046, "value1":0.5674622805765219, "value2":4791162537839256650, "value3":false, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.542843, "value1":0.19807908776500033, "value2":6807107659814067311, "value3":true, "__name__":"metric_00000033", "key_1":"h","key_9":"f","key_0":"i", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703834, "value1":0.5788042510612239, "value2":5007564675868943302, "value3":true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945291, "value1":0.3701740719620502, "value2":2459541142030887319, "value3":false, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070410, "value1":0.2627776276203505, "value2":9110732671786561088, "value3":false, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.305776, "value1":0.24403108521016115, "value2":3139654072672113070, "value3":true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.417910, "value1":0.549110839930364, "value2":2406040915101429610, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410216, "value1":0.8784609507591513, "value2":3467112924963980779, "value3":false, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186300, "value1":0.13500098796508653, "value2":4869858265003017538, "value3":false, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934357, "value1":0.7572145499314314, "value2":3619701318200032332, "value3":true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871110, "value1":0.4402673304510715, "value2":5018886154821992035, "value3":false, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.493200, "value1":0.1348835909121323, "value2":3845890133642201057, "value3":false, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875968, "value1":0.1364875892800979, "value2":570804136752641105, "value3":true, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.469938, "value1":0.5776925075890766, "value2":8313002338216576665, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.667608, "value1":0.9950302066492306, "value2":1913698187097493343, "value3":false, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497144, "value1":0.7591363060077458, "value2":4304322015184203986, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731147, "value1":0.743025674257032, "value2":3480752580915944185, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878760, "value1":0.5947269209561762, "value2":7998247484475194678, "value3":true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.606309, "value1":0.663213778377581, "value2":3067341192985682212, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.207494, "value1":0.8601914108563131, "value2":256790036338793553, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.144417, "value1":0.30286956006124033, "value2":5377965606661311252, "value3":false, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.134377, "value1":0.609971735912583, "value2":3774215546443285103, "value3":false, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.434483, "value1":0.986170039019965, "value2":5064754870308124270, "value3":true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.509040, "value1":0.5522321797777098, "value2":5012812814115566068, "value3":false, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903746, "value1":0.20330422291674133, "value2":6986910292987554191, "value3":false, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282547, "value1":0.04694512830804489, "value2":6329857494201458927, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354275, "value1":0.16833606088021735, "value2":3136536256813456866, "value3":true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774256, "value1":0.5410085627082251, "value2":3587475793865363544, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.762751, "value1":0.806841924275979, "value2":7358998298622408616, "value3":true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584082, "value1":0.6563492986452016, "value2":387593409104388354, "value3":true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.080055, "value1":0.5509982288474451, "value2":6251069533288673484, "value3":true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695506, "value1":0.8632479367125148, "value2":2061552818267451584, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489099, "value1":0.6265315870043634, "value2":4304737020249244240, "value3":true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098523, "value1":0.5339406052485941, "value2":4381041775949231103, "value3":false, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945882, "value1":0.23785010902188283, "value2":6729562900414289120, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.167154, "value1":0.3701120650676118, "value2":622381534969203518, "value3":true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775199, "value1":0.6184507954864329, "value2":5210419683928766855, "value3":true, "__name__":"metric_00000068", "key_9":"j","key_3":"f","key_4":"g", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.730074, "value1":0.38121631002699524, "value2":419350096079979395, "value3":false, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.013737, "value1":0.8747949109587233, "value2":1018960453244065142, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.091852, "value1":0.629803885859078, "value2":859062102274314528, "value3":true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166974, "value1":0.47395530253799945, "value2":6411055007256508887, "value3":true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.524472, "value1":0.9186585978945081, "value2":3492305084628232737, "value3":true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241236, "value1":0.3787867675534582, "value2":4760531059679736084, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918712, "value1":0.05290704362542312, "value2":1966436853098488271, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636045, "value1":0.9510956232200477, "value2":8704302710827007347, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187946, "value1":0.6635616634234764, "value2":1433605612315108919, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180096, "value1":0.173261531213523, "value2":3306945339384428282, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.453459, "value1":0.11094008182846338, "value2":5724966973695631624, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562127, "value1":0.3576118834730829, "value2":3463667757785807622, "value3":true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137341, "value1":0.6255329096036213, "value2":1381955744525597553, "value3":false, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913181, "value1":0.29162415237583095, "value2":2202030046648119531, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.443734, "value1":0.3209051100037936, "value2":3274712051745788108, "value3":false, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.302887, "value1":0.38589486316701666, "value2":1489273904023316120, "value3":true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205658, "value1":0.07820373463872678, "value2":7665901880569799873, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.211623, "value1":0.32477633025580727, "value2":6808357456829544888, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.697819, "value1":0.26244647649977987, "value2":6232754250974380613, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853360, "value1":0.3310597786987246, "value2":7565924568483154342, "value3":true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.135295, "value1":0.049613381396899345, "value2":6155724101399152110, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701749, "value1":0.9716574010135762, "value2":3704214151049295904, "value3":false, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527554, "value1":0.6532141955951636, "value2":2786469958733724605, "value3":false, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866043, "value1":0.03422399382505321, "value2":5496039574170303988, "value3":true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.415307, "value1":0.682109162147481, "value2":4924263240012975638, "value3":false, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631078, "value1":0.7274897566111301, "value2":8520095366826456731, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.549725, "value1":0.7077865773466269, "value2":2977543758339035237, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.154150, "value1":0.14439730391589514, "value2":7810498955756317569, "value3":false, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007570, "value1":0.8087681893470107, "value2":4017899853890170979, "value3":false, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.055629, "value1":0.44977291321386215, "value2":3453485470431150033, "value3":false, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189056, "value1":0.012064619774392913, "value2":3564594888238224552, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.545459, "value1":0.412297537176583, "value2":7021178975362987850, "value3":true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.116215, "value1":0.3276649448479836, "value2":3143364922694929733, "value3":true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529456, "value1":0.4708787441244093, "value2":2999813825732785635, "value3":true, "__name__":"metric_00000102", "key_8":"d","key_1":"i", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.966668, "value1":0.04918146120370962, "value2":870818154766064140, "value3":false, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601524, "value1":0.4723442759712491, "value2":5773596887056998694, "value3":false, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712431, "value1":0.04260252713403969, "value2":4497318416983710614, "value3":true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452968, "value1":0.03699415334331326, "value2":3190104546355404213, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.394341, "value1":0.05586282401938057, "value2":2019354346274899062, "value3":true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.084367, "value1":0.35383668982583555, "value2":14041413767842483, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227017, "value1":0.7278732120270098, "value2":8901905120302064125, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494493, "value1":0.0910027308098097, "value2":3614458258424175747, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059547, "value1":0.2958045744232146, "value2":5813283620430694858, "value3":true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.423389, "value1":0.9221509815116762, "value2":6742495498273189642, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501855, "value1":0.44198471706869047, "value2":7662744388645110665, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293650, "value1":0.512834114916101, "value2":5984393328938374467, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614743, "value1":0.8714098760733937, "value2":937734757736172502, "value3":false, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406459, "value1":0.17711840072487878, "value2":749048739837121472, "value3":false, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711112, "value1":0.5465711310503617, "value2":2943773951314932011, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335697, "value1":0.8515828942170133, "value2":7615557480020888685, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548405, "value1":0.13026413723454033, "value2":4513263208882824306, "value3":true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.351448, "value1":0.28264554746804893, "value2":1701229548455066314, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501558, "value1":0.7474271542447126, "value2":4300562906938627060, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107125, "value1":0.5220893826927367, "value2":3433346642469038441, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968552, "value1":0.32921550916944375, "value2":4204134061287881453, "value3":false, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.438315, "value1":0.5786006279022659, "value2":7016921001360584415, "value3":false, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.834753, "value1":0.15159677478724748, "value2":3404421917920731295, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454269, "value1":0.5292362536342466, "value2":6241633082661695873, "value3":false, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137463, "value1":0.4976726181585776, "value2":7634359338923847776, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744484, "value1":0.7709542443985684, "value2":2984932547540133289, "value3":false, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.653649, "value1":0.40168424113567774, "value2":9208485552143870835, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420303, "value1":0.8834331384252292, "value2":6602388659890449882, "value3":false, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.995276, "value1":0.35489802779155194, "value2":3627299399432728520, "value3":false, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.609717, "value1":0.32106453173709887, "value2":6997589541390821763, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.764384, "value1":0.448976213692369, "value2":2970007228856945436, "value3":true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548393, "value1":0.15204462152678552, "value2":7646037909450468928, "value3":false, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379101, "value1":0.7807002939639928, "value2":783999596388900221, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.278309, "value1":0.014487780492841778, "value2":1057643982300790379, "value3":false, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.016491, "value1":0.40903973612790867, "value2":453932377401006766, "value3":true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347001, "value1":0.5622168932310977, "value2":5705011066230310463, "value3":true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.086911, "value1":0.6494793319729653, "value2":861844073039964568, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.997904, "value1":0.8338228010696771, "value2":1349114457177401043, "value3":false, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266213, "value1":0.43023507437229846, "value2":2397780341037792730, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068462, "value1":0.6058806411980994, "value2":78381759723174420, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358282, "value1":0.2854157607268251, "value2":9047800747480831869, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845468, "value1":0.009988708005244632, "value2":2440593977353687165, "value3":true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780415, "value1":0.6187822246822943, "value2":4482602300599359335, "value3":true, "__name__":"metric_00000145", "key_1":"g","key_4":"k","key_0":"c", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700223, "value1":0.018684194191095932, "value2":6529685586040726217, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.792345, "value1":0.9111940168010332, "value2":1575039147844860051, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275696, "value1":0.01410734034670126, "value2":8535583193313743852, "value3":false, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458977, "value1":0.23636701604224677, "value2":314843807550918715, "value3":true, "__name__":"metric_00000148", "key_9":"d","key_3":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711597, "value1":0.1315269488618045, "value2":914748291669479577, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003734, "value1":0.48566417416368673, "value2":5809881749485776988, "value3":false, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.587116, "value1":0.411572893699433, "value2":582776826400065747, "value3":true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000207, "value1":0.25509895950578154, "value2":5802214523620869062, "value3":false, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082969, "value1":0.48435989736978513, "value2":7352735247711683766, "value3":false, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.042350, "value1":0.16157141460949973, "value2":1607874054697622469, "value3":true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.765374, "value1":0.15664293959132114, "value2":3675493379037113802, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.359224, "value1":0.887036256695556, "value2":7786059543942584229, "value3":true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293533, "value1":0.267785564160632, "value2":2723125994900979360, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964294, "value1":0.716239726237387, "value2":8651041612345523274, "value3":true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076213, "value1":0.9749455558859894, "value2":5297461003834502762, "value3":false, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.896241, "value1":0.7102104646771853, "value2":2795778846362770396, "value3":true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903120, "value1":0.14414851964547873, "value2":7334142904098197, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.843431, "value1":0.8361555205773405, "value2":7976205336110881786, "value3":true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731615, "value1":0.4179919061811569, "value2":1554283199870929425, "value3":true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.953374, "value1":0.7537780463182495, "value2":2591672750977463865, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400999, "value1":0.7471726171159019, "value2":7300350647929857982, "value3":false, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232296, "value1":0.7720730243870363, "value2":1060384392061020703, "value3":true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219798, "value1":0.16805543570986609, "value2":1462620146278820176, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364330, "value1":0.7255148218772242, "value2":713753994079127610, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152830, "value1":0.13410318940561705, "value2":9140011796352402602, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585566, "value1":0.4130103851752984, "value2":2970454230901035575, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.718094, "value1":0.5500539331076559, "value2":8305617631847453693, "value3":false, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.624058, "value1":0.9500416769324029, "value2":2935568409193402069, "value3":false, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539303, "value1":0.05177136725388866, "value2":7442175860869173746, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.172986, "value1":0.4456449462659514, "value2":137655062794989901, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631420, "value1":0.9356892224700029, "value2":8946945917645913041, "value3":true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948627, "value1":0.9988907819529511, "value2":7978995836658846886, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946323, "value1":0.5613474012283862, "value2":648588536937617868, "value3":true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361054, "value1":0.4585620050578102, "value2":5573215420419545492, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.817729, "value1":0.3241298418342054, "value2":2657202509964663855, "value3":false, "__name__":"metric_00000179", "key_6":"i","key_7":"h","key_2":"b", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713670, "value1":0.7338301625220077, "value2":7217765288314019300, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.521076, "value1":0.5030076209647331, "value2":3777966789579493364, "value3":true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.855317, "value1":0.7172152098421418, "value2":8060266815999977878, "value3":true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849239, "value1":0.9561486509896251, "value2":266727525021228946, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008199, "value1":0.11740640397171373, "value2":3689690733597943394, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936773, "value1":0.795334335887194, "value2":4733404788345141419, "value3":true, "__name__":"metric_00000185", "key_5":"d","key_7":"k","key_1":"g", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888974, "value1":0.9298030625919304, "value2":6088282595701162644, "value3":true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247054, "value1":0.1558941325474189, "value2":460928608228918469, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853182, "value1":0.8511278083029166, "value2":5598797292292719359, "value3":false, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.252753, "value1":0.16156992511009938, "value2":4368635811765896864, "value3":true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309449, "value1":0.5849973411963314, "value2":683197153873995001, "value3":true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.222187, "value1":0.6637363404506509, "value2":201431956551451886, "value3":false, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805299, "value1":0.03082292423828661, "value2":3135350466088347105, "value3":false, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489193, "value1":0.47023015827549575, "value2":301183049195776618, "value3":true, "__name__":"metric_00000194", "key_6":"b","key_1":"b","key_2":"c", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.058774, "value1":0.7620086190415307, "value2":2178333587571299612, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098933, "value1":0.696442402358351, "value2":427216344556980093, "value3":true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497026, "value1":0.18415993654905655, "value2":8786493859199944811, "value3":true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257640, "value1":0.6674105512961352, "value2":395753979079709412, "value3":true, "__name__":"metric_00000197", "key_9":"g","key_1":"d","key_4":"d", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490944, "value1":0.5114071451614219, "value2":2042645288775439509, "value3":true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455445, "value1":0.7431063214848364, "value2":8165863124378358117, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.796800, "value1":0.6414858212054014, "value2":2442066548061613486, "value3":true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616766, "value1":0.17517079750483502, "value2":2134452812266479738, "value3":true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060207, "value1":0.5103041206672632, "value2":6515531846051600666, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677324, "value1":0.2542456767596465, "value2":3968114223247313347, "value3":false, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.487523, "value1":0.6903207412936282, "value2":1424416428597884416, "value3":false, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601220, "value1":0.7953800597000228, "value2":3247496941215159385, "value3":false, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.642959, "value1":0.2541441152051556, "value2":7121652419723619875, "value3":true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268832, "value1":0.025435318199453064, "value2":8277309445690585191, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591576, "value1":0.98502525758626, "value2":3261712292707439368, "value3":false, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696131, "value1":0.602844278741062, "value2":9150721424292798196, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.799050, "value1":0.6397730903214256, "value2":322841440821332477, "value3":false, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703698, "value1":0.7179811740811757, "value2":6911091771320522730, "value3":false, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.276968, "value1":0.16623158682521033, "value2":5763688015485978321, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.932237, "value1":0.9953773456479392, "value2":1648531648235955285, "value3":true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562810, "value1":0.4786357292651346, "value2":3349388012721867031, "value3":true, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496221, "value1":0.10819101220823944, "value2":5211051877655840528, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720431, "value1":0.552698162040604, "value2":7021843927329324470, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.087213, "value1":0.8472571863880398, "value2":7745061885728922306, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.039167, "value1":0.21533794157440472, "value2":3121229544727240655, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309356, "value1":0.1590321315931928, "value2":710479249184112134, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.191661, "value1":0.040375310871212516, "value2":1055365248220450266, "value3":false, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189905, "value1":0.8222460270476257, "value2":9109111690041797580, "value3":true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517425, "value1":0.7168387126170568, "value2":6945048447223853682, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916881, "value1":0.37193142727878464, "value2":2747126651524473231, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.395094, "value1":0.6414572510838905, "value2":2905014786439561176, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.163189, "value1":0.7311236407483467, "value2":5070208982215207577, "value3":false, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827333, "value1":0.19235777061627696, "value2":1042609530408390018, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107479, "value1":0.11539837467269841, "value2":7629188841800355160, "value3":false, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186100, "value1":0.23799692868617986, "value2":3114291156883869525, "value3":false, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790442, "value1":0.4974549500784167, "value2":3142531162087829163, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.797183, "value1":0.252808246395566, "value2":7867371410034903023, "value3":true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269019, "value1":0.4459897405815389, "value2":5769972800266784623, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299688, "value1":0.7900084672305991, "value2":4400086899085085411, "value3":true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.295251, "value1":0.4992500267841033, "value2":8698415822608825668, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.992903, "value1":0.9703688948654993, "value2":699484569689785049, "value3":false, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694010, "value1":0.24590491974037657, "value2":6378096972799959498, "value3":true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916779, "value1":0.03667685970441408, "value2":8236016681410946158, "value3":false, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.018930, "value1":0.7762719211488223, "value2":5653992369569699642, "value3":true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673243, "value1":0.9005498092358467, "value2":2541259280078576557, "value3":false, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.403564, "value1":0.7066442393226117, "value2":7468478788293853391, "value3":true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912162, "value1":0.4479784585493673, "value2":2451129253364350697, "value3":false, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622846, "value1":0.6985884362308626, "value2":1822728881940552123, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707469, "value1":0.052057120254889885, "value2":640515677830640949, "value3":true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433576, "value1":0.4097754058261397, "value2":2475392911974322330, "value3":false, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374459, "value1":0.1353067477342801, "value2":7368967313707141232, "value3":false, "__name__":"metric_00000245", "key_7":"k","key_3":"g","key_5":"g", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993651, "value1":0.06711618906287087, "value2":1750993237587336816, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219084, "value1":0.5316119999916927, "value2":5790597415731545344, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527236, "value1":0.6470232539834205, "value2":5706809590317379695, "value3":false, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566174, "value1":0.9745838789323467, "value2":774234656544468494, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803752, "value1":0.8708197206158679, "value2":3775873873449515760, "value3":true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.388879, "value1":0.4862649086023325, "value2":8962983929778901050, "value3":true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471944, "value1":0.25578979603582636, "value2":7732635688969517720, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017485, "value1":0.4968624551232297, "value2":202940911271524303, "value3":true, "__name__":"metric_00000254", "key_8":"c","key_1":"f","key_3":"b", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.333780, "value1":0.8299780273551013, "value2":895214629529935810, "value3":true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.164082, "value1":0.9321061863901389, "value2":7407307236544459993, "value3":true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225068, "value1":0.9163062543062077, "value2":8739271132887488724, "value3":true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304547, "value1":0.45077586944816295, "value2":1933302507685789986, "value3":false, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.538110, "value1":0.4128481092985977, "value2":3178182543294139274, "value3":true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541173, "value1":0.7483924955860858, "value2":212944257013368531, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255386, "value1":0.14438260405328707, "value2":7645966424158688784, "value3":true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.073000, "value1":0.7628193459385832, "value2":4973597494190231390, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408853, "value1":0.5564603800067697, "value2":4401411283807625564, "value3":true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447512, "value1":0.8939878640344981, "value2":2693396599053914337, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048017, "value1":0.34230394362828026, "value2":3182689568942606164, "value3":true, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057151, "value1":0.8042880864594656, "value2":2051969816473375198, "value3":false, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746476, "value1":0.7332341667442952, "value2":461488002348760082, "value3":false, "__name__":"metric_00000267", "key_6":"d","key_1":"i","key_4":"h", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180890, "value1":0.31049348146155, "value2":1020023460971269956, "value3":true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550108, "value1":0.4362790210713837, "value2":7901529415931806303, "value3":false, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345598, "value1":0.11823802615751057, "value2":8356166230167153323, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804625, "value1":0.5643553567149021, "value2":2154734444228835687, "value3":true, "__name__":"metric_00000268", "key_8":"c","key_9":"d","key_6":"j", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247313, "value1":0.6975496564473892, "value2":3820052334665018388, "value3":true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805735, "value1":0.31084346436090265, "value2":4462974662487547374, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364439, "value1":0.1744566217814337, "value2":607910964577919864, "value3":true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589313, "value1":0.37010824428777245, "value2":5796079422541877022, "value3":true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931293, "value1":0.5772458152930102, "value2":1027336122468786461, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527679, "value1":0.2977670251954975, "value2":3769527680254605539, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.404475, "value1":0.38496439646172187, "value2":2284351654690866371, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772590, "value1":0.07775425303960136, "value2":8477700234367515830, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722168, "value1":0.17927054370632736, "value2":7192360571641392544, "value3":true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712775, "value1":0.576069645932529, "value2":4594581975868935835, "value3":false, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.126652, "value1":0.5061094013712955, "value2":3728550795113093627, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321145, "value1":0.13357161718693558, "value2":588806792143157793, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.101550, "value1":0.6394371793311785, "value2":3045802217092318405, "value3":true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.547834, "value1":0.5303880277608923, "value2":244812322497139670, "value3":true, "__name__":"metric_00000287", "key_5":"f","key_6":"f","key_2":"k", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.022447, "value1":0.813105541632356, "value2":4932627202785941294, "value3":false, "__name__":"metric_00000285", "key_8":"b","key_9":"k","key_5":"b", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382289, "value1":0.3508103161551219, "value2":2376325264202595095, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.611464, "value1":0.9618639190404236, "value2":2197184281244482985, "value3":false, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.911985, "value1":0.3625352195059055, "value2":211893969967676184, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268188, "value1":0.6899041268799436, "value2":3263903700628732234, "value3":true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229932, "value1":0.7830409765107325, "value2":3581295224187821255, "value3":true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921675, "value1":0.15132831757259205, "value2":8941349375775958121, "value3":true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.738768, "value1":0.28818505448700765, "value2":5311365998633458584, "value3":true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.670459, "value1":0.7891122563117983, "value2":5569764792017894880, "value3":false, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.893664, "value1":0.863874518524225, "value2":4812640494718764785, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484101, "value1":0.6065100518640419, "value2":3350972220626617309, "value3":false, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048572, "value1":0.15248106917852358, "value2":6593923637797377418, "value3":false, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.043823, "value1":0.6456350127530551, "value2":3888964250592362865, "value3":false, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109495, "value1":0.6845812758644145, "value2":167613517010834758, "value3":true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.328450, "value1":0.42643027428907515, "value2":8129659398848977292, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.429304, "value1":0.14015794848590404, "value2":4681962038728020905, "value3":true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.776267, "value1":0.3289665984746857, "value2":3318083720052608693, "value3":true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496512, "value1":0.3624675397742157, "value2":6289410424653796398, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441324, "value1":0.9094233421689666, "value2":2912289985138568601, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.650459, "value1":0.3752516427539381, "value2":4398293649161961566, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.350383, "value1":0.1741603700667637, "value2":3124160867383954120, "value3":true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.051974, "value1":0.5242492694985263, "value2":6760802540060503552, "value3":false, "__name__":"metric_00000306", "key_9":"i","key_5":"d","key_8":"f", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.291840, "value1":0.5576830705012547, "value2":4155187805029620559, "value3":true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853505, "value1":0.13344813988799406, "value2":7220245991757698014, "value3":false, "__name__":"metric_00000308", "key_4":"b","key_1":"f","key_2":"f", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.480822, "value1":0.34390971406763976, "value2":4047856794594370906, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192205, "value1":0.46496172552445103, "value2":2904944897204510913, "value3":false, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668616, "value1":0.9800566015891223, "value2":3768818727218430516, "value3":true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.940469, "value1":0.8502278446477077, "value2":5643864232780485221, "value3":false, "__name__":"metric_00000312", "key_4":"c","key_9":"i","key_0":"a", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775796, "value1":0.26560945264856717, "value2":4386556505603475589, "value3":false, "__name__":"metric_00000313", "key_4":"c","key_9":"g","key_2":"f", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484224, "value1":0.10363743205437767, "value2":7594526116903338826, "value3":true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.889136, "value1":0.4627114619785784, "value2":4680730415934145352, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739229, "value1":0.1719548250883568, "value2":6803717844294234894, "value3":true, "__name__":"metric_00000316", "key_8":"i","key_4":"f","key_6":"k", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057765, "value1":0.3081555912049451, "value2":8499651141401941047, "value3":true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.450693, "value1":0.8168675860307004, "value2":7949382588313472478, "value3":true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770167, "value1":0.272960033553786, "value2":1543554906153054869, "value3":true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411881, "value1":0.5252975573521564, "value2":7223448923269287286, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.892105, "value1":0.15152104472471403, "value2":3531358618763240055, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.882618, "value1":0.45516649656010977, "value2":2288125701245215412, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913392, "value1":0.6620512228658078, "value2":4363674990969315932, "value3":false, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.835393, "value1":0.7513525395048825, "value2":5883431192279251653, "value3":true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.563057, "value1":0.9008981984383352, "value2":6465724162584986904, "value3":false, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007001, "value1":0.9502622268086611, "value2":6989543064471041687, "value3":false, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688255, "value1":0.07451084769674933, "value2":2957263551734111550, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751629, "value1":0.2617947338456188, "value2":684618542420066244, "value3":false, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930787, "value1":0.6308597904360509, "value2":6601900249320434792, "value3":true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.363243, "value1":0.8139409689083822, "value2":1497160325123262680, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676085, "value1":0.9711732869605988, "value2":1463546357374490230, "value3":true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.161858, "value1":0.6374277110955211, "value2":6435065381978315913, "value3":false, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757941, "value1":0.910443361797238, "value2":2854035357843167779, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.331899, "value1":0.6895825352637633, "value2":621471764809456095, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473746, "value1":0.2844187612168192, "value2":5704986401969616545, "value3":true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048060, "value1":0.9984260267119236, "value2":4771344600707329825, "value3":false, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.030112, "value1":0.9637634957448155, "value2":5568130719531472451, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257160, "value1":0.5940271387669745, "value2":142865511754737819, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760326, "value1":0.4334581523321684, "value2":6413870941968139402, "value3":true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827687, "value1":0.8136794802681319, "value2":746965665674196204, "value3":true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447680, "value1":0.3168283761550826, "value2":7144161377877398366, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690596, "value1":0.3996428785716089, "value2":3598801694226963836, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.009204, "value1":0.21753660582669873, "value2":1790271338088923331, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.432520, "value1":0.8760470338201594, "value2":2329654518443264597, "value3":false, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558450, "value1":0.5241764449812006, "value2":8062760419700144076, "value3":true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.999165, "value1":0.4211253053940594, "value2":6241537222130424287, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.492382, "value1":0.19419908429465416, "value2":8132562796409382183, "value3":true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968488, "value1":0.8740668460370395, "value2":5462263580744941212, "value3":false, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566020, "value1":0.3116629548456918, "value2":1660846180499855539, "value3":true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899292, "value1":0.4455952655758791, "value2":5641720897227183391, "value3":true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213999, "value1":0.7107285930376039, "value2":320006539559036783, "value3":false, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177859, "value1":0.6189254938745993, "value2":3916861895859871884, "value3":false, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.942059, "value1":0.3464916414346185, "value2":1739217875853687994, "value3":false, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637964, "value1":0.4027137396066144, "value2":498622351647583844, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994610, "value1":0.7530828834500288, "value2":5890892402021233384, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.727487, "value1":0.28182955033112955, "value2":4630833361364112333, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.915590, "value1":0.5470194104324091, "value2":1666728635470642191, "value3":true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757932, "value1":0.5371051329548212, "value2":1493513093655584392, "value3":true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.254302, "value1":0.06977656180754654, "value2":5979425073849108121, "value3":false, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.475877, "value1":0.8550799756217883, "value2":624522672976326814, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247089, "value1":0.39743079305559637, "value2":4830779617580090488, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015595, "value1":0.2577347123153436, "value2":2406835300488043720, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945719, "value1":0.3129335496013211, "value2":820286347352483926, "value3":false, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081969, "value1":0.6051672450326654, "value2":8261558360059795606, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948207, "value1":0.7815897187663656, "value2":3429909037501105433, "value3":false, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.935696, "value1":0.909809980891043, "value2":6385383912360129741, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.248852, "value1":0.9247408795385116, "value2":9175721939467544545, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934738, "value1":0.6334884200328613, "value2":327826698570505025, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102837, "value1":0.44571438142580455, "value2":4747306485524936018, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.885921, "value1":0.35994074928816305, "value2":1395907621788196782, "value3":false, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.949368, "value1":0.7895021421389461, "value2":7949753331449968910, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.633040, "value1":0.5458972679689471, "value2":4245189254820710416, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.150358, "value1":0.7548359500337994, "value2":8481176564399105584, "value3":true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.557167, "value1":0.37427466994480085, "value2":1133639012241894316, "value3":true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.093134, "value1":0.29043113295756745, "value2":6947797053182077820, "value3":true, "__name__":"metric_00000375", "key_7":"d","key_1":"j","key_6":"a", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.136231, "value1":0.7872857259243544, "value2":2466956534940008420, "value3":true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.329501, "value1":0.01079935258997052, "value2":4968522926518293278, "value3":false, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.825131, "value1":0.6767719760068325, "value2":8529334985450219291, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807971, "value1":0.7351117134307579, "value2":3019346910002173461, "value3":true, "__name__":"metric_00000379", "key_9":"k","key_3":"f","key_7":"a", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017959, "value1":0.6253725541571399, "value2":8835344453005145545, "value3":false, "__name__":"metric_00000380", "key_1":"f","key_4":"j","key_0":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.280599, "value1":0.16481336771423033, "value2":1417072867431315310, "value3":false, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673036, "value1":0.8178932673930933, "value2":6530240293167023011, "value3":false, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192297, "value1":0.7505738676572953, "value2":6861957449422419505, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517236, "value1":0.639000099787322, "value2":5815570450456127187, "value3":true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739883, "value1":0.17493563192408712, "value2":7179940437785343272, "value3":true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.531912, "value1":0.08360001327419847, "value2":2755543678440921421, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452307, "value1":0.8726532834145122, "value2":187164282595967676, "value3":false, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517272, "value1":0.9444444024596198, "value2":5607680802025895328, "value3":true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289453, "value1":0.15619031798053584, "value2":810705789880173134, "value3":false, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397508, "value1":0.9544477853731341, "value2":7641027031682219689, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812107, "value1":0.9448725135844366, "value2":5145031592575451663, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.063548, "value1":0.922662738511914, "value2":2569888172146766439, "value3":true, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.959005, "value1":0.7947728545231653, "value2":4794325670292386969, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.570203, "value1":0.9426031560649749, "value2":8759901536059040166, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.514218, "value1":0.47519036709767065, "value2":1247765910819397634, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541370, "value1":0.3774216285225394, "value2":7667033726860743169, "value3":false, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.938282, "value1":0.020047520852319107, "value2":7849085976338159942, "value3":false, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874020, "value1":0.47048043771282444, "value2":2196635070664843443, "value3":false, "__name__":"metric_00000407", "key_8":"c","key_3":"a","key_6":"e", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513588, "value1":0.012232986779526358, "value2":8478348320905895865, "value3":true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109700, "value1":0.7338145643706604, "value2":6797416603289211588, "value3":false, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.488779, "value1":0.6031093288566471, "value2":5300319596464382751, "value3":true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931594, "value1":0.42339447548233033, "value2":969762807855801088, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.553860, "value1":0.676076748018672, "value2":1192139536623982501, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.132006, "value1":0.6728558661354047, "value2":1380355336108501777, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.669226, "value1":0.1577792131508693, "value2":4512376080159929307, "value3":true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921683, "value1":0.24615483388690507, "value2":5797393083888487885, "value3":false, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.800085, "value1":0.768303750055719, "value2":6971653838071612447, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468961, "value1":0.45803974634578515, "value2":1199483949895079721, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356489, "value1":0.49471640885346424, "value2":4138046086256944727, "value3":true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225992, "value1":0.6741558407603926, "value2":5216648174057731100, "value3":true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.779458, "value1":0.11692394331277675, "value2":9011198363894466670, "value3":true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.683290, "value1":0.48772002686762633, "value2":7212512979389072906, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053472, "value1":0.5939009672642634, "value2":1146669122452133702, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790089, "value1":0.8261492714018603, "value2":6290707817110359790, "value3":false, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145153, "value1":0.6394833078139336, "value2":7849898590496421466, "value3":false, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.367040, "value1":0.6490663681468899, "value2":3967326627567255091, "value3":true, "__name__":"metric_00000415", "key_7":"k","key_9":"e","key_4":"j", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930388, "value1":0.6376629127038135, "value2":3197193413798224716, "value3":true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471623, "value1":0.06021884177520976, "value2":4773928340410962251, "value3":false, "__name__":"metric_00000418", "key_7":"e","key_1":"c","key_6":"i", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201531, "value1":0.7136591237382468, "value2":6260011441045155926, "value3":true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064553, "value1":0.2818642070197672, "value2":2495939851413003901, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282911, "value1":0.1452427125541155, "value2":8365230104234658997, "value3":true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279591, "value1":0.8354266947002554, "value2":1254307733558365753, "value3":false, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849657, "value1":0.6912085697845711, "value2":1824933885382646361, "value3":false, "__name__":"metric_00000424", "key_2":"d","key_8":"b","key_1":"i", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.092000, "value1":0.053974617818740134, "value2":8465803905922172931, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408792, "value1":0.3821176162488874, "value2":4135301501032886839, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.728999, "value1":0.9460512775720927, "value2":5232062325949200093, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.706765, "value1":0.05808795299595995, "value2":3187473873599955497, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689611, "value1":0.07108495627051868, "value2":8720638688812490559, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.686555, "value1":0.11535487696002504, "value2":3943663806047625686, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815986, "value1":0.6951101939977824, "value2":2103945278295322543, "value3":false, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991212, "value1":0.9405521412873721, "value2":3787480236551923641, "value3":false, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665833, "value1":0.21165663807431181, "value2":3904059491287927557, "value3":true, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183315, "value1":0.5795855388985459, "value2":1854253684581147206, "value3":false, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344965, "value1":0.8721578775869344, "value2":2246100236955108961, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671953, "value1":0.31034394563958956, "value2":193813594216774336, "value3":true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.127315, "value1":0.4080665667532778, "value2":360930115537783747, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.947191, "value1":0.901998732832304, "value2":1708333379014319670, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894585, "value1":0.9195635726221545, "value2":8084121448593915190, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004792, "value1":0.9591461919610487, "value2":8675745838539187803, "value3":true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900181, "value1":0.4274569433819623, "value2":6055452196459435249, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948861, "value1":0.954396067309185, "value2":2700832246849384276, "value3":true, "__name__":"metric_00000450", "key_8":"k","key_9":"g","key_3":"h", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656208, "value1":0.4983661330975837, "value2":3291693517576289438, "value3":false, "__name__":"metric_00000451", "key_4":"h","key_5":"c","key_1":"a", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.937253, "value1":0.2620189984734357, "value2":8181737649910995938, "value3":true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867100, "value1":0.06224158128949376, "value2":4580692356241748765, "value3":false, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827749, "value1":0.03871136162211054, "value2":7089705182458313228, "value3":true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241402, "value1":0.4482242754297532, "value2":9114765255689214165, "value3":true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.602059, "value1":0.29735202117340814, "value2":3162172555777612351, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.249371, "value1":0.23918886296603434, "value2":9067747281684729145, "value3":true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725148, "value1":0.8102090769702104, "value2":4626069739522513533, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.928403, "value1":0.3838571333237863, "value2":8647736244601572769, "value3":true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358211, "value1":0.32922133279873583, "value2":8174411590575614648, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347825, "value1":0.7167150468238062, "value2":8311537141574084420, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875897, "value1":0.26919747077534717, "value2":9053609159881163713, "value3":true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392772, "value1":0.2800052196883224, "value2":7203447031464069407, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849662, "value1":0.5009846634201038, "value2":939832331440858045, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.024201, "value1":0.9391704477843268, "value2":2885798586864811264, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952269, "value1":0.2664477356829927, "value2":847075590517072325, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.307777, "value1":0.6456685214669929, "value2":3218983200036055999, "value3":true, "__name__":"metric_00000435", "key_5":"a","key_9":"h","key_3":"j", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.832715, "value1":0.2591746701265793, "value2":3426357136923132270, "value3":true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.510375, "value1":0.5688430382533638, "value2":7025749889591390600, "value3":false, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.850762, "value1":0.6438552947023466, "value2":1197489157433338286, "value3":false, "__name__":"metric_00000438", "key_6":"f","key_1":"j", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.905930, "value1":0.54095928495375, "value2":67603635153932551, "value3":true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636437, "value1":0.5854506016456503, "value2":5128251159570110333, "value3":false, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836009, "value1":0.4067978187295729, "value2":6225162557978433985, "value3":true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.781439, "value1":0.8809907639802731, "value2":1945481468549859562, "value3":false, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722225, "value1":0.6928638550983661, "value2":7984089816823103368, "value3":false, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631076, "value1":0.0305606186093776, "value2":3704333676945193488, "value3":true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.515877, "value1":0.03574457632343526, "value2":8116748669267665996, "value3":true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.062029, "value1":0.8880344844442181, "value2":7205869492721817899, "value3":true, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.753934, "value1":0.5309770343574366, "value2":3276732533413239990, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598973, "value1":0.9458250274999828, "value2":4588112807086674988, "value3":false, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284154, "value1":0.6864088859089572, "value2":8516198907569397198, "value3":false, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.337595, "value1":0.45387405891882765, "value2":2878690183576728505, "value3":false, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743636, "value1":0.5411320007572493, "value2":7926291410401458702, "value3":true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751569, "value1":0.29015226479219164, "value2":550259652101408680, "value3":false, "__name__":"metric_00000476", "key_6":"b","key_9":"f","key_4":"g", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696737, "value1":0.19318283293628083, "value2":8059270019889405703, "value3":false, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558974, "value1":0.5527141281446059, "value2":4981805365285955545, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.499157, "value1":0.3593055667554787, "value2":8763899128009608550, "value3":true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725437, "value1":0.38132949820890993, "value2":2593622500148824140, "value3":true, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575337, "value1":0.27925006017882537, "value2":4225105729105614711, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.168187, "value1":0.8455269167046058, "value2":4706079022430056900, "value3":false, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.223652, "value1":0.14768410638621007, "value2":2848574409182395374, "value3":false, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.655719, "value1":0.26992469602574687, "value2":4754721503564673004, "value3":false, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972901, "value1":0.22308996900209793, "value2":4121883961065445917, "value3":true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513268, "value1":0.763713042109811, "value2":1389619580667003266, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335708, "value1":0.9988121809461258, "value2":8201212630370607829, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874837, "value1":0.37662051730821194, "value2":1239286216512679885, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.761409, "value1":0.08352300660036191, "value2":788197175065166874, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993753, "value1":0.08212627471454952, "value2":2431495613386540436, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513306, "value1":0.09225555422185021, "value2":6434312790658522957, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899387, "value1":0.2747071960330611, "value2":3838090571407685488, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498947, "value1":0.19881833202736185, "value2":645491056031774982, "value3":false, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851971, "value1":0.9432399401550566, "value2":3792284814302154775, "value3":true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912337, "value1":0.6898029391029513, "value2":8695381256661600831, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060802, "value1":0.4672690888050405, "value2":509110642459148760, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.988275, "value1":0.6949519126514042, "value2":2125445866842945317, "value3":false, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.537780, "value1":0.9644299177479598, "value2":2844641921809856363, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.129171, "value1":0.2870831250353682, "value2":5356758362083554180, "value3":true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.699225, "value1":0.7476204022496519, "value2":4681919372415529338, "value3":true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.798856, "value1":0.5178345021294356, "value2":8950909328927419020, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356529, "value1":0.9689016091606751, "value2":778967583500360424, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.740297, "value1":0.48229504021118547, "value2":319778397891873168, "value3":false, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082023, "value1":0.8672498335471771, "value2":2761738695533665878, "value3":false, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.737582, "value1":0.21539065762206674, "value2":1173702480391317018, "value3":true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.203204, "value1":0.04428967569336166, "value2":6796467582468653168, "value3":false, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197343, "value1":0.5038157716364858, "value2":5419554006840711744, "value3":false, "__name__":"metric_00000506", "key_5":"g","key_6":"g","key_3":"e", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166454, "value1":0.4520211285816449, "value2":4140960458683685597, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.810425, "value1":0.9556399667218088, "value2":2345821074968960700, "value3":false, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708611, "value1":0.36562484484170277, "value2":7482183983302020104, "value3":true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614909, "value1":0.010690625631515951, "value2":4802163912353727023, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529200, "value1":0.23500509922901588, "value2":1518336865591090789, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957588, "value1":0.8999494296908893, "value2":5272580661109552970, "value3":false, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900198, "value1":0.3844554073160095, "value2":4170365040112200144, "value3":true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.579116, "value1":0.16747277518786968, "value2":6804977291625507700, "value3":false, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250708, "value1":0.8577877748125339, "value2":590691746141274872, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721810, "value1":0.7147800394287138, "value2":4035652902850373019, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.922402, "value1":0.0627394141655773, "value2":4185528067707010977, "value3":true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.198128, "value1":0.08114002676829289, "value2":2239252166892691326, "value3":false, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803033, "value1":0.0019520065249596544, "value2":1312973954738137976, "value3":false, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.174637, "value1":0.09909477348410087, "value2":93134409084713078, "value3":true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250984, "value1":0.49581172361364056, "value2":9059996107352612477, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381289, "value1":0.5781636662002811, "value2":3051612121774242926, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.046509, "value1":0.758530400177533, "value2":67989907532933127, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.933345, "value1":0.9339175035199608, "value2":377586629316226209, "value3":false, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963510, "value1":0.7004527478348754, "value2":1390034556068597934, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918093, "value1":0.17000680115237268, "value2":4215018843369459603, "value3":true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.569630, "value1":0.20175634343868196, "value2":8381338885007503558, "value3":true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.883668, "value1":0.06695323081792075, "value2":1589024156310802370, "value3":false, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.178981, "value1":0.37314887727591756, "value2":518499450705948775, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187208, "value1":0.4599180344115974, "value2":4887550148987301900, "value3":true, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952934, "value1":0.8319538072999415, "value2":3671167315474130299, "value3":false, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.588275, "value1":0.5426790448893685, "value2":6523903627951338788, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.128001, "value1":0.37530546288075156, "value2":5616976404881722053, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004862, "value1":0.8346907316991129, "value2":4019033006145719569, "value3":false, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826250, "value1":0.5254457071340125, "value2":972903594117286740, "value3":true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344623, "value1":0.6999472357226462, "value2":1701492823670778543, "value3":false, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639041, "value1":0.2248828857383105, "value2":4810585903388633809, "value3":false, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.571165, "value1":0.22946219998225406, "value2":8055176566662404630, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117315, "value1":0.7792786347828345, "value2":8503346841194050818, "value3":false, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.646178, "value1":0.29610828190559946, "value2":6707117461343778078, "value3":true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804384, "value1":0.8994527553958773, "value2":3055603759085197982, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.503237, "value1":0.038643663095115535, "value2":97929435615191250, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.917565, "value1":0.06570063912325382, "value2":7659557506835508250, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.194133, "value1":0.7280519586189449, "value2":607993872873285299, "value3":true, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197005, "value1":0.34702978428971004, "value2":2119181822487481024, "value3":false, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578715, "value1":0.4531709992663944, "value2":1719249508626109183, "value3":true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.221581, "value1":0.5476991652235295, "value2":8006956843834072386, "value3":true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177885, "value1":0.5137634978637854, "value2":5009892345362966157, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596214, "value1":0.2368591303155911, "value2":8792401284340375040, "value3":true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182004, "value1":0.5444756866358181, "value2":1210706677095755681, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972249, "value1":0.9909726775988351, "value2":1108183309913410572, "value3":false, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225911, "value1":0.47194616841751835, "value2":3490783413049237610, "value3":true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321843, "value1":0.5905642407899484, "value2":4893835600551421123, "value3":true, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.552992, "value1":0.8654702264058397, "value2":5106927081449112108, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782381, "value1":0.8285431175840194, "value2":9168936255854371872, "value3":true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410849, "value1":0.31656794437122665, "value2":2071986113321022167, "value3":false, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780581, "value1":0.051832044232056354, "value2":6484981440389875558, "value3":true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.072700, "value1":0.698857333675772, "value2":3443549406704488268, "value3":true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689951, "value1":0.5808073564555776, "value2":5048983612386176894, "value3":true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.733768, "value1":0.8029317643963917, "value2":7359595780765172048, "value3":true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364358, "value1":0.34011958562739014, "value2":999331274737013831, "value3":false, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.512816, "value1":0.34358105240557685, "value2":6714721903573983722, "value3":false, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711860, "value1":0.9821515464895118, "value2":5477696623940889135, "value3":true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.857930, "value1":0.697732523770638, "value2":6661494860267552531, "value3":false, "__name__":"metric_00000573", "key_8":"k","key_4":"d","key_5":"d", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114607, "value1":0.8236051117026364, "value2":4748182480100358673, "value3":true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.628228, "value1":0.168082038026373, "value2":7102000252406916356, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.687012, "value1":0.3638576608952103, "value2":8540460395700338240, "value3":true, "__name__":"metric_00000576", "key_9":"c","key_2":"k","key_5":"b", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169601, "value1":0.3111748596945155, "value2":8571165720141489367, "value3":false, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498080, "value1":0.6187426468242411, "value2":2451438877433776073, "value3":false, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866907, "value1":0.5131483959448067, "value2":8869138925582259535, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.396171, "value1":0.28292656018705137, "value2":5391262893377663378, "value3":true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114785, "value1":0.0649996585433473, "value2":3978180949228199287, "value3":true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671381, "value1":0.13833508369091968, "value2":5430930553797657293, "value3":false, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.923816, "value1":0.8519197270200854, "value2":9127083562371495095, "value3":false, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786218, "value1":0.02969493094352536, "value2":7671441472993176714, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774561, "value1":0.2102387127527105, "value2":4039185760762390304, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192344, "value1":0.144819186927003, "value2":378491549245728191, "value3":false, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076506, "value1":0.3187033255519536, "value2":8885020385104236250, "value3":false, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213717, "value1":0.8128476179407741, "value2":2181064442818506560, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528442, "value1":0.5936064804578587, "value2":5648031374498714859, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.767476, "value1":0.1348520099403021, "value2":4074505568425329493, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.449677, "value1":0.09144593933086395, "value2":8719212717808997934, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411794, "value1":0.8990540517914553, "value2":8624156888058647365, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.457637, "value1":0.3197628975782331, "value2":16763285997559821, "value3":false, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.330356, "value1":0.8095806602517515, "value2":4384365921475656332, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584148, "value1":0.16829289997155406, "value2":380241693738836834, "value3":false, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455567, "value1":0.7529538853574959, "value2":1681989614930760275, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.844936, "value1":0.6074576588778431, "value2":7722066461223215119, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926267, "value1":0.5797545736876416, "value2":7883616781252732331, "value3":true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238789, "value1":0.014736618507623488, "value2":4669225171297242340, "value3":true, "__name__":"metric_00000590", "key_9":"g","key_0":"k", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361896, "value1":0.17876601404118195, "value2":5115665659478830691, "value3":false, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756049, "value1":0.04403510631259223, "value2":7660735889331080164, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.031957, "value1":0.33256068212920453, "value2":4453878565561794031, "value3":false, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.828157, "value1":0.39237914319665085, "value2":7249969180646160990, "value3":false, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952636, "value1":0.372690613847562, "value2":6192025452112268327, "value3":true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991675, "value1":0.5494769946982794, "value2":90791903671198766, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240830, "value1":0.9786829569747533, "value2":5978162907079513697, "value3":false, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279525, "value1":0.1494973578442592, "value2":6869066553359800738, "value3":false, "__name__":"metric_00000607", "key_8":"h","key_5":"d", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.939830, "value1":0.37432428722372013, "value2":4226472512491590319, "value3":false, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.564559, "value1":0.7159468168647539, "value2":1512961341899043326, "value3":true, "__name__":"metric_00000609", "key_7":"e","key_6":"k", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048658, "value1":0.5401236641884074, "value2":8347872809477134111, "value3":false, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.340095, "value1":0.4533223235157631, "value2":1670286383442367211, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293295, "value1":0.4258630060523724, "value2":5674260405699571928, "value3":true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.239916, "value1":0.9150796129317534, "value2":8474870548791768473, "value3":true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744309, "value1":0.6470478918019383, "value2":7704476841600519211, "value3":false, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458393, "value1":0.05603920506738755, "value2":1942285693271412968, "value3":true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720119, "value1":0.6222144605927992, "value2":5009900948732974588, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435194, "value1":0.6528463342414516, "value2":2372394551953155351, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.847658, "value1":0.12821161771950726, "value2":1481519210091358924, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.083324, "value1":0.3648356544882518, "value2":15076769986125901, "value3":false, "__name__":"metric_00000619", "key_5":"h","key_0":"h","key_1":"i", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696866, "value1":0.5460078372418026, "value2":8482317071463194706, "value3":false, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878357, "value1":0.9158893232321995, "value2":4112830082593015136, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815751, "value1":0.7832798664333082, "value2":6621546193633794291, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.036509, "value1":0.7258315650178256, "value2":2697123926599878172, "value3":false, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786542, "value1":0.050969235189444476, "value2":960522431282362262, "value3":false, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293909, "value1":0.1637483242537312, "value2":4063809715912746117, "value3":true, "__name__":"metric_00000599", "key_6":"d","key_1":"g","key_2":"e", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960513, "value1":0.15412859037505927, "value2":3609332046279378552, "value3":true, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845902, "value1":0.9308703210994502, "value2":6609460372849170625, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750261, "value1":0.12234055832157559, "value2":8912704681070762337, "value3":false, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458834, "value1":0.9318510467298435, "value2":8026158416345347023, "value3":false, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859979, "value1":0.9301579137721765, "value2":7925354941740095610, "value3":true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.783146, "value1":0.4471773356574157, "value2":8481603802647763313, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.505375, "value1":0.5948200403445978, "value2":52874652951483269, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005495, "value1":0.829951436170396, "value2":3721373420722024974, "value3":false, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867411, "value1":0.6049905245498157, "value2":8221468316724652544, "value3":false, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435857, "value1":0.8205378674820365, "value2":4321340481613778781, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.618588, "value1":0.6552813364407445, "value2":3804139748589484208, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.027076, "value1":0.23841518035800105, "value2":7013010516589186218, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285111, "value1":0.5613630899205873, "value2":1517169132984045486, "value3":false, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117978, "value1":0.3226349150452925, "value2":5930250973192218816, "value3":false, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238673, "value1":0.398077529845209, "value2":8495752825324479706, "value3":true, "__name__":"metric_00000640", "key_7":"d","key_8":"j","key_3":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258260, "value1":0.3301633034128501, "value2":3240643823549476632, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.373493, "value1":0.3529131970207001, "value2":3035971070657682432, "value3":true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665486, "value1":0.1772430533425256, "value2":5999482327853244380, "value3":false, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358796, "value1":0.18816869038874143, "value2":5134838279453664617, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140177, "value1":0.8387362327402264, "value2":2310199526324609410, "value3":false, "__name__":"metric_00000645", "key_9":"j","key_0":"a","key_1":"c", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059616, "value1":0.8521328351977854, "value2":8719119695607623072, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121795, "value1":0.3605246283964201, "value2":1540950724302287517, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760661, "value1":0.21327358886251288, "value2":5483717677008529333, "value3":true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.019253, "value1":0.7234842919041711, "value2":3995891303198614565, "value3":true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468371, "value1":0.026572272284047523, "value2":2431068728438057213, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397658, "value1":0.22598294609730654, "value2":2698499054966120972, "value3":true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755700, "value1":0.09282245685151808, "value2":1822149750384470026, "value3":true, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304716, "value1":0.6358956357038871, "value2":1038515895167880998, "value3":false, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673410, "value1":0.01619219159654624, "value2":999540262235584439, "value3":false, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406798, "value1":0.2801610335950404, "value2":5737646822014765045, "value3":true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472994, "value1":0.14109660013219333, "value2":5384594772062199779, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601871, "value1":0.291815873607527, "value2":1847519318868439297, "value3":false, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994725, "value1":0.6730562186913949, "value2":5489090928652625049, "value3":false, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205639, "value1":0.42334867153762545, "value2":1823044221508369143, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.904945, "value1":0.598173556849254, "value2":1029832085092338611, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507464, "value1":0.8462370553826191, "value2":8938945324991684299, "value3":true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105783, "value1":0.4362851240411875, "value2":8230814679005715011, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.130819, "value1":0.5311370726009594, "value2":2799369947502242194, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.436621, "value1":0.5406911040823768, "value2":5927551572729154830, "value3":false, "__name__":"metric_00000655", "key_6":"e","key_0":"b", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081043, "value1":0.9897107711937903, "value2":1409810039307886257, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.360124, "value1":0.8898589942584547, "value2":1182666162202869439, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473406, "value1":0.9785032169444227, "value2":7933797459692204307, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711755, "value1":0.8440901040738314, "value2":1266347640739805258, "value3":true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836845, "value1":0.13202825619743236, "value2":6494324863003452768, "value3":true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.814171, "value1":0.22371325434459086, "value2":3679612694652027947, "value3":true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707264, "value1":0.43001516107021553, "value2":4022260893191171434, "value3":false, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585576, "value1":0.9028218878505339, "value2":3312996140868409539, "value3":false, "__name__":"metric_00000662", "key_8":"h","key_2":"j","key_3":"d", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.253947, "value1":0.5596795145791367, "value2":9053991858276197521, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.793355, "value1":0.655432892066219, "value2":2171965578246650022, "value3":false, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003349, "value1":0.32642930281521576, "value2":4724329284506985861, "value3":true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532693, "value1":0.47384645211929527, "value2":166053326506373649, "value3":true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258148, "value1":0.8755464478705562, "value2":1367687629182661630, "value3":true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284672, "value1":0.969983045672162, "value2":7709886208589610791, "value3":false, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656648, "value1":0.7178009692517222, "value2":378528392759859903, "value3":true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.138190, "value1":0.17961568288211016, "value2":6086554307721297156, "value3":false, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755588, "value1":0.7476161070292083, "value2":3339991721000583202, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.160231, "value1":0.4667023272863336, "value2":8520356596039544954, "value3":false, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888583, "value1":0.20049555103413333, "value2":2444783554472337568, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008015, "value1":0.08008213078983399, "value2":3073434633965527342, "value3":true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094964, "value1":0.7132542439294741, "value2":5312205610465131789, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.234374, "value1":0.3178507521158574, "value2":1341935169276159022, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772701, "value1":0.9799507521983674, "value2":4679043722393447880, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.742787, "value1":0.8436883610291797, "value2":8206108321797356735, "value3":false, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285496, "value1":0.02119049506813719, "value2":5643416354242611531, "value3":true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041734, "value1":0.8593746773495375, "value2":7177072845753511018, "value3":false, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102405, "value1":0.258959468834628, "value2":5506364318868451234, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.617097, "value1":0.5213134879577571, "value2":7820417886370593685, "value3":true, "__name__":"metric_00000683", "key_9":"c","key_1":"k","key_6":"a", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614285, "value1":0.8586887753126803, "value2":146426955523604107, "value3":false, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094475, "value1":0.3100760743003405, "value2":3216906708607385583, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400149, "value1":0.40305327199936813, "value2":4063666163388559666, "value3":true, "__name__":"metric_00000686", "key_4":"a","key_8":"g","key_1":"d", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.741707, "value1":0.07472258132971145, "value2":4583642984114208044, "value3":true, "__name__":"metric_00000688", "key_8":"c","key_2":"b","key_7":"b", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289818, "value1":0.21965034622340104, "value2":548484284878385021, "value3":false, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788770, "value1":0.8645491106416102, "value2":6428112142936498186, "value3":true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589911, "value1":0.8352505870700639, "value2":8807067943589232363, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980194, "value1":0.4222199616754522, "value2":1431823521353243628, "value3":true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347713, "value1":0.3792858568507692, "value2":5699682291213510899, "value3":false, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790574, "value1":0.48263080283098414, "value2":3899441712388650234, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.586360, "value1":0.39598813053011617, "value2":8951581838322949271, "value3":true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147821, "value1":0.696249186695191, "value2":5997811343683052027, "value3":true, "__name__":"metric_00000695", "key_9":"g","key_6":"a","key_8":"d", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375267, "value1":0.30594268061015806, "value2":7589869429394133120, "value3":false, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.662739, "value1":0.8061361815679017, "value2":5904903629197529499, "value3":true, "__name__":"metric_00000696", "key_4":"h","key_1":"d","key_2":"d", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447051, "value1":0.37750215516426633, "value2":8371518067848222420, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121391, "value1":0.6696130432092745, "value2":3528483712948148367, "value3":false, "__name__":"metric_00000701", "key_7":"a","key_2":"h","key_5":"i", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114395, "value1":0.9303974176092045, "value2":539479413974722614, "value3":true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.440024, "value1":0.33141470193064243, "value2":492711817812030768, "value3":false, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830103, "value1":0.7998316524208386, "value2":4274003622712511224, "value3":true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.343594, "value1":0.948502518427523, "value2":1092062054441297015, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.820370, "value1":0.6468806610248711, "value2":5700638394649707417, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169752, "value1":0.427387565910367, "value2":7968698722232869415, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.352398, "value1":0.008797452976395297, "value2":5389560588922529986, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.574184, "value1":0.6925729377012323, "value2":7921127567234590177, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708589, "value1":0.6874360024316648, "value2":7727336634389454587, "value3":false, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.647250, "value1":0.2053256936728798, "value2":8730436625808972720, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522110, "value1":0.7058378270168358, "value2":5838333049897839477, "value3":false, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894605, "value1":0.3141493596388785, "value2":2076790219296988613, "value3":true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008782, "value1":0.590566630449788, "value2":571655682516576680, "value3":true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.424761, "value1":0.6098530285996386, "value2":7846500926381318806, "value3":false, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591196, "value1":0.33824124551827933, "value2":646303391431941929, "value3":true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964204, "value1":0.7366285135213106, "value2":4137944794465811468, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643081, "value1":0.3218928504433246, "value2":9121706025453589663, "value3":false, "__name__":"metric_00000717", "key_9":"e","key_0":"b","key_4":"f", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804724, "value1":0.014940637092911, "value2":7451060690570027498, "value3":true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694428, "value1":0.21460306796333273, "value2":2982813118003123474, "value3":true, "__name__":"metric_00000718", "key_6":"g","key_9":"e","key_5":"b", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.122258, "value1":0.49140759630280006, "value2":8832276444519664682, "value3":true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.996796, "value1":0.2474526024511197, "value2":7613005431630356063, "value3":true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238269, "value1":0.8267663978098108, "value2":2724285849945025892, "value3":true, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.644529, "value1":0.4649514742650587, "value2":7589769671176102226, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.376035, "value1":0.7135202605627624, "value2":4131507979199286159, "value3":false, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958098, "value1":0.5742451996367589, "value2":7655662522626761261, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478186, "value1":0.30259713893200063, "value2":7366343481003162441, "value3":false, "__name__":"metric_00000725", "key_6":"g","key_9":"d","key_3":"a", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299728, "value1":0.5700777203840441, "value2":8077274544410191414, "value3":false, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.685643, "value1":0.9403232361639245, "value2":809458262071959965, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478225, "value1":0.7234775762369925, "value2":1256131637116314402, "value3":false, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.445423, "value1":0.17591193935592764, "value2":6211570042356723652, "value3":true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.362681, "value1":0.09007902579380467, "value2":5590217114587646147, "value3":false, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140908, "value1":0.3872422494802931, "value2":3824505998771737796, "value3":true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.312510, "value1":0.26492926145602147, "value2":2580605589364820036, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639094, "value1":0.8695708814560201, "value2":828997494663746747, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532778, "value1":0.8659642362585516, "value2":489767661854306328, "value3":false, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.332818, "value1":0.5085353492466338, "value2":7131627003266463948, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616148, "value1":0.6665482572291218, "value2":9183191141357087143, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807708, "value1":0.03703617074385954, "value2":1634911105603876624, "value3":true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.990782, "value1":0.8721197542810023, "value2":829062251335888311, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.678575, "value1":0.8518891534609417, "value2":5490128933307846252, "value3":false, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676580, "value1":0.032403043603468545, "value2":1860775688299500894, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.011776, "value1":0.3310374930985375, "value2":6454291476355278297, "value3":false, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.902226, "value1":0.03335506845841715, "value2":4281002708262490682, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.884426, "value1":0.26587448707952976, "value2":2888077545939138944, "value3":false, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721322, "value1":0.9352434968753679, "value2":7587503328880398082, "value3":false, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674491, "value1":0.13470131213667097, "value2":733064028083204302, "value3":false, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.422073, "value1":0.34800250717845166, "value2":5657610491232907815, "value3":false, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232687, "value1":0.8974445402000435, "value2":5670871752075567567, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.603175, "value1":0.8044440847017621, "value2":8279540241317103114, "value3":false, "__name__":"metric_00000757", "key_5":"b","key_6":"j","key_3":"e", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354995, "value1":0.9370809546532062, "value2":2794379758286031831, "value3":true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598521, "value1":0.03162604483850602, "value2":8898609192002715752, "value3":false, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489635, "value1":0.09343989179985779, "value2":6354680197733735029, "value3":true, "__name__":"metric_00000760", "key_5":"k","key_2":"d","key_4":"i", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.766631, "value1":0.6557065025532866, "value2":8867906113412686551, "value3":false, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.110067, "value1":0.37336348255028134, "value2":8359961637534300323, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.734771, "value1":0.6467506386274888, "value2":1558727129876488311, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751073, "value1":0.5011707406625697, "value2":165276254030404304, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903806, "value1":0.883542811993649, "value2":5713497784481830170, "value3":false, "__name__":"metric_00000765", "key_9":"b","key_3":"a","key_5":"h", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068830, "value1":0.41287620615552584, "value2":110550725395415731, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.481998, "value1":0.9176610757654294, "value2":3543653242038441064, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.206639, "value1":0.6881985031716263, "value2":3991865736711226608, "value3":true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682333, "value1":0.7354938188464508, "value2":2336942143484925177, "value3":true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.909363, "value1":0.6550859218093187, "value2":7848025848676081478, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.439129, "value1":0.4735897904277586, "value2":1730101214885033885, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788447, "value1":0.7029400400744759, "value2":7348647096857266019, "value3":false, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656519, "value1":0.8226813316731755, "value2":8565628989265102630, "value3":true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.286018, "value1":0.1715470955708772, "value2":281139728200192436, "value3":false, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.316461, "value1":0.970829440713552, "value2":8266115606666039974, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.131938, "value1":0.06029126490427053, "value2":6088969321338942970, "value3":false, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936137, "value1":0.16630484422294922, "value2":368627481682565524, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.632956, "value1":0.4597748954914025, "value2":614883467360042989, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441534, "value1":0.3169134259464135, "value2":4894244187289767341, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.536363, "value1":0.21988172452701402, "value2":3848642916965059480, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722069, "value1":0.0035036640993316324, "value2":1784910019753576066, "value3":true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189179, "value1":0.5999084267877618, "value2":589928855378135734, "value3":false, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070147, "value1":0.2816071736469606, "value2":1978576359562742327, "value3":true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.474328, "value1":0.12233813345274026, "value2":7757125810591739471, "value3":true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756802, "value1":0.8734910453077231, "value2":2858305999840238155, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830462, "value1":0.7809071944843656, "value2":6070440638222443699, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.158081, "value1":0.8007440801809812, "value2":3893225795949780633, "value3":false, "__name__":"metric_00000778", "key_7":"g","key_1":"b","key_3":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266044, "value1":0.5291100849459248, "value2":8570983485516004790, "value3":false, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375100, "value1":0.43060898088087474, "value2":5072556087103000232, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433713, "value1":0.4065332038272028, "value2":7498601821703546789, "value3":false, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293147, "value1":0.2008523459507916, "value2":3469755278652299519, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.464083, "value1":0.7193413705182999, "value2":8360590737420028572, "value3":true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182168, "value1":0.12869784027227613, "value2":2436732966596869973, "value3":true, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152849, "value1":0.915235247509805, "value2":6575468662860533653, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.106299, "value1":0.6574795299867987, "value2":5839146977664216835, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.842382, "value1":0.6684180756051546, "value2":6255233029731307668, "value3":true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.976893, "value1":0.9921834371308854, "value2":286112572755130788, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.747197, "value1":0.6687884895336262, "value2":5026368687057544137, "value3":true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227443, "value1":0.8444750407098351, "value2":3238045843783198862, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575432, "value1":0.04959245001196866, "value2":4763760968874648742, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484574, "value1":0.8794886478305205, "value2":2929187105321944273, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.613332, "value1":0.4649488124035846, "value2":8038984692264433406, "value3":false, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.156389, "value1":0.013056913554735015, "value2":1073629030912865258, "value3":false, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614657, "value1":0.21062410476220902, "value2":6415897327379032774, "value3":true, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238729, "value1":0.10820884210252545, "value2":9083864450943421396, "value3":false, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241678, "value1":0.22605677165183255, "value2":191747579342742576, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958794, "value1":0.08847610578305673, "value2":7897538472323716766, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335228, "value1":0.7181578715519717, "value2":7450823110233453413, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408629, "value1":0.007049664704797943, "value2":1505104843936344580, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558681, "value1":0.17466226992103584, "value2":4472498115626926601, "value3":true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682989, "value1":0.9645272985758268, "value2":1982644188431990793, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.179200, "value1":0.6439691808346755, "value2":8186363896835482044, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.823991, "value1":0.6478071089364331, "value2":8599197823923500800, "value3":false, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109730, "value1":0.43770117688691335, "value2":8534783631376673343, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.659077, "value1":0.2388079100822681, "value2":5323293096195049116, "value3":false, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.120405, "value1":0.7373368673794832, "value2":8559881035375574344, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.355361, "value1":0.6166673498020887, "value2":5456145421934358189, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.824086, "value1":0.4534358999465994, "value2":6384310233648456918, "value3":true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871013, "value1":0.5576949556795737, "value2":1428733241052481930, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507168, "value1":0.14733425604177453, "value2":4932780474472793744, "value3":true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.399272, "value1":0.7455596536595573, "value2":7959578558660693956, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015542, "value1":0.3661445195957949, "value2":5873286557095287645, "value3":false, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.103786, "value1":0.04914833432556202, "value2":3473465672536909160, "value3":true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201038, "value1":0.09243610446589845, "value2":317387159009776683, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637510, "value1":0.05764711949860248, "value2":683074398669967457, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000515, "value1":0.27555419865710506, "value2":1427932259617448392, "value3":true, "__name__":"metric_00000816", "key_6":"d","key_4":"e", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455871, "value1":0.4421289415542984, "value2":2179707427691179241, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.816179, "value1":0.0315328103269199, "value2":8955771787657622788, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.012467, "value1":0.2583652156574764, "value2":3152397034013397022, "value3":false, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.323088, "value1":0.6137213421267609, "value2":6194154265710187463, "value3":false, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361691, "value1":0.2728847707358428, "value2":1410878768807423021, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507208, "value1":0.40681095070477, "value2":2031127740762892841, "value3":true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.943788, "value1":0.4688930142953297, "value2":4135242725922960343, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528649, "value1":0.6786255340569864, "value2":5287486612492903502, "value3":false, "__name__":"metric_00000824", "key_3":"f","key_5":"a","key_0":"f", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225718, "value1":0.4601298705932887, "value2":9052049426787719860, "value3":true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.308880, "value1":0.4295083103049244, "value2":5651329885903965921, "value3":true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759618, "value1":0.16482488772851353, "value2":7857083585674436753, "value3":true, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416052, "value1":0.9111867810311565, "value2":6723934981662609251, "value3":true, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.962464, "value1":0.4935260151955853, "value2":3632302155175562257, "value3":true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.771117, "value1":0.5157133736913164, "value2":2578240794064018102, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690480, "value1":0.6806227775589547, "value2":478506980366332127, "value3":false, "__name__":"metric_00000832", "key_3":"h","key_0":"e", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.819739, "value1":0.8780417506767497, "value2":5796860147973283852, "value3":true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.393173, "value1":0.42102685254727695, "value2":655129410007603946, "value3":true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.648983, "value1":0.7792161980083644, "value2":3901696553081135725, "value3":true, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746096, "value1":0.4415965960336573, "value2":5128962497088709971, "value3":false, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550611, "value1":0.9164668411003939, "value2":8252426555782396131, "value3":false, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.026169, "value1":0.7150672668373745, "value2":4764749266652855333, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.684226, "value1":0.7136962442214909, "value2":5607090679190953591, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382560, "value1":0.4327657862928645, "value2":2199828856833521565, "value3":true, "__name__":"metric_00000840", "key_7":"c","key_2":"b","key_3":"i", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.467808, "value1":0.22363679307659168, "value2":2768232122997302049, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575206, "value1":0.7602213610975361, "value2":2779299954374668017, "value3":true, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591751, "value1":0.6270567668817301, "value2":8638942281069975034, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374899, "value1":0.3628294711980554, "value2":2256958770162190672, "value3":true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490274, "value1":0.7562635891941201, "value2":828105904736409259, "value3":true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.170118, "value1":0.5429488039839387, "value2":2772778506127755922, "value3":true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240769, "value1":0.2820088361870865, "value2":5639438631779439142, "value3":false, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.052386, "value1":0.43149632078191064, "value2":1748438881846630926, "value3":true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507233, "value1":0.961520455982136, "value2":538372936759435184, "value3":true, "__name__":"metric_00000849", "key_4":"e","key_3":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705937, "value1":0.5267821110187133, "value2":2615203611933488638, "value3":true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750925, "value1":0.12125960803758137, "value2":9007375284837027095, "value3":false, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631618, "value1":0.07157188871273036, "value2":8155465841105503917, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695843, "value1":0.7797612580730544, "value2":174508687346238960, "value3":true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269295, "value1":0.7420818036952436, "value2":4380281606468315484, "value3":true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566418, "value1":0.31419999154507045, "value2":1448626920753535043, "value3":true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.125439, "value1":0.23644172282231143, "value2":6454497651916669000, "value3":false, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260012, "value1":0.7675706994384546, "value2":5131372429329822614, "value3":false, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674484, "value1":0.024443192261290648, "value2":1060802392645575205, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870912, "value1":0.40232027987742836, "value2":7398514903536659789, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.193840, "value1":0.5573502520764032, "value2":2891807756570316806, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496711, "value1":0.0561679292808207, "value2":7275940738251409744, "value3":true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007546, "value1":0.4548616529539407, "value2":3391843142730199327, "value3":false, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532516, "value1":0.5229547783776293, "value2":306719517858726688, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.879419, "value1":0.738955824150393, "value2":4551796041817468625, "value3":false, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.037090, "value1":0.16862399179431758, "value2":4308060124699455866, "value3":false, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.748879, "value1":0.537219693168335, "value2":464744057316162498, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.610036, "value1":0.35423811263377813, "value2":8058264218672212470, "value3":false, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.212290, "value1":0.8726353918052155, "value2":7894596078524999317, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.876606, "value1":0.6546517022010405, "value2":6758959777685985912, "value3":true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140504, "value1":0.347953522010423, "value2":6223000072814932347, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.829533, "value1":0.7297577355885363, "value2":7098834979468843426, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770147, "value1":0.5158958137849098, "value2":7721786354129545110, "value3":true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960566, "value1":0.08984803039947419, "value2":7183642332674708296, "value3":false, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.002511, "value1":0.7991212300299283, "value2":1907144646133978540, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.969474, "value1":0.8490958586264525, "value2":2475634499457331111, "value3":true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757044, "value1":0.08525717165380897, "value2":6126821447855017519, "value3":false, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826913, "value1":0.3832744407117778, "value2":7549472912902563562, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713539, "value1":0.040929240199395034, "value2":8725477692408636856, "value3":true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.235038, "value1":0.21655074702892513, "value2":6340750264489982261, "value3":false, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713507, "value1":0.09698561541865447, "value2":6976532341223849395, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926365, "value1":0.6869464705551463, "value2":1770406309916093131, "value3":false, "__name__":"metric_00000890", "key_8":"h","key_0":"a", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.159778, "value1":0.37025745483297273, "value2":1667176166944781722, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964175, "value1":0.3460815492793407, "value2":8584884445165219279, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.789212, "value1":0.5752530695794612, "value2":7518543931942002634, "value3":true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946193, "value1":0.9389444654094391, "value2":4798866777286425255, "value3":true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183973, "value1":0.42689788996190486, "value2":2707564478214913458, "value3":false, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381159, "value1":0.19968138225970908, "value2":8498929039632010906, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.535645, "value1":0.9768502318817727, "value2":6273017261118239502, "value3":true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957340, "value1":0.1584492922790321, "value2":1071530234423723880, "value3":false, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622320, "value1":0.6014449518439046, "value2":5905866866815287771, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.839825, "value1":0.6412078194818629, "value2":8581664149275929445, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.925054, "value1":0.6499072539736884, "value2":3213784800648371267, "value3":true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.654382, "value1":0.9107398509050536, "value2":6290871463312648713, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.795697, "value1":0.8789558932554133, "value2":9118966352973366626, "value3":false, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299533, "value1":0.18449853142128472, "value2":6924501956061845944, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993498, "value1":0.10358257618685637, "value2":949727120096577179, "value3":true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192327, "value1":0.14944601875044164, "value2":3650903859653754777, "value3":false, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115925, "value1":0.7654222794387283, "value2":1807303188151583637, "value3":false, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290372, "value1":0.022817260035797225, "value2":3306384333491756709, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.402670, "value1":0.5073298240798392, "value2":4756233885952638662, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682572, "value1":0.44581853587828163, "value2":8977717914108281095, "value3":true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.176704, "value1":0.0686762673056191, "value2":418784156784186861, "value3":true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676421, "value1":0.9540024144812602, "value2":5040838132393700704, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851525, "value1":0.4501599756607321, "value2":7291682933774791859, "value3":false, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.593343, "value1":0.506206698395899, "value2":622013338659516904, "value3":true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454600, "value1":0.4498330838292854, "value2":2669650474822317581, "value3":false, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527805, "value1":0.10877922841749062, "value2":2131131607855096268, "value3":true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.196472, "value1":0.04264644660597854, "value2":2656288085872651609, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743799, "value1":0.2248320170707705, "value2":6330060840606217610, "value3":false, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137446, "value1":0.5799142292246671, "value2":4848799016740596142, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.405919, "value1":0.3378516880177889, "value2":3532844197361780130, "value3":false, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.085405, "value1":0.11768848717895854, "value2":3011043909687794254, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494421, "value1":0.45496539093555294, "value2":1968423041425508639, "value3":false, "__name__":"metric_00000901", "key_5":"f","key_0":"b","key_4":"h", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.846925, "value1":0.23911281903346318, "value2":1229857216789047456, "value3":false, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.320172, "value1":0.7911586278138405, "value2":145982991700455098, "value3":false, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688187, "value1":0.9272721413003945, "value2":5752958588041193324, "value3":false, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.069720, "value1":0.006222818414235851, "value2":7251719396221186260, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.974434, "value1":0.7327419748667513, "value2":8892941322690231954, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.891519, "value1":0.2559114110516683, "value2":8453661615833220464, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596204, "value1":0.8329616164402396, "value2":6548226786173702319, "value3":true, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539908, "value1":0.7401840588881007, "value2":2823865774842747097, "value3":true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865375, "value1":0.23950780446408748, "value2":6387790804623201730, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121268, "value1":0.4818898421020435, "value2":5213834205313537165, "value3":true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865802, "value1":0.3275415536098801, "value2":2802490407634150279, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105641, "value1":0.6210077776605651, "value2":7224328772436945002, "value3":true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592291, "value1":0.821033330430236, "value2":3130486219263774432, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497809, "value1":0.5925199380020667, "value2":341686057277963037, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097734, "value1":0.7071847698151175, "value2":3681912566029517435, "value3":true, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.763574, "value1":0.5357729458141934, "value2":4576393986863014129, "value3":false, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626276, "value1":0.4467400550839313, "value2":8832239215373133572, "value3":false, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626952, "value1":0.8566744061055088, "value2":2681684884383601627, "value3":false, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980666, "value1":0.620719669726131, "value2":3866315867973394400, "value3":false, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.970686, "value1":0.492660470887635, "value2":8584579737939983838, "value3":true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205981, "value1":0.41980273151462594, "value2":1272721076115788334, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643071, "value1":0.9850402079127909, "value2":6866475251892359276, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.520241, "value1":0.021531538514302357, "value2":3680419636166133733, "value3":false, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689781, "value1":0.7246711061095096, "value2":9156928465977729149, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936886, "value1":0.30593890582238353, "value2":8650088989944818585, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064882, "value1":0.9419400037260646, "value2":5139525288533697167, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255204, "value1":0.6562362511494719, "value2":5377088753944330401, "value3":true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147096, "value1":0.12576537131594923, "value2":696189011986700613, "value3":false, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.199366, "value1":0.8703136109731814, "value2":8758678284656449473, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782148, "value1":0.21588917591543605, "value2":7157001910263594492, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053985, "value1":0.9864976225072658, "value2":3999466034196061068, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.409718, "value1":0.4052988292496616, "value2":7239010836693528474, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918878, "value1":0.9460279119002769, "value2":8629146616157881411, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744254, "value1":0.12263478198823631, "value2":8698623269262602240, "value3":false, "__name__":"metric_00000937", "key_9":"i","key_4":"i","key_8":"h", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701868, "value1":0.3178667417931335, "value2":8069435536591676411, "value3":true, "__name__":"metric_00000938", "key_6":"k","key_7":"h","key_5":"b", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.794445, "value1":0.7546856348043545, "value2":1056257035145843208, "value3":false, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.318803, "value1":0.8976234734615749, "value2":7848445975050376420, "value3":false, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.146350, "value1":0.31748244669278847, "value2":2926268572806049868, "value3":false, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392349, "value1":0.655725592207486, "value2":4190046781739044777, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.749690, "value1":0.6239130127907853, "value2":5206683397328774242, "value3":true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744300, "value1":0.6764999956977692, "value2":1477381647586507515, "value3":true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.758303, "value1":0.5258481908865111, "value2":5235026862755190183, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379762, "value1":0.890935177217856, "value2":6322796509408703364, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592912, "value1":0.6886535422027805, "value2":3968885283796366911, "value3":true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859203, "value1":0.3806206868492738, "value2":6588306706789304138, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.565175, "value1":0.823280011831958, "value2":1840222942885696142, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.095858, "value1":0.18658533157232132, "value2":3040729957448861027, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.567863, "value1":0.1544724870114228, "value2":5739061219420103119, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739840, "value1":0.5915071280708923, "value2":6986464734573649185, "value3":false, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.200755, "value1":0.4500618730121175, "value2":4400580498849630870, "value3":false, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358028, "value1":0.8986628799372285, "value2":3320447208954210899, "value3":false, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017181, "value1":0.4972731292168526, "value2":5687205385943047330, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229539, "value1":0.5257635254456374, "value2":1562089048942850119, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.171044, "value1":0.8736127789799626, "value2":7151271249368522054, "value3":true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.384626, "value1":0.2370246172687121, "value2":2826036906774872004, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522883, "value1":0.6013577976454938, "value2":890649018657102134, "value3":true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.877768, "value1":0.6077458920362099, "value2":8966895480129060070, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.224033, "value1":0.0076863111785604905, "value2":6569047840639595510, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.311682, "value1":0.07593318166504835, "value2":3253540135578122766, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490459, "value1":0.4609451528285016, "value2":2994036544656680091, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.378898, "value1":0.9545401129487334, "value2":3668542248171100045, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416785, "value1":0.35541532907488144, "value2":94039845626229270, "value3":false, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275554, "value1":0.421200916650073, "value2":5818521501894570082, "value3":false, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619269, "value1":0.30772799684273167, "value2":4876402407267868155, "value3":false, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.702725, "value1":0.15927682148224467, "value2":1903456341847904299, "value3":false, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232604, "value1":0.7711772747260455, "value2":3866638254479737513, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.217772, "value1":0.8540182257688077, "value2":7309608218029816756, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705505, "value1":0.48163362162995105, "value2":1550283929814058088, "value3":false, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900193, "value1":0.23760307179291948, "value2":8646186763820142769, "value3":true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.208626, "value1":0.38931880434870053, "value2":5483530341831301025, "value3":false, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.568347, "value1":0.2801336860166125, "value2":9164397534745927432, "value3":false, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117458, "value1":0.981405746517363, "value2":6802896340427379587, "value3":false, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963035, "value1":0.7214166769862517, "value2":8440456704015434339, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700137, "value1":0.5934611351717767, "value2":6333818613628602547, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145303, "value1":0.6922047329264945, "value2":3009832451157760657, "value3":true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345925, "value1":0.7232346453151443, "value2":4791014890475627712, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619327, "value1":0.9932237622196707, "value2":3624811523135333449, "value3":false, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269730, "value1":0.7866373764087783, "value2":3746435711914598124, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.265782, "value1":0.05991121042342668, "value2":1951849832097700576, "value3":false, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.175543, "value1":0.5444637318639066, "value2":5220283889625062347, "value3":false, "__name__":"metric_00000991", "key_8":"h","key_5":"i", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677825, "value1":0.904620506580413, "value2":4669605336529676512, "value3":false, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668600, "value1":0.17366696171797397, "value2":8249554071220606769, "value3":true, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041222, "value1":0.3254693615668756, "value2":5262353711290630554, "value3":true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.243762, "value1":0.8759302538831941, "value2":7311945236881805486, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958625, "value1":0.5492731854343449, "value2":5435276136514133338, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290213, "value1":0.7949670770789082, "value2":7111766404096808279, "value3":false, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025988, "value1":0.1026128579407924, "value2":1755616401770874869, "value3":true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595099, "value1":0.18775955592545962, "value2":1656134813479826833, "value3":true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.671098, "value1":0.8079739199189518, "value2":3665225572712988915, "value3":false, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.542084, "value1":0.5181312401815146, "value2":1481804189008155276, "value3":true, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.784956, "value1":0.40357178109169595, "value2":9035114082415618141, "value3":true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.276517, "value1":0.24301724893563487, "value2":4602891919019880467, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064974, "value1":0.37706581069603073, "value2":7053383221578853097, "value3":false, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698510, "value1":0.45184858839742487, "value2":2258520036203869092, "value3":true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.218133, "value1":0.3113727295782559, "value2":6841422843138500205, "value3":false, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668881, "value1":0.25909755219748914, "value2":1185172632185740020, "value3":false, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.049479, "value1":0.9359900704241361, "value2":8443878057371326705, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.674394, "value1":0.5330986303950987, "value2":4597086230952392696, "value3":true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195073, "value1":0.23095513043718424, "value2":886484372856556987, "value3":false, "__name__":"metric_00000010", "key_6":"a","key_1":"e","key_5":"c", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.380352, "value1":0.9222632147295795, "value2":2011984636052581606, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.302156, "value1":0.7967664965647748, "value2":1936857391665696956, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673174, "value1":0.8085420465867998, "value2":6974688906284239573, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.013224, "value1":0.01982064999844403, "value2":3955016096628456095, "value3":true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090855, "value1":0.01304461484505885, "value2":8778318340262765164, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.795784, "value1":0.12217080131703652, "value2":2014947645475742270, "value3":false, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284442, "value1":0.42738762978686595, "value2":2584606560267930457, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.741760, "value1":0.5561052082820417, "value2":7263332550390986826, "value3":false, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035524, "value1":0.908173032025188, "value2":5685807943763240264, "value3":false, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025764, "value1":0.20925710637592332, "value2":6562668387404626293, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.530188, "value1":0.6447955243050295, "value2":3888614173619999623, "value3":false, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424157, "value1":0.17487330381635957, "value2":5301061833157028268, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026934, "value1":0.0422857387218024, "value2":8303721108125129087, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841847, "value1":0.5206472086096076, "value2":74348635869177909, "value3":true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.414469, "value1":0.4278094238360654, "value2":7010327929309184782, "value3":false, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491788, "value1":0.747156270778759, "value2":4790193170750716406, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399141, "value1":0.47388249427808166, "value2":8836306836420904183, "value3":false, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.160250, "value1":0.29496035919361546, "value2":1937679940636334558, "value3":true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787511, "value1":0.40198955034703965, "value2":5313833463463440565, "value3":true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684988, "value1":0.6177512623201419, "value2":1449869359073755283, "value3":false, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803253, "value1":0.8970896940692594, "value2":6178067064755804966, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964550, "value1":0.8199555931693112, "value2":140777809986023618, "value3":true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456407, "value1":0.2401625723920763, "value2":4030355648738753679, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849585, "value1":0.2629587147468046, "value2":7076050191155576404, "value3":false, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.824331, "value1":0.6892589926802947, "value2":6399755277819070915, "value3":false, "__name__":"metric_00000046", "key_4":"e","key_6":"h","key_2":"k", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388915, "value1":0.8127879446503501, "value2":5588278675440700300, "value3":false, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782971, "value1":0.22771808116681516, "value2":6261899644042013732, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712419, "value1":0.2905286238822736, "value2":5614218841386062674, "value3":false, "__name__":"metric_00000049", "key_3":"h","key_9":"j","key_1":"d", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249696, "value1":0.9445558673553629, "value2":1404486883141712140, "value3":true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601814, "value1":0.6990549633969345, "value2":7198014296771075924, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543813, "value1":0.07857387425059083, "value2":1217712980435541695, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123893, "value1":0.07983470162468347, "value2":7171526612504871775, "value3":false, "__name__":"metric_00000053", "key_4":"a","key_7":"k","key_1":"f", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703648, "value1":0.36447696895349185, "value2":5146632909527815894, "value3":true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166353, "value1":0.7098874569397392, "value2":5324193865568739754, "value3":true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399594, "value1":0.7745072356158436, "value2":7831862313374173936, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.439724, "value1":0.2037239485962775, "value2":3755069786706852308, "value3":true, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839998, "value1":0.6732355725193216, "value2":2639271267504971364, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228435, "value1":0.897840654678523, "value2":5116662644663764475, "value3":false, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973287, "value1":0.9461886626837847, "value2":5279391211817478931, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.441360, "value1":0.9419535299333034, "value2":2098864241711696584, "value3":false, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.780978, "value1":0.7212145622817746, "value2":8825784157879516761, "value3":true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.342306, "value1":0.40763808526506856, "value2":615440271373894957, "value3":true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964992, "value1":0.8428995412660205, "value2":5458378648341607708, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386901, "value1":0.3242872455426699, "value2":1659876068547712992, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.960102, "value1":0.2464238639205716, "value2":699750419933475491, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585099, "value1":0.11986390379255613, "value2":3678288424535052375, "value3":false, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.154037, "value1":0.7785920099800233, "value2":8275166075671164173, "value3":false, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211892, "value1":0.7853941226491261, "value2":6369532531235861115, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.746677, "value1":0.6664817644749024, "value2":4658328841108884689, "value3":true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.941862, "value1":0.846994201539624, "value2":1274645788690532949, "value3":true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.733827, "value1":0.13169457879746155, "value2":3221287182313964764, "value3":true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870127, "value1":0.37466507316122794, "value2":8856106508914542017, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798298, "value1":0.22462880312469063, "value2":6798911070080579970, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993366, "value1":0.9961738964349245, "value2":1193549068611970903, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.157412, "value1":0.4452570786392375, "value2":3487651503828456956, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.529021, "value1":0.26905395113611574, "value2":2025155869621977272, "value3":true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.701922, "value1":0.8730175680632801, "value2":997347931310956157, "value3":true, "__name__":"metric_00000078", "key_1":"a","key_9":"h","key_0":"b", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234577, "value1":0.7588051800901922, "value2":5319874977525060911, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943591, "value1":0.47603601658101, "value2":5946410545801626105, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.292747, "value1":0.03173464098642328, "value2":7226691638812613138, "value3":false, "__name__":"metric_00000057", "key_3":"b","key_5":"j","key_2":"a", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608463, "value1":0.2957253043607161, "value2":122362680185330457, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.643245, "value1":0.7167827619023723, "value2":5186237060875400709, "value3":false, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456051, "value1":0.2188755629521717, "value2":1845588654842798990, "value3":false, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231176, "value1":0.6418253580281563, "value2":3690123298454825, "value3":false, "__name__":"metric_00000061", "key_4":"d","key_5":"d","key_3":"f", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158253, "value1":0.7945876103242618, "value2":9004322525035166688, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234079, "value1":0.5202371436378874, "value2":1591535835129382659, "value3":false, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.875339, "value1":0.37493709867362873, "value2":8328159676060260451, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907034, "value1":0.20583691616211497, "value2":3631192064418800289, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.518876, "value1":0.16335464977880865, "value2":4643059165781979124, "value3":false, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185787, "value1":0.018176867671764584, "value2":3270163687436586406, "value3":true, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308787, "value1":0.4773555846410255, "value2":3253306657765008963, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297154, "value1":0.246942999840671, "value2":6872861714776356785, "value3":true, "__name__":"metric_00000084", "key_3":"k","key_9":"j","key_1":"c", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.817582, "value1":0.8165235911801363, "value2":8310469403306376604, "value3":true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493041, "value1":0.38500411997916434, "value2":6747567659837060435, "value3":false, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224106, "value1":0.9994238326374754, "value2":7099587552820760691, "value3":false, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031476, "value1":0.09066884358892856, "value2":5945790442760569800, "value3":true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311356, "value1":0.5806991883130863, "value2":6895176067401972683, "value3":true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653725, "value1":0.15503072276739358, "value2":7449412216901454949, "value3":false, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.598231, "value1":0.20373471056711764, "value2":1437658094331870355, "value3":false, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798108, "value1":0.6097451467316525, "value2":7133130757829716058, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709054, "value1":0.7371923755130838, "value2":7543405219351917468, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104790, "value1":0.21083306139991156, "value2":7929604552171608387, "value3":false, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.187877, "value1":0.5608708053678212, "value2":3368623446076987813, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749130, "value1":0.657437311445075, "value2":8949805346712651308, "value3":false, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582481, "value1":0.42362389624895797, "value2":8717288037606444575, "value3":true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.558438, "value1":0.370755803762819, "value2":1107999166234163628, "value3":false, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802675, "value1":0.8677299277452752, "value2":5915000168574046056, "value3":true, "__name__":"metric_00000108", "key_3":"j","key_0":"j","key_1":"c", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.208039, "value1":0.38781051864576976, "value2":3375198001539598542, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680278, "value1":0.886048145499295, "value2":4081527324669320081, "value3":false, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162287, "value1":0.7118801419064047, "value2":2390093012206667849, "value3":true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.092015, "value1":0.9942758150465311, "value2":4638994738698984564, "value3":true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039345, "value1":0.641165676244722, "value2":2976540841974698815, "value3":true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.408130, "value1":0.5042712547968512, "value2":8947846924433036861, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259088, "value1":0.514500988249709, "value2":2770449998520564629, "value3":false, "__name__":"metric_00000092", "key_4":"b","key_0":"j","key_3":"i", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293773, "value1":0.36521689212647834, "value2":67527608081891502, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646136, "value1":0.04048325982754749, "value2":4743191660759249600, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.047497, "value1":0.5502497475964416, "value2":7169106217640232030, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876800, "value1":0.3799501707452096, "value2":6757458314368411993, "value3":false, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968057, "value1":0.7499783045753542, "value2":1200059614173097893, "value3":true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.871681, "value1":0.6987940576652765, "value2":3787158661733344974, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.998419, "value1":0.7974184353878208, "value2":3011281681090708365, "value3":true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663978, "value1":0.1875336848417326, "value2":7682956450262415743, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799777, "value1":0.13431952299825461, "value2":8472008336913527353, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224565, "value1":0.7432835827699946, "value2":5363247813194187485, "value3":true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896272, "value1":0.1238866666650729, "value2":1249324477450532329, "value3":true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630720, "value1":0.15283582580094462, "value2":1819923308050662735, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405798, "value1":0.9413697480070442, "value2":6170928651103063980, "value3":true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249174, "value1":0.3594131764729996, "value2":6196624819261733193, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628627, "value1":0.5009434579166352, "value2":4913727766417338788, "value3":true, "__name__":"metric_00000130", "key_6":"d","key_9":"g","key_2":"e", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949656, "value1":0.677579274449204, "value2":4321897610912596942, "value3":false, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.452302, "value1":0.8465391778403211, "value2":8767695810058898547, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224741, "value1":0.9419255467202373, "value2":3942608565088456699, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832311, "value1":0.1844332411242929, "value2":7897621420940762914, "value3":false, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364450, "value1":0.6062623207548122, "value2":3048338263269528616, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458762, "value1":0.8964243000562225, "value2":6980106557138925502, "value3":true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090342, "value1":0.9353992150482529, "value2":3024974544306240484, "value3":true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680282, "value1":0.32391766175077186, "value2":855225884463637074, "value3":false, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949004, "value1":0.8992840018801868, "value2":5343754317476027418, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.564072, "value1":0.04591172153502947, "value2":15119713343690272, "value3":false, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.015169, "value1":0.2775250663576938, "value2":8329492228577328961, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035102, "value1":0.8955268703368635, "value2":1413978015063203049, "value3":true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.922310, "value1":0.5335365949387284, "value2":7752415045685811238, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.862402, "value1":0.328052481246616, "value2":1049471938369225232, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.632372, "value1":0.702701110691373, "value2":736041481181376478, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886888, "value1":0.8185435772395079, "value2":6423113939133946645, "value3":true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087072, "value1":0.4700081672384728, "value2":4642202730229975792, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.936650, "value1":0.18762496890040795, "value2":3355401686890613388, "value3":false, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032044, "value1":0.12307064568471555, "value2":5991756874915773532, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093295, "value1":0.22401921911686248, "value2":6120641321792430058, "value3":false, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.777098, "value1":0.29744828015347885, "value2":3084065102603938152, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.437450, "value1":0.2884592585353877, "value2":4044254904136084327, "value3":false, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.017135, "value1":0.7608822818277434, "value2":1205572766308158879, "value3":false, "__name__":"metric_00000143", "key_7":"a","key_4":"g","key_6":"d", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907629, "value1":0.7081298253215954, "value2":7688268403734929948, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924934, "value1":0.7192898599410195, "value2":1421354730652485196, "value3":false, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.736050, "value1":0.7502596301905261, "value2":2582210329738662673, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515235, "value1":0.09310016531489594, "value2":5123514648664672005, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.117213, "value1":0.07556633746315061, "value2":3501827326756011617, "value3":false, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.485929, "value1":0.9272991096008794, "value2":5159915940333201308, "value3":false, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296263, "value1":0.11108812397005534, "value2":8753717861080819638, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990089, "value1":0.527982415726353, "value2":5638704340740792965, "value3":false, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835455, "value1":0.062168150269998286, "value2":1944114908616022523, "value3":true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297201, "value1":0.30562887567201646, "value2":2937626694886920620, "value3":false, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974272, "value1":0.7221303752638616, "value2":4841762343070531864, "value3":true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.477606, "value1":0.011147194457821816, "value2":7938710785429203576, "value3":true, "__name__":"metric_00000152", "key_9":"h","key_0":"d","key_2":"k", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129918, "value1":0.5408385148641213, "value2":9195358942783956586, "value3":true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364995, "value1":0.0773648729840119, "value2":2987992944358913043, "value3":false, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.776833, "value1":0.8273262149027085, "value2":3110126394788152042, "value3":true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.610517, "value1":0.6781728258681893, "value2":4793737503812454856, "value3":true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896976, "value1":0.37188254064396126, "value2":3144974162911084078, "value3":true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202562, "value1":0.5271477787382647, "value2":6411589275983706377, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906770, "value1":0.4476129119528191, "value2":424159247916997254, "value3":true, "__name__":"metric_00000159", "key_9":"j","key_0":"e","key_3":"c", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289456, "value1":0.541155755467242, "value2":2999265325007368522, "value3":false, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.860878, "value1":0.4758705080993404, "value2":6690006483968082589, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.365448, "value1":0.027420735129761006, "value2":825754437394215422, "value3":true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212551, "value1":0.16839070098231337, "value2":1942763578810483030, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870900, "value1":0.4994077584006896, "value2":5993483889801645858, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.455124, "value1":0.26138016691309135, "value2":4427691071851307843, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.569012, "value1":0.4778238812129043, "value2":8556858598621845387, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.362641, "value1":0.2203054974232932, "value2":3477406531290022646, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493850, "value1":0.4787384781915333, "value2":2198408185614770688, "value3":false, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.513028, "value1":0.6429388402116987, "value2":7593443154020639176, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386729, "value1":0.17475131034566962, "value2":3647348637356138676, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053464, "value1":0.6091636844532319, "value2":5763485176228727883, "value3":false, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628348, "value1":0.023668653461306262, "value2":6178270391876561387, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913114, "value1":0.637992847841559, "value2":8248068010165238913, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.413158, "value1":0.5132505144116425, "value2":3488568643829613122, "value3":true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517474, "value1":0.21449952384417484, "value2":4829637720497890669, "value3":false, "__name__":"metric_00000177", "key_8":"d","key_4":"i","key_7":"c", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.659125, "value1":0.35348852774479717, "value2":929980299829000703, "value3":false, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071642, "value1":0.5598789974197187, "value2":7463399784012985552, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993416, "value1":0.6475276350489354, "value2":3252439000616730752, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003658, "value1":0.897666564581801, "value2":508174253835823210, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.629566, "value1":0.49532352920736916, "value2":3384274756820723223, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.084906, "value1":0.40471552937588173, "value2":2358566109120320700, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.108543, "value1":0.8814759719911462, "value2":940297244819349653, "value3":false, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138889, "value1":0.48057446422385713, "value2":5651682080133821762, "value3":false, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.468024, "value1":0.6607643026338559, "value2":3567273041505939519, "value3":false, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.278010, "value1":0.05185404011612328, "value2":9174087231001182435, "value3":false, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.095328, "value1":0.4998002617147629, "value2":4028395944074542796, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.962889, "value1":0.7966578010574328, "value2":907724426763786137, "value3":false, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.434695, "value1":0.596944451662117, "value2":6599947217349296930, "value3":false, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.695515, "value1":0.23670726317490795, "value2":2923186045419354751, "value3":false, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830693, "value1":0.43683284863611044, "value2":9110976313628947704, "value3":true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311809, "value1":0.18924644468054935, "value2":2283072307874475430, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405030, "value1":0.5026514010742291, "value2":7499615277819756210, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.985393, "value1":0.2214787811403981, "value2":1167173892132804942, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123017, "value1":0.24050302556695696, "value2":3512829503479749438, "value3":true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.275825, "value1":0.8989518053152799, "value2":7332940306078599750, "value3":false, "__name__":"metric_00000207", "key_3":"g","key_0":"k","key_1":"e", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.935513, "value1":0.422470893258827, "value2":1965139573445622612, "value3":true, "__name__":"metric_00000208", "key_4":"e","key_2":"a", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697239, "value1":0.9715985007012725, "value2":2441260821113474270, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297829, "value1":0.4121340516781959, "value2":7327020552945137669, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.488215, "value1":0.22392461646214185, "value2":5911938621135820572, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.206243, "value1":0.020325317058324376, "value2":9207086370751028426, "value3":true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183572, "value1":0.23206347801714602, "value2":6792315880866236659, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524533, "value1":0.9124779133103079, "value2":3221463708308920869, "value3":false, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.893528, "value1":0.4848003839426891, "value2":3814492558819679029, "value3":true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.752371, "value1":0.8170507918067189, "value2":6527155317426998401, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916551, "value1":0.342137272729241, "value2":1602041662641372111, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.037592, "value1":0.6500639685761466, "value2":3567065955735150085, "value3":false, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902253, "value1":0.7531115895009749, "value2":2543364566980292046, "value3":false, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683579, "value1":0.30275430657154007, "value2":7640650810082868472, "value3":true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956614, "value1":0.5403322951696091, "value2":332198111972672347, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202908, "value1":0.3832878669702297, "value2":880756788127038920, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.820586, "value1":0.34286605495032285, "value2":6526500882733787563, "value3":true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930022, "value1":0.3539159969493719, "value2":370073818816792799, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023457, "value1":0.047546161290410185, "value2":5398553854250083599, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303770, "value1":0.7186586727923895, "value2":9169471131017267259, "value3":false, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.756889, "value1":0.31103630013784395, "value2":3594098461981248414, "value3":false, "__name__":"metric_00000218", "key_8":"a","key_2":"k","key_7":"j", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212371, "value1":0.3882294279634762, "value2":2428299028227660876, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.472113, "value1":0.19389271171622363, "value2":7737936344522555466, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051213, "value1":0.12920469147183916, "value2":7837057207029745863, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907093, "value1":0.017956122904157993, "value2":4878755396476119431, "value3":true, "__name__":"metric_00000232", "key_4":"e","key_7":"a","key_2":"g", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.525258, "value1":0.49782305829726053, "value2":2466466392623808778, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913492, "value1":0.4704351769152427, "value2":7908561685667216260, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290702, "value1":0.4198914182070395, "value2":6819583912532498504, "value3":false, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125217, "value1":0.04410603241691789, "value2":6832161683827996201, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004937, "value1":0.13145801093930243, "value2":5314457688060891725, "value3":true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.656896, "value1":0.41192120722520353, "value2":5457893604153821743, "value3":false, "__name__":"metric_00000238", "key_5":"k","key_7":"e","key_4":"a", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.700497, "value1":0.8047394500892703, "value2":8180560267702238361, "value3":true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553716, "value1":0.7484558334038893, "value2":1003966921986785454, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966183, "value1":0.8167158291471999, "value2":2250434045285686743, "value3":false, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663750, "value1":0.7127353578019723, "value2":3741760060639503306, "value3":true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686363, "value1":0.9712701187599743, "value2":3622116724705792870, "value3":true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.103723, "value1":0.4724493721726449, "value2":8181583911299132150, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.484649, "value1":0.18448759636607337, "value2":4053994543260017135, "value3":false, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.925440, "value1":0.06382834538328273, "value2":6916231632621539461, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939805, "value1":0.22424399111258897, "value2":4351472309231431136, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792665, "value1":0.33436392964606154, "value2":2696421529224538356, "value3":true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054274, "value1":0.9272819232934056, "value2":7264556561094806515, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.983823, "value1":0.7989127551970725, "value2":5239695063979297186, "value3":true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714766, "value1":0.6892792624795272, "value2":270252571976728726, "value3":false, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.263106, "value1":0.64122047400478, "value2":3421795383581628735, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720260, "value1":0.8087795736115097, "value2":8426217720617766298, "value3":true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839070, "value1":0.5917446619321536, "value2":390571547986932413, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966990, "value1":0.2432955344040808, "value2":8461111581798764026, "value3":false, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.808072, "value1":0.3216500857620197, "value2":4440941489040196963, "value3":false, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091958, "value1":0.3995806295500445, "value2":1180768478663237716, "value3":true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139366, "value1":0.11507703668400583, "value2":4516311843825616388, "value3":false, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.251512, "value1":0.7632852678548858, "value2":6418768969046377858, "value3":true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963533, "value1":0.8213013619231073, "value2":6101164365690068241, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.466433, "value1":0.28350424710105476, "value2":5563469458031181794, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831271, "value1":0.8821293582204699, "value2":3430349356503249989, "value3":true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003553, "value1":0.7323647142880576, "value2":5739925461894436779, "value3":true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.241144, "value1":0.3850622637508932, "value2":6137203197960360313, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.872100, "value1":0.3184658084196574, "value2":2061373587207508318, "value3":false, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.900532, "value1":0.2888524955130242, "value2":6600516250776964704, "value3":false, "__name__":"metric_00000256", "key_9":"c","key_3":"g","key_4":"b", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142058, "value1":0.7572218923237818, "value2":5437617110907280837, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.167933, "value1":0.20087813542991115, "value2":8497531403193171323, "value3":false, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977657, "value1":0.49027491644969856, "value2":6846007575513730281, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.337497, "value1":0.7535889949814326, "value2":4191915280375173080, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091305, "value1":0.41827531366800946, "value2":3553824659208645760, "value3":false, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.260475, "value1":0.39071909544867817, "value2":3828966522631037754, "value3":false, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421117, "value1":0.6070501451115596, "value2":637677502764251310, "value3":false, "__name__":"metric_00000263", "key_6":"d","key_3":"a","key_5":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949074, "value1":0.8719785824911194, "value2":739855289497042435, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005847, "value1":0.564214486086032, "value2":3879428464931256295, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608985, "value1":0.8120552257475274, "value2":6222621751128276018, "value3":true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.505971, "value1":0.44491083173920337, "value2":2966233843840872053, "value3":false, "__name__":"metric_00000269", "key_6":"b","key_1":"d","key_2":"f", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686524, "value1":0.910573536000882, "value2":7259657495340557823, "value3":false, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386181, "value1":0.5690887314353372, "value2":7696797346941049468, "value3":true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004437, "value1":0.2837421940276855, "value2":3630048754825451133, "value3":true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197835, "value1":0.3478392366591849, "value2":8924518323717745322, "value3":true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404754, "value1":0.14818758453959469, "value2":3841840031846521039, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.847397, "value1":0.10611042816524652, "value2":3749767974486199531, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773493, "value1":0.16710420925739453, "value2":2156267597938791257, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253265, "value1":0.39770730374803015, "value2":8078016986748379936, "value3":true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720298, "value1":0.3023956927195079, "value2":2183933305568131133, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.306860, "value1":0.13870609943567333, "value2":5783110454957795198, "value3":true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189190, "value1":0.3825441585131095, "value2":2493897460883724129, "value3":true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842956, "value1":0.712749117246484, "value2":1337306979661530320, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045437, "value1":0.9278250040101162, "value2":6116936767056337825, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.606083, "value1":0.4732072236705937, "value2":6967738612762397759, "value3":false, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094844, "value1":0.9269374913746184, "value2":2591132307871233919, "value3":false, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369784, "value1":0.09045009486516034, "value2":7834710052863465672, "value3":false, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032637, "value1":0.16606039490952637, "value2":2737666598235173614, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.394163, "value1":0.47507719343340277, "value2":313537922371752138, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.987576, "value1":0.5780445815848603, "value2":4108970207058212125, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906463, "value1":0.03465970899762972, "value2":6841259554497483309, "value3":false, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.266096, "value1":0.6742131298888875, "value2":7025803245311237704, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963062, "value1":0.5904971275955457, "value2":2254362569087327315, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.130304, "value1":0.868734941900672, "value2":1538504759997860509, "value3":true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.955104, "value1":0.5994289988135004, "value2":3881947199005484386, "value3":false, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.480748, "value1":0.49853130606809243, "value2":727938692981473193, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.682757, "value1":0.38003013064353247, "value2":8479678523613035623, "value3":false, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792753, "value1":0.16576560662385773, "value2":7797109252869584262, "value3":false, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.510440, "value1":0.6049726150607012, "value2":8669482832353600257, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.638136, "value1":0.15014964585075946, "value2":6225706505785608080, "value3":true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.522674, "value1":0.3512111717300567, "value2":3641814037368457348, "value3":false, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785473, "value1":0.4544503987969124, "value2":4635364210597620767, "value3":true, "__name__":"metric_00000308", "key_2":"f","key_4":"b","key_1":"f", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110745, "value1":0.5221211963164464, "value2":4903849187407455848, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195405, "value1":0.08594736781118724, "value2":2859597321511204836, "value3":true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051298, "value1":0.8256101721258909, "value2":290470632378621656, "value3":true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284220, "value1":0.27211501738111415, "value2":2134420797562982003, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792536, "value1":0.9372077558354763, "value2":7927244557057049645, "value3":false, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.681776, "value1":0.4477409740262762, "value2":3995977364357841936, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231810, "value1":0.6078487538143919, "value2":3261959160546744959, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.131613, "value1":0.7340176818759, "value2":3565658759712151137, "value3":true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.908393, "value1":0.28199967714832963, "value2":3350748733246898915, "value3":false, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698846, "value1":0.043919603046489966, "value2":4515802998753688491, "value3":false, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803882, "value1":0.9014217952934828, "value2":8345388974528376469, "value3":true, "__name__":"metric_00000291", "key_2":"a","key_3":"i","key_0":"d", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501652, "value1":0.45069419646062914, "value2":4400331011414194610, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.240902, "value1":0.34849466754791153, "value2":3738192721082184330, "value3":false, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.950263, "value1":0.5119937423488377, "value2":8919897235633479114, "value3":false, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.443897, "value1":0.08318284858972097, "value2":8878946431124018702, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653076, "value1":0.2711114129019499, "value2":4904832384185588956, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199562, "value1":0.5179430768249581, "value2":2338622690587545147, "value3":true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902254, "value1":0.8456128690164353, "value2":3512723913452353513, "value3":true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424568, "value1":0.358235963026908, "value2":1951463002559117594, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.891020, "value1":0.899730727943035, "value2":384925737643795292, "value3":false, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242327, "value1":0.8194521049409685, "value2":1651591731883214718, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064957, "value1":0.5094969888256025, "value2":1137790773894543912, "value3":false, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032406, "value1":0.3455331481317415, "value2":2958859262258749395, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553817, "value1":0.484360827667417, "value2":3796071195831724862, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197274, "value1":0.4874943149488065, "value2":1044578434864087492, "value3":true, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.384646, "value1":0.3107814255944672, "value2":1519235564474675279, "value3":true, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286271, "value1":0.42853641527210906, "value2":4722523490137371754, "value3":false, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175308, "value1":0.3946055220580569, "value2":6954032044064595773, "value3":true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.793904, "value1":0.06297728493572644, "value2":5259172994496601557, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.062223, "value1":0.013288825435356888, "value2":6140160587654305146, "value3":true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.768622, "value1":0.39922765836242, "value2":6629678017270316706, "value3":false, "__name__":"metric_00000339", "key_5":"b","key_9":"g","key_4":"k", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.203380, "value1":0.9727752066385791, "value2":960040697676306380, "value3":true, "__name__":"metric_00000340", "key_8":"i","key_0":"i","key_3":"d", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.845192, "value1":0.9218669222513459, "value2":8161692681657646744, "value3":true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625176, "value1":0.7818690860470178, "value2":6286486510404426651, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943535, "value1":0.004848635402133272, "value2":8498014405632225004, "value3":true, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.204456, "value1":0.5833134681081519, "value2":7410659829376107064, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.576114, "value1":0.9192372118815992, "value2":6657353407785586037, "value3":true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118284, "value1":0.8313517670516266, "value2":7764880758391828110, "value3":true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.535569, "value1":0.24827312684994104, "value2":2040106521001592824, "value3":true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.549323, "value1":0.8166897779780931, "value2":1564430022376434258, "value3":true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881818, "value1":0.09926733907609549, "value2":3471220368243877954, "value3":false, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374110, "value1":0.5149369120619428, "value2":2786214942305343351, "value3":true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770127, "value1":0.7580522569953325, "value2":4637323015723129783, "value3":true, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178499, "value1":0.146709310096475, "value2":7936042836162333987, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274162, "value1":0.4372447015799375, "value2":9083170531856859501, "value3":true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491205, "value1":0.022796016225098064, "value2":8123705023600354614, "value3":true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016451, "value1":0.9256503393490508, "value2":6970208660239633828, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087308, "value1":0.4462899441715343, "value2":1749521530182375024, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094183, "value1":0.3475409881711034, "value2":5390535090747181247, "value3":true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.735505, "value1":0.53372762504523, "value2":8372678000097135865, "value3":true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.114823, "value1":0.029693120979849812, "value2":6638472287910475525, "value3":true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242928, "value1":0.3050035672228757, "value2":1792917800424529821, "value3":true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703291, "value1":0.31694612370345837, "value2":2183523770285388049, "value3":true, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.331904, "value1":0.47208239473287594, "value2":6204971001548635721, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.345432, "value1":0.4951178136444847, "value2":3855507055566939240, "value3":true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136145, "value1":0.6933172444157865, "value2":1571591012651895136, "value3":false, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303587, "value1":0.05226903625691088, "value2":6994464805121874342, "value3":true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.604568, "value1":0.14657045529476723, "value2":2698838307937260948, "value3":true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376282, "value1":0.8133509413304958, "value2":8456674940436282992, "value3":false, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843511, "value1":0.6514108937273068, "value2":7171882174493021649, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888039, "value1":0.34434154548426427, "value2":5546478000801091154, "value3":false, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663092, "value1":0.2145932369532521, "value2":1924108819168508153, "value3":true, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178424, "value1":0.3811642005135661, "value2":5277232346465674780, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.982420, "value1":0.509903018966164, "value2":1366973490493465213, "value3":false, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.789914, "value1":0.5590817594064089, "value2":2545283978114147454, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.858245, "value1":0.2990011327668893, "value2":7402472987921684090, "value3":true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045358, "value1":0.4729812652892598, "value2":6922132388747645101, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212994, "value1":0.6311902604147308, "value2":4414936740257073237, "value3":true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357341, "value1":0.06207502931784455, "value2":4475293686563973956, "value3":false, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.975194, "value1":0.41868388111482774, "value2":1854059337783469990, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841014, "value1":0.21676180216590327, "value2":4591196137073503812, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.978140, "value1":0.3174552368999545, "value2":1901352595947591853, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.267125, "value1":0.18049579402909974, "value2":4625546630316797154, "value3":true, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916965, "value1":0.5612711107070505, "value2":6029229351915879062, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.927166, "value1":0.40285609194050886, "value2":5993132840270480501, "value3":true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791545, "value1":0.4639522865587751, "value2":7463548107279858536, "value3":true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770063, "value1":0.26886367149111556, "value2":3097459774572125541, "value3":true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077189, "value1":0.7572977836842385, "value2":8298753827870878088, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.287375, "value1":0.621000003066286, "value2":5005851683916784910, "value3":true, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517180, "value1":0.6660153705066499, "value2":8387087977559252312, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.675034, "value1":0.9813881401454424, "value2":3741676304938251291, "value3":true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.554896, "value1":0.7715073370679849, "value2":6589060977722716090, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319288, "value1":0.12279901479898898, "value2":3422770912327659239, "value3":true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703895, "value1":0.8358860148624778, "value2":8257794540400854550, "value3":false, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493064, "value1":0.048118109532117544, "value2":4134373887037968422, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799328, "value1":0.11593192650373021, "value2":1597084067805135642, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.848966, "value1":0.3986720724086452, "value2":2405658508790196207, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792447, "value1":0.5248994141540482, "value2":6715402324504357457, "value3":false, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376734, "value1":0.7992683845538152, "value2":636776368616526730, "value3":false, "__name__":"metric_00000366", "key_8":"b","key_2":"j","key_6":"i", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646082, "value1":0.7040359351145827, "value2":8792341972386350386, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.358717, "value1":0.15149972457126396, "value2":8525880284771508381, "value3":false, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038854, "value1":0.6775307407388963, "value2":4497884453622086771, "value3":false, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313173, "value1":0.6408664944912432, "value2":9170385529548968990, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.209148, "value1":0.8824832965439914, "value2":6994782182036294339, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.349829, "value1":0.6332729838723153, "value2":8895300460695986955, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877356, "value1":0.9629884067987685, "value2":5874587318078108417, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175639, "value1":0.2215504394755138, "value2":1645626442028267495, "value3":true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.867334, "value1":0.17245948139280137, "value2":7635160136876279780, "value3":true, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.304089, "value1":0.4417775050588339, "value2":4663940359268319500, "value3":true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.377260, "value1":0.3699273314519082, "value2":3098095013643812538, "value3":false, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.687645, "value1":0.49899496293822504, "value2":3483082050221607102, "value3":true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.764258, "value1":0.5854336662763441, "value2":9096847150126021331, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.395360, "value1":0.4621587456489886, "value2":4751829496417501448, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702903, "value1":0.41598100403233135, "value2":3929201336867975918, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107090, "value1":0.7914140452826858, "value2":3589076543012868088, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.487563, "value1":0.4474861481593704, "value2":2779456470910748695, "value3":true, "__name__":"metric_00000406", "key_6":"j","key_7":"f","key_2":"i", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512119, "value1":0.5021386207405886, "value2":5678715906223819621, "value3":false, "__name__":"metric_00000402", "key_4":"k","key_7":"j","key_2":"h", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346070, "value1":0.55706872148054, "value2":2039287696528568559, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.574280, "value1":0.3713448367655447, "value2":3929829805998447613, "value3":true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.892079, "value1":0.6942759872823322, "value2":7729227663415283404, "value3":false, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.344002, "value1":0.42092378887559295, "value2":4794883782773019081, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078411, "value1":0.6154878270099419, "value2":6707716797669517904, "value3":false, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.571429, "value1":0.1378751349573672, "value2":8074751840319910755, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.040055, "value1":0.6408943986817053, "value2":1079211030051415138, "value3":true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162869, "value1":0.5000621894976172, "value2":5126515351324361210, "value3":false, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.390768, "value1":0.5289053805740587, "value2":7495442762687396988, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259180, "value1":0.5649142133543891, "value2":4328071162032108601, "value3":true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842704, "value1":0.21495927580985347, "value2":2932451180465678632, "value3":true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650755, "value1":0.6984634581926075, "value2":5766024562422825714, "value3":false, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612080, "value1":0.22851599558447847, "value2":5344041318095855549, "value3":true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990865, "value1":0.7354893379622883, "value2":9025665650100041943, "value3":false, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787926, "value1":0.33568512834020503, "value2":6804173344109850974, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172355, "value1":0.9139245088414795, "value2":6155807217840657379, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.979541, "value1":0.24199196392953898, "value2":5042790445163279739, "value3":false, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.314666, "value1":0.3110154918771059, "value2":8906652806658780575, "value3":true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357408, "value1":0.8063999303874789, "value2":1651751776595868392, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.288676, "value1":0.9293974960471005, "value2":7670397636585827003, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293262, "value1":0.7818217756805212, "value2":8606918715162923765, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.083534, "value1":0.9670843857632683, "value2":5887729702260327460, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149981, "value1":0.6552447379991068, "value2":6725961755903338686, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.492842, "value1":0.7092364133108918, "value2":7796995215212529618, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944648, "value1":0.6256326288941183, "value2":4831063196252879183, "value3":true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.895337, "value1":0.4995752655149396, "value2":1408597026195247818, "value3":false, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944023, "value1":0.634711000263352, "value2":5584697240807054271, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809484, "value1":0.7426498850085607, "value2":8020687455958940501, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289743, "value1":0.008820277060811819, "value2":6337782472142563494, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.537111, "value1":0.14436676295527084, "value2":1783901321543975972, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749042, "value1":0.9467021656046419, "value2":279529980240550876, "value3":true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.261740, "value1":0.20334209964882055, "value2":3677408684840846866, "value3":false, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.272200, "value1":0.5140769192808836, "value2":3965788713393972569, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391452, "value1":0.5620808926889774, "value2":3432925103242153786, "value3":false, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.797201, "value1":0.24306630830309864, "value2":2212488327265931040, "value3":false, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296686, "value1":0.4958413853936345, "value2":2745010941408731624, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.508150, "value1":0.1764099610149601, "value2":6087888705783067229, "value3":true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.150030, "value1":0.6475061379725164, "value2":4152410772475568903, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406997, "value1":0.8552543065924282, "value2":6943121315248371505, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798595, "value1":0.4482032174944134, "value2":4591949889203793788, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427353, "value1":0.23427032816286456, "value2":8455901174434850105, "value3":true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.426947, "value1":0.2157937977395769, "value2":4812396978108020858, "value3":true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553964, "value1":0.6569888020417926, "value2":4304633456065882312, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830280, "value1":0.6753558122351391, "value2":5709360383833750840, "value3":true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335737, "value1":0.6229413264866215, "value2":774141312393774432, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934503, "value1":0.3184401780744072, "value2":992485213419150513, "value3":false, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286280, "value1":0.19701195292124982, "value2":5131579316119627914, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.027527, "value1":0.5595644624118316, "value2":7324008691917344768, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.445835, "value1":0.3878002605330551, "value2":5283397242149349986, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.317903, "value1":0.34865399024368093, "value2":919918660600814082, "value3":true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.988293, "value1":0.520877683344731, "value2":1362287203583282026, "value3":true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451918, "value1":0.17797518451009597, "value2":8069259017842130934, "value3":true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.416614, "value1":0.49438798293161773, "value2":8221530176741871409, "value3":true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930143, "value1":0.11046223267799947, "value2":1569220098038190448, "value3":false, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841002, "value1":0.2351074608491269, "value2":6987153620614536862, "value3":true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172031, "value1":0.30378326240838155, "value2":3438948688273621480, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427034, "value1":0.3726321157197697, "value2":3892410333661915590, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.996688, "value1":0.06148106153681236, "value2":7012776674540552021, "value3":true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178139, "value1":0.24652480255103007, "value2":7264782641186044646, "value3":true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.641890, "value1":0.18896088271258277, "value2":5281363384350384982, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020470, "value1":0.5427453076719968, "value2":4211018551963613162, "value3":true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548627, "value1":0.9467235670877983, "value2":1356346215173760766, "value3":true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923887, "value1":0.16868132666008845, "value2":2419587552380208808, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388273, "value1":0.34607395217688514, "value2":6360018437765039879, "value3":false, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199475, "value1":0.48700761251474795, "value2":4207130869528000977, "value3":false, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212632, "value1":0.7298887109862953, "value2":3855225371708173636, "value3":false, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869081, "value1":0.3040832001677732, "value2":1636252957386132843, "value3":false, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458358, "value1":0.25499479265263875, "value2":7951534915845928364, "value3":false, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673427, "value1":0.3560914688953808, "value2":4486505974707646981, "value3":false, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934304, "value1":0.9086314287420252, "value2":6454407382033994812, "value3":false, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077472, "value1":0.272412312785447, "value2":4055844210948982160, "value3":false, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391470, "value1":0.4094306269697527, "value2":8737172085844354362, "value3":false, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.507196, "value1":0.4293932426928758, "value2":7752579176339832618, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.354922, "value1":0.8782836749723913, "value2":3092074071663841238, "value3":false, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290283, "value1":0.6466038651665884, "value2":6445966108572410248, "value3":true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930993, "value1":0.2534798517547614, "value2":2212538268968967382, "value3":false, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078571, "value1":0.766186870171555, "value2":2276988139922811436, "value3":true, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512143, "value1":0.26492881415396713, "value2":7960469334919381452, "value3":false, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.664663, "value1":0.9293891578195701, "value2":8363525489715122179, "value3":false, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.859809, "value1":0.5317709711783597, "value2":5646146960428890136, "value3":false, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755810, "value1":0.5033479266486215, "value2":1442438201922477703, "value3":false, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.014944, "value1":0.2736949295034319, "value2":4543402002213224798, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.019842, "value1":0.06301734338783647, "value2":7848421926410748778, "value3":true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.006460, "value1":0.7283128915704968, "value2":8696821133231766119, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.495665, "value1":0.22357458621454834, "value2":5013935622275517771, "value3":true, "__name__":"metric_00000490", "key_5":"j","key_9":"c","key_0":"k", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.829438, "value1":0.08516153321035196, "value2":6346747105122069788, "value3":false, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120479, "value1":0.6330301971897442, "value2":3074605270601257967, "value3":false, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406377, "value1":0.9646902889346889, "value2":9116844958473667659, "value3":false, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258382, "value1":0.9290313183449712, "value2":7925410787408077827, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340705, "value1":0.7460138699765932, "value2":1834174806073680705, "value3":false, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.660263, "value1":0.9250659950303504, "value2":6999248688568055694, "value3":false, "__name__":"metric_00000496", "key_6":"g","key_0":"h","key_5":"f", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252290, "value1":0.718659365662016, "value2":414864145410727491, "value3":false, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404912, "value1":0.5143974391105166, "value2":6178244170296726344, "value3":false, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.355231, "value1":0.7285417682392774, "value2":4099314188219635809, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232613, "value1":0.9628251247534773, "value2":2753347640662156751, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042238, "value1":0.40260777202369225, "value2":6405687740970757268, "value3":false, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.059614, "value1":0.6706802975294311, "value2":155105754975301806, "value3":false, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335427, "value1":0.03875943420721546, "value2":2621496673126808846, "value3":true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791994, "value1":0.4069226188854093, "value2":8911123850930154192, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.152311, "value1":0.001001700535447483, "value2":8160805255392424399, "value3":false, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.806126, "value1":0.09279090865200551, "value2":5165528109888127485, "value3":false, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181230, "value1":0.34297422042642206, "value2":5952935302808125089, "value3":false, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308758, "value1":0.39399952685268963, "value2":2291587630268260397, "value3":true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438229, "value1":0.008760206538330227, "value2":4899262456651209905, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253853, "value1":0.7708129134752543, "value2":5591299897276344295, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388133, "value1":0.4749440177332357, "value2":1978415349188199488, "value3":true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612871, "value1":0.07573141562457633, "value2":5953047090784791116, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.481277, "value1":0.12674805322449534, "value2":8950535310944284362, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026417, "value1":0.13620073115890582, "value2":5838422267573225081, "value3":true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491782, "value1":0.6697421435610075, "value2":5470323240276494634, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599889, "value1":0.2218252683274155, "value2":3863404473966363047, "value3":false, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028452, "value1":0.8531657224160458, "value2":1800467248475944001, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234267, "value1":0.2783481987951185, "value2":6997165241525177240, "value3":false, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369705, "value1":0.5636516367093837, "value2":4958650393346745090, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201031, "value1":0.628281679414999, "value2":3983210515283198963, "value3":true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.519582, "value1":0.950842451530494, "value2":8333725115642196357, "value3":false, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.800667, "value1":0.47362573201727787, "value2":5531339313881866007, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.022715, "value1":0.7456986400451252, "value2":6375649752818307396, "value3":true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026963, "value1":0.013441045212029653, "value2":8879122156886303116, "value3":false, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849167, "value1":0.7209297073484169, "value2":5571977114960029707, "value3":true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916214, "value1":0.07478238462060836, "value2":4957974530052652715, "value3":false, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.801842, "value1":0.5497676040427942, "value2":7068572572815929980, "value3":true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.180884, "value1":0.16934771421243305, "value2":4129529446885753081, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742335, "value1":0.17996128783258833, "value2":1612889506435491518, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.124131, "value1":0.6211285761421723, "value2":6936437753872174981, "value3":false, "__name__":"metric_00000531", "key_4":"j","key_2":"c","key_3":"h", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.626664, "value1":0.1412142921060844, "value2":7340286601023012024, "value3":true, "__name__":"metric_00000532", "key_9":"k","key_7":"e", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534599, "value1":0.1824543867650024, "value2":2710596260731528329, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.198428, "value1":0.563515863993397, "value2":8658671925349980639, "value3":true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939981, "value1":0.5602221119588965, "value2":1332664463057731952, "value3":true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749870, "value1":0.27530155912076365, "value2":3294589175109845443, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501991, "value1":0.28276078133217086, "value2":6074437039967216201, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886807, "value1":0.3878328960046618, "value2":6751632027412168403, "value3":true, "__name__":"metric_00000536", "key_8":"e","key_3":"d","key_7":"f", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.911289, "value1":0.6411363324230984, "value2":8252067497790790379, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.696807, "value1":0.43753398310350267, "value2":3790908907624313334, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319319, "value1":0.8437649419939369, "value2":4536112071098596928, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.734427, "value1":0.7768336856428384, "value2":577814262800235815, "value3":true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232482, "value1":0.4639181971383516, "value2":915233793101361418, "value3":true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033045, "value1":0.6840737140076493, "value2":6077817095892263388, "value3":true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427451, "value1":0.7412294644798527, "value2":5067651533453787714, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.086307, "value1":0.884916959850894, "value2":6213157740517107791, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032588, "value1":0.7692661642362982, "value2":3255905758159811447, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129133, "value1":0.9868818324607139, "value2":2840159926392322636, "value3":true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005933, "value1":0.4856659261691267, "value2":5401836471710156191, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078678, "value1":0.43872499297363293, "value2":7610348477992750069, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.101709, "value1":0.6553052079447295, "value2":5590613299241898040, "value3":false, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.691765, "value1":0.5500719022328135, "value2":2311787078857040546, "value3":true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.609841, "value1":0.20437362444327167, "value2":5976709159513367946, "value3":false, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.262383, "value1":0.988890785552926, "value2":5073782211732867167, "value3":false, "__name__":"metric_00000554", "key_5":"g","key_9":"g","key_1":"e", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382099, "value1":0.7056400640121613, "value2":1362786206799762754, "value3":false, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.910924, "value1":0.740118438943497, "value2":5204241364776707180, "value3":false, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526567, "value1":0.33926045568377805, "value2":8282435047009117311, "value3":true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259549, "value1":0.6968073477407744, "value2":5508984811117632066, "value3":false, "__name__":"metric_00000567", "key_6":"a","key_2":"i","key_4":"b", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.575379, "value1":0.9204583134801098, "value2":7328623658034172463, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.021181, "value1":0.8037794671210114, "value2":8763012613490416638, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666802, "value1":0.735290222214112, "value2":5898346647756703163, "value3":true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836023, "value1":0.9774841401477888, "value2":262883488175479807, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085239, "value1":0.2006523822964799, "value2":6927806981819194391, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722778, "value1":0.29750991877070054, "value2":7269146382542606407, "value3":true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585823, "value1":0.7790122698056057, "value2":5454870737254023995, "value3":false, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966399, "value1":0.9342783027092735, "value2":1736810747318512793, "value3":true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.363378, "value1":0.5421736888212692, "value2":8680345426115136675, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548518, "value1":0.6180502927475818, "value2":576215651567739722, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169252, "value1":0.8808696517971845, "value2":3266464067570846970, "value3":true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032461, "value1":0.3854322648479189, "value2":5351736110848205547, "value3":false, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418352, "value1":0.9282836374954182, "value2":4950655664039540565, "value3":true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.567245, "value1":0.06221685577996024, "value2":6376294794655595743, "value3":false, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.607825, "value1":0.8284877360321535, "value2":8836357162970050736, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.002720, "value1":0.9429757239354525, "value2":3712886085226721402, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772087, "value1":0.8045038867848936, "value2":4759138460922621996, "value3":true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515031, "value1":0.2984015395169622, "value2":2069584478283962675, "value3":false, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.141150, "value1":0.6666900797494949, "value2":268184099180001198, "value3":false, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.102173, "value1":0.08736073078451814, "value2":3669960405293392951, "value3":false, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.825677, "value1":0.883585003117732, "value2":4900464079888733154, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.613326, "value1":0.2003226722632331, "value2":3796686759340146598, "value3":true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.965080, "value1":0.24041472099907274, "value2":7454089011589464567, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139395, "value1":0.8846967167208682, "value2":3451086131264105278, "value3":false, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709618, "value1":0.40080113749800483, "value2":7347906568423400039, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.490979, "value1":0.10232414632757307, "value2":6898384399737921493, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.419810, "value1":0.9521194603097142, "value2":4165710553126975748, "value3":true, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142251, "value1":0.3656369505236563, "value2":2656042717790241170, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578775, "value1":0.42161623774225904, "value2":7959202830983535548, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340081, "value1":0.47208224231080925, "value2":1119972655203312039, "value3":true, "__name__":"metric_00000597", "key_3":"e","key_6":"j","key_2":"k", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201301, "value1":0.686282703890728, "value2":1347056528311053195, "value3":true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.905450, "value1":0.1504198445233754, "value2":4153183079049561671, "value3":true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.774660, "value1":0.04197441149840889, "value2":2892741854923361469, "value3":false, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.223828, "value1":0.9913278205978917, "value2":2634921019433133187, "value3":true, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145135, "value1":0.9764589636119458, "value2":1863333686324511619, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051457, "value1":0.9664880803510422, "value2":5468675095205300977, "value3":false, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.471202, "value1":0.9438594012185837, "value2":8711551131933841419, "value3":false, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438432, "value1":0.7332734301616686, "value2":6736601196370014167, "value3":false, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.018444, "value1":0.6166410041335356, "value2":7430067770903432071, "value3":false, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042785, "value1":0.3083845760668554, "value2":7529797289952981250, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.500528, "value1":0.41360737530731173, "value2":527946418421906650, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.639029, "value1":0.28120003518234893, "value2":7333246850823447562, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118602, "value1":0.22780592472876143, "value2":5562449967641903101, "value3":true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773447, "value1":0.34556648930141964, "value2":3454919255414457289, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.751050, "value1":0.5691145183911931, "value2":3420985612149611112, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.967122, "value1":0.010774686710818307, "value2":1857363403020110299, "value3":false, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211203, "value1":0.5071144540997159, "value2":3201415543625346596, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966034, "value1":0.10396047428400347, "value2":9014157472738440520, "value3":true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253319, "value1":0.43609058248995825, "value2":1183645881443278471, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104187, "value1":0.4324653404306875, "value2":2865322183450789108, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185518, "value1":0.8897028996083935, "value2":7566651396044601727, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930391, "value1":0.8041371194633276, "value2":7871648174569158725, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163568, "value1":0.9301616644769536, "value2":7748526202149251971, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869278, "value1":0.88349271636991, "value2":554007324522749523, "value3":true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.213408, "value1":0.664832495384472, "value2":2758153864408048603, "value3":true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451136, "value1":0.3702265627491039, "value2":494580882184796857, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915009, "value1":0.6519575592422996, "value2":6479471856073874185, "value3":true, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.581364, "value1":0.0703141245253348, "value2":899491222964044388, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142057, "value1":0.9957531092014932, "value2":5996532724034214303, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618561, "value1":0.31275866348570913, "value2":4261560321529985020, "value3":true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526426, "value1":0.8632813331612903, "value2":2280841193196363442, "value3":false, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.640741, "value1":0.4671010753968709, "value2":2523475617435456826, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686774, "value1":0.37684937337218677, "value2":6568317226871778750, "value3":false, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.547284, "value1":0.9412532971618462, "value2":6396494980432088364, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630465, "value1":0.10697292615695254, "value2":6790210135628014818, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156319, "value1":0.10115374361141051, "value2":2774605952238778168, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742080, "value1":0.4345761934130475, "value2":6896687242300333410, "value3":false, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662119, "value1":0.3886014719849511, "value2":2171378209788759856, "value3":false, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158805, "value1":0.4473032074837117, "value2":586039993535786710, "value3":false, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025398, "value1":0.0651048756141861, "value2":3136658443010387115, "value3":true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189314, "value1":0.11574796728414168, "value2":5756732023607570312, "value3":false, "__name__":"metric_00000631", "key_9":"d","key_1":"i","key_8":"g", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469000, "value1":0.3187212262703272, "value2":4880377874531438837, "value3":true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698910, "value1":0.9021085577061128, "value2":8314934978791640839, "value3":true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228153, "value1":0.7939593629687762, "value2":5721748474729655997, "value3":false, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.174833, "value1":0.47180057752688764, "value2":74850886504396865, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.373682, "value1":0.9894401032981868, "value2":2443814427210362559, "value3":false, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.230070, "value1":0.840516361148932, "value2":5295744989016708862, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595645, "value1":0.2788762608865926, "value2":898774770804749270, "value3":false, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583440, "value1":0.38764558337295657, "value2":4879715356875855821, "value3":false, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.818979, "value1":0.08550208893092236, "value2":7063465379400979205, "value3":true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990629, "value1":0.9735905702213057, "value2":9121417987396463994, "value3":false, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.904697, "value1":0.06007475161136332, "value2":1399260524832392153, "value3":true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832693, "value1":0.9560818456694564, "value2":6313771830503199817, "value3":false, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.273330, "value1":0.5911156951363433, "value2":6957059227202260296, "value3":false, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.069691, "value1":0.43486640701864104, "value2":5573463606213032969, "value3":true, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107074, "value1":0.28194511495703845, "value2":5008401247473276713, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.133554, "value1":0.6572805795366599, "value2":1430282776047161882, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.994425, "value1":0.7317293344785567, "value2":2161864319170825977, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703844, "value1":0.05169084281485511, "value2":7389845001390437940, "value3":true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667112, "value1":0.912614805848696, "value2":7494408396730454879, "value3":false, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175192, "value1":0.5654088358678295, "value2":7414573375849683263, "value3":true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.205436, "value1":0.5764549187518897, "value2":4315302504250390302, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.816073, "value1":0.3021595298462732, "value2":7835612651483818643, "value3":true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465800, "value1":0.7224511737802353, "value2":1475815608612889931, "value3":true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836860, "value1":0.13043025990273924, "value2":8725370796816365579, "value3":true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110112, "value1":0.2182570483232002, "value2":4212625314776126266, "value3":false, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.401613, "value1":0.3660560702023913, "value2":7136594683517469774, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772305, "value1":0.6420871129325768, "value2":346199260296482441, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142556, "value1":0.18492252698973582, "value2":2410884666534888495, "value3":true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.710999, "value1":0.03498389655141747, "value2":3857815905441742992, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588844, "value1":0.6843616917435196, "value2":925508006135613377, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902010, "value1":0.7892717951683124, "value2":7453201615304707478, "value3":true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071475, "value1":0.8597541524726022, "value2":5247483360729978449, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886399, "value1":0.896477399685476, "value2":3245685435537096080, "value3":false, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683519, "value1":0.7125828488615462, "value2":9221328263547905112, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650600, "value1":0.8976371405804583, "value2":6562020796772331341, "value3":false, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.316077, "value1":0.8989654150875539, "value2":2315139769444859937, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.844010, "value1":0.027932169490326376, "value2":8266191397214577403, "value3":false, "__name__":"metric_00000666", "key_3":"b","key_1":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.732165, "value1":0.9768852287807686, "value2":7871357409427659276, "value3":true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.644252, "value1":0.3652027068824268, "value2":4777422302644847840, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016877, "value1":0.4774488788727276, "value2":1359637185519527638, "value3":true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.121368, "value1":0.07637107971325026, "value2":225307864650781063, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.661741, "value1":0.9526993459994305, "value2":1751890497684728484, "value3":false, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534867, "value1":0.5672837246804853, "value2":3948370290656301760, "value3":false, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.400506, "value1":0.312527577003677, "value2":3200609992091774001, "value3":true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023204, "value1":0.9498281171738407, "value2":1775257641053296989, "value3":false, "__name__":"metric_00000674", "key_6":"c","key_7":"f","key_1":"a", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650859, "value1":0.7200243703698949, "value2":6459489446120996262, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125434, "value1":0.7513756216852647, "value2":6502722962310909561, "value3":false, "__name__":"metric_00000676", "key_5":"k","key_0":"k","key_2":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.861634, "value1":0.22132024628333183, "value2":4213082281103637151, "value3":true, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.616718, "value1":0.7492815151768418, "value2":8785198245169443750, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618495, "value1":0.5367662090666573, "value2":311068731789452160, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090805, "value1":0.6461806005354355, "value2":1567126231539851190, "value3":false, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.704866, "value1":0.6742157058257261, "value2":8258394004417852586, "value3":false, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178045, "value1":0.9347807954648532, "value2":5391643132437615590, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.903981, "value1":0.04205578457439769, "value2":4267367541005494603, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038592, "value1":0.9632414356901877, "value2":7847330231249131972, "value3":true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753078, "value1":0.025682952887214512, "value2":4407969986032631485, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.551387, "value1":0.9127538521307936, "value2":5532887277300530226, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340315, "value1":0.357568695323674, "value2":5839322022227051810, "value3":false, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253088, "value1":0.37455084749823464, "value2":919408856650004667, "value3":false, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.061612, "value1":0.08807754228035809, "value2":7674978240888726818, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350886, "value1":0.9187800273401259, "value2":3985979173089262193, "value3":false, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.997832, "value1":0.3332675485121975, "value2":6884650671561667205, "value3":false, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376107, "value1":0.882485292013383, "value2":980580007402095853, "value3":true, "__name__":"metric_00000691", "key_5":"h","key_6":"h","key_0":"b", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666200, "value1":0.543727834482791, "value2":7683502472635473948, "value3":true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.690345, "value1":0.9227402770528923, "value2":5963421474121503099, "value3":false, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831896, "value1":0.8899731417411955, "value2":3119002755449635021, "value3":true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583901, "value1":0.46723052632915907, "value2":4738569017001620646, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254234, "value1":0.07205706866452416, "value2":1464209096722519553, "value3":true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.940932, "value1":0.4880590550999508, "value2":3249029228406137695, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.819723, "value1":0.9180382594935601, "value2":5531777988899534137, "value3":true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.899553, "value1":0.5017555686141989, "value2":3526732457056334830, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364242, "value1":0.726219019872585, "value2":7463421321863859910, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591997, "value1":0.18380773538675038, "value2":6489374408484526555, "value3":true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268627, "value1":0.43767329135389277, "value2":7591255891965320507, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668426, "value1":0.8905993514345967, "value2":8025287728753470033, "value3":false, "__name__":"metric_00000713", "key_4":"c","key_2":"a","key_3":"b", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980089, "value1":0.9629455057472474, "value2":8613343366791814003, "value3":true, "__name__":"metric_00000714", "key_6":"g","key_3":"i","key_4":"d", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.561494, "value1":0.11038616417110533, "value2":5718549668479051731, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031316, "value1":0.5572446441362652, "value2":9073355869031665981, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913579, "value1":0.40513173097972344, "value2":1392267543875933322, "value3":true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469802, "value1":0.7197223556739805, "value2":8312522501295372114, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216239, "value1":0.38424010939993836, "value2":3429512093250859609, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053128, "value1":0.12167032010014549, "value2":2500133776481013473, "value3":false, "__name__":"metric_00000719", "key_9":"g","key_1":"d","key_4":"b", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755093, "value1":0.1432079670350677, "value2":4460076765766388831, "value3":true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939117, "value1":0.022396858430100586, "value2":1677736517466813137, "value3":true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730771, "value1":0.5975163721595746, "value2":8564236690483063617, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.763913, "value1":0.11111651544548351, "value2":2128389252611471929, "value3":false, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225483, "value1":0.1857169909861624, "value2":1459664257900170929, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.557521, "value1":0.10580492023702627, "value2":9211924519949290338, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.398250, "value1":0.8985634717833737, "value2":2690743666091944740, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.523236, "value1":0.8737716900990798, "value2":3717036232567418933, "value3":true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465788, "value1":0.8763409298655005, "value2":1486778905892801100, "value3":true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185627, "value1":0.4834833602007871, "value2":8283437701946785636, "value3":true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129438, "value1":0.4506377660858689, "value2":8577339860951120030, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.937075, "value1":0.6005010987020736, "value2":4798896188528165681, "value3":true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.387681, "value1":0.8777928078176396, "value2":4677891530420984908, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.074680, "value1":0.015131030176877407, "value2":2652924707092709319, "value3":true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032439, "value1":0.7762590461709376, "value2":5555968285860120909, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107619, "value1":0.0013683674938652786, "value2":8724767287173452904, "value3":true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199777, "value1":0.2705573241568207, "value2":5731172518242608315, "value3":true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.883455, "value1":0.440980354957089, "value2":717450023283511313, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.065153, "value1":0.7805040031236675, "value2":8249819238828017763, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028771, "value1":0.4266508385350679, "value2":3525220539697375602, "value3":false, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417030, "value1":0.9344530676800743, "value2":2840008251192450241, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169783, "value1":0.4944137362673325, "value2":8143727104076341090, "value3":true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309365, "value1":0.8233798931001785, "value2":4143752746550875358, "value3":true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405563, "value1":0.36473772798628423, "value2":8368477347330201864, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.080493, "value1":0.45312759136632275, "value2":5757367910796479605, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591589, "value1":0.5574497820130245, "value2":4692942231335997930, "value3":false, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770417, "value1":0.20880923062348417, "value2":4939040557737590500, "value3":false, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923414, "value1":0.8006055497749137, "value2":6979326850811422524, "value3":false, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249012, "value1":0.022621459630026125, "value2":6570512292324514521, "value3":false, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787566, "value1":0.5692664113038692, "value2":3307063324843670478, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.236668, "value1":0.706981031305401, "value2":6240895241125781235, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590979, "value1":0.9325091299580149, "value2":5714058795401825599, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782982, "value1":0.8159590900452445, "value2":3981533983292904484, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449106, "value1":0.2421958898621009, "value2":8819094433559346153, "value3":true, "__name__":"metric_00000746", "key_3":"d","key_8":"c","key_1":"i", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.721732, "value1":0.7320938316381037, "value2":1840371677400026121, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417920, "value1":0.3777544081714696, "value2":2517494055558796825, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.467240, "value1":0.07736271112081412, "value2":3355581288729910614, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802307, "value1":0.06164218213969723, "value2":6962215396306940249, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599871, "value1":0.2948698648725146, "value2":1065059716477496547, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.109220, "value1":0.11737528756979723, "value2":4468678220083400352, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156222, "value1":0.6294465098457002, "value2":2835781199255992964, "value3":false, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697592, "value1":0.9317953948461911, "value2":2221981181398322475, "value3":false, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785752, "value1":0.4235402956347778, "value2":5956435747949559681, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943686, "value1":0.28355998213096456, "value2":3638916038813769869, "value3":false, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.010104, "value1":0.22773696044954786, "value2":2360395284842638669, "value3":false, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.239178, "value1":0.14419564768805598, "value2":5315118297850048019, "value3":true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136093, "value1":0.3244550022291072, "value2":164222067931855660, "value3":false, "__name__":"metric_00000758", "key_6":"i","key_7":"b","key_2":"j", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968733, "value1":0.7486844786821671, "value2":6287914493893452990, "value3":false, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252999, "value1":0.24537563867085085, "value2":1626574969302079091, "value3":false, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.885743, "value1":0.9307243682752692, "value2":2858030878483913913, "value3":true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.631011, "value1":0.7084136481624964, "value2":2044467985498488215, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.853026, "value1":0.04425979450750048, "value2":1653452661642691666, "value3":false, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.991236, "value1":0.40886253057738015, "value2":7871621921656029433, "value3":true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.207156, "value1":0.37687587966259445, "value2":1943347287070225574, "value3":true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.708313, "value1":0.5785225454675275, "value2":1380941757138302107, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770913, "value1":0.9188180749639426, "value2":3884885787383779114, "value3":true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418857, "value1":0.9998412287287592, "value2":1886115929568662728, "value3":true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.850257, "value1":0.9796804114853653, "value2":4397109710918249601, "value3":true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339971, "value1":0.5089698391144044, "value2":8237713822108701891, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913742, "value1":0.9646995309003717, "value2":1173067889283022862, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067798, "value1":0.6899692743360695, "value2":1520636933221389194, "value3":false, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533696, "value1":0.8606713340189989, "value2":5737213954118900238, "value3":false, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212415, "value1":0.39075235526474594, "value2":6592432468398327445, "value3":false, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054486, "value1":0.34784858262092483, "value2":7843113252021213719, "value3":true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479147, "value1":0.5614107726843738, "value2":8690368331747046125, "value3":false, "__name__":"metric_00000785", "key_6":"d","key_9":"k","key_5":"j", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.622976, "value1":0.06775340365098863, "value2":5564636394288500497, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.315983, "value1":0.33546786831791003, "value2":3930900696554115736, "value3":true, "__name__":"metric_00000787", "key_6":"j","key_7":"g","key_3":"h", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.370627, "value1":0.779233269421853, "value2":3716543858406653649, "value3":false, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.969610, "value1":0.006014494345107452, "value2":8710808689518483016, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.724048, "value1":0.07768942947746518, "value2":496409043717037404, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590602, "value1":0.8288610653786054, "value2":1102610595846648990, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297543, "value1":0.37578423768641916, "value2":5536977376920161050, "value3":true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.155394, "value1":0.7083524353328224, "value2":4002932567237644088, "value3":false, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.459881, "value1":0.4873646265203241, "value2":5916271319076231574, "value3":false, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791505, "value1":0.5808298895011346, "value2":7841892156688279555, "value3":true, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067541, "value1":0.09121699649729094, "value2":8699612203169033522, "value3":true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138436, "value1":0.8850890106644654, "value2":9069485496265005186, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.812668, "value1":0.7098616610527902, "value2":8132915524494658053, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.190668, "value1":0.2761598493860578, "value2":651770389362313830, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.220527, "value1":0.17936744276680264, "value2":310375881133353905, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428315, "value1":0.49075022964169346, "value2":6493345006460964913, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595999, "value1":0.2755442227044263, "value2":4862778030716507189, "value3":false, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.719052, "value1":0.21758648924177262, "value2":9168976451471290619, "value3":true, "__name__":"metric_00000777", "key_4":"c","key_8":"i","key_0":"j", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166119, "value1":0.5187037640605351, "value2":7936720027148899998, "value3":false, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.361344, "value1":0.7201073761748152, "value2":8725593566170852352, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254415, "value1":0.07641284618924576, "value2":7978515171853518726, "value3":true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.981898, "value1":0.027541030165662613, "value2":6121611839007131360, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625954, "value1":0.22759978693770738, "value2":2989185053134172992, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350547, "value1":0.566849398731457, "value2":4327099372915065254, "value3":false, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799208, "value1":0.11746266545752859, "value2":7451515689230242023, "value3":true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.322064, "value1":0.26907348347198895, "value2":262434464387240754, "value3":true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.919974, "value1":0.6405108138252255, "value2":7284316699831173435, "value3":true, "__name__":"metric_00000803", "key_9":"f","key_3":"k","key_7":"c", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534467, "value1":0.9814271230191226, "value2":9214896283898892364, "value3":true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.291823, "value1":0.7119821636486502, "value2":4137668561036791084, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773725, "value1":0.05006867968524374, "value2":4350082823034910080, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.642706, "value1":0.8326077028765216, "value2":6301127653055186823, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.624652, "value1":0.6013140948710498, "value2":7150151669380473161, "value3":false, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663653, "value1":0.6471420893487286, "value2":2842026748176794650, "value3":false, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533665, "value1":0.40332997931033865, "value2":8939087210798770840, "value3":false, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915959, "value1":0.20834521315367086, "value2":7895351758501362895, "value3":false, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.238383, "value1":0.5265667505785955, "value2":2676293106518263900, "value3":true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582878, "value1":0.3420886897063641, "value2":1233598437288047014, "value3":false, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.489464, "value1":0.6718160824398975, "value2":4468628860088387255, "value3":true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.649028, "value1":0.7293826474439736, "value2":3127060444172041547, "value3":true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254010, "value1":0.8457444172560414, "value2":6200016452741633767, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.536370, "value1":0.16874770298054162, "value2":7575698105239291032, "value3":false, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809584, "value1":0.343753090691686, "value2":4854239248106071203, "value3":true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.334375, "value1":0.5540567040963329, "value2":2646076573445947055, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826524, "value1":0.7273180695893363, "value2":6564419565708307637, "value3":false, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071563, "value1":0.26385268835278003, "value2":4070457585628644142, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225965, "value1":0.9321283899737617, "value2":7702159669474089190, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458779, "value1":0.10195409940683517, "value2":5987602857042259052, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501592, "value1":0.07688705594071264, "value2":5259725214710983339, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.385240, "value1":0.8254376912417642, "value2":6760238975249610189, "value3":true, "__name__":"metric_00000818", "key_6":"f","key_3":"c","key_4":"g", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.760329, "value1":0.7530171422146449, "value2":1097173296466796516, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.597269, "value1":0.2403481802375547, "value2":5050041855629937310, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149349, "value1":0.43132198586517284, "value2":3287400838900732872, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.499626, "value1":0.737892470781718, "value2":4461721794139931373, "value3":false, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449953, "value1":0.8639558735483506, "value2":2903703899322529602, "value3":true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.912253, "value1":0.01605090964788225, "value2":5781647102967996198, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.4054065908363409, "value2":6440329633133258472, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038844, "value1":0.3763702930516151, "value2":1302491802347098686, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.453739, "value1":0.6265318797324549, "value2":819075302260626614, "value3":false, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590829, "value1":0.5479451127212884, "value2":8557856578535483978, "value3":true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714164, "value1":0.8513319596110408, "value2":3981392010073089983, "value3":false, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.767546, "value1":0.9500048124431112, "value2":2193055476612398128, "value3":false, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.685203, "value1":0.9241278767545394, "value2":5146494836784446946, "value3":false, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171752, "value1":0.05193751579817584, "value2":2407542541423061968, "value3":false, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620585, "value1":0.9449477088124408, "value2":293003481714135625, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197927, "value1":0.9081143587822024, "value2":336470947794228646, "value3":false, "__name__":"metric_00000841", "key_6":"b","key_1":"i","key_2":"d", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730165, "value1":0.4106378302625472, "value2":2305406403284031522, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993455, "value1":0.5015628664485756, "value2":3942941750281172936, "value3":false, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376150, "value1":0.08846010494510266, "value2":451409939393211294, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578966, "value1":0.5184884584460681, "value2":5013311361860229047, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980632, "value1":0.6414794039381226, "value2":4661775189685128413, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543153, "value1":0.7779597819383781, "value2":1678453405944867509, "value3":false, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.728157, "value1":0.4026694195455883, "value2":1247937331490084335, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039033, "value1":0.6078130901686731, "value2":6572519819620597387, "value3":false, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020089, "value1":0.16501689332992148, "value2":2135299756770180375, "value3":true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159279, "value1":0.06812322597368, "value2":1853312335979487824, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225355, "value1":0.31761717143878, "value2":1547635249117783016, "value3":false, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.579591, "value1":0.45080215259792744, "value2":830280592135153306, "value3":true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.989155, "value1":0.3246650009893744, "value2":3832412454221483471, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.898794, "value1":0.9353227689941251, "value2":5019805485542488656, "value3":true, "__name__":"metric_00000854", "key_4":"g","key_1":"d","key_2":"f", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.325402, "value1":0.40452074217950085, "value2":3779976710102146862, "value3":false, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.846682, "value1":0.8078556003817017, "value2":4563926776477066326, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.538107, "value1":0.6916244463696708, "value2":2616499684497809047, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601707, "value1":0.0005961819411869942, "value2":8421789457526552579, "value3":false, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.474772, "value1":0.40247038234287125, "value2":691809847413019723, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835874, "value1":0.24276779175523852, "value2":6899854861109897252, "value3":false, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.366826, "value1":0.8778001737651254, "value2":4321676554804962434, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826448, "value1":0.8031889549998205, "value2":5554188972077808449, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.432681, "value1":0.15017093541000798, "value2":7513115457016757405, "value3":true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.353975, "value1":0.23702527674889587, "value2":1810989557365945672, "value3":true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303934, "value1":0.2677306312444, "value2":430799962318472061, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163809, "value1":0.8138323281483592, "value2":8246241283973870498, "value3":false, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662355, "value1":0.9144882781553422, "value2":4359470640610896382, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120568, "value1":0.8056479342778862, "value2":6170593949410616781, "value3":true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.541822, "value1":0.248047143451303, "value2":8344606419798048221, "value3":false, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.147354, "value1":0.35385149944297156, "value2":6006537614889971434, "value3":true, "__name__":"metric_00000868", "key_8":"j","key_3":"i","key_4":"i", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.116235, "value1":0.6774547184266492, "value2":5503266725598369859, "value3":false, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.563395, "value1":0.978792023344712, "value2":8139355376515169099, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202304, "value1":0.42453838538082794, "value2":8767087562530261434, "value3":false, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553168, "value1":0.813931872578598, "value2":838817774250793063, "value3":true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.476339, "value1":0.2705647850316454, "value2":6004383053661805919, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067348, "value1":0.23127791420697405, "value2":614040873255409897, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.063340, "value1":0.04494057741571453, "value2":7945808906854676018, "value3":true, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877369, "value1":0.188685408716376, "value2":5653852277013510880, "value3":false, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145318, "value1":0.6394711610065485, "value2":6391486639411558835, "value3":false, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171038, "value1":0.9902721723985418, "value2":3163235560331954355, "value3":true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773084, "value1":0.40123815528773027, "value2":6772197516975478771, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.976169, "value1":0.608513336182585, "value2":3898075135558136236, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.460540, "value1":0.027884469593113505, "value2":1806632468596114826, "value3":false, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.411909, "value1":0.6437101253853217, "value2":4628507915366383990, "value3":true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.055892, "value1":0.4209111843259915, "value2":8498738002897413709, "value3":true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684315, "value1":0.3608627471653678, "value2":8911984339628672502, "value3":true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181887, "value1":0.04106030144724243, "value2":912973087706429869, "value3":true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.127221, "value1":0.8723975492127802, "value2":4688994221703849298, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881735, "value1":0.8013556912079172, "value2":643258613153646569, "value3":true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093171, "value1":0.9983142632275949, "value2":2242609892773836818, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702516, "value1":0.9951260414368723, "value2":9110402286568364937, "value3":true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712584, "value1":0.10494403348655837, "value2":4745554958401980793, "value3":false, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.699467, "value1":0.9010413033988799, "value2":9007036540377523860, "value3":false, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.132646, "value1":0.06467587338211019, "value2":1060827532336676254, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.552157, "value1":0.4330362437559917, "value2":7496787358324320393, "value3":false, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888799, "value1":0.2576583993305694, "value2":4018360458617386281, "value3":true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.498269, "value1":0.2224059001256479, "value2":4065063745853139952, "value3":false, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588721, "value1":0.14516718870277823, "value2":3512331805580372297, "value3":true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583233, "value1":0.7489522543167118, "value2":5610699138607696953, "value3":true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290155, "value1":0.44273304014865483, "value2":427744180869204041, "value3":true, "__name__":"metric_00000910", "key_9":"e","key_5":"a","key_8":"c", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183829, "value1":0.6345267428968718, "value2":1846381248733979645, "value3":false, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.323098, "value1":0.012439339538142696, "value2":8869073391299009493, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634145, "value1":0.9267919956221178, "value2":2118172993893609219, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794582, "value1":0.8353549954749553, "value2":1351674467301929869, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.008550, "value1":0.6040059322178531, "value2":753170312236461842, "value3":true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268043, "value1":0.31922185771526074, "value2":2365194637514291305, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.857933, "value1":0.8196275302693926, "value2":4478227043901542114, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.645585, "value1":0.17157442528490047, "value2":3055405974571687053, "value3":true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.277791, "value1":0.8508212157393535, "value2":475170132890875463, "value3":false, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123450, "value1":0.35015180460716977, "value2":7931493690032322348, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.918435, "value1":0.30789613557477924, "value2":4634204529169845529, "value3":false, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.747396, "value1":0.7119174217020244, "value2":5113967535733338451, "value3":true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346547, "value1":0.4105703222986521, "value2":5634326652610778742, "value3":false, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.873833, "value1":0.5726619572582797, "value2":4834039219162125836, "value3":true, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839553, "value1":0.3474160702051541, "value2":5912402851303338785, "value3":true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924943, "value1":0.25193375971882537, "value2":3157228766646586798, "value3":false, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309520, "value1":0.4856065585160012, "value2":12512784025771269, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612385, "value1":0.06742962293903079, "value2":5561454087904294314, "value3":true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634744, "value1":0.8158802826640159, "value2":4407063105915178233, "value3":false, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524442, "value1":0.05056002141365119, "value2":1069310785755082516, "value3":false, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.122461, "value1":0.8648592352685478, "value2":2607760955410972754, "value3":false, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308553, "value1":0.8340608354185821, "value2":5091217112123016682, "value3":true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.073014, "value1":0.412293314616913, "value2":1754188674597018639, "value3":false, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382506, "value1":0.7914084871078703, "value2":3113230598922027169, "value3":true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.494019, "value1":0.22683994673921629, "value2":345509562584529791, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770833, "value1":0.36946692588064045, "value2":3634543347045698698, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374724, "value1":0.7674203492653544, "value2":409087874753033650, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.33760436450813625, "value2":6378216485000854161, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753898, "value1":0.08146706117124274, "value2":4619087622758440080, "value3":true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.058417, "value1":0.0895913841094616, "value2":7775148933653129917, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.889748, "value1":0.9025957275835974, "value2":9060698938664882878, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.429099, "value1":0.45935248830386766, "value2":5813431686543708617, "value3":false, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.415280, "value1":0.8845089283715811, "value2":6384109853170568567, "value3":false, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956220, "value1":0.6163283411653032, "value2":7939487848850431764, "value3":false, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.813125, "value1":0.3306736514104797, "value2":717112968732186535, "value3":false, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.464126, "value1":0.23519387446018714, "value2":2061991374504688132, "value3":false, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.336194, "value1":0.6334359034532542, "value2":5464410230276800258, "value3":false, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722970, "value1":0.6104432497384834, "value2":3298076355265173187, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.246842, "value1":0.8069861206050694, "value2":7793368021912415700, "value3":false, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465645, "value1":0.6879012241509205, "value2":4196934016416565115, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876004, "value1":0.06724546167889978, "value2":7203308436755461322, "value3":true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667298, "value1":0.6826712318664662, "value2":6402839614176163878, "value3":true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.329694, "value1":0.7040366253423608, "value2":1262392626609016984, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591566, "value1":0.9312106031712911, "value2":6293442498282971529, "value3":true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.559072, "value1":0.4448009704050545, "value2":2669159126008899565, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431852, "value1":0.8238849009316198, "value2":1021818236630467563, "value3":true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722762, "value1":0.5948459056041813, "value2":6253806025921861085, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339501, "value1":0.14816011629555237, "value2":2437202342857552183, "value3":false, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313849, "value1":0.14517234363613402, "value2":3032490535727906731, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974064, "value1":0.5282428591902778, "value2":7306937940345849875, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930949, "value1":0.6595519497499615, "value2":2818227237339798252, "value3":false, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.995836, "value1":0.41346641045143234, "value2":457793590293209131, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.527926, "value1":0.17089664849686367, "value2":5825724904068162921, "value3":false, "__name__":"metric_00000955", "key_7":"d","key_1":"e","key_6":"j", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258906, "value1":0.5587901020638738, "value2":2247971244413895345, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232248, "value1":0.9182710496214209, "value2":2114818465791862400, "value3":false, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.075755, "value1":0.36489084685089246, "value2":294204895726685309, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.726685, "value1":0.6513663450138257, "value2":741257611118540876, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794966, "value1":0.07728058455296861, "value2":6115920270410886698, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.954148, "value1":0.6576477115837028, "value2":8442517348621889353, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.305446, "value1":0.8417215600157095, "value2":3960819244598684350, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749210, "value1":0.905397326364665, "value2":7392448658664215427, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479499, "value1":0.8192850944227448, "value2":1109448037735485747, "value3":true, "__name__":"metric_00000963", "key_8":"b","key_4":"h","key_5":"a", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228850, "value1":0.7549298352802428, "value2":2475887390541907125, "value3":false, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.256344, "value1":0.6191375565525401, "value2":6629221972448932333, "value3":true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843715, "value1":0.4048974244394111, "value2":2052496354441233400, "value3":true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.257771, "value1":0.11131625744600283, "value2":5527432282921695963, "value3":false, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033616, "value1":0.18479886541727453, "value2":9028612437715086928, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832552, "value1":0.6959404377768793, "value2":2173084661524542898, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431933, "value1":0.7816557853866045, "value2":7424533234329790131, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.264080, "value1":0.7382347817860379, "value2":5213561271316161382, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085647, "value1":0.4523244522101, "value2":7889921400021958159, "value3":true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004893, "value1":0.6556889205138186, "value2":3025561804921198898, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339616, "value1":0.2999169933424259, "value2":8184676717410157627, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216679, "value1":0.42313229610434316, "value2":4008780382662754625, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418495, "value1":0.49555776117242517, "value2":1018253790078366875, "value3":false, "__name__":"metric_00000985", "key_7":"j","key_0":"a", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033786, "value1":0.9209475895817386, "value2":5569575246133894540, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620702, "value1":0.6494468637932249, "value2":4953667604093058532, "value3":true, "__name__":"metric_00000987", "key_5":"b","key_7":"e","key_2":"a", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.706504, "value1":0.12760690080118256, "value2":6290715519152270329, "value3":true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.483588, "value1":0.5095141564714446, "value2":6862972202733799902, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159429, "value1":0.90417660237305, "value2":8461823351624136781, "value3":false, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.868532, "value1":0.20009311827615117, "value2":2269303027016825167, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.030853, "value1":0.07427016371278365, "value2":6524107618104112365, "value3":false, "__name__":"metric_00000983", "key_7":"c","key_9":"e","key_0":"b", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625191, "value1":0.3705374970999065, "value2":8940487375802886850, "value3":false, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274579, "value1":0.43745692673872216, "value2":7664930411577220439, "value3":false, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428630, "value1":0.527310522539046, "value2":6084859156132058763, "value3":true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.594595, "value1":0.21322470607998698, "value2":3416879710307779950, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.920870, "value1":0.47127845699263055, "value2":4564621285620268941, "value3":false, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930512, "value1":0.41802427118606883, "value2":8549501771062307593, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977996, "value1":0.1610907563584878, "value2":6578022034712623821, "value3":true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973727, "value1":0.9224145581231933, "value2":8668945457313971984, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421891, "value1":0.40172827078543016, "value2":2817865116264178423, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683482, "value1":0.8428158825760677, "value2":1087743773064912864, "value3":false, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.001802, "value1":0.5075672608227982, "value2":3748937813625482336, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924538, "value1":0.6184538471340483, "value2":8049426804122873881, "value3":false, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427400, "value1":0.23484522887445405, "value2":360795956748828928, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.371453, "value1":0.14713938237360089, "value2":2782450449945692027, "value3":false, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105401, "value1":0.7787234859135739, "value2":7103390400724679707, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608837, "value1":0.9083492888699478, "value2":591947190434298439, "value3":true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839636, "value1":0.20678896707817307, "value2":1816652983048222981, "value3":true, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.798496, "value1":0.27959153848419394, "value2":6453301245071446957, "value3":true, "__name__":"metric_00000002", "key_8":"f","key_1":"a","key_2":"g", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079294, "value1":0.9921346825877959, "value2":2610009567825131673, "value3":true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.932174, "value1":0.616531393243407, "value2":5793240958146632836, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352378, "value1":0.5817027431697587, "value2":1001435181122015935, "value3":false, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.892678, "value1":0.09115452403074104, "value2":5100882205034466347, "value3":false, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.976748, "value1":0.223689480612581, "value2":4832346905388106952, "value3":true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348707, "value1":0.4785116775888147, "value2":1474405574460091730, "value3":true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285197, "value1":0.2545105724094511, "value2":2069757700743153534, "value3":true, "__name__":"metric_00000007", "key_8":"a","key_3":"e","key_4":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886496, "value1":0.6267242677514145, "value2":5390965713876988864, "value3":true, "__name__":"metric_00000009", "key_3":"k","key_8":"b","key_0":"h", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.106653, "value1":0.29720904047717794, "value2":3776030286792185916, "value3":true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.360157, "value1":0.5546347320924953, "value2":330359320098225031, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229710, "value1":0.36539460584181066, "value2":2852256551896569615, "value3":false, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.107272, "value1":0.7519447135299673, "value2":479042341091737730, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.653206, "value1":0.055664103121505173, "value2":4715105139689145584, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339724, "value1":0.65279778916842, "value2":5051290581444035933, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605595, "value1":0.7185823471699857, "value2":6840084614633305448, "value3":true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923752, "value1":0.4375825544173804, "value2":5512879974255226786, "value3":true, "__name__":"metric_00000017", "key_5":"d","key_7":"e","key_0":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259124, "value1":0.1976251344530473, "value2":3865204942280496136, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833243, "value1":0.1327195116975274, "value2":25036382112222312, "value3":true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181954, "value1":0.03893696644874014, "value2":8620048848181171214, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241452, "value1":0.08765547856551259, "value2":478115772614972254, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.346035, "value1":0.8594527006005106, "value2":7112082294884150846, "value3":false, "__name__":"metric_00000021", "key_3":"f","key_9":"c","key_2":"i", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171651, "value1":0.03206744023244803, "value2":1423544456047900511, "value3":false, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863325, "value1":0.8536761474388656, "value2":6060590565202496674, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.441838, "value1":0.172156773217081, "value2":5954759661339252215, "value3":true, "__name__":"metric_00000025", "key_4":"j","key_5":"b","key_0":"a", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960980, "value1":0.32980876505437323, "value2":8725964433492010874, "value3":false, "__name__":"metric_00000026", "key_7":"d","key_1":"f","key_2":"c", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000707, "value1":0.24709283071336913, "value2":6676514692655151346, "value3":false, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.618373, "value1":0.9009517094483285, "value2":3595072304978000605, "value3":true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.170508, "value1":0.9280407408957537, "value2":5488749324303924425, "value3":true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863252, "value1":0.12661256202486373, "value2":8664732043269436820, "value3":true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.849403, "value1":0.16431470929214623, "value2":7019138744327165116, "value3":false, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464027, "value1":0.09323138351556215, "value2":7725624855401413056, "value3":false, "__name__":"metric_00000034", "key_7":"e","key_9":"c","key_6":"b", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.236399, "value1":0.8723974444748736, "value2":4852776678069519279, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748433, "value1":0.955013940926474, "value2":1647976132077310759, "value3":false, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519277, "value1":0.9878333472759755, "value2":1234045371574302701, "value3":false, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098817, "value1":0.01452665827731407, "value2":1567075824080468956, "value3":true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.086812, "value1":0.507976874584688, "value2":7088565570837880582, "value3":true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.442740, "value1":0.05234017389082447, "value2":6744325328771713826, "value3":true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200886, "value1":0.26486555470244233, "value2":7063921451775904700, "value3":true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542647, "value1":0.44652572463811874, "value2":881040498652574470, "value3":false, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403848, "value1":0.15638403887386948, "value2":881480808583221270, "value3":false, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.456051, "value1":0.8059172501323808, "value2":7272729849107639168, "value3":false, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529307, "value1":0.34486654778342174, "value2":392684728493023069, "value3":true, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171436, "value1":0.3729515434430422, "value2":3944928681545148043, "value3":false, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308638, "value1":0.7632957285940241, "value2":92442464729948625, "value3":false, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515220, "value1":0.6925288324736131, "value2":8180048442965609063, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584191, "value1":0.021892941959738798, "value2":8483219855752350520, "value3":false, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.062243, "value1":0.4180089249967854, "value2":6354312606503764778, "value3":true, "__name__":"metric_00000049", "key_9":"j","key_1":"d","key_3":"h", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341779, "value1":0.7379607607767219, "value2":5150239222005969615, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843413, "value1":0.6687750316343205, "value2":2316529591090571268, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.402005, "value1":0.9141363896275393, "value2":6270246938486716288, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128379, "value1":0.5150067708119237, "value2":678394980878451314, "value3":true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108482, "value1":0.3129644434816646, "value2":8384633845422523721, "value3":false, "__name__":"metric_00000052", "key_4":"a","key_9":"g","key_3":"c", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.594874, "value1":0.2519824660631957, "value2":6886759477377939773, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445205, "value1":0.903159708784011, "value2":7034171128847343546, "value3":false, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824198, "value1":0.6204070552794421, "value2":1059263092990498226, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.159620, "value1":0.8686561618779289, "value2":5404253444446120581, "value3":true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.365387, "value1":0.4572890031222697, "value2":7934471101097198056, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521775, "value1":0.36902024174358433, "value2":2258863231460147617, "value3":false, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.553157, "value1":0.44513850096108715, "value2":6217475414573430294, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141433, "value1":0.21220874698804243, "value2":93944351046750038, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.811456, "value1":0.8670934692477834, "value2":7403345830074134000, "value3":false, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003892, "value1":0.4239005643048954, "value2":7762289626422779835, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.067882, "value1":0.18864986614817061, "value2":2907310668904743062, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.750570, "value1":0.6373780024843834, "value2":3686123513663951346, "value3":false, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.386866, "value1":0.2003273945539795, "value2":5992425398301380670, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515245, "value1":0.516857446811694, "value2":4876668434287590384, "value3":false, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389071, "value1":0.9340489387136409, "value2":7504250901331893981, "value3":true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098316, "value1":0.29683731723021556, "value2":7480572590262205533, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906295, "value1":0.7596148064915852, "value2":2005574389285214762, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.469199, "value1":0.3474919541045514, "value2":4513611972684272623, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735221, "value1":0.44321964186413737, "value2":284056865289241089, "value3":false, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382330, "value1":0.4698238258381697, "value2":6544780705530113911, "value3":false, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644228, "value1":0.34547957427274706, "value2":5643558758908257061, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025533, "value1":0.9831418878764304, "value2":34382825795091331, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564530, "value1":0.1974264877985391, "value2":7202346148984902707, "value3":true, "__name__":"metric_00000076", "key_8":"h","key_4":"e", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.582038, "value1":0.6658499962262118, "value2":4779411307587076137, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.048223, "value1":0.28118182352648263, "value2":2406664371776445226, "value3":false, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.277281, "value1":0.3293088133074837, "value2":3125287577587960916, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711593, "value1":0.6513318644160435, "value2":6304109109655854952, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.136044, "value1":0.15479605681668362, "value2":4561542839689997911, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.143594, "value1":0.03881037667597162, "value2":5145183720039449728, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766270, "value1":0.5913740900789849, "value2":2880750585927778789, "value3":true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.330272, "value1":0.03345193869225257, "value2":5177206809108614692, "value3":true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893588, "value1":0.9545617077887952, "value2":1994751413642673810, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200065, "value1":0.7615854624227617, "value2":2555245387882162412, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.637143, "value1":0.8697167932948927, "value2":3163242108044176287, "value3":true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.640389, "value1":0.5659009743641423, "value2":7954937488899706120, "value3":true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.558497, "value1":0.928513605246244, "value2":1207841403052993714, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.244339, "value1":0.9485629290996225, "value2":8066024682557174519, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.368292, "value1":0.1771040772082859, "value2":5158146682102067527, "value3":false, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883369, "value1":0.26796315495485606, "value2":500188854907829642, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.603831, "value1":0.695040513930677, "value2":7670486818087410343, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635402, "value1":0.17089841071375036, "value2":2188177332357604180, "value3":false, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246584, "value1":0.638326848995027, "value2":2467157573588247428, "value3":false, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297753, "value1":0.4028871413838985, "value2":549392368948968713, "value3":true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061298, "value1":0.28566196711825703, "value2":4345219419785285366, "value3":true, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468603, "value1":0.15137756224435941, "value2":5857705640176057374, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705772, "value1":0.9900652870935787, "value2":4160023456685330050, "value3":true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628560, "value1":0.47755802914007767, "value2":2886069569872931687, "value3":true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.201527, "value1":0.3765200270511815, "value2":8192740497964394547, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405600, "value1":0.16568713498682006, "value2":807681981221494025, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.070083, "value1":0.5411063354146068, "value2":6815859051389366015, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339848, "value1":0.09610559077312085, "value2":5872072911320364504, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571126, "value1":0.2497057720663749, "value2":924893218806837423, "value3":true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659128, "value1":0.026188484488162033, "value2":8931856805131420281, "value3":true, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.127455, "value1":0.3964141576554178, "value2":2662798392903843365, "value3":false, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780076, "value1":0.15040609462861912, "value2":7522359721454017998, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108734, "value1":0.6578186778027336, "value2":2079921522814480295, "value3":true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.476594, "value1":0.5388243902425934, "value2":1295082543190183614, "value3":true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377611, "value1":0.4899107617022405, "value2":1810844261495017790, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.980760, "value1":0.052036560099483146, "value2":4808591658339710229, "value3":true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.662657, "value1":0.739342249293886, "value2":4457016924188070830, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573154, "value1":0.043565004929414355, "value2":2934360743315418976, "value3":false, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.792186, "value1":0.7194337918198027, "value2":8482147182498656703, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687869, "value1":0.5159874448621534, "value2":5870856463348491189, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.522440, "value1":0.8824876555801262, "value2":2534178016362914654, "value3":false, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132515, "value1":0.8599678891940701, "value2":8395948226155967317, "value3":true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.096926, "value1":0.9245913183435254, "value2":2358549994381165596, "value3":true, "__name__":"metric_00000120", "key_3":"e","key_7":"d","key_2":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486752, "value1":0.2842029485021584, "value2":2801356184403378281, "value3":false, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.475238, "value1":0.22293224326621178, "value2":443811864922674372, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132657, "value1":0.9371037777036672, "value2":4385576391999515299, "value3":true, "__name__":"metric_00000121", "key_5":"a","key_7":"h","key_3":"e", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.549121, "value1":0.03204732941506213, "value2":8714811939867543319, "value3":true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796275, "value1":0.21070900388130828, "value2":8667819799137478248, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927343, "value1":0.9159887689007843, "value2":7689292400204431331, "value3":false, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685507, "value1":0.7348923960578333, "value2":330857683592078866, "value3":false, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.865459, "value1":0.24066397037006473, "value2":203169051837200909, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.827976, "value1":0.4488303067264579, "value2":3646968045887419113, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900145, "value1":0.31889183176683844, "value2":5999122195912397909, "value3":true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.251489, "value1":0.5593865051855272, "value2":5772018587784459447, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191087, "value1":0.2891985736646055, "value2":4399230413942175266, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087488, "value1":0.5425699812510858, "value2":3965285909604337765, "value3":true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680134, "value1":0.18266599397918218, "value2":5571070206339456089, "value3":true, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551156, "value1":0.8876027936303559, "value2":6010745863587972423, "value3":false, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.803161, "value1":0.9394184911877158, "value2":571658979107886488, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.054336, "value1":0.03957759543141472, "value2":3918244459790702522, "value3":true, "__name__":"metric_00000144", "key_3":"b","key_9":"e","key_0":"g", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685390, "value1":0.5264564714839759, "value2":2232108588919897592, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.161082, "value1":0.32998817001502934, "value2":8460874534662447333, "value3":false, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778535, "value1":0.06811024060980113, "value2":7835773491677016452, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832955, "value1":0.028669939437680664, "value2":4299578964430488535, "value3":true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004645, "value1":0.8188081000238006, "value2":9221293293613853860, "value3":false, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205474, "value1":0.40545870686171354, "value2":1575377844766226698, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639268, "value1":0.8908404617416188, "value2":1540158992274106467, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.139661, "value1":0.3668772992670227, "value2":6483649363648949947, "value3":true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924909, "value1":0.16190741337415684, "value2":5257368157508278570, "value3":true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297500, "value1":0.4330469601614637, "value2":4632520395480129342, "value3":false, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101765, "value1":0.4685291307083107, "value2":1614399458247460286, "value3":false, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291465, "value1":0.6213831794241761, "value2":7754123020970248013, "value3":true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369394, "value1":0.5153486584517465, "value2":2263160774029528717, "value3":false, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719680, "value1":0.5383036464533613, "value2":3589028105702605678, "value3":false, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262982, "value1":0.02336562179973987, "value2":2500396082328773649, "value3":false, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.790458, "value1":0.7125005920534319, "value2":8453423485925643300, "value3":true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.937011, "value1":0.9029577396659265, "value2":3493444148478497784, "value3":false, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815751, "value1":0.34331005780857, "value2":4896770691763395578, "value3":true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.732239, "value1":0.5781653462685661, "value2":2815554048397289034, "value3":true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073195, "value1":0.6318398759739543, "value2":1443128294631046938, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.335280, "value1":0.2900799960272814, "value2":319439076295666505, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375292, "value1":0.8416100389726322, "value2":1104228901216289966, "value3":true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543223, "value1":0.30539264418826334, "value2":2529051273510780065, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850347, "value1":0.9475422757893678, "value2":3815335775316587120, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010486, "value1":0.9233328326166781, "value2":1132448560381334143, "value3":true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509503, "value1":0.43078826324145314, "value2":2585538196535057958, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440516, "value1":0.36575868540627, "value2":764298189191108299, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.576334, "value1":0.9992971397552057, "value2":8607203945347812848, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.970781, "value1":0.43725394836722203, "value2":8039899278253467744, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746791, "value1":0.5653040341344706, "value2":6676447946324724349, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636245, "value1":0.8420862976065959, "value2":1562613302672270132, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285262, "value1":0.32142916834577895, "value2":5289001385309911194, "value3":false, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.417392, "value1":0.049995803596490404, "value2":6607894685909801752, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584296, "value1":0.2409216093884783, "value2":1637965971530230928, "value3":true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325720, "value1":0.028777043283946674, "value2":963675377332588860, "value3":false, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566004, "value1":0.9002123352889874, "value2":4604125644132750665, "value3":true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.882236, "value1":0.33330368402055666, "value2":156824449049228711, "value3":true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.767642, "value1":0.13763101488639068, "value2":8064121309553799557, "value3":true, "__name__":"metric_00000152", "key_2":"k","key_9":"h","key_0":"d", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719598, "value1":0.8986180062766446, "value2":1742955026017794842, "value3":false, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886873, "value1":0.010775899740746808, "value2":1351720915728762737, "value3":false, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484279, "value1":0.5158913513515657, "value2":6695543448765155807, "value3":true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.120818, "value1":0.38000311042969326, "value2":8387696033998291, "value3":false, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.471169, "value1":0.5449952216814127, "value2":7977583153767662789, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.692704, "value1":0.644651424950153, "value2":1344814886763463146, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434917, "value1":0.5336620566427158, "value2":7974250102837601477, "value3":true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.231472, "value1":0.18429722211173638, "value2":239717814769298968, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351874, "value1":0.3302574440246314, "value2":116725965202686758, "value3":false, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.652256, "value1":0.8832602328157859, "value2":2443207413955073666, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392058, "value1":0.3666546013150111, "value2":5949156227430002451, "value3":false, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940471, "value1":0.038777989285728266, "value2":2871780422161853348, "value3":true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412070, "value1":0.5197207843322247, "value2":406042026849178989, "value3":false, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571008, "value1":0.7027610078055025, "value2":1383746034138340292, "value3":false, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900415, "value1":0.17910816553895145, "value2":401749706711859248, "value3":false, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.718726, "value1":0.2276914412180115, "value2":9221447498523708978, "value3":false, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.381086, "value1":0.9677704240195594, "value2":1303001585301279474, "value3":true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893466, "value1":0.2579622668275943, "value2":3971563895105696581, "value3":false, "__name__":"metric_00000193", "key_9":"e","key_2":"h","key_4":"a", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.400237, "value1":0.23652480428212666, "value2":6009422694154041952, "value3":true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481908, "value1":0.625595272995919, "value2":2204722405376257638, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.936595, "value1":0.2680275935509345, "value2":324738255121972023, "value3":false, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.983646, "value1":0.3650479381459544, "value2":6655499432411034428, "value3":true, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.489964, "value1":0.6666717547984726, "value2":2225549269590513897, "value3":true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641209, "value1":0.578327662161616, "value2":6547750503066251015, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134486, "value1":0.67209109592655, "value2":2885152331577461710, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073911, "value1":0.08294753363850706, "value2":7590236594974272032, "value3":true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181170, "value1":0.5500056163737558, "value2":6771546940482209584, "value3":true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.029045, "value1":0.6184324246909119, "value2":8362437134419045707, "value3":true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162843, "value1":0.4243438443026443, "value2":8729664805668824103, "value3":true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.808018, "value1":0.9740191657641797, "value2":811057613633969138, "value3":true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509186, "value1":0.5860682567407373, "value2":4715370569695436166, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664283, "value1":0.9354682602818816, "value2":4771667149292350278, "value3":true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220707, "value1":0.9895245632099, "value2":7384968594835669289, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389342, "value1":0.5213334242916959, "value2":9064052527115170337, "value3":true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977547, "value1":0.9968413793490465, "value2":1018668683107683896, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122093, "value1":0.22725801593699438, "value2":471678573457243344, "value3":false, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474960, "value1":0.6106407332478643, "value2":6142821675445007328, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770612, "value1":0.38366813133853406, "value2":4771686788392964587, "value3":false, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166046, "value1":0.6731562334089229, "value2":6765933549203517189, "value3":true, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000435, "value1":0.7725469520964668, "value2":1526254098614108268, "value3":true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.596567, "value1":0.33143109459424, "value2":6988963074743099921, "value3":false, "__name__":"metric_00000215", "key_2":"f","key_0":"f","key_1":"h", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832348, "value1":0.5082718126919952, "value2":7366669828299681751, "value3":true, "__name__":"metric_00000216", "key_5":"j","key_7":"b","key_1":"j", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.933461, "value1":0.2177858524105436, "value2":1927680313692641282, "value3":true, "__name__":"metric_00000217", "key_4":"a","key_9":"h","key_1":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800386, "value1":0.37752422688739345, "value2":3623415676274096956, "value3":true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602632, "value1":0.9591290646171173, "value2":149144865605034806, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805853, "value1":0.46005104238615857, "value2":5399494644643471050, "value3":false, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023064, "value1":0.2589954384060476, "value2":2161234059148173478, "value3":true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.663052, "value1":0.4340276216577473, "value2":1651362675477804080, "value3":false, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425648, "value1":0.5284587146267693, "value2":2204338428823542625, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297837, "value1":0.9930278173662268, "value2":1153109222914733395, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.661230, "value1":0.3189372627835483, "value2":8950530915947476877, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997529, "value1":0.9931406569786665, "value2":3171376634461957806, "value3":true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785991, "value1":0.6115707472871859, "value2":4480386318218501447, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909727, "value1":0.29199552201380746, "value2":3686357880544622730, "value3":false, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155119, "value1":0.5470008889969509, "value2":7858820522318450111, "value3":false, "__name__":"metric_00000238", "key_7":"e","key_4":"a","key_5":"k", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.530629, "value1":0.9873625937302516, "value2":5471095001682678938, "value3":false, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512132, "value1":0.14465321479404997, "value2":8852396394387214520, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.643903, "value1":0.81607168234715, "value2":7750000008183770029, "value3":false, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088371, "value1":0.9834997161657205, "value2":8265123019714389041, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.840291, "value1":0.31720704974294733, "value2":8838522133270072113, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607394, "value1":0.6158114593013463, "value2":3920524769932432062, "value3":false, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.891225, "value1":0.7260707014711243, "value2":8545845285837175547, "value3":false, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155524, "value1":0.5324837252402642, "value2":6050562650984829349, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463368, "value1":0.3636429297340659, "value2":7366387480034568615, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.051267, "value1":0.21240586224005645, "value2":3520112151921939439, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.713935, "value1":0.6148054965944443, "value2":4435199374933360163, "value3":true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229994, "value1":0.877849746015665, "value2":4328772383519778289, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.394088, "value1":0.7419214909083843, "value2":1533570891823401045, "value3":false, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311200, "value1":0.6547222273874411, "value2":603076909255570468, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593460, "value1":0.33723617074445394, "value2":4429464301654712087, "value3":false, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016002, "value1":0.7850512802013139, "value2":9218293660844009579, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293893, "value1":0.9024192531760731, "value2":874814059564514510, "value3":false, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197015, "value1":0.6346018344137921, "value2":2275287342249871781, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444015, "value1":0.07476763645648904, "value2":6294420894284163263, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.137036, "value1":0.7149572150590823, "value2":5890716035999924313, "value3":false, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382203, "value1":0.2373007200794848, "value2":4237441552449229366, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927338, "value1":0.5067232894842505, "value2":7166709656823836078, "value3":false, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.856975, "value1":0.7551172442205076, "value2":3114426699624764241, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290618, "value1":0.461652630545808, "value2":6870191814019559455, "value3":true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194719, "value1":0.28972301156720964, "value2":2266210405139781750, "value3":false, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.930146, "value1":0.8041428167626514, "value2":3758081748375193976, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.876259, "value1":0.8601408243336739, "value2":8153013110856241749, "value3":true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544542, "value1":0.4588432483941638, "value2":1078220017088765885, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861984, "value1":0.33210767773987954, "value2":6423637803003548004, "value3":false, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725768, "value1":0.13821724766383348, "value2":2685436575194046233, "value3":true, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422624, "value1":0.6174306852475553, "value2":622178413482526248, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.749470, "value1":0.7804428540523304, "value2":6875918761098582661, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.903503, "value1":0.7328633216838554, "value2":3622697451467218798, "value3":true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.116528, "value1":0.9193757905595477, "value2":6857103271674313745, "value3":false, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318972, "value1":0.35822522818261543, "value2":8829601973990551809, "value3":true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433538, "value1":0.5317306440222773, "value2":2556858263334639046, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497594, "value1":0.1404686557588262, "value2":5182788796116063454, "value3":false, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.731708, "value1":0.08352722566425953, "value2":57349948699707298, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052659, "value1":0.5013772188087474, "value2":4075548779807119976, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680970, "value1":0.7475466041318248, "value2":804362071276609030, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644235, "value1":0.13991370513704515, "value2":8012476351028838411, "value3":false, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.430709, "value1":0.07096170614998987, "value2":810033186472659399, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.283029, "value1":0.49501003784947184, "value2":5335533898113400696, "value3":false, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.213964, "value1":0.06985861165056967, "value2":3236599269070483646, "value3":false, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899939, "value1":0.8432631992057813, "value2":8331656940556283499, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447604, "value1":0.4633606486187383, "value2":3282862092550905414, "value3":false, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986819, "value1":0.31082999792855004, "value2":5075360107904774840, "value3":true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375656, "value1":0.6617351656082016, "value2":8930874302935418560, "value3":false, "__name__":"metric_00000278", "key_5":"j","key_1":"c","key_4":"h", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787774, "value1":0.5242892599116226, "value2":4186364548758456678, "value3":false, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918288, "value1":0.3268848136936015, "value2":2614642728372801254, "value3":true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535191, "value1":0.176268908097137, "value2":653040557920632015, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529661, "value1":0.35212458808041835, "value2":1525600014751099598, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.361165, "value1":0.4714568379001177, "value2":4598533494631122989, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898032, "value1":0.22769956750018774, "value2":5631589457641828317, "value3":false, "__name__":"metric_00000283", "key_7":"c","key_1":"f","key_5":"f", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483853, "value1":0.4317121876757055, "value2":564210499810148536, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512418, "value1":0.14841146100443744, "value2":6375007633580482412, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172575, "value1":0.4457725144006962, "value2":940411418055250112, "value3":true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109679, "value1":0.17465206890579033, "value2":619954528132439983, "value3":false, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016784, "value1":0.6172432789077167, "value2":8694611599799563, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405055, "value1":0.9156938137116516, "value2":1197968795715040120, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.153068, "value1":0.6193599211433017, "value2":3101256038412073082, "value3":false, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041261, "value1":0.0029965595633104653, "value2":1337872063966920132, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.852796, "value1":0.021368920853982197, "value2":7987040863826958067, "value3":false, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921132, "value1":0.47595655072486737, "value2":1700855145658274194, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.383399, "value1":0.26371142871266584, "value2":6832011082222328508, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.332123, "value1":0.8824714441278855, "value2":4889159017232292596, "value3":true, "__name__":"metric_00000294", "key_5":"a","key_7":"j","key_0":"f", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847690, "value1":0.5825485094693434, "value2":8301349950043802905, "value3":false, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004022, "value1":0.963420472883698, "value2":4714062367056435117, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178319, "value1":0.21282157659328568, "value2":5741804038235426956, "value3":true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226035, "value1":0.614117409288991, "value2":7916317937107113091, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188200, "value1":0.6715893678504037, "value2":7136230473959043902, "value3":true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709476, "value1":0.7492754619759127, "value2":6914951663040962797, "value3":false, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879839, "value1":0.9373684799176515, "value2":3333434004719572493, "value3":true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174084, "value1":0.14484188554360505, "value2":2035242236177706450, "value3":false, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.505125, "value1":0.57329431354178, "value2":5119683575480371621, "value3":false, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742853, "value1":0.7629472821449486, "value2":5737967107040353117, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226226, "value1":0.17730887465740228, "value2":4673386690110010077, "value3":false, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560859, "value1":0.9766459300795906, "value2":1996912721458005085, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262060, "value1":0.05094335959612934, "value2":1245357682330480297, "value3":true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432269, "value1":0.8390455137158405, "value2":1878222408225434076, "value3":true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113677, "value1":0.7363256343069756, "value2":2885521712019062646, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444967, "value1":0.7955117384805845, "value2":264780152192144506, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162729, "value1":0.6712123871834628, "value2":1119778679298263778, "value3":false, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.911266, "value1":0.9891407670464004, "value2":9103652755878870157, "value3":true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379007, "value1":0.2889256375011702, "value2":5327006092087239478, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.092408, "value1":0.21632368500444782, "value2":3442219206298318473, "value3":true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032852, "value1":0.524214031146113, "value2":6530887848447144232, "value3":false, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382209, "value1":0.7113041138812657, "value2":933816269978698137, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.334638, "value1":0.7695287307455443, "value2":7489040093668157103, "value3":true, "__name__":"metric_00000318", "key_4":"d","key_7":"f","key_1":"e", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372671, "value1":0.7439130206835773, "value2":4959199765009131376, "value3":true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900273, "value1":0.9052840812209041, "value2":965461062104746729, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.654196, "value1":0.8121385984212723, "value2":5665296699467259244, "value3":true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178571, "value1":0.19948342559708582, "value2":5100240681564871548, "value3":false, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770922, "value1":0.3886846963277322, "value2":619589477307797843, "value3":false, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.623448, "value1":0.010709272715713756, "value2":2405417022197491275, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.501430, "value1":0.49322743442420075, "value2":4840878662359361238, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268501, "value1":0.9623480970104531, "value2":7490234459966550271, "value3":true, "__name__":"metric_00000326", "key_7":"b","key_9":"c","key_4":"g", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403202, "value1":0.7953121912476196, "value2":6585060526257826515, "value3":true, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078706, "value1":0.6781344722473363, "value2":3204877843782827661, "value3":true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.343846, "value1":0.7227864686817432, "value2":3271852738757870545, "value3":true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.026648, "value1":0.47137422416076813, "value2":3734918729615183416, "value3":true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.691254, "value1":0.7660887564670621, "value2":790090719852690753, "value3":true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262583, "value1":0.666841123381339, "value2":2785906959504169261, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.234934, "value1":0.28467055157973764, "value2":5751420159217569904, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110004, "value1":0.6775088488593821, "value2":3553287660015547262, "value3":false, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766422, "value1":0.23227174275211976, "value2":623571748203397381, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800936, "value1":0.31353934764817454, "value2":4270432397290565810, "value3":false, "__name__":"metric_00000336", "key_7":"e","key_0":"k","key_3":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709087, "value1":0.45637800363407555, "value2":6299983262198737196, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756471, "value1":0.4807492151985086, "value2":7279712801686741955, "value3":false, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188086, "value1":0.36885137660554584, "value2":7300782007111153833, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.948483, "value1":0.37900022039799186, "value2":659834049221243616, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.589486, "value1":0.790378840630986, "value2":5696488550645422430, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783617, "value1":0.7580291430060272, "value2":3320953914407054988, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321932, "value1":0.6227238552522466, "value2":8556762730182657496, "value3":true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585825, "value1":0.8340787864264869, "value2":2874883662014238947, "value3":true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425246, "value1":0.4329750194122766, "value2":6011155334107441536, "value3":true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023097, "value1":0.474467449495024, "value2":9096564500742879948, "value3":false, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625423, "value1":0.1682038891195739, "value2":4431847776626169211, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289832, "value1":0.9685809921059807, "value2":1593648503211538110, "value3":false, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613185, "value1":0.10083138549838662, "value2":6012379324790188749, "value3":true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720225, "value1":0.22688014499451828, "value2":533494308206781076, "value3":true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.925929, "value1":0.3145810371686551, "value2":2521880979655219825, "value3":false, "__name__":"metric_00000360", "key_5":"j","key_9":"f","key_0":"a", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073559, "value1":0.8510652378781342, "value2":2771699030834759801, "value3":true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540880, "value1":0.19703714704465763, "value2":2229904355406996428, "value3":true, "__name__":"metric_00000362", "key_1":"d","key_0":"c", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358372, "value1":0.47865214913286414, "value2":991745116280869642, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.985891, "value1":0.5712880806403865, "value2":7108232781064624938, "value3":false, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787182, "value1":0.10623777547287752, "value2":5757177518524775613, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725343, "value1":0.4881539523572101, "value2":9008775172737657154, "value3":true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.276129, "value1":0.7963976864976252, "value2":4233311333311619132, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.296009, "value1":0.9362536518865696, "value2":6472012225611436212, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572284, "value1":0.5256827892320936, "value2":6967364447645723612, "value3":true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490559, "value1":0.5321215426269078, "value2":6272951345279554009, "value3":false, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.897929, "value1":0.9148681755770618, "value2":8519086271296505773, "value3":true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438881, "value1":0.9657537469163081, "value2":1825676918798292475, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.878233, "value1":0.44284605438890395, "value2":948269923000784878, "value3":false, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.240505, "value1":0.2373413838092857, "value2":510374534141290052, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041417, "value1":0.17592672383423122, "value2":3473808065501665173, "value3":true, "__name__":"metric_00000345", "key_8":"k","key_9":"a","key_1":"j", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191879, "value1":0.3751219146996357, "value2":7106698191372720163, "value3":true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694969, "value1":0.8445486295845246, "value2":672504742133677359, "value3":true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974994, "value1":0.6654201856510823, "value2":4512443800515268537, "value3":false, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.265020, "value1":0.3977309051692021, "value2":987280456712575618, "value3":true, "__name__":"metric_00000350", "key_9":"d","key_0":"f","key_7":"k", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.837282, "value1":0.6667916772962499, "value2":2649317741597325862, "value3":false, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584752, "value1":0.8184998685164273, "value2":6853243233010504233, "value3":true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956207, "value1":0.20740750871245983, "value2":3701599145151889680, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785977, "value1":0.42059150301577825, "value2":4300206568693523992, "value3":false, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206794, "value1":0.4065971742289675, "value2":5055750687883280777, "value3":true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.466356, "value1":0.24692397834834437, "value2":6503690772355582431, "value3":false, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374485, "value1":0.27439926954774063, "value2":2257827689674118963, "value3":false, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859317, "value1":0.15734285025101918, "value2":2188339349886556422, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.888728, "value1":0.2275635074824085, "value2":3959472993254545104, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552286, "value1":0.8140316248314068, "value2":28562363718590634, "value3":false, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101732, "value1":0.9986972418263012, "value2":6625998065774179710, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.064398, "value1":0.4719546471748007, "value2":5016107261432878232, "value3":false, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540095, "value1":0.05564657854486056, "value2":2892021965316178106, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052034, "value1":0.6085312696710864, "value2":267981962928859635, "value3":true, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479153, "value1":0.1864971302120239, "value2":4274890765687650257, "value3":true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285637, "value1":0.8618179927379825, "value2":98273395449385741, "value3":true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160949, "value1":0.1665709809492651, "value2":3549197933610586412, "value3":true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479807, "value1":0.9792753435060556, "value2":544874846395596701, "value3":true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.715679, "value1":0.955575677088658, "value2":7351930305259561505, "value3":true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655220, "value1":0.7716326625888155, "value2":490280663710319116, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799754, "value1":0.26438748028137354, "value2":2889376751051793411, "value3":false, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625103, "value1":0.1721904108442133, "value2":7991543743244408523, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.690861, "value1":0.007633155868195746, "value2":3073219921658712367, "value3":false, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308852, "value1":0.12391196742809665, "value2":1115580635235450437, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.013227, "value1":0.7267200692886595, "value2":1429240573369570759, "value3":false, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268822, "value1":0.6684636760714637, "value2":4339881778709198457, "value3":false, "__name__":"metric_00000396", "key_5":"f","key_2":"i","key_3":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160477, "value1":0.5396173638949372, "value2":4906545989468275563, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599858, "value1":0.07893489799204395, "value2":410345370022207222, "value3":true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584892, "value1":0.8068626341311561, "value2":4979004915398024459, "value3":true, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.987465, "value1":0.8935603600443448, "value2":8149710572361609309, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227032, "value1":0.3687576237691273, "value2":1391051851919204878, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833201, "value1":0.32203470987301264, "value2":2081318068679055279, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291063, "value1":0.034071004249405484, "value2":5342977357585868177, "value3":false, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290178, "value1":0.3044083195032883, "value2":4841141697891251443, "value3":true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109841, "value1":0.9228991323104464, "value2":1691775942527270832, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587504, "value1":0.5332728718746681, "value2":1334529969932810461, "value3":true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.920462, "value1":0.18350284649923848, "value2":2590094361323084553, "value3":true, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.966666, "value1":0.25923537198192, "value2":6116285159120956507, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480100, "value1":0.8741323699909098, "value2":1633286453079731557, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.175764, "value1":0.46174408884851265, "value2":5588956365079776400, "value3":true, "__name__":"metric_00000410", "key_7":"j","key_1":"i","key_5":"h", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357949, "value1":0.9471421832392611, "value2":2020538223652392065, "value3":false, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087613, "value1":0.8391096943297279, "value2":1883071449916289062, "value3":true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883487, "value1":0.5961374967262479, "value2":6748555704132728387, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.514904, "value1":0.43082646153613147, "value2":3827154946167603504, "value3":true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899576, "value1":0.926252451999718, "value2":6912098010507428750, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.708901, "value1":0.5214095428253637, "value2":8239976493295716264, "value3":false, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.784267, "value1":0.7119074977387614, "value2":5875315435697946397, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318238, "value1":0.2857683698624022, "value2":2168440855638261641, "value3":false, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434779, "value1":0.728469975577827, "value2":4027120624532411802, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432104, "value1":0.9159655292186285, "value2":1266477315347168519, "value3":true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.453551, "value1":0.9820731220635184, "value2":472902679452044251, "value3":false, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.239100, "value1":0.05657570742681277, "value2":1082500030056759801, "value3":true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918697, "value1":0.5552870083342735, "value2":3402695849932536811, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689901, "value1":0.372073062318352, "value2":4891331242898882829, "value3":false, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924258, "value1":0.5283619937827609, "value2":559122472611611400, "value3":false, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172686, "value1":0.4344219735503553, "value2":1248792109975059381, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894193, "value1":0.43426860175202114, "value2":3497813026288966485, "value3":true, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.327909, "value1":0.3878605121485491, "value2":5047756483827083362, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220825, "value1":0.07655058465784402, "value2":5252985818354538541, "value3":false, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742213, "value1":0.38430981024473343, "value2":5620787295533246773, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.733232, "value1":0.5555223164703845, "value2":631270712021422829, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940662, "value1":0.3098977203238577, "value2":5954879855194614083, "value3":true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.649184, "value1":0.3354932964670884, "value2":6051779552546236650, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832762, "value1":0.9063274954717605, "value2":3724265762255192701, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.385652, "value1":0.5036946685314467, "value2":5109085475601380385, "value3":false, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.722579, "value1":0.772640832685559, "value2":3780594965713240464, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655329, "value1":0.29356054638145535, "value2":4596604760791157242, "value3":false, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.459104, "value1":0.8777447945394818, "value2":1186981345641271839, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.399975, "value1":0.025674080171056102, "value2":8964167090414725476, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.123918, "value1":0.9272811881045043, "value2":7218529778562540903, "value3":true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747920, "value1":0.9264629051459597, "value2":8464913590880556582, "value3":false, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914960, "value1":0.32245869851475545, "value2":9109063864698425563, "value3":false, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.328190, "value1":0.7652891376156531, "value2":3024374861601192187, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593397, "value1":0.17904346621073747, "value2":3899031855173978925, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229837, "value1":0.5384685110923013, "value2":5235158233416481121, "value3":true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.698328, "value1":0.5982638259890453, "value2":6740778632490166175, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812614, "value1":0.1666081159883058, "value2":2800734738787135789, "value3":true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352669, "value1":0.33792886940688555, "value2":7496164581714310216, "value3":true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128711, "value1":0.2991783007626336, "value2":7338677993190210677, "value3":false, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552539, "value1":0.06968528327878533, "value2":8300480680068139329, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.581708, "value1":0.35231544880285015, "value2":1384864101595986439, "value3":false, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778952, "value1":0.8415964882413857, "value2":1745115595251159346, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.695991, "value1":0.5753223257206643, "value2":1886601101964605359, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664273, "value1":0.6725491194351899, "value2":5574699884022596470, "value3":true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711367, "value1":0.2595939895449639, "value2":7814193120722379636, "value3":true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.600545, "value1":0.944128951867823, "value2":6711650903238481221, "value3":false, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452146, "value1":0.9554010846729131, "value2":3169390990279464268, "value3":true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.140642, "value1":0.16345235341026784, "value2":2148518750069655597, "value3":true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388132, "value1":0.5979152819074895, "value2":7002872128153528233, "value3":true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078323, "value1":0.5697531979956287, "value2":980455479278929153, "value3":true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499938, "value1":0.388179798040964, "value2":2192712143484318261, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761109, "value1":0.2600517751619912, "value2":647733851046806648, "value3":false, "__name__":"metric_00000462", "key_9":"d","key_6":"d","key_8":"g", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088032, "value1":0.2162359479396891, "value2":7960761017907576840, "value3":true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886480, "value1":0.21950291960892157, "value2":7881441150954657573, "value3":true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.538255, "value1":0.36627249562354336, "value2":2697185681710870928, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956144, "value1":0.09482810661100612, "value2":3142048329987868522, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.610158, "value1":0.7330550786147262, "value2":4897289423318138878, "value3":true, "__name__":"metric_00000466", "key_5":"g","key_7":"h","key_0":"b", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641699, "value1":0.27867439634792107, "value2":621712637914424018, "value3":true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.218950, "value1":0.1329184934440547, "value2":3565116595465876544, "value3":true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.247888, "value1":0.7166338804544903, "value2":5367509956565676215, "value3":false, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.214416, "value1":0.3332699601933243, "value2":4699030477990361670, "value3":true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023652, "value1":0.5060732821933588, "value2":1384885559250896583, "value3":true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907328, "value1":0.1694015036022409, "value2":3592921690010847117, "value3":true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.642558, "value1":0.737568796988017, "value2":2406767446522063712, "value3":false, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273826, "value1":0.8447613680154935, "value2":6080438111270024688, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.984506, "value1":0.28262866688772725, "value2":6480278416681893463, "value3":false, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.868136, "value1":0.8910201029103536, "value2":6867531047341490726, "value3":false, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687877, "value1":0.8829884505686191, "value2":2249445178190997362, "value3":false, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850677, "value1":0.6276341510695712, "value2":2252184121487700107, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737726, "value1":0.12597210262772834, "value2":1348511802277935140, "value3":false, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.810752, "value1":0.2102822839010783, "value2":4831043683200695850, "value3":true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317105, "value1":0.817061302570146, "value2":8324179065679406640, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.426069, "value1":0.09982797254913842, "value2":6519984536301213031, "value3":true, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.121073, "value1":0.5197148618479782, "value2":3010915938164137229, "value3":false, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.264976, "value1":0.4225812260629186, "value2":5303229136813956536, "value3":true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572439, "value1":0.07380111555778088, "value2":4428887607089365223, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526101, "value1":0.2368198468823957, "value2":5676247054773494672, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.794040, "value1":0.8488571887152928, "value2":7926265788784400661, "value3":true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682501, "value1":0.6419011526811798, "value2":6127958830288275598, "value3":false, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.508833, "value1":0.08505936393863323, "value2":6956791791775335476, "value3":true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.485450, "value1":0.24440134625086388, "value2":8675077803244715182, "value3":false, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341184, "value1":0.7204080689080778, "value2":7288860462608592695, "value3":false, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300497, "value1":0.587674524187528, "value2":2136914643668219995, "value3":false, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646074, "value1":0.5687487761791935, "value2":5117858885820847437, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405215, "value1":0.63703146651586, "value2":8601605955587466200, "value3":true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040361, "value1":0.9056350649746174, "value2":4395139702119619272, "value3":true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042190, "value1":0.25258468888663055, "value2":6275160558122438530, "value3":true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436823, "value1":0.3052140189316623, "value2":1176588437754725430, "value3":true, "__name__":"metric_00000502", "key_9":"c","key_2":"d","key_4":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.885585, "value1":0.996098364991821, "value2":7425087259003526051, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.667291, "value1":0.01567259898271256, "value2":688757342279745926, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560861, "value1":0.8815928724937075, "value2":5127701164424889645, "value3":false, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921488, "value1":0.5280525030917018, "value2":8702967906065448733, "value3":false, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906624, "value1":0.204252621066632, "value2":7869988567138398448, "value3":false, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.855702, "value1":0.956230561465019, "value2":157946491688041108, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367342, "value1":0.6684623495570281, "value2":329650868812150871, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445102, "value1":0.6894360816538613, "value2":4920916512388188612, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.609045, "value1":0.0027756305160863397, "value2":5637928110041308291, "value3":true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.632075, "value1":0.04166500110036427, "value2":6157318199778992604, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972254, "value1":0.31480609981503055, "value2":4364758259634809973, "value3":true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.776493, "value1":0.13104837145124149, "value2":4732079918184171170, "value3":true, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238222, "value1":0.6559225899889957, "value2":3576156993215909053, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434447, "value1":0.24817502461178453, "value2":7546679218765777183, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040709, "value1":0.1860292914746309, "value2":1580899927050287482, "value3":false, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273335, "value1":0.9526744406123745, "value2":5086267667987270481, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052913, "value1":0.33164061518472737, "value2":6494564179935068135, "value3":true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093729, "value1":0.8324601543372498, "value2":4840824205416424699, "value3":false, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.167756, "value1":0.8292588795543363, "value2":926691352131787790, "value3":true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602891, "value1":0.7468923005026717, "value2":1499084894678948825, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939687, "value1":0.9093420550399373, "value2":7624268916058145495, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859436, "value1":0.018264976223293757, "value2":8966202800040990455, "value3":false, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540853, "value1":0.15398968210956118, "value2":1169217513145417422, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927943, "value1":0.6497527028232101, "value2":8964902118693660858, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032864, "value1":0.5440150103669782, "value2":3398396102381979841, "value3":true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440858, "value1":0.5805592700724092, "value2":598543749015746918, "value3":false, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220522, "value1":0.7634462305829404, "value2":4744586200886838973, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350814, "value1":0.8753641286301881, "value2":3594959380973533940, "value3":true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689230, "value1":0.11698455183819528, "value2":3362979353229964321, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369880, "value1":0.9959022990686378, "value2":913069268809019275, "value3":true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499935, "value1":0.42435230326696954, "value2":5642082683494178997, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.126493, "value1":0.24855180288733794, "value2":2553926153696879833, "value3":false, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241812, "value1":0.19715345400514303, "value2":7966035951898297683, "value3":true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.282709, "value1":0.26568871987724985, "value2":5614603759761174315, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634974, "value1":0.5611667195638738, "value2":719923428562838874, "value3":false, "__name__":"metric_00000533", "key_9":"f","key_2":"i","key_3":"b", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.437429, "value1":0.044098595487335565, "value2":7598428545433157434, "value3":false, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986084, "value1":0.15643080062821366, "value2":876340157580973000, "value3":false, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350215, "value1":0.5350647081004255, "value2":2039458830523373741, "value3":false, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.263790, "value1":0.13663992546983536, "value2":4887595679013193650, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761252, "value1":0.31778054800300737, "value2":4963237153707981360, "value3":false, "__name__":"metric_00000547", "key_5":"k","key_0":"k","key_1":"c", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358509, "value1":0.4977744678036884, "value2":2417156657544845857, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.825502, "value1":0.7927543298826962, "value2":7803607945365925463, "value3":true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463820, "value1":0.25407613728207273, "value2":7298874514736284603, "value3":true, "__name__":"metric_00000550", "key_7":"d","key_5":"f", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815985, "value1":0.03419039527127868, "value2":2983524479323907938, "value3":false, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648209, "value1":0.5111390924658735, "value2":4469943396931617311, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560198, "value1":0.3027175063306067, "value2":8582749896805213520, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293707, "value1":0.7381247559612524, "value2":267296433337218156, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436607, "value1":0.1678795867934854, "value2":2882160820519571353, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516040, "value1":0.02439990616294445, "value2":3459503829097802670, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861717, "value1":0.8549050337904404, "value2":5330118776081944380, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181801, "value1":0.014687212605387332, "value2":1741497345355098408, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398203, "value1":0.3791649455492745, "value2":7827883443566154722, "value3":true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004152, "value1":0.4627140852343998, "value2":5127693472090561791, "value3":false, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289305, "value1":0.9473077760244797, "value2":1972308969161346102, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.407619, "value1":0.22478804543381714, "value2":4393947727507143193, "value3":true, "__name__":"metric_00000553", "key_8":"g","key_5":"c","key_7":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500355, "value1":0.9148638467317476, "value2":2930801640270658847, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390931, "value1":0.03756752782865913, "value2":6450238118865267877, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436518, "value1":0.7302171451231968, "value2":8892033149379515881, "value3":false, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352126, "value1":0.6345536180749114, "value2":8076417892766647115, "value3":false, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211476, "value1":0.7869548859510007, "value2":2137142575900025284, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.620465, "value1":0.04777740137317064, "value2":2493346722675519040, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166994, "value1":0.5908047667032886, "value2":2133480026426783424, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799110, "value1":0.7447928303219921, "value2":6367274514210109366, "value3":true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.258756, "value1":0.5720021685376423, "value2":7819250772844902436, "value3":true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178485, "value1":0.22728962622611987, "value2":5810329748220466185, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756847, "value1":0.09118290676810949, "value2":2438579961381457911, "value3":true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447596, "value1":0.2813020748043202, "value2":4446085339672311465, "value3":false, "__name__":"metric_00000572", "key_6":"b","key_7":"i","key_1":"d", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.507148, "value1":0.8635348228135215, "value2":7235169187543778395, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412107, "value1":0.6168652299086078, "value2":8460560637760416654, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.916980, "value1":0.20506020965429841, "value2":1044309486752695204, "value3":true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358540, "value1":0.5251096014647666, "value2":6638181287796109637, "value3":false, "__name__":"metric_00000576", "key_5":"b","key_9":"c","key_2":"k", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.199047, "value1":0.6942630163772678, "value2":3410541935645339151, "value3":true, "__name__":"metric_00000577", "key_5":"h","key_6":"j","key_1":"g", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.754827, "value1":0.9837922147041048, "value2":3690131697027385202, "value3":false, "__name__":"metric_00000578", "key_4":"b","key_5":"h","key_2":"a", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.329509, "value1":0.8271404181330777, "value2":4227974754451880800, "value3":false, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.513229, "value1":0.672862073275747, "value2":3810363550665232995, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910574, "value1":0.9390557529752215, "value2":5133678849432201178, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181846, "value1":0.49307852075676845, "value2":6137025771035845818, "value3":false, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572559, "value1":0.2408765387072259, "value2":8298872358883983458, "value3":true, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078241, "value1":0.08393417297668128, "value2":8837983950648920911, "value3":true, "__name__":"metric_00000561", "key_9":"e","key_4":"k", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.056551, "value1":0.9755466271988745, "value2":2207490383904730952, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.814175, "value1":0.5043585864165596, "value2":5350947527006645760, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.066736, "value1":0.5177229679064868, "value2":2703347438865819514, "value3":false, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560402, "value1":0.4478041012053205, "value2":117552149576626326, "value3":false, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079515, "value1":0.133747701259907, "value2":5643720599026556636, "value3":false, "__name__":"metric_00000581", "key_9":"h","key_2":"f","key_3":"e", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246265, "value1":0.8787847052412423, "value2":8596628070583162060, "value3":true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.011510, "value1":0.3733460731999948, "value2":6601446014598669145, "value3":true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735379, "value1":0.9116679871732262, "value2":5605502556155572766, "value3":true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.574257, "value1":0.3506079241295861, "value2":1800090074854288106, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900580, "value1":0.8930823791186644, "value2":1843480248185265031, "value3":false, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.744935, "value1":0.754485340555253, "value2":1527411985430067052, "value3":false, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369195, "value1":0.7284643223167715, "value2":2551804296404453319, "value3":false, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500109, "value1":0.1715238288854149, "value2":3468590424262559119, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196638, "value1":0.4131391791311713, "value2":2036494112995161182, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660034, "value1":0.5162081878070082, "value2":683174465375000181, "value3":false, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.324311, "value1":0.6392052032119463, "value2":6854067358310903979, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477405, "value1":0.9223995048777075, "value2":8678575474954356403, "value3":true, "__name__":"metric_00000593", "key_5":"k","key_6":"b","key_3":"e", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894983, "value1":0.2879379950878977, "value2":1760029872301364905, "value3":true, "__name__":"metric_00000596", "key_8":"b","key_0":"a","key_4":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404612, "value1":0.5964910019926303, "value2":5352075776414002891, "value3":false, "__name__":"metric_00000595", "key_6":"e","key_1":"h","key_2":"j", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.406123, "value1":0.9911934759109439, "value2":2505572631856818583, "value3":false, "__name__":"metric_00000598", "key_5":"c","key_0":"a","key_4":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098160, "value1":0.29265324156352274, "value2":673331618670269188, "value3":true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317156, "value1":0.5284207406945622, "value2":5801235018377104530, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.278954, "value1":0.8339844494736254, "value2":142783045099103297, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144316, "value1":0.5981739630713123, "value2":4376705426457644175, "value3":false, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.533900, "value1":0.7048057967645066, "value2":7668900249344587055, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655047, "value1":0.9572743468227054, "value2":8615116979377517835, "value3":false, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.164063, "value1":0.12955196466904195, "value2":8720744967395487030, "value3":false, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211413, "value1":0.4645239019949383, "value2":1341011821671689315, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.254073, "value1":0.9282656526373411, "value2":5675449932048560089, "value3":true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939659, "value1":0.3774328021259814, "value2":7756643154892601396, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.789551, "value1":0.7757594275737888, "value2":8309777310097157940, "value3":false, "__name__":"metric_00000617", "key_6":"a","key_4":"c","key_5":"j", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.256014, "value1":0.7599489616826112, "value2":210625827724423434, "value3":true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883947, "value1":0.13842755442158153, "value2":5303830768512577958, "value3":true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516070, "value1":0.17914029301657106, "value2":1056091608874344104, "value3":false, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.446988, "value1":0.27173267141582086, "value2":6180352463990379834, "value3":false, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.622518, "value1":0.8672352642207358, "value2":6931279306288268314, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.002662, "value1":0.25623553544743577, "value2":4114415105877902794, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.250822, "value1":0.1691133676662619, "value2":8315894568290440082, "value3":true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148935, "value1":0.5989068031663785, "value2":916798823552491100, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404397, "value1":0.3854563629426573, "value2":1729425108588138938, "value3":false, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155114, "value1":0.42561573956387977, "value2":3412690494545228649, "value3":false, "__name__":"metric_00000627", "key_6":"f","key_4":"e","key_5":"k", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.185911, "value1":0.7064647650423553, "value2":6268941778110729170, "value3":false, "__name__":"metric_00000628", "key_4":"i","key_7":"b","key_1":"g", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.349779, "value1":0.9255472682273418, "value2":1166345991907977547, "value3":false, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.982678, "value1":0.6891602632892445, "value2":1740831383850752805, "value3":false, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914669, "value1":0.4165113125409203, "value2":1826795354292330747, "value3":true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871167, "value1":0.7086334187759732, "value2":5543325343112219047, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.762690, "value1":0.8505896053437381, "value2":5927978325036218166, "value3":false, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.292162, "value1":0.6672535143315814, "value2":1327915299266347941, "value3":false, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311132, "value1":0.47941008598606394, "value2":7437258572827323548, "value3":false, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523170, "value1":0.7404188223668129, "value2":5505665204112874175, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357326, "value1":0.9538155003965737, "value2":2702198679896121556, "value3":false, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.364597, "value1":0.17680019955559403, "value2":1753707621503560793, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452318, "value1":0.2992929781674707, "value2":8362411304781786581, "value3":true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518625, "value1":0.8647905294441987, "value2":5431066155038766988, "value3":true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.355524, "value1":0.6116936780714586, "value2":1234552975837284329, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542514, "value1":0.8351923165774915, "value2":1733954151005989610, "value3":false, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979827, "value1":0.2892109226113143, "value2":6049795733820800226, "value3":true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883185, "value1":0.011744889551642781, "value2":141159892430595908, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.223860, "value1":0.4073953516179897, "value2":6458792499529133548, "value3":false, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049058, "value1":0.12482723086065665, "value2":9009567363074084595, "value3":true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.298389, "value1":0.4827417824302383, "value2":4841281853843733128, "value3":false, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.629693, "value1":0.6591071306029623, "value2":3355181768485869219, "value3":true, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.147730, "value1":0.34536474665893674, "value2":3191505597112674507, "value3":false, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.179092, "value1":0.7637896583466016, "value2":1179889369576374895, "value3":true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.658439, "value1":0.9209377727035774, "value2":3422449294756383390, "value3":false, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.071622, "value1":0.28143127213081115, "value2":7766511653375780058, "value3":false, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537938, "value1":0.6024534595610256, "value2":8547099742960318907, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646788, "value1":0.20378541750257417, "value2":7978976103929383445, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747803, "value1":0.10859041477345463, "value2":1109588376642858177, "value3":false, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564874, "value1":0.23647658766118684, "value2":6863917686945126643, "value3":false, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042244, "value1":0.3078158837684314, "value2":4419488498667340703, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.284250, "value1":0.9387293705744738, "value2":6769306683790898588, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479123, "value1":0.4851789805412465, "value2":6507016270927900733, "value3":true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025704, "value1":0.8538216321323174, "value2":4020894753483999383, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833612, "value1":0.2626123684082926, "value2":5033990782030407302, "value3":false, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.881634, "value1":0.935752736774716, "value2":6384512508877447191, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474427, "value1":0.8815963618627093, "value2":8929077315195001710, "value3":true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321721, "value1":0.8079053030903706, "value2":7509708901019477626, "value3":false, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.854474, "value1":0.6950357389111533, "value2":6524710922297840763, "value3":false, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229889, "value1":0.6795085741944863, "value2":5609613021607849758, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.988904, "value1":0.5223586661895601, "value2":89977508053442787, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374340, "value1":0.755589597933608, "value2":7955537057313043569, "value3":false, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639189, "value1":0.7408346971613415, "value2":1902500838340597252, "value3":false, "__name__":"metric_00000669", "key_9":"d","key_4":"c","key_5":"e", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481547, "value1":0.5004511492728786, "value2":164083172923357290, "value3":false, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518019, "value1":0.729605886464792, "value2":9068451454110926149, "value3":false, "__name__":"metric_00000671", "key_7":"b","key_1":"c","key_5":"d", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.363805, "value1":0.7382344993077278, "value2":7045722716338732866, "value3":true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.630886, "value1":0.688064557117447, "value2":296768996976239880, "value3":false, "__name__":"metric_00000673", "key_8":"k","key_4":"g","key_7":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.845466, "value1":0.13213135497669065, "value2":4879659946939633689, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660200, "value1":0.2229370900047786, "value2":292379935200020145, "value3":true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073732, "value1":0.8115899146610889, "value2":8577867893469971833, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.019908, "value1":0.49801059455223595, "value2":7683222481005280397, "value3":false, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.347447, "value1":0.9276573673104045, "value2":2091335224644651701, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321409, "value1":0.6126771681057066, "value2":1893407571657392417, "value3":true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.738125, "value1":0.3569881382718092, "value2":4029761814756595416, "value3":false, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.665842, "value1":0.05170172426704011, "value2":8174581603042814007, "value3":false, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.449347, "value1":0.7466823594518665, "value2":6257307241351449210, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.588421, "value1":0.4753472682300776, "value2":4728406440983577982, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.257494, "value1":0.1386821629125743, "value2":332192645116862048, "value3":false, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.728108, "value1":0.43976058351020564, "value2":1985435892430278941, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.209628, "value1":0.8966889425339122, "value2":5787236380684241489, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.638851, "value1":0.9617304712840572, "value2":4707670589409962460, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484422, "value1":0.020940662405038862, "value2":1545134502951937463, "value3":false, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440131, "value1":0.17343525627253645, "value2":6009630316902812876, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229422, "value1":0.02146978522387302, "value2":2111892615484907893, "value3":true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780591, "value1":0.6377996056471382, "value2":8896668703689799570, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.124061, "value1":0.7970190759525996, "value2":7452754965853045737, "value3":true, "__name__":"metric_00000663", "key_2":"i","key_4":"g","key_0":"j", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.450958, "value1":0.8563337518192949, "value2":1606820680288550386, "value3":true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194998, "value1":0.8092586961600053, "value2":1104653518579575826, "value3":false, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746061, "value1":0.888221791516023, "value2":6881584298104407367, "value3":true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.550870, "value1":0.45584434378222843, "value2":7394447363524184561, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.684022, "value1":0.3067771551792505, "value2":1245627384806850593, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289452, "value1":0.4582593754084005, "value2":5465272523370779338, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235677, "value1":0.8840876989101745, "value2":551093905717196985, "value3":false, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.890313, "value1":0.09198263253620918, "value2":6028451346704401787, "value3":true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.168914, "value1":0.40840563117271195, "value2":3931349792536134233, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.717876, "value1":0.7410792258916561, "value2":5169350845698434472, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.202605, "value1":0.5290546996961457, "value2":3475693437398263961, "value3":true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636633, "value1":0.824130818871441, "value2":5903284123492948390, "value3":true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.615008, "value1":0.04083331973299552, "value2":7577674506757874978, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036559, "value1":0.5696805792367668, "value2":4633771544842561674, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367723, "value1":0.355783015228386, "value2":5675601540429514938, "value3":false, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907306, "value1":0.5698956093759964, "value2":9122457118633339454, "value3":true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685953, "value1":0.4276033626449204, "value2":7616472412744453525, "value3":true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587193, "value1":0.9520532910468696, "value2":2039460897796769985, "value3":false, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.322100, "value1":0.9372598841781506, "value2":8025865217929956411, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155211, "value1":0.2725517811852112, "value2":7680103519195376114, "value3":false, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964045, "value1":0.49419337991273576, "value2":8605447057400658561, "value3":false, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.423455, "value1":0.28782941494899417, "value2":7631463815755268967, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433388, "value1":0.6722143467332055, "value2":4743404486443339981, "value3":false, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073044, "value1":0.9919783220125814, "value2":3546009072923452494, "value3":true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060903, "value1":0.09931972482798261, "value2":3396822536822006988, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.952732, "value1":0.3775686488599951, "value2":7121618792612707640, "value3":true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521690, "value1":0.5851273414170087, "value2":4662473906180780029, "value3":false, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227468, "value1":0.11958534947502218, "value2":3788112644660727919, "value3":false, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196785, "value1":0.8966564462732959, "value2":4357533587354590088, "value3":false, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901366, "value1":0.5571789697475874, "value2":7759726578542538969, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.633498, "value1":0.9521589976099354, "value2":3412636563124759184, "value3":true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477196, "value1":0.6345321514502629, "value2":2968047284799701894, "value3":true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.245970, "value1":0.8602524837567829, "value2":239564384927138993, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923521, "value1":0.1090517952569125, "value2":8200798429042044530, "value3":false, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648242, "value1":0.5170091370557403, "value2":2725553204805480592, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812069, "value1":0.8884437199979248, "value2":9154279472010215199, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962857, "value1":0.2140414709499265, "value2":5080056778998450413, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.561764, "value1":0.6433379926959468, "value2":1246414196804721612, "value3":true, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288689, "value1":0.8910605919928337, "value2":8094720078703937078, "value3":true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.419833, "value1":0.6705423885099231, "value2":6631047350002617679, "value3":true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.085812, "value1":0.03831227212231745, "value2":2735730201464302258, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.046896, "value1":0.09940708362873886, "value2":600170391537990700, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.884000, "value1":0.5332485748572247, "value2":5278799395808920103, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.942254, "value1":0.6825570254740958, "value2":3505708958239891309, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010039, "value1":0.007884797092278711, "value2":3745225427592325971, "value3":true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.712360, "value1":0.2908960224193339, "value2":9147580185923316377, "value3":true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109158, "value1":0.3741670076640341, "value2":1005531285446043532, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680722, "value1":0.010722209112304005, "value2":826737562607372966, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.439540, "value1":0.4713372686055574, "value2":6754252699406092830, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392281, "value1":0.9898544758145399, "value2":2353492636354645276, "value3":false, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205843, "value1":0.6788358030320953, "value2":712272724855746801, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003145, "value1":0.7114317643988356, "value2":3716874444298014530, "value3":true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.133514, "value1":0.37586422274898074, "value2":2416361460381490648, "value3":false, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.904105, "value1":0.18756095304630882, "value2":6928964256284052382, "value3":false, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171387, "value1":0.6820596249231228, "value2":172259903859496202, "value3":true, "__name__":"metric_00000736", "key_6":"e","key_0":"i","key_4":"k", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.510204, "value1":0.06734756409519246, "value2":7819819964334132673, "value3":true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422546, "value1":0.1465987153240839, "value2":3085770726305992826, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.039220, "value1":0.39594354666033404, "value2":6309816101603497959, "value3":false, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.739730, "value1":0.8530985778840952, "value2":7379770578925142492, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.672747, "value1":0.7865961192571739, "value2":7774331371169359686, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.769896, "value1":0.802526565041674, "value2":3838614021162591095, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.304485, "value1":0.6949310789221889, "value2":7501685449967872481, "value3":false, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.678592, "value1":0.7504248678928968, "value2":7218481023042482207, "value3":true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764761, "value1":0.20348098581370722, "value2":5988835601145003290, "value3":true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.968921, "value1":0.21895244723583734, "value2":6407608012483731307, "value3":true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480313, "value1":0.3910222274917921, "value2":935230396616102910, "value3":true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680717, "value1":0.7679413590875351, "value2":6737105603635406237, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843792, "value1":0.928935788622385, "value2":8032690240756723112, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544961, "value1":0.3803375288284628, "value2":808064213573953017, "value3":false, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974943, "value1":0.2873027943668844, "value2":420303650121753052, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351352, "value1":0.9011332762961933, "value2":5787708950335762277, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049110, "value1":0.920403506389589, "value2":3542056558848585785, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.473874, "value1":0.20499658001386006, "value2":6473623955229286943, "value3":true, "__name__":"metric_00000749", "key_9":"e","key_5":"f","key_6":"c", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705527, "value1":0.3369786416226785, "value2":5565389573084278676, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977283, "value1":0.5753145958771464, "value2":5509805379215659395, "value3":true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.969055, "value1":0.7944991097395847, "value2":5227490476089768493, "value3":true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.260822, "value1":0.37247993171967786, "value2":9100748121410350084, "value3":false, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392300, "value1":0.9170516775216153, "value2":3337181235028553814, "value3":false, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.917458, "value1":0.7005088931842404, "value2":4249344436835260528, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398079, "value1":0.4412594689419285, "value2":355338602844953023, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605493, "value1":0.14953313696992326, "value2":7522955956227719413, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.947612, "value1":0.24504972733419753, "value2":19900356272350185, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.657164, "value1":0.7554151847082919, "value2":2083009730678034907, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.303410, "value1":0.9047440140278514, "value2":2665949305315254597, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720136, "value1":0.02233304294884983, "value2":2148351116720732280, "value3":false, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605342, "value1":0.06576250199061187, "value2":3149195196917185744, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.249421, "value1":0.10981969871005968, "value2":1489417694228791945, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490002, "value1":0.2365349300258502, "value2":5413143921150744151, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537535, "value1":0.056469545325025704, "value2":2942905392018720189, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748755, "value1":0.5086783291906242, "value2":4857376881146282553, "value3":false, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235928, "value1":0.6784255332464089, "value2":3029395751611067716, "value3":true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.401233, "value1":0.753375539931517, "value2":8169340970187425814, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960595, "value1":0.5301594317579601, "value2":3908827718931366422, "value3":true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113359, "value1":0.7603991477763598, "value2":2119709520548445193, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.875780, "value1":0.49587314465392923, "value2":8685987546300538378, "value3":false, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.938401, "value1":0.9152835192021896, "value2":1076417540165823423, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.869630, "value1":0.7973627027092416, "value2":2126356962780308110, "value3":false, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388566, "value1":0.6382739532774706, "value2":159979324198275701, "value3":false, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.275145, "value1":0.5927046376681012, "value2":7080867181269495541, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523278, "value1":0.5267214411412359, "value2":1728237597853111534, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584632, "value1":0.5083502145633392, "value2":7227399514745308579, "value3":true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964267, "value1":0.6658213068549924, "value2":5519764789585080716, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824564, "value1":0.9264931547852524, "value2":3998899671907293853, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986213, "value1":0.027645492661625626, "value2":3185626691636394685, "value3":true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634127, "value1":0.858418006506526, "value2":5813619502423233874, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.099856, "value1":0.5750664046005645, "value2":7125055104530490351, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685619, "value1":0.3275765619786257, "value2":3507230098472264785, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174236, "value1":0.36236865121536765, "value2":2097215901740320742, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093835, "value1":0.8733207807127527, "value2":5480486627610559582, "value3":false, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687722, "value1":0.42815747753178873, "value2":2463297564216975626, "value3":false, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015930, "value1":0.035417712050281464, "value2":6849011969803492014, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181519, "value1":0.9822679258679019, "value2":408242883289676228, "value3":false, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370465, "value1":0.9916450645571185, "value2":3540916962390370495, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.557960, "value1":0.814086825170977, "value2":975702025360269586, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785667, "value1":0.18117401736664177, "value2":8964384630261548826, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014126, "value1":0.6836156690970117, "value2":5150749937802379773, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310715, "value1":0.8554125170001826, "value2":6356794838014021178, "value3":false, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000193, "value1":0.43591866134299384, "value2":3781675008214883017, "value3":true, "__name__":"metric_00000800", "key_5":"k","key_7":"e","key_3":"d", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310472, "value1":0.20656655062551638, "value2":2155883927623577685, "value3":true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348781, "value1":0.1874457704512197, "value2":3156445090401189095, "value3":true, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737752, "value1":0.4107965308340836, "value2":6831391234474352496, "value3":false, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.864771, "value1":0.7087377270632053, "value2":3852759604751603741, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377175, "value1":0.302211293966416, "value2":3671141882815450126, "value3":true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035362, "value1":0.11868658008033231, "value2":4402958858442974997, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206005, "value1":0.9050640840246325, "value2":1571018380191846388, "value3":false, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607719, "value1":0.5276900307886362, "value2":2938372021759409514, "value3":true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847851, "value1":0.23589306520821895, "value2":8130082209207560079, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181853, "value1":0.3318615047159984, "value2":2814234275379593797, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093707, "value1":0.8735019416998069, "value2":7839357347839415099, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.050872, "value1":0.045154096442812874, "value2":1387216811650422069, "value3":false, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635715, "value1":0.7372871363614325, "value2":2653034997948603272, "value3":true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.807390, "value1":0.06360776205204009, "value2":8705617981394656683, "value3":true, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893750, "value1":0.033458142425902254, "value2":2550969485909611240, "value3":true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599717, "value1":0.9713785482184008, "value2":1528762568291094955, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544670, "value1":0.0942081326101987, "value2":5208646740238503092, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.460630, "value1":0.30899325368963804, "value2":6955190987114158639, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348159, "value1":0.8338319484328468, "value2":382561011167415899, "value3":false, "__name__":"metric_00000820", "key_5":"f","key_4":"d", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.866844, "value1":0.4559148181796855, "value2":4921190547959215054, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.779864, "value1":0.8408779744303566, "value2":5151627244176235100, "value3":false, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087888, "value1":0.9444160436820308, "value2":2636681313922893724, "value3":true, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422949, "value1":0.45533312838966733, "value2":8719456700476032056, "value3":false, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909073, "value1":0.4871022495216044, "value2":7470894518121500308, "value3":true, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526308, "value1":0.6556527723106481, "value2":2061496006850665904, "value3":true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483995, "value1":0.01917969989791312, "value2":6594173174887504625, "value3":false, "__name__":"metric_00000836", "key_9":"g","key_1":"k","key_3":"h", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.704096, "value1":0.7335839167893562, "value2":7683462035683402688, "value3":false, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796245, "value1":0.775721209508694, "value2":1762994312464074505, "value3":false, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635089, "value1":0.08634955165534107, "value2":5435729701416464796, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.146656, "value1":0.904887030211023, "value2":3845389016759808576, "value3":true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.975641, "value1":0.06011625753112877, "value2":2167561119971885977, "value3":true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.027722, "value1":0.7606346737865884, "value2":7806658536996851894, "value3":true, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.255754, "value1":0.8709289771166605, "value2":2946981794368069885, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682006, "value1":0.164502402573321, "value2":5375762287724896423, "value3":false, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901286, "value1":0.32721693073724145, "value2":1600743249293213477, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.844121, "value1":0.4592683330441191, "value2":5355971233932247551, "value3":false, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756958, "value1":0.06556927134836166, "value2":2324918632583888734, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.686437, "value1":0.5613972309334896, "value2":1964277954793319022, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.280286, "value1":0.210108301927216, "value2":4762745887015662113, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300106, "value1":0.3451768742896802, "value2":5756080702879459521, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008833, "value1":0.8860643993351358, "value2":5562038867537669443, "value3":true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964560, "value1":0.5679670350248304, "value2":7270655908716143592, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259598, "value1":0.1779142648395649, "value2":1989791279542261185, "value3":false, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289316, "value1":0.27910110845131275, "value2":9108304266352627509, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.380786, "value1":0.03821154554853608, "value2":5351444755572082025, "value3":false, "__name__":"metric_00000845", "key_3":"d","key_8":"b","key_0":"c", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.619088, "value1":0.014955635682113628, "value2":1506343674187146948, "value3":true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.991863, "value1":0.4507410741879421, "value2":700372855016807730, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.819751, "value1":0.47361005562943986, "value2":397573677309043949, "value3":false, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962980, "value1":0.5040052190830692, "value2":7266525751259633187, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.451541, "value1":0.1442134636404948, "value2":561640599502466121, "value3":false, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.387998, "value1":0.5301085061271325, "value2":9134048967019410288, "value3":true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300013, "value1":0.7373601287356207, "value2":342878676774549894, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008200, "value1":0.7728904842469639, "value2":6539748237741452295, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.697595, "value1":0.02404028274324282, "value2":5913828782152003967, "value3":false, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.726484, "value1":0.45889774925126936, "value2":8474247967864647236, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.696138, "value1":0.8390712125923527, "value2":8640587498190477408, "value3":false, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.053954, "value1":0.3443648011087006, "value2":6899937907932201153, "value3":true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.702864, "value1":0.09419193983638496, "value2":1509375003037818758, "value3":false, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379925, "value1":0.10612710322810476, "value2":7235965724589430822, "value3":true, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.065471, "value1":0.03922963726603652, "value2":5920886218601602154, "value3":false, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087291, "value1":0.4838942531586474, "value2":535199259720805783, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977724, "value1":0.6375345212159459, "value2":4619853255519698247, "value3":false, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566854, "value1":0.21378747510647453, "value2":3467239940916600883, "value3":true, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110862, "value1":0.6797718237586716, "value2":7914262457466633074, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440246, "value1":0.30120840846619557, "value2":3776365109572133674, "value3":false, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.797803, "value1":0.7797287504617298, "value2":5037915665887516284, "value3":false, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.586577, "value1":0.1709193163609849, "value2":5198104224414552394, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879594, "value1":0.7450124911318364, "value2":8773848846938448870, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886367, "value1":0.551654132389125, "value2":358026416808351667, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464288, "value1":0.7461865171369122, "value2":8085418698319569937, "value3":true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197248, "value1":0.8352566024408136, "value2":5751810192674740519, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.217337, "value1":0.19284687675681011, "value2":7864252915572676984, "value3":true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448294, "value1":0.42640546323275524, "value2":168095719577724557, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.591254, "value1":0.037748263913037684, "value2":2863811536714848498, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372170, "value1":0.6660431538229836, "value2":8567505344595141514, "value3":false, "__name__":"metric_00000885", "key_6":"g","key_9":"f","key_3":"k", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317243, "value1":0.18445456220952994, "value2":4074456277125729011, "value3":false, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.267738, "value1":0.2691217808870719, "value2":159261331127552033, "value3":false, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585076, "value1":0.21078242008152992, "value2":1239129613801043878, "value3":false, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979016, "value1":0.5956340156534701, "value2":9183614150234962352, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447286, "value1":0.10494392100402684, "value2":3770086671966420800, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105069, "value1":0.2794799410983503, "value2":3469689160956336636, "value3":false, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.173597, "value1":0.6860882854882017, "value2":6360441874685617414, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.835787, "value1":0.7543174676127191, "value2":6129127352506150313, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.772599, "value1":0.7516717167947417, "value2":8027123646004983290, "value3":false, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.115846, "value1":0.3569203207491387, "value2":8576278949637442153, "value3":true, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551211, "value1":0.9246333110829817, "value2":898669648143321253, "value3":false, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.230459, "value1":0.04953444106043317, "value2":2309701856093198744, "value3":true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060038, "value1":0.7972052364471702, "value2":2490346122315313464, "value3":false, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512371, "value1":0.09068245544499785, "value2":3847395208192505439, "value3":true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783506, "value1":0.7070750057247079, "value2":451775490327949567, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000319, "value1":0.9107891782905447, "value2":4820343479119697886, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628626, "value1":0.016138857727039164, "value2":6452512787294668321, "value3":false, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.063279, "value1":0.39758013318800806, "value2":2380900021298276106, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480402, "value1":0.014403992075819017, "value2":6901656754464212004, "value3":false, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.462954, "value1":0.20570595420817753, "value2":4308695238369802814, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308777, "value1":0.46154273334740287, "value2":533630259579000143, "value3":true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.006531, "value1":0.41769996010310556, "value2":1003539425076610908, "value3":true, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.606413, "value1":0.8392935388898198, "value2":7138000920468723420, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390008, "value1":0.9457032273317953, "value2":8816745864308273066, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.545540, "value1":0.9980077865004171, "value2":3431998048103898458, "value3":true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.771291, "value1":0.9269154791448218, "value2":2868685560372963447, "value3":false, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035244, "value1":0.15635792222626285, "value2":745780555759481875, "value3":true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.802078, "value1":0.49326419263144117, "value2":7061825021138296948, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141863, "value1":0.6394029445292002, "value2":4052750041261099856, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210519, "value1":0.4447889112985626, "value2":4491375706281220187, "value3":true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014644, "value1":0.5845639893426985, "value2":8504751940020912407, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.457311, "value1":0.39232937571361776, "value2":8948083137978870620, "value3":false, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497330, "value1":0.0554441906288732, "value2":8552872946126060990, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778960, "value1":0.3021332369906045, "value2":5664412036675630494, "value3":false, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151648, "value1":0.7180248405530221, "value2":4322565027390602745, "value3":false, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.286112, "value1":0.7877440826847747, "value2":6988799327817384951, "value3":false, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.131604, "value1":0.8048208447034889, "value2":7951591877124931974, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122845, "value1":0.26609349854031233, "value2":1608820578917270500, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.059135, "value1":0.8932421413801345, "value2":6863045131855298597, "value3":false, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.119289, "value1":0.6625474883897694, "value2":3938725492057076257, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.114546, "value1":0.1285766450638473, "value2":3866539491041344116, "value3":true, "__name__":"metric_00000908", "key_7":"e","key_6":"i", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134976, "value1":0.9092879262740383, "value2":3783610158682087147, "value3":true, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.676253, "value1":0.24518334285373874, "value2":5037863892800296388, "value3":true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613311, "value1":0.5221731263503148, "value2":6112150162991672535, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448627, "value1":0.09620166167463585, "value2":7262951598026367151, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370708, "value1":0.9034926573578455, "value2":6264024668154911646, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694827, "value1":0.4562036337293121, "value2":5132858458452108988, "value3":false, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.409213, "value1":0.2400040814231921, "value2":5692228309415552332, "value3":true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.272515, "value1":0.512618446397202, "value2":4245362498789762636, "value3":true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398497, "value1":0.9796136787594169, "value2":2854381927687220104, "value3":true, "__name__":"metric_00000926", "key_3":"b","key_5":"g","key_1":"k", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573421, "value1":0.25717373724181153, "value2":6071818930618476006, "value3":false, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499699, "value1":0.5015171773350958, "value2":6590026308842608922, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390872, "value1":0.15108026004608982, "value2":4054685864574240684, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.187721, "value1":0.11524534470163803, "value2":4129310514528385220, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486124, "value1":0.9297806553036565, "value2":2076740459139610542, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.506484, "value1":0.48473773192721936, "value2":5279382479053069968, "value3":true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095317, "value1":0.5302104024468535, "value2":1006511502199967697, "value3":true, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659995, "value1":0.7173304922658666, "value2":7272260713937588084, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206009, "value1":0.21353139637435428, "value2":4614697991046896838, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049549, "value1":0.9736103016824452, "value2":713014430698092731, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764793, "value1":0.5877828915841818, "value2":2743371147788843703, "value3":false, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095972, "value1":0.7802447999330678, "value2":1247537151507568078, "value3":false, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210056, "value1":0.13206812499303777, "value2":5119237508780620544, "value3":false, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477450, "value1":0.6302618943375251, "value2":3505999540579430287, "value3":false, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.182076, "value1":0.5504514321159135, "value2":2799023815129557532, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262688, "value1":0.9279334258185475, "value2":3960329791115549039, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601942, "value1":0.9333313963421735, "value2":8488152958276516414, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095627, "value1":0.2423663973685341, "value2":3147375434225291522, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325183, "value1":0.9698662748883076, "value2":8348906475599130816, "value3":true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535337, "value1":0.9108408536175355, "value2":548221601502028995, "value3":false, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.465394, "value1":0.20049512890521093, "value2":6918344754347341650, "value3":false, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805179, "value1":0.04912411768594746, "value2":7498908729880858823, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635014, "value1":0.5227012296024662, "value2":6384611575231594987, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481932, "value1":0.2492598145582209, "value2":7063605156324549396, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.458503, "value1":0.3120534424622104, "value2":219352200445992919, "value3":false, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144186, "value1":0.6043714345133836, "value2":5343022569080095368, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148408, "value1":0.7539243961545181, "value2":8912511444056366708, "value3":true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357993, "value1":0.9968001216010738, "value2":3427619790755264753, "value3":true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519356, "value1":0.2148487452898843, "value2":387702087539510555, "value3":true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631948, "value1":0.4990937312036263, "value2":936214783188826913, "value3":true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631191, "value1":0.03578918325727108, "value2":2415177266799716415, "value3":false, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.915725, "value1":0.6672040350791707, "value2":7911702372383761591, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898111, "value1":0.2610115750745007, "value2":6604319492118960080, "value3":false, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871853, "value1":0.700295668840288, "value2":4280817186446527854, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601683, "value1":0.48046941918756864, "value2":4803180873024223807, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036853, "value1":0.059813074330959816, "value2":4421336201693669164, "value3":true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.870282, "value1":0.09570589043315565, "value2":2716621407072678469, "value3":false, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543069, "value1":0.31121787546995405, "value2":1276512731094194139, "value3":false, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367830, "value1":0.09434652170549433, "value2":2256603050688647078, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468827, "value1":0.8402294714187805, "value2":299914447978500025, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910520, "value1":0.6890435117531037, "value2":2224012689460052929, "value3":true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291500, "value1":0.2581942233541333, "value2":94642526795868830, "value3":true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972763, "value1":0.39798358703466546, "value2":711133600064208053, "value3":true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398873, "value1":0.007238280134283497, "value2":414930643350925093, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003583, "value1":0.3709596459020853, "value2":7158538565937127848, "value3":true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288974, "value1":0.0033772263476846164, "value2":3505644260463770547, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.989217, "value1":0.7234413065529468, "value2":6102862603791704696, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151126, "value1":0.962243743917813, "value2":2796295067408464675, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.313076, "value1":0.4725232716261173, "value2":5800110791744456113, "value3":false, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.169414, "value1":0.10364598232077965, "value2":8013370755699143486, "value3":false, "__name__":"metric_00000980", "key_4":"d","key_7":"f","key_0":"d", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641143, "value1":0.3874568519220252, "value2":1865317425913683745, "value3":false, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960408, "value1":0.6444887127846117, "value2":2021509390881885067, "value3":true, "__name__":"metric_00000977", "key_7":"f","key_1":"h", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188655, "value1":0.9671102461325771, "value2":4499002439335266005, "value3":false, "__name__":"metric_00000978", "key_8":"h","key_9":"a","key_7":"e", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438594, "value1":0.23438356733095106, "value2":5880329494012383454, "value3":false, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.043142, "value1":0.8137602822079195, "value2":5072736510183548575, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566884, "value1":0.3037390436862004, "value2":1427943088173652054, "value3":true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.528095, "value1":0.06744248220950605, "value2":2617279385140322030, "value3":false, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.075656, "value1":0.6248261098594748, "value2":3102580216650467270, "value3":false, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644300, "value1":0.11077089127760144, "value2":252971746503923271, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061068, "value1":0.18796880019702752, "value2":1108553918392505010, "value3":true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.996996, "value1":0.5752434742054809, "value2":8128574670569848168, "value3":false, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015917, "value1":0.06928565567804344, "value2":6283279207443485305, "value3":false, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238217, "value1":0.29468486256767473, "value2":4879109468201269125, "value3":true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.302561, "value1":0.16374126420414117, "value2":1267840542849218667, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.879665, "value1":0.06081772906299747, "value2":7735358902544562393, "value3":true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.436414, "value1":0.9145077406345081, "value2":725040121642504252, "value3":false, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845550, "value1":0.6355646354104764, "value2":2412964869373168042, "value3":true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918386, "value1":0.3307533061620177, "value2":4567564826294647319, "value3":true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310744, "value1":0.5825111373150833, "value2":6000767898591093764, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206794, "value1":0.8760021186600452, "value2":3359095928804782209, "value3":true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.237359, "value1":0.7515468979685085, "value2":4507098732286155978, "value3":false, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247625, "value1":0.9212103536180063, "value2":1210374309278913705, "value3":true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.531076, "value1":0.9294917705494895, "value2":6008294356031388873, "value3":false, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258474, "value1":0.4801885775502877, "value2":4809296905208303773, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687251, "value1":0.5503479981842399, "value2":1159057662539665025, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294328, "value1":0.579524047000884, "value2":8591427275775614498, "value3":false, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030329, "value1":0.3897745465299769, "value2":8195510768796559399, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591813, "value1":0.4311576035013457, "value2":57188656194062821, "value3":true, "__name__":"metric_00000013", "key_7":"e","key_3":"f","key_6":"k", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258460, "value1":0.19968970719511664, "value2":6006706732580329556, "value3":true, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153359, "value1":0.6163826197572554, "value2":8075354818413450461, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977991, "value1":0.47088424321623057, "value2":1437620300807340754, "value3":true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.554048, "value1":0.1312819884002532, "value2":2639581932761358442, "value3":true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.829720, "value1":0.9986118788512084, "value2":3816923447231661055, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648184, "value1":0.9987080219677139, "value2":7493304631161246772, "value3":false, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171705, "value1":0.6479656070812448, "value2":940500361907630226, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452962, "value1":0.7379240782521741, "value2":8607242240788556904, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422715, "value1":0.29055682202125643, "value2":2609014884729200526, "value3":false, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997247, "value1":0.03493075047200949, "value2":1341694184124582251, "value3":false, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.454967, "value1":0.4083872840808467, "value2":7327414310312171249, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268085, "value1":0.9598378795022791, "value2":9180787761997570552, "value3":false, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.185765, "value1":0.869366023599698, "value2":6065348312952390593, "value3":false, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954707, "value1":0.29066212545734305, "value2":6500179214577769106, "value3":false, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134999, "value1":0.0746058596733823, "value2":3954294580008112843, "value3":true, "__name__":"metric_00000019", "key_7":"j","key_1":"b","key_4":"h", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.056701, "value1":0.08887774822608922, "value2":2624540757429278414, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503714, "value1":0.7939975169235949, "value2":1543165523760841555, "value3":false, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.407434, "value1":0.5263350258249397, "value2":6207484012298867707, "value3":false, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.216887, "value1":0.6135120862192558, "value2":6706252188296828465, "value3":true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950767, "value1":0.6258842267416237, "value2":6943474392706233335, "value3":true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409499, "value1":0.39019457036616395, "value2":5543165311657950313, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.064967, "value1":0.25118206025688317, "value2":4933404061205221336, "value3":false, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503859, "value1":0.6260829223715964, "value2":2553798156705274576, "value3":false, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.239884, "value1":0.0646597153433716, "value2":5357886478310061690, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.560239, "value1":0.025368350153306324, "value2":1834804451034467956, "value3":false, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021890, "value1":0.4250913620299391, "value2":7901663683744276123, "value3":true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.720024, "value1":0.7102429898685084, "value2":8848853087379978689, "value3":false, "__name__":"metric_00000040", "key_1":"b","key_6":"f","key_0":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721708, "value1":0.267345226146574, "value2":8204105067810932755, "value3":false, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.688701, "value1":0.8124970825519422, "value2":5072280932532136578, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227719, "value1":0.19432503349760097, "value2":2763726347992488568, "value3":false, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398501, "value1":0.6725319245300637, "value2":2029542129845625509, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220707, "value1":0.07663396351127535, "value2":6862161694126124253, "value3":true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464464, "value1":0.8033021261492177, "value2":500776341255510367, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.369494, "value1":0.6874411246839779, "value2":5472077652556256405, "value3":false, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429691, "value1":0.7205162840867915, "value2":2379554410345555228, "value3":true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.003693, "value1":0.5301205012344029, "value2":7339147645001280473, "value3":false, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095495, "value1":0.0031080960946583136, "value2":3948518200809126624, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.649621, "value1":0.9796250725140294, "value2":5076285879574984318, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.701028, "value1":0.4664187080974912, "value2":8061147057085676917, "value3":false, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024773, "value1":0.7030139999067455, "value2":8201442408389043582, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570617, "value1":0.6283295712663881, "value2":7893659755244222642, "value3":false, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780271, "value1":0.6223497445918998, "value2":4413418515678745434, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904547, "value1":0.28169993275009253, "value2":1702651611066126658, "value3":false, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.039402, "value1":0.42131084562328136, "value2":3560241450438323083, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.612707, "value1":0.19465390726157364, "value2":3161717756488318325, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729159, "value1":0.25669937307380997, "value2":1479715553122219203, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813686, "value1":0.049715056995271165, "value2":2079826454687289078, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747376, "value1":0.7275373884923515, "value2":2435962270026982603, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354388, "value1":0.8636816800782944, "value2":6027722589314572513, "value3":false, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505532, "value1":0.1663611987871669, "value2":668594150031699725, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087699, "value1":0.09888054207021492, "value2":8612173434517336899, "value3":false, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.118211, "value1":0.9755664430251321, "value2":3670406131876856616, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918355, "value1":0.2320424600489978, "value2":8375963807463485059, "value3":false, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106082, "value1":0.2496270382815134, "value2":3218140633568902603, "value3":false, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.356200, "value1":0.22905701139904852, "value2":4580449910037652861, "value3":true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272156, "value1":0.08246894655199775, "value2":8520120895086499443, "value3":true, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.445288, "value1":0.13062843592283024, "value2":1544733522378030955, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189215, "value1":0.023728801354414537, "value2":7950194843845547325, "value3":true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525278, "value1":0.8120711270842138, "value2":6926776820889632931, "value3":true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532234, "value1":0.2803691083519268, "value2":2750609942680894663, "value3":false, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169390, "value1":0.721935860273012, "value2":323165999073131824, "value3":true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465184, "value1":0.14065806381420595, "value2":6373433541582043482, "value3":false, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211347, "value1":0.5119890992803398, "value2":437881008737730578, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.326310, "value1":0.8982944342798628, "value2":7550941116286319434, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767054, "value1":0.058056603449873775, "value2":8172199064443221808, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098754, "value1":0.802732462605452, "value2":5531373846445331861, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.626701, "value1":0.9658445273400498, "value2":6144529458195478799, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382019, "value1":0.6200649361393201, "value2":74370737764383341, "value3":true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.162075, "value1":0.773701976515762, "value2":3415021911489973103, "value3":false, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452128, "value1":0.4165931264471384, "value2":1592331302702594637, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546168, "value1":0.9818897837453856, "value2":2770075967419206536, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794037, "value1":0.6329308941123749, "value2":1855788356162220067, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267186, "value1":0.5183944510585616, "value2":2971715621482767583, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.386007, "value1":0.7925028746106562, "value2":5519530031313056293, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194807, "value1":0.9560872102829179, "value2":8899546063621825706, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786449, "value1":0.25289379431092024, "value2":3323685962729555247, "value3":true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722033, "value1":0.791566877264509, "value2":8569419440028565177, "value3":false, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.527726, "value1":0.5123053424894405, "value2":5410088201011323097, "value3":true, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991248, "value1":0.00835285017592764, "value2":7876085652270552879, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971469, "value1":0.7516893928865965, "value2":1615677872423733394, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457203, "value1":0.2802029059377931, "value2":3774880897765407379, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429963, "value1":0.4064794421282188, "value2":7288375695455796287, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807745, "value1":0.1346588472416809, "value2":7295335243976756666, "value3":false, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247446, "value1":0.18437680636683432, "value2":2763564622590822709, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505494, "value1":0.36367826246909607, "value2":279237135102501393, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570322, "value1":0.807767632694814, "value2":3489957523135662213, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.770864, "value1":0.12825646293816148, "value2":559050377523308246, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723654, "value1":0.48061854827645123, "value2":2625015563945376475, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939025, "value1":0.31592719165163524, "value2":1661597397006530804, "value3":true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935809, "value1":0.16925319668616004, "value2":8449044609778795224, "value3":true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336027, "value1":0.7017500784708112, "value2":8428599073105193397, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904159, "value1":0.7924061448124604, "value2":4266452884457119467, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847458, "value1":0.27361557683078364, "value2":3345769475113460399, "value3":false, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378261, "value1":0.7380019020377752, "value2":432938298263940108, "value3":false, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799977, "value1":0.25999970547607815, "value2":7119589132321429408, "value3":true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786435, "value1":0.812994267772834, "value2":2642960787391817015, "value3":true, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.587571, "value1":0.5078453077544904, "value2":7413095241087719577, "value3":true, "__name__":"metric_00000100", "key_2":"a","key_3":"f","key_1":"e", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189572, "value1":0.33029349839366906, "value2":182717731110127548, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128493, "value1":0.3005744554380485, "value2":1813809549706584614, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.751426, "value1":0.43021252312166497, "value2":1048575931846703963, "value3":true, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.824593, "value1":0.9962985612048794, "value2":3451457313304562908, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913497, "value1":0.19667409773488792, "value2":324965724484330012, "value3":true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280371, "value1":0.6558253377067158, "value2":3238482029594886215, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953879, "value1":0.11691624934505847, "value2":3461549771600466582, "value3":true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189513, "value1":0.9019750283572279, "value2":7575759643142018621, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465264, "value1":0.15533711497405195, "value2":8078121216216233303, "value3":true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857857, "value1":0.2731351567904969, "value2":6433553377051334332, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.114548, "value1":0.9868338794536605, "value2":6036587149016739858, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734426, "value1":0.4227117888273388, "value2":1455241552057769722, "value3":true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075679, "value1":0.2701709613339434, "value2":3857058056766739646, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104255, "value1":0.981970010025583, "value2":3682624724640104305, "value3":false, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653572, "value1":0.008103832880105273, "value2":16321050615205215, "value3":false, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.790159, "value1":0.6178089725023603, "value2":2820923101957050608, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274981, "value1":0.28628503444016773, "value2":5532572594206077059, "value3":false, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274235, "value1":0.01138331892670461, "value2":4146948379716629700, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280495, "value1":0.19902196639234795, "value2":7564853449905093412, "value3":false, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272293, "value1":0.017257126721765374, "value2":91860645774780383, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447052, "value1":0.6776141143437516, "value2":5897324288007611430, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.209396, "value1":0.14657117940198489, "value2":3976211106646613048, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850761, "value1":0.5201711080016932, "value2":7354069105129952055, "value3":false, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.300233, "value1":0.1860571279843043, "value2":470481327297902326, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310707, "value1":0.25674808205958144, "value2":2662172304685648211, "value3":false, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766353, "value1":0.3692045132202804, "value2":7757321942348800299, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214351, "value1":0.6936702854905948, "value2":8248415695462939576, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.005940, "value1":0.9564433372149446, "value2":8800540347032950677, "value3":true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009379, "value1":0.5129617748107491, "value2":2230603715203037945, "value3":false, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867507, "value1":0.6850688842908961, "value2":4341430047057401636, "value3":true, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640527, "value1":0.8542276865628686, "value2":6337085201397795074, "value3":true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186612, "value1":0.2520653493154805, "value2":3597439324405314019, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168374, "value1":0.9686674775113324, "value2":5086009893745928969, "value3":true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.930110, "value1":0.9065446675547199, "value2":9084895148279172527, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385337, "value1":0.5310871446244799, "value2":3634604672435791412, "value3":false, "__name__":"metric_00000137", "key_6":"k","key_1":"k","key_4":"a", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273397, "value1":0.6978323941286791, "value2":6270777752769360572, "value3":false, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.644440, "value1":0.7390124331493296, "value2":5633558971491993000, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692014, "value1":0.5735092134024398, "value2":4018757059763515593, "value3":true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307407, "value1":0.9523210688679733, "value2":4664291077378357176, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.864207, "value1":0.2504856063622225, "value2":5125796995165750818, "value3":true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.891946, "value1":0.8905322399035085, "value2":3790600090207208565, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.196262, "value1":0.02637661052612273, "value2":3345947703698603019, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609259, "value1":0.9964561610316119, "value2":4297199572790537149, "value3":false, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.479133, "value1":0.573177630994298, "value2":4352137818057928535, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.212122, "value1":0.9374174976714222, "value2":2883686750556684691, "value3":true, "__name__":"metric_00000146", "key_7":"i","key_1":"c", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143194, "value1":0.5493618690016547, "value2":8675397367930704936, "value3":true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.911116, "value1":0.8323148117854684, "value2":3727739579448171557, "value3":true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.100156, "value1":0.7882098153425944, "value2":6270723454145661182, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159904, "value1":0.5891414895820671, "value2":7892306327256327213, "value3":true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.561342, "value1":0.575909380567069, "value2":2087294011397500886, "value3":true, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.542768, "value1":0.7419123808449162, "value2":5892135507379591161, "value3":false, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977295, "value1":0.2537719509110862, "value2":23083079770681507, "value3":true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.690704, "value1":0.08136832639928646, "value2":8018440792149947537, "value3":true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.681388, "value1":0.6173470752976815, "value2":4725502695033185312, "value3":false, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045833, "value1":0.9052130764103621, "value2":2335561799986333486, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.621313, "value1":0.5578705772605427, "value2":5423853801041319612, "value3":false, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362143, "value1":0.058506006415061124, "value2":1952601720041809520, "value3":true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.019544, "value1":0.2454839257683829, "value2":5439837990435419484, "value3":false, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414507, "value1":0.8898999762410835, "value2":9399980232748751, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.946393, "value1":0.5231814332055934, "value2":8044307915684488670, "value3":false, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640533, "value1":0.1401932575466956, "value2":3489584691123939236, "value3":false, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.641620, "value1":0.7793690564908118, "value2":6745229093735363060, "value3":false, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256954, "value1":0.9792655297711959, "value2":8340241164033802673, "value3":true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.578274, "value1":0.6786422376056035, "value2":2268577288780537910, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168131, "value1":0.7011560783500894, "value2":5754899077173699372, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261168, "value1":0.41241864943729767, "value2":6926145892542625665, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.142353, "value1":0.026897262060108916, "value2":3407492707727317952, "value3":false, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453717, "value1":0.9480329042934568, "value2":5909842310603335433, "value3":true, "__name__":"metric_00000169", "key_6":"d","key_9":"b","key_4":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.498253, "value1":0.2696408414861873, "value2":9093122739488143709, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152193, "value1":0.829008206269259, "value2":7872930012981859271, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147195, "value1":0.40564509426795664, "value2":388387455399110782, "value3":true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.756476, "value1":0.08296241719338777, "value2":8819867312676478909, "value3":false, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472139, "value1":0.10622575406945087, "value2":2389837147036501120, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553362, "value1":0.4640810980824681, "value2":9129773762938561987, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138568, "value1":0.27153851873742235, "value2":3922539618739320056, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629854, "value1":0.06560725045609712, "value2":5242856550347246211, "value3":false, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532533, "value1":0.6685532806932011, "value2":4403464406211940116, "value3":false, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.102100, "value1":0.30269603070417783, "value2":560972484995423105, "value3":false, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.594212, "value1":0.3436785616541505, "value2":6118258015340076117, "value3":true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182345, "value1":0.5829131946786057, "value2":316161802601703652, "value3":false, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865350, "value1":0.9510270552078569, "value2":5815202444060769343, "value3":true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.417157, "value1":0.07805271825644472, "value2":1671886271645798176, "value3":false, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453428, "value1":0.07722736953435244, "value2":5062577774198478117, "value3":true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923875, "value1":0.3869476521576109, "value2":7644772358804477804, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170062, "value1":0.771360835653392, "value2":8029933370964328416, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.341390, "value1":0.538519995660261, "value2":6637857221246165272, "value3":true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267222, "value1":0.4666279704151431, "value2":716146669296183590, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562383, "value1":0.35102191528024995, "value2":8428363533739190009, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767969, "value1":0.6173626174937589, "value2":6290463280158957319, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451339, "value1":0.546379610337649, "value2":1547962669719183472, "value3":true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464199, "value1":0.24671185662577808, "value2":3561721323195761229, "value3":true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.154391, "value1":0.45596856325620033, "value2":6560907831281448338, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.709403, "value1":0.6109692366076535, "value2":3486468545935536219, "value3":false, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570798, "value1":0.7335030585660903, "value2":1572258901091495894, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.198469, "value1":0.7425214958348296, "value2":3904425769094656355, "value3":true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315912, "value1":0.9245437278546984, "value2":3047808486948520796, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186365, "value1":0.8721078635865479, "value2":6305488287785962360, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759050, "value1":0.3487806615217454, "value2":5791750426523247447, "value3":true, "__name__":"metric_00000199", "key_5":"j","key_9":"j","key_2":"f", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991504, "value1":0.5895957831739977, "value2":6282966318152098641, "value3":true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355229, "value1":0.507857444797784, "value2":295494213133771851, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551655, "value1":0.4307208831464999, "value2":7265410065885482832, "value3":true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525491, "value1":0.6404292728685821, "value2":1060635693413562315, "value3":true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008415, "value1":0.6916001000931801, "value2":2353592246165483773, "value3":false, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400680, "value1":0.36661865825780576, "value2":9104767548881126156, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376594, "value1":0.3981424874885838, "value2":7118511131751820911, "value3":true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553143, "value1":0.6460744131761798, "value2":1887476981066696157, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473253, "value1":0.8348910772414322, "value2":7455007366058138980, "value3":true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457579, "value1":0.4708610456466168, "value2":4593338028669686470, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433566, "value1":0.16598645827046257, "value2":5257287213674383265, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.357007, "value1":0.7860515830836307, "value2":2972497673348348004, "value3":true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.461293, "value1":0.35370822925539525, "value2":355204159426372577, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.975565, "value1":0.6491904274322989, "value2":2226145731856801788, "value3":true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951226, "value1":0.8792346577420341, "value2":5756323624906456991, "value3":true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143238, "value1":0.9818710036776283, "value2":4180983964934518595, "value3":true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874423, "value1":0.3287251148209575, "value2":381928105696021842, "value3":false, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648299, "value1":0.1842978254216889, "value2":5567179544229053293, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055528, "value1":0.2649298342733898, "value2":3063386077105020906, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551525, "value1":0.008929798118195076, "value2":1932022596647743318, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078106, "value1":0.6249963076831733, "value2":3775614794809061222, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595048, "value1":0.6556888188689401, "value2":263555282139003693, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270487, "value1":0.15652423199780005, "value2":8005167724994681049, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986526, "value1":0.44168299905409525, "value2":3616684978189935787, "value3":false, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648015, "value1":0.6935400265504532, "value2":3554010850084269714, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082643, "value1":0.4464596047653067, "value2":7393355921025489008, "value3":false, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.297812, "value1":0.8411130245970256, "value2":3060908524323233968, "value3":false, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.023535, "value1":0.8979883834169406, "value2":3955075066530261651, "value3":false, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775710, "value1":0.803115138047979, "value2":1956023105923958421, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276355, "value1":0.35534087510852963, "value2":70554362711904210, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227561, "value1":0.3475937884638202, "value2":5770217876057854108, "value3":false, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.997058, "value1":0.9920212102015171, "value2":4179786645072950325, "value3":true, "__name__":"metric_00000233", "key_9":"i","key_3":"e","key_4":"j", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330001, "value1":0.7117987721576648, "value2":1416148178901813673, "value3":false, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.859492, "value1":0.07088427876105753, "value2":5628499584516778303, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.758514, "value1":0.5315142010423931, "value2":3999275359211756172, "value3":false, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567291, "value1":0.5976846753929923, "value2":3568009536081310459, "value3":true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914132, "value1":0.8979176888984427, "value2":4778372357738152583, "value3":true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.470387, "value1":0.2136647847272661, "value2":4632422950916722092, "value3":true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.180567, "value1":0.7349455061346736, "value2":4238367012667671950, "value3":false, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444766, "value1":0.35029664162897833, "value2":4149159141941833270, "value3":true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.351876, "value1":0.719839928132912, "value2":8311035304279403168, "value3":true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.343283, "value1":0.5403744774398413, "value2":7831018924301288402, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202600, "value1":0.07323861368397915, "value2":7512140161978615708, "value3":true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953729, "value1":0.4072430540967121, "value2":6162477020927495810, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206750, "value1":0.7276384026815652, "value2":3867378392166470570, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313521, "value1":0.19760060640640484, "value2":1845943836170488560, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774075, "value1":0.9632484532837077, "value2":2186178755453106945, "value3":true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228120, "value1":0.1138891785769252, "value2":2119625663561385331, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355712, "value1":0.12321943936422765, "value2":5732137663228356419, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.004820, "value1":0.1110884226197388, "value2":2265343653685159151, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.630751, "value1":0.6269969213669998, "value2":3359610757917670818, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.015500, "value1":0.20581219173107831, "value2":452968813923389028, "value3":false, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646040, "value1":0.14066092038312122, "value2":6343927209062846133, "value3":false, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220902, "value1":0.6865529990306172, "value2":1648919416099165702, "value3":false, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.700811, "value1":0.4621690976315305, "value2":3551937779376072568, "value3":true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607198, "value1":0.8914891421096152, "value2":246252330281719887, "value3":false, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.574035, "value1":0.7865743679720645, "value2":5713754277685792561, "value3":true, "__name__":"metric_00000255", "key_7":"e","key_9":"e","key_1":"i", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473497, "value1":0.13845795308548414, "value2":259488391880499746, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944779, "value1":0.9609062478569093, "value2":5705930854457323614, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775443, "value1":0.1259913054905557, "value2":5080990926603035088, "value3":true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010365, "value1":0.4903104589413477, "value2":3314614042137410585, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913172, "value1":0.7864192987989552, "value2":8344919384469197086, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.796557, "value1":0.037094581426906255, "value2":9011770804417776349, "value3":true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.314769, "value1":0.8877112795540667, "value2":6683425457896895076, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.806104, "value1":0.5962447685862431, "value2":5435268152791547936, "value3":false, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579388, "value1":0.6885293468734471, "value2":9064761548833010698, "value3":false, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.391022, "value1":0.5900734892123236, "value2":4669259037774416678, "value3":false, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762980, "value1":0.7685532202125763, "value2":2315953471058650483, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.135687, "value1":0.32474681440848086, "value2":5755607255133419339, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104934, "value1":0.14589803567481788, "value2":126178294464524762, "value3":true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.052690, "value1":0.593421743400015, "value2":5119197034484564414, "value3":false, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870007, "value1":0.4673411129128983, "value2":2860543686633489085, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629308, "value1":0.4498616996591619, "value2":5328613189051877451, "value3":true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711415, "value1":0.8849559918426787, "value2":5903543494874730971, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901063, "value1":0.5349473283169682, "value2":4797032563086773247, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.651731, "value1":0.3127106327077303, "value2":5700343521566497813, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825358, "value1":0.7283436835171048, "value2":7844311848127264742, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.956544, "value1":0.8853976152911399, "value2":5661939812745981161, "value3":true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235737, "value1":0.26141150008124164, "value2":1680083033155058708, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.537374, "value1":0.4021256295776066, "value2":412217863647126208, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238448, "value1":0.20048610554803628, "value2":6383291537020572135, "value3":false, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506422, "value1":0.3812390668975348, "value2":184701845674968962, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.456772, "value1":0.12479606082790685, "value2":7770738457381111114, "value3":true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.789484, "value1":0.7412786023699602, "value2":492664655073083816, "value3":false, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931307, "value1":0.2536425784942127, "value2":3030103981590386173, "value3":true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610031, "value1":0.5842291904640166, "value2":5028658995770055300, "value3":false, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.068107, "value1":0.6578126156370956, "value2":4352952521986270134, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477620, "value1":0.16120256759137608, "value2":8502715215777515370, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.124418, "value1":0.2689168702571952, "value2":404701828083799050, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.779829, "value1":0.4388821199327946, "value2":4899529018433185166, "value3":false, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800101, "value1":0.2714233939432663, "value2":584157730559703393, "value3":false, "__name__":"metric_00000273", "key_7":"i","key_0":"g","key_2":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860807, "value1":0.4135766617320191, "value2":3872756991712201227, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.917667, "value1":0.2149289211585331, "value2":1070350813804397058, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214604, "value1":0.6792613272879433, "value2":2691750153580462441, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245813, "value1":0.6913342601180135, "value2":6216928639197666035, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087487, "value1":0.9301668438033497, "value2":6232780192940116715, "value3":false, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.069143, "value1":0.6293446581971062, "value2":7108419812354921929, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635591, "value1":0.9539262040674916, "value2":2954917330417044389, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672064, "value1":0.8389645710809651, "value2":3571122666192720690, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818753, "value1":0.0576648702049322, "value2":450051931472461981, "value3":true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270657, "value1":0.28617962738097397, "value2":7674542934415081242, "value3":false, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726745, "value1":0.2340744755556526, "value2":82189729669291091, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.897029, "value1":0.2638345630149327, "value2":8238518585440831118, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721815, "value1":0.7091199604872773, "value2":6366310396364171134, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.130557, "value1":0.49724111456500786, "value2":5852311186811706792, "value3":true, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447829, "value1":0.6230142507141527, "value2":937535691807945647, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.802396, "value1":0.6394461194574114, "value2":7242948197847068875, "value3":false, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.424144, "value1":0.6000896127174374, "value2":8028530164864078540, "value3":true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.426986, "value1":0.7003205100320555, "value2":2322106140405745021, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942361, "value1":0.016353745384231323, "value2":4629676907790639064, "value3":false, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653679, "value1":0.37062297658466353, "value2":8208981100462487617, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.992030, "value1":0.7072168752768998, "value2":3820991654654691772, "value3":false, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.346953, "value1":0.4620611667835753, "value2":4760146202938852105, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321463, "value1":0.9135126280713665, "value2":8417775919956392807, "value3":false, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168908, "value1":0.05729213648419359, "value2":8181061490241314804, "value3":true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.230852, "value1":0.17085551083436254, "value2":3688016011930153595, "value3":true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.066979, "value1":0.33115248245947465, "value2":398734641986366420, "value3":false, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825110, "value1":0.29481497745263574, "value2":7623192343447837676, "value3":false, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.979130, "value1":0.18961515202216345, "value2":6037869636552702023, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692641, "value1":0.4102441559202999, "value2":2079215473977001869, "value3":true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459527, "value1":0.38418621904384953, "value2":8455919021737289595, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924379, "value1":0.831174718392, "value2":8455941435197243693, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433952, "value1":0.6847864876960775, "value2":6021961695231181621, "value3":true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195096, "value1":0.9137826039265785, "value2":1951159443698721147, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.392032, "value1":0.7140691740516873, "value2":7387270195217478014, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711603, "value1":0.30553043110809663, "value2":5447610537284294754, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024988, "value1":0.636153947206049, "value2":6011662688198487566, "value3":true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140076, "value1":0.9829818790209834, "value2":1992400557046890911, "value3":false, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591682, "value1":0.734180708126354, "value2":8165041750078897088, "value3":false, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.254962, "value1":0.4530139927706582, "value2":792701874387399785, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653418, "value1":0.6853469695508817, "value2":2445415548541311714, "value3":true, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.406935, "value1":0.49945705352987035, "value2":4326827190201670587, "value3":true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140154, "value1":0.19200910536315854, "value2":7788114053604406936, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774910, "value1":0.775205817556658, "value2":1778011502106771934, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224005, "value1":0.4109368947961367, "value2":5090636250181538696, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025070, "value1":0.8639998386130291, "value2":4097020029155871346, "value3":true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.110800, "value1":0.46475105418540896, "value2":7930781849523032659, "value3":true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502796, "value1":0.19123829931308983, "value2":4926709577891896623, "value3":false, "__name__":"metric_00000346", "key_8":"g","key_9":"c","key_4":"e", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.483311, "value1":0.21080006327277256, "value2":1394785449775081074, "value3":false, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194479, "value1":0.7780668170411426, "value2":3729639098545975029, "value3":false, "__name__":"metric_00000337", "key_1":"c","key_4":"b","key_0":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385848, "value1":0.4299644347219865, "value2":7191790268699040877, "value3":true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211519, "value1":0.32521468058588077, "value2":2791560527192035681, "value3":false, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228801, "value1":0.83757037869911, "value2":7191574037900046957, "value3":false, "__name__":"metric_00000336", "key_3":"e","key_7":"e","key_0":"k", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.752930, "value1":0.8040696176893675, "value2":1942969218455296906, "value3":true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951018, "value1":0.07747976080254226, "value2":1630225104393013299, "value3":true, "__name__":"metric_00000339", "key_9":"g","key_4":"k","key_5":"b", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.750256, "value1":0.816008410350239, "value2":6724933613410762759, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583564, "value1":0.4508316797935446, "value2":6917556384524072348, "value3":true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697470, "value1":0.8171672522091267, "value2":2060386169134326462, "value3":false, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.974221, "value1":0.7951082158998446, "value2":153021997721922670, "value3":true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134841, "value1":0.655263428147034, "value2":6943977496854991893, "value3":false, "__name__":"metric_00000333", "key_4":"b","key_9":"e","key_1":"f", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.673465, "value1":0.16441674918821836, "value2":3664243165148767961, "value3":false, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152562, "value1":0.26528241288165916, "value2":1119343360475809102, "value3":true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.420604, "value1":0.2336675869017092, "value2":3279324159460647609, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.622036, "value1":0.6846769209119129, "value2":7351022317760422092, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687001, "value1":0.06651364619191985, "value2":4264970580734046753, "value3":false, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106345, "value1":0.7730270909594562, "value2":6259010316309812350, "value3":true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018243, "value1":0.39163099713401783, "value2":2052648194611616106, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576138, "value1":0.9602266945474953, "value2":5355810905366063269, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398686, "value1":0.6503809877595728, "value2":8153069195959796847, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.367754, "value1":0.6043902819598663, "value2":3335160234939115824, "value3":true, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.089576, "value1":0.7877453973247192, "value2":7617166012064737014, "value3":false, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.112529, "value1":0.42004273729990893, "value2":2084869905228279863, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262886, "value1":0.6795217484416415, "value2":8937461554206260283, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652433, "value1":0.15404027924570762, "value2":16114496573329591, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.285612, "value1":0.36648611710364465, "value2":4517711436918873094, "value3":true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.213223, "value1":0.283682948749034, "value2":1960872184236727731, "value3":false, "__name__":"metric_00000374", "key_9":"a","key_5":"k","key_8":"j", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096677, "value1":0.1578537002758255, "value2":7008414275581832390, "value3":true, "__name__":"metric_00000375", "key_6":"a","key_7":"d","key_1":"j", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.900633, "value1":0.993822419434507, "value2":3303474850432526688, "value3":true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862995, "value1":0.2925883436045491, "value2":4890951234376924728, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.038886, "value1":0.18850439286651008, "value2":7964663822924622191, "value3":false, "__name__":"metric_00000378", "key_8":"d","key_0":"e","key_6":"f", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742880, "value1":0.8063312642154098, "value2":8220352304335077837, "value3":true, "__name__":"metric_00000379", "key_7":"a","key_9":"k","key_3":"f", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545517, "value1":0.023471717745134867, "value2":5984820965643364202, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.079979, "value1":0.6088556886509169, "value2":8961288376687773441, "value3":false, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409232, "value1":0.6445899053375183, "value2":992192951010465319, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224358, "value1":0.13573773649022736, "value2":1165195770092489984, "value3":false, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211580, "value1":0.8798345576210722, "value2":2259526712170036956, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.137602, "value1":0.5470546586447269, "value2":2196698895749095736, "value3":true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950011, "value1":0.5109493788687889, "value2":7352128446573662528, "value3":true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098941, "value1":0.972587647595324, "value2":4702835811693986573, "value3":true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036022, "value1":0.7940355478266105, "value2":3489006783771959904, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.257286, "value1":0.6629731808259061, "value2":6607686064851284173, "value3":false, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283256, "value1":0.4735441861177576, "value2":690002256366468336, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729363, "value1":0.5152988869648636, "value2":7191544072945571987, "value3":false, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.671685, "value1":0.7719299111978523, "value2":8611060565403129118, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585682, "value1":0.23795953447079246, "value2":7946332098229864997, "value3":true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.187797, "value1":0.09090886526352324, "value2":470342401691160594, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.617428, "value1":0.5223785659162559, "value2":5891162691146013349, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.053786, "value1":0.9753973493719631, "value2":1732499838549362952, "value3":true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.830052, "value1":0.2400470925507307, "value2":7569204709479310112, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566485, "value1":0.9031315965779712, "value2":1637870392526502935, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104166, "value1":0.8068833505635166, "value2":5557577539110699594, "value3":true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457581, "value1":0.31154508147136156, "value2":3306826589254236098, "value3":false, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740978, "value1":0.6517923386066168, "value2":6929527434345999196, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398554, "value1":0.7163746669385104, "value2":6895927508923594471, "value3":true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174253, "value1":0.37937402384141433, "value2":1418065909790834443, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168613, "value1":0.8552112788507821, "value2":1201668238332263018, "value3":false, "__name__":"metric_00000403", "key_6":"i","key_1":"k","key_2":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376103, "value1":0.7428463858250676, "value2":9210155210384521495, "value3":false, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.798751, "value1":0.16215960908856522, "value2":4916478620663149838, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.588227, "value1":0.7494081995223059, "value2":1969994590907491188, "value3":false, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270909, "value1":0.8815916954125361, "value2":2264317244685656881, "value3":false, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.466310, "value1":0.09496768292246209, "value2":5092383460942749308, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543412, "value1":0.9329719667021983, "value2":5844328151679816255, "value3":true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.834516, "value1":0.9248915367132075, "value2":5869247588035820642, "value3":false, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.916922, "value1":0.052803170907048405, "value2":105752679780838825, "value3":false, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583307, "value1":0.04534276867452472, "value2":1081545276806182474, "value3":false, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.957758, "value1":0.1893001037569564, "value2":4530859851365168030, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723091, "value1":0.22925973916873355, "value2":5896808329469448104, "value3":false, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.719902, "value1":0.5038773612977684, "value2":5428690516490754673, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025587, "value1":0.7184099655587061, "value2":4108913234892685798, "value3":true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907580, "value1":0.24492386825368684, "value2":1156962833669313399, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958121, "value1":0.9818175985414233, "value2":3857893912179537366, "value3":false, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.657555, "value1":0.17952027005330515, "value2":3409773579631767064, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829662, "value1":0.9863896179595923, "value2":7017412361402418209, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382356, "value1":0.13377486891358994, "value2":2261611253942664564, "value3":false, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.735596, "value1":0.3414280491857885, "value2":1656516704640942493, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.365266, "value1":0.08360581432684039, "value2":751089660510969379, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.236047, "value1":0.6527140947106955, "value2":7257801694535836049, "value3":false, "__name__":"metric_00000424", "key_8":"b","key_1":"i","key_2":"d", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.563096, "value1":0.4425122630203769, "value2":8109435442419953734, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870160, "value1":0.7097748830786019, "value2":6394016408878658736, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853311, "value1":0.7865716590956611, "value2":9110725490965710227, "value3":false, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.103882, "value1":0.9153251610630555, "value2":5340001815935707095, "value3":false, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.319691, "value1":0.32338872565142723, "value2":9182777252636363483, "value3":true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971522, "value1":0.5621974159988574, "value2":6803454508761360177, "value3":true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893225, "value1":0.5781000116088169, "value2":3230113190945788558, "value3":true, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.910026, "value1":0.6955188069638236, "value2":3766609848094762934, "value3":true, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.680766, "value1":0.24230859153208487, "value2":5617809008491748996, "value3":true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.493463, "value1":0.1697890841270668, "value2":8556153147754124462, "value3":true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.849699, "value1":0.7552917967316483, "value2":2395400148768820203, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091646, "value1":0.5496353344673094, "value2":8512525001992418076, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104484, "value1":0.4174390270830882, "value2":8422771009020841833, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210780, "value1":0.8954537390033984, "value2":4547239076695924457, "value3":true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.882314, "value1":0.5142219879570834, "value2":7919864591569880176, "value3":false, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.983791, "value1":0.42885136319724737, "value2":7969816758490279476, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747300, "value1":0.4699241249924091, "value2":4491569276746468237, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939701, "value1":0.6800950779540557, "value2":3198358625582737117, "value3":true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261442, "value1":0.4575968148914831, "value2":6477186909414818810, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256111, "value1":0.693918133492274, "value2":1509539897942915838, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893394, "value1":0.31628967903294547, "value2":5501152751179175693, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742786, "value1":0.9162968056386509, "value2":9070857057311832045, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.642040, "value1":0.9183764345745785, "value2":1183434137529208194, "value3":true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045625, "value1":0.9635794700047972, "value2":8991017817592303034, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847013, "value1":0.07723071092189422, "value2":6180620156015906931, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532846, "value1":0.5943541460929466, "value2":2462231763403080171, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514157, "value1":0.7898774054205837, "value2":9059207414611927294, "value3":true, "__name__":"metric_00000452", "key_9":"h","key_0":"i", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.044776, "value1":0.9033139820464094, "value2":6478344621702437696, "value3":false, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043799, "value1":0.8974102840071708, "value2":4685725453168510298, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138572, "value1":0.5940072973710391, "value2":9150962581658325077, "value3":true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.215236, "value1":0.35464434283570645, "value2":2728457088873951162, "value3":false, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545063, "value1":0.4562894763140951, "value2":3160291795997224895, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.277361, "value1":0.9620522510409097, "value2":2528750129940465586, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477478, "value1":0.5247853848955957, "value2":7252619165572966740, "value3":false, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.940911, "value1":0.4288772636544275, "value2":1739941452377564494, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.488981, "value1":0.07801581197604958, "value2":3785496084335402474, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.037817, "value1":0.20985274999200432, "value2":3757407018866875990, "value3":false, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477559, "value1":0.26133928393072436, "value2":4699711244003996922, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.107051, "value1":0.5035109860681579, "value2":2406818127100942026, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692353, "value1":0.44174492856895586, "value2":5849623391254459092, "value3":false, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.948736, "value1":0.6131752371911697, "value2":8355568973826156875, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562536, "value1":0.5314554437999132, "value2":2325922727970832181, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336595, "value1":0.8477764608771982, "value2":2899112579581457318, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609668, "value1":0.7984731186839654, "value2":1035702079046481045, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.446414, "value1":0.8536378880203014, "value2":1450863797869138094, "value3":false, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.345841, "value1":0.005289676671501025, "value2":9055262701472338992, "value3":false, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.665956, "value1":0.5725507407225954, "value2":2388492838265413703, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.972916, "value1":0.6568884959214099, "value2":4110169872571045957, "value3":true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.370968, "value1":0.9171054847132539, "value2":9195948027770005199, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109600, "value1":0.6976295242986092, "value2":3260150427416206815, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722444, "value1":0.756396263144578, "value2":5473984714896425934, "value3":true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.011414, "value1":0.35518020333634837, "value2":5432640933038728748, "value3":false, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.442479, "value1":0.9135774445018765, "value2":1658872104403529722, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860707, "value1":0.1419422977742929, "value2":5745627115489673481, "value3":true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.925502, "value1":0.19711051262279727, "value2":4597556979704859536, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.636414, "value1":0.8593669141574183, "value2":3782467267185587990, "value3":false, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157759, "value1":0.4115235495685266, "value2":3223572108511746336, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399657, "value1":0.24629418245237128, "value2":4491685955930394692, "value3":false, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.941994, "value1":0.17062270403788604, "value2":3592023630547953731, "value3":false, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282191, "value1":0.8843261061566057, "value2":6691527928807377675, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938895, "value1":0.7167094005927336, "value2":1624441217258843945, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492025, "value1":0.09243530358924243, "value2":1564407566445130438, "value3":false, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837030, "value1":0.7918808546682014, "value2":6330206212330805579, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.123530, "value1":0.594358060271596, "value2":1086840755761366155, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.515405, "value1":0.8774866546466452, "value2":6471463500843130242, "value3":false, "__name__":"metric_00000466", "key_7":"h","key_0":"b","key_5":"g", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814527, "value1":0.46973250770262637, "value2":617895520080191082, "value3":true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.712491, "value1":0.8366069393700927, "value2":8124214803409242134, "value3":true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.598705, "value1":0.13529834276697159, "value2":4048137400971693133, "value3":false, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.115099, "value1":0.013374612744122648, "value2":4497239763994228002, "value3":false, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.871918, "value1":0.5477513867264745, "value2":6929880807654898865, "value3":false, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.600230, "value1":0.06263242199896547, "value2":7099533171627178830, "value3":false, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.832053, "value1":0.9159517384120022, "value2":6068004913866310526, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497324, "value1":0.7077689276515862, "value2":8496454581274124817, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912067, "value1":0.9966402570764005, "value2":6508382435383037912, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394175, "value1":0.4056622098754871, "value2":7483291783499825043, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931738, "value1":0.7229091089946994, "value2":6319491958713592651, "value3":false, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.778862, "value1":0.3546846056067059, "value2":5823244617744051085, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.571315, "value1":0.32603322678558494, "value2":617900084765432670, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.022106, "value1":0.09634190796880056, "value2":1538435119318752571, "value3":false, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777759, "value1":0.9991415497234644, "value2":6469274665586859562, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813159, "value1":0.6009731720422214, "value2":5681496788640905646, "value3":false, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990295, "value1":0.866629146873839, "value2":454920947769684349, "value3":false, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577898, "value1":0.4068627360956291, "value2":8354382964548721361, "value3":false, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652554, "value1":0.25976343730256174, "value2":4258649045980411819, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031275, "value1":0.4835922460947118, "value2":1965749848088972278, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535728, "value1":0.0741614372394035, "value2":2365543114658538202, "value3":true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.725451, "value1":0.08897749963143643, "value2":2568717794069312737, "value3":true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517633, "value1":0.31412074219954, "value2":3756199754285644161, "value3":false, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170595, "value1":0.31501680305424884, "value2":835864889280146724, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.839334, "value1":0.8860221287236891, "value2":7263801537900469082, "value3":false, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874807, "value1":0.37435025928000243, "value2":7922739501753662605, "value3":false, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914081, "value1":0.7766285758384013, "value2":8262805777914063347, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.668141, "value1":0.6740415300239384, "value2":5791687526396481300, "value3":false, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.302177, "value1":0.7738583815242437, "value2":4861664383009090324, "value3":false, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711096, "value1":0.314472985957645, "value2":8509002774444373125, "value3":false, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321149, "value1":0.44826985902573663, "value2":5704588376153110492, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.438786, "value1":0.9611261969930821, "value2":8417802456083541652, "value3":true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894152, "value1":0.47571072484930443, "value2":22587301710957954, "value3":true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939556, "value1":0.9226156153333962, "value2":5568759137623866023, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159870, "value1":0.6737683215814319, "value2":23284244914642959, "value3":true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273159, "value1":0.7150778132424469, "value2":7795879578675436289, "value3":true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.116050, "value1":0.7176538032312242, "value2":6236044932175094049, "value3":true, "__name__":"metric_00000526", "key_4":"c","key_0":"k", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274847, "value1":0.2989651835291124, "value2":2096206477458603718, "value3":true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764340, "value1":0.5947263013675924, "value2":949703240129275173, "value3":true, "__name__":"metric_00000507", "key_3":"i","key_1":"a", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.073476, "value1":0.1345576666087966, "value2":2417122739684294792, "value3":true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418445, "value1":0.8647442373572726, "value2":4787011153359227960, "value3":true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721780, "value1":0.31241733378602804, "value2":2882725760348945089, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338609, "value1":0.30106180327338455, "value2":4145007555047407199, "value3":true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109899, "value1":0.7817730958310003, "value2":5914588161487452188, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.547269, "value1":0.29274410504915255, "value2":3689334620249634917, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741862, "value1":0.6895772402009777, "value2":4687778247837546339, "value3":true, "__name__":"metric_00000527", "key_5":"b","key_9":"k","key_3":"a", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.744160, "value1":0.7533050071878981, "value2":183319309026503278, "value3":false, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.880890, "value1":0.8337197335863336, "value2":5434700933804867558, "value3":false, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467450, "value1":0.7739450951598513, "value2":8969122970020127825, "value3":false, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036765, "value1":0.05604892859180446, "value2":5414621267015229038, "value3":false, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458853, "value1":0.3999783567682559, "value2":7084993602991838778, "value3":true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.144253, "value1":0.945939865721221, "value2":9151551595196013169, "value3":false, "__name__":"metric_00000533", "key_3":"b","key_9":"f","key_2":"i", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.519078, "value1":0.28562220040037745, "value2":3466327056358787385, "value3":false, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.425448, "value1":0.1561471042921615, "value2":8555609213379362480, "value3":false, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.071015, "value1":0.867937414704974, "value2":7843796710543352776, "value3":true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936460, "value1":0.8572536817998362, "value2":3953769786680934648, "value3":false, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886358, "value1":0.2289778721765615, "value2":7617650297619769040, "value3":false, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.029905, "value1":0.0015729191075447354, "value2":2294660572707247909, "value3":false, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777761, "value1":0.9646894479758846, "value2":530724906601952281, "value3":true, "__name__":"metric_00000539", "key_6":"e","key_3":"i", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294885, "value1":0.6419477950120894, "value2":3626154680940200317, "value3":true, "__name__":"metric_00000545", "key_6":"g","key_3":"c", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210708, "value1":0.9629469371323777, "value2":1825251501924013732, "value3":true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354794, "value1":0.9560891897559984, "value2":7402084087841622412, "value3":true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.007599, "value1":0.8488357845508476, "value2":9129548229641808509, "value3":true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.949198, "value1":0.3153555717941824, "value2":7434568718980757764, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373802, "value1":0.3277196318709405, "value2":3450473392216255589, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.815211, "value1":0.19640953195392433, "value2":6991753246444170561, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532006, "value1":0.8004824485602414, "value2":6264834568239277372, "value3":true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.745472, "value1":0.20136170467216696, "value2":5514377914099787271, "value3":true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.296097, "value1":0.9552161212557331, "value2":1452145138149816534, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350623, "value1":0.4834905958063239, "value2":3177892637454509046, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894862, "value1":0.26274803670489, "value2":5904879356296841126, "value3":false, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663989, "value1":0.4617229653712207, "value2":5839673609391532826, "value3":true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920142, "value1":0.4202645422145252, "value2":6322617270554106528, "value3":false, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174524, "value1":0.7616258127751694, "value2":465160921360548556, "value3":true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829492, "value1":0.9553481357587528, "value2":1445392053101953093, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.419698, "value1":0.7725216991307152, "value2":4285351248890366822, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235468, "value1":0.9784345273084265, "value2":3668509819811778630, "value3":true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935515, "value1":0.5182554276152802, "value2":3295309486606700899, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539034, "value1":0.37544115144770396, "value2":7306450144895271272, "value3":true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729718, "value1":0.8597111754781319, "value2":3238304594948466970, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538038, "value1":0.32408768170822555, "value2":2186207806521299657, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566784, "value1":0.4892156459829451, "value2":8596226688481136458, "value3":false, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.980628, "value1":0.8327067880954203, "value2":8561137068463144173, "value3":true, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153522, "value1":0.15467129518207473, "value2":8602088790336601513, "value3":false, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748640, "value1":0.18204671422495655, "value2":6797988693835752844, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247682, "value1":0.09249535842888819, "value2":5067988675442272893, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936986, "value1":0.9192193881284415, "value2":5123985662043884348, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378293, "value1":0.815846067412878, "value2":6070693823656037795, "value3":false, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734247, "value1":0.49161995996865615, "value2":624464230830574462, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.661797, "value1":0.2056464067499432, "value2":6156118294329348494, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.033052, "value1":0.35405594685495784, "value2":520776464703947921, "value3":false, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.575933, "value1":0.9335712679806217, "value2":425476291317360309, "value3":true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.732828, "value1":0.4700815629744225, "value2":8550463063060596020, "value3":true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.899613, "value1":0.4385944862087238, "value2":7654768751774889730, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165683, "value1":0.16899343532059313, "value2":5688582074969409894, "value3":false, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986964, "value1":0.253512537787597, "value2":8166110434329136902, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766787, "value1":0.6248495968383331, "value2":8035594691046262470, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338227, "value1":0.6533144293221477, "value2":3922786069578896774, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912364, "value1":0.888149605868496, "value2":7372081699617979857, "value3":true, "__name__":"metric_00000589", "key_8":"f","key_1":"a","key_4":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.223399, "value1":0.21329948027080436, "value2":8415945829680890770, "value3":false, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.922973, "value1":0.6987095535905381, "value2":8304144963505893865, "value3":false, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.484292, "value1":0.6786625244755433, "value2":595838710724334047, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977202, "value1":0.8656898577077133, "value2":3529785897914282727, "value3":true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.231129, "value1":0.9035229811237685, "value2":2477457900157688998, "value3":false, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.565357, "value1":0.40499024658110055, "value2":1356497801667980388, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.691153, "value1":0.5098765403884086, "value2":2450630350992474672, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459909, "value1":0.07976566115765178, "value2":3732709013710166299, "value3":true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104179, "value1":0.3445131016649922, "value2":528513889221012225, "value3":true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764200, "value1":0.1348858138724795, "value2":409727276676244812, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288182, "value1":0.46853874054271116, "value2":6728001446241512599, "value3":true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502298, "value1":0.31577497007213595, "value2":8859554815564990855, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990152, "value1":0.3102588503519357, "value2":5723742327078842798, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520176, "value1":0.15090023272414005, "value2":5438285255049886987, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.155921, "value1":0.6850691286515489, "value2":7357616240486203605, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350044, "value1":0.8819289168133723, "value2":4656926419487785926, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.383040, "value1":0.8254228746494352, "value2":2036646364854508134, "value3":false, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451480, "value1":0.9507122214170446, "value2":4089393907406910121, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516899, "value1":0.7707382433506376, "value2":5274734231531309553, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467177, "value1":0.603964532592086, "value2":1716897544081104224, "value3":false, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264870, "value1":0.9573825263948318, "value2":3412285010274210994, "value3":true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.556842, "value1":0.17542611082882992, "value2":1043370942216175800, "value3":false, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009549, "value1":0.21407922779664104, "value2":5715745607073532896, "value3":false, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462287, "value1":0.8132271496462169, "value2":7467232826614980432, "value3":true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538632, "value1":0.7627133328751289, "value2":2219523491939336599, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.217749, "value1":0.9344741716490049, "value2":8166313022724995390, "value3":true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811552, "value1":0.9190437296914767, "value2":5885018846694177935, "value3":true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814548, "value1":0.8390440310277583, "value2":5493133492272371657, "value3":false, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.817102, "value1":0.6598894636593393, "value2":8537596977440503466, "value3":false, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.067200, "value1":0.263575315223174, "value2":2125629267937428501, "value3":false, "__name__":"metric_00000619", "key_1":"i","key_5":"h","key_0":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082507, "value1":0.6268250434274911, "value2":9058686535109969227, "value3":false, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734195, "value1":0.3167788157158676, "value2":7432987222538757392, "value3":true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.795085, "value1":0.4488172699242242, "value2":4473525864538142538, "value3":false, "__name__":"metric_00000622", "key_6":"b","key_8":"i","key_4":"d", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.332775, "value1":0.8505535842615966, "value2":6047205731470921861, "value3":true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727338, "value1":0.8876718108712484, "value2":1531222095185940701, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.387878, "value1":0.5788599279069843, "value2":8604920509885625807, "value3":false, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.233954, "value1":0.6310172938155442, "value2":8925576745188878375, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.805738, "value1":0.9330396354121838, "value2":1424947937953535481, "value3":true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726267, "value1":0.32286499179099665, "value2":3766446177817631940, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229570, "value1":0.9916323313225612, "value2":3888482529661188248, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167559, "value1":0.08178718223938254, "value2":7520256881259787943, "value3":false, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845474, "value1":0.6808433459458884, "value2":7844247683138918907, "value3":false, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400844, "value1":0.39452808620697233, "value2":7357453563119638881, "value3":false, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206854, "value1":0.9379245513406866, "value2":6598367899784187154, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640341, "value1":0.11560748654575319, "value2":414342029075207599, "value3":false, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497616, "value1":0.26256799403609016, "value2":7366324474884701561, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.822083, "value1":0.6049405008185211, "value2":6275944038502753172, "value3":true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.819954, "value1":0.373947872013572, "value2":3355608188213748317, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739894, "value1":0.06143600864431769, "value2":309096149476303416, "value3":true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.684612, "value1":0.7692595721499597, "value2":2629612954063953162, "value3":true, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958165, "value1":0.6185066606297357, "value2":2824340682692220488, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944290, "value1":0.3864309642344103, "value2":6500366333535522458, "value3":false, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576232, "value1":0.6603219832087598, "value2":7174737056650755203, "value3":true, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.047735, "value1":0.4501836136879949, "value2":7043766590114442160, "value3":false, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282665, "value1":0.3628575018664628, "value2":3673366331802014852, "value3":false, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.590235, "value1":0.30332064856581503, "value2":5473820163155790307, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995108, "value1":0.1854914610326227, "value2":3604467487566582522, "value3":true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270312, "value1":0.9632251653093951, "value2":4816282908135882739, "value3":false, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541752, "value1":0.2581250654809775, "value2":6451610961696076892, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535737, "value1":0.3163976646275719, "value2":2322386485534186745, "value3":false, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610988, "value1":0.49088683208094147, "value2":4479251775302671, "value3":true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.111345, "value1":0.016089752487428322, "value2":4062002565116218346, "value3":true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.380428, "value1":0.8878960076695425, "value2":7482838775142667379, "value3":true, "__name__":"metric_00000652", "key_5":"a","key_9":"i","key_4":"f", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.286191, "value1":0.5281244570210905, "value2":4742826162899756346, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.633864, "value1":0.28655049708989444, "value2":2281246548041472554, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517718, "value1":0.8760337085208192, "value2":6876760617398624261, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275264, "value1":0.8763572333888452, "value2":2040006649964380357, "value3":false, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.133688, "value1":0.03683553631174895, "value2":879315984438783745, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.178775, "value1":0.7921271023115032, "value2":8974452178625132408, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.559621, "value1":0.6842414958291543, "value2":4473030242509416772, "value3":false, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.933147, "value1":0.42656042046254644, "value2":6875784776532081765, "value3":true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.265632, "value1":0.6577298952553573, "value2":4117148991099157160, "value3":false, "__name__":"metric_00000661", "key_6":"e","key_1":"d","key_5":"k", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.291705, "value1":0.35063088013808896, "value2":6022154315116551060, "value3":false, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.486399, "value1":0.9066996869066141, "value2":3890435706555388518, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539080, "value1":0.6597581253474917, "value2":7149394823928068405, "value3":false, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.172124, "value1":0.8972702809236033, "value2":5889011931565346539, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399820, "value1":0.22420487156545466, "value2":3632173066141869984, "value3":false, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857491, "value1":0.658023411947534, "value2":5044439445561970794, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.960983, "value1":0.8072960706443908, "value2":4799697070254932404, "value3":true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672380, "value1":0.6734321372300892, "value2":7080832382541851744, "value3":false, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.699212, "value1":0.7095725169760027, "value2":1829131464356530776, "value3":false, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893830, "value1":0.2302554104612147, "value2":9188062021573390442, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520337, "value1":0.16075557090608333, "value2":219133269848494495, "value3":false, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173477, "value1":0.6892053752715795, "value2":2151318569219964336, "value3":true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229017, "value1":0.9691975492787838, "value2":1366609608627051856, "value3":true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659208, "value1":0.7460794334063994, "value2":817106321355418532, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.361925, "value1":0.05687743883366658, "value2":7921175806067397073, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.523858, "value1":0.7866547024502999, "value2":4071070330904780510, "value3":true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920001, "value1":0.5740949320374887, "value2":487306006337280717, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775196, "value1":0.1341448972829432, "value2":7272783478294294123, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000330, "value1":0.1260519175980482, "value2":9041625229579242321, "value3":false, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313318, "value1":0.4146999970169919, "value2":6937035767989914959, "value3":true, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147057, "value1":0.5790734522844233, "value2":1184517563846638764, "value3":true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.331707, "value1":0.5826950424479324, "value2":7357100402735593295, "value3":true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.476945, "value1":0.09337572406541937, "value2":3528304154285323988, "value3":true, "__name__":"metric_00000684", "key_4":"i","key_5":"d","key_2":"e", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.660357, "value1":0.9211165869524813, "value2":6490630229378478544, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.978786, "value1":0.24445422171222897, "value2":6316462580906113749, "value3":true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031375, "value1":0.9435579557750952, "value2":3775564649397357096, "value3":true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786489, "value1":0.19831604296196023, "value2":4519338814156406008, "value3":false, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845707, "value1":0.0733198408835947, "value2":6043258187561628613, "value3":false, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607379, "value1":0.49574791837525684, "value2":2511607289885086712, "value3":true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541151, "value1":0.9239913172285873, "value2":8906754784907447893, "value3":true, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409187, "value1":0.347121325872609, "value2":4022279358676190140, "value3":false, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179171, "value1":0.9901703571238136, "value2":1971122679978818549, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807000, "value1":0.9794430062314226, "value2":4140173177331652200, "value3":true, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514307, "value1":0.3368836183017325, "value2":6646214711319824003, "value3":true, "__name__":"metric_00000695", "key_8":"d","key_9":"g","key_6":"a", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.430324, "value1":0.6783643243115747, "value2":8700387930430110357, "value3":false, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245716, "value1":0.5777847683942137, "value2":1772788621154859179, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825752, "value1":0.8229486912139793, "value2":3218217186376093534, "value3":true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.550373, "value1":0.44484600976320454, "value2":1214217372893594474, "value3":true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810847, "value1":0.6410630380778536, "value2":1107368725861373561, "value3":true, "__name__":"metric_00000700", "key_7":"e","key_3":"g","key_4":"b", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258640, "value1":0.3699636915738955, "value2":3778046401086414746, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354190, "value1":0.2787621713653213, "value2":7905127146501563870, "value3":true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276300, "value1":0.38144875313061755, "value2":279189270439050245, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030267, "value1":0.09524338073167776, "value2":624319768896255519, "value3":false, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395031, "value1":0.017165936825319472, "value2":1839096275149454289, "value3":false, "__name__":"metric_00000659", "key_3":"c","key_8":"i","key_1":"k", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920079, "value1":0.3695378285006256, "value2":3642800979833317953, "value3":false, "__name__":"metric_00000681", "key_3":"f","key_9":"f","key_1":"e", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055296, "value1":0.5098042037064671, "value2":1508579590160432139, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128994, "value1":0.8793028393199556, "value2":5718919044160675205, "value3":false, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418223, "value1":0.7389480744126634, "value2":5430202156875304420, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458004, "value1":0.7589090833562657, "value2":7035246718540970070, "value3":false, "__name__":"metric_00000682", "key_6":"k","key_0":"f","key_3":"b", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492634, "value1":0.3450062925676948, "value2":289715553515433637, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965681, "value1":0.9053649041017894, "value2":2294309054504906499, "value3":true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234931, "value1":0.0732565141042219, "value2":2755939599945126397, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939989, "value1":0.8451432422468432, "value2":2656355912924482468, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862255, "value1":0.3071521473170743, "value2":4029052487902069438, "value3":true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027859, "value1":0.5418227496728708, "value2":5130775345772381962, "value3":false, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016707, "value1":0.5241900456864456, "value2":1703509093458181611, "value3":true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.129222, "value1":0.45379187899537626, "value2":8646708304907761402, "value3":false, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167400, "value1":0.4118569437426427, "value2":1933032658312744458, "value3":false, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.448154, "value1":0.9880224154363099, "value2":1958598830665944865, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095756, "value1":0.7737922723291526, "value2":1554270443791925823, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.669733, "value1":0.05998044134164805, "value2":4233304263515802857, "value3":false, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360452, "value1":0.021777318888489317, "value2":1462350357926539483, "value3":true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.791525, "value1":0.24698569461323847, "value2":6782513953965205791, "value3":false, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.136814, "value1":0.13312227926717296, "value2":2973379075607635624, "value3":false, "__name__":"metric_00000715", "key_5":"a","key_2":"i", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.611322, "value1":0.06800531937511199, "value2":2253428998305264546, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000702, "value1":0.40505920041369187, "value2":7444308564790803777, "value3":false, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.317483, "value1":0.5921456568045917, "value2":1159233331687655633, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.207714, "value1":0.03623125609776327, "value2":4050612595565072329, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739699, "value1":0.1967627573726683, "value2":5128621003255593133, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938178, "value1":0.8074789236886436, "value2":5329361240090086591, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075581, "value1":0.36183126859676484, "value2":8380824340175200195, "value3":false, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.840434, "value1":0.36564465907617033, "value2":4138989352760782569, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697538, "value1":0.8089117668002023, "value2":2706669265618979267, "value3":true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810618, "value1":0.9822561514547918, "value2":6363847253975708775, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453492, "value1":0.7521891954121415, "value2":3415954419502830788, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.246679, "value1":0.291250645374521, "value2":6787758008675320095, "value3":false, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.717578, "value1":0.21885147116003636, "value2":8249910093492361560, "value3":false, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.584635, "value1":0.267173208281922, "value2":6350566579717368150, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288596, "value1":0.8034133849819789, "value2":3057165042866020348, "value3":false, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043225, "value1":0.8891915956366093, "value2":1172085376591869222, "value3":true, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514557, "value1":0.7993701790896678, "value2":3530709071532758425, "value3":false, "__name__":"metric_00000734", "key_8":"h","key_3":"k","key_4":"d", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727549, "value1":0.7581402483767911, "value2":2882149021889454970, "value3":true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944767, "value1":0.03125902081908963, "value2":5930819025045052350, "value3":false, "__name__":"metric_00000735", "key_9":"c","key_6":"c","key_7":"k", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635131, "value1":0.5563692033064526, "value2":3139117266540547008, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901432, "value1":0.08725832596843655, "value2":2516400281682537507, "value3":false, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965447, "value1":0.24555624361218226, "value2":8420031154067673423, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653005, "value1":0.6015476287649423, "value2":7179718486091286242, "value3":true, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.251270, "value1":0.6303898680513278, "value2":1976335650844570524, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.902243, "value1":0.42829690884430943, "value2":7763758072113593479, "value3":true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748351, "value1":0.5394339139974091, "value2":1947184765867041268, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.496788, "value1":0.0776914163879659, "value2":4120402058111565140, "value3":false, "__name__":"metric_00000742", "key_6":"j","key_4":"a","key_5":"g", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202592, "value1":0.3950986449717796, "value2":7867550414487529994, "value3":false, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.475860, "value1":0.5847506627916105, "value2":1395577688027634835, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307279, "value1":0.66953692642835, "value2":1884810173101069714, "value3":false, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447867, "value1":0.30049434187635843, "value2":2021494635928386331, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371456, "value1":0.9583003775042523, "value2":8408660678843228458, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321951, "value1":0.673208957772375, "value2":4484934316922285119, "value3":false, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.654317, "value1":0.34301031853223307, "value2":1377167639028225502, "value3":false, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258833, "value1":0.5754272342729656, "value2":3672436572873719874, "value3":false, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894800, "value1":0.27153721678184795, "value2":6771295009871301863, "value3":false, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886915, "value1":0.4211098948169063, "value2":327314688390276361, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464071, "value1":0.8401442859431041, "value2":7237906700870505791, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179548, "value1":0.35527262545135796, "value2":6929428209196710992, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348853, "value1":0.7390477554491065, "value2":6017303276519989534, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228799, "value1":0.7190691653435217, "value2":3466847624797872404, "value3":false, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.676189, "value1":0.8701176990093307, "value2":8139200890464740272, "value3":true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780301, "value1":0.31331610868424237, "value2":7970934533227613450, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.763684, "value1":0.6117316228567153, "value2":1342308618516991103, "value3":true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.510130, "value1":0.26253831176449904, "value2":223167711418991037, "value3":false, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.054906, "value1":0.30113096258283606, "value2":9029709188884415357, "value3":true, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.101830, "value1":0.6535137630775729, "value2":8047896116508601877, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.816267, "value1":0.6779088610569585, "value2":402736258268648092, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794465, "value1":0.6385825716360082, "value2":647146760807700438, "value3":true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.342961, "value1":0.08518129482765982, "value2":3079348210933738649, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503723, "value1":0.5000975675164184, "value2":3087069466664576618, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.552740, "value1":0.46623198603217897, "value2":5991311418948024851, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.634526, "value1":0.1623989693910568, "value2":880550310997257962, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409701, "value1":0.18776946541701378, "value2":4672037814003639717, "value3":false, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928628, "value1":0.5544683022868725, "value2":8786609556345104074, "value3":false, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422305, "value1":0.3339761609696324, "value2":2037434470333278554, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497260, "value1":0.21504973222613463, "value2":8744267736203582723, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348792, "value1":0.7267165846871734, "value2":2029452385366977955, "value3":false, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767788, "value1":0.7268443539701483, "value2":5120204611450631315, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313022, "value1":0.3937734460488266, "value2":8789465407671191549, "value3":true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373011, "value1":0.5815734596824621, "value2":7433149153695507174, "value3":false, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.403959, "value1":0.19547385196106162, "value2":4536904833017601954, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702004, "value1":0.787480554646893, "value2":6826871522585433391, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238346, "value1":0.3158260333418349, "value2":4890876749664266809, "value3":false, "__name__":"metric_00000781", "key_7":"h","key_9":"d","key_2":"g", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.184806, "value1":0.1658734659451667, "value2":2103368389172466197, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646125, "value1":0.7611256397426871, "value2":3779878057324979416, "value3":false, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.963683, "value1":0.6784369755611176, "value2":7198319968843792884, "value3":true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264784, "value1":0.6944677799630911, "value2":1361718740081763677, "value3":true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.606235, "value1":0.9767553574110414, "value2":8172694375569168120, "value3":true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.555524, "value1":0.0757100455425729, "value2":6369412142150141094, "value3":true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.683577, "value1":0.23488908846392256, "value2":7496440285542052380, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741203, "value1":0.6152800537206294, "value2":4336041330603516829, "value3":false, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.076957, "value1":0.46349128481126883, "value2":1995149205719248029, "value3":true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360568, "value1":0.18125853395781905, "value2":5023669000017633099, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.127735, "value1":0.2064987900720652, "value2":1090691939971047635, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182440, "value1":0.18035240548503653, "value2":2377567143003176008, "value3":false, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173358, "value1":0.9386364543056086, "value2":6498536821606623122, "value3":true, "__name__":"metric_00000785", "key_9":"k","key_5":"j","key_6":"d", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.749875, "value1":0.9851379736422945, "value2":9068316297781013801, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262923, "value1":0.6215747239294878, "value2":3444654966165617334, "value3":true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.471762, "value1":0.9687744980352465, "value2":4008980207107557770, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.449800, "value1":0.43369424684152247, "value2":1796590317391088733, "value3":false, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799032, "value1":0.7408688395558841, "value2":1701592597588021265, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727561, "value1":0.14048934174685151, "value2":1964669731520342830, "value3":false, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393461, "value1":0.22927403588573142, "value2":6637169452408438633, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577492, "value1":0.9899876418682123, "value2":417258805492276216, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.149873, "value1":0.2451895383714133, "value2":6783012468215477739, "value3":false, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794818, "value1":0.2978983436952165, "value2":6898137216027059682, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826276, "value1":0.03031011408512658, "value2":6243267082953045335, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853432, "value1":0.8319004837974954, "value2":6916103698866916518, "value3":true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336756, "value1":0.6573141098461979, "value2":7552682537825884423, "value3":true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280458, "value1":0.9944062499165491, "value2":4515137518875248587, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723258, "value1":0.8389015105022624, "value2":4385100343250672688, "value3":false, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.518510, "value1":0.262846594625773, "value2":8956848533096958437, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398441, "value1":0.29945983222092276, "value2":7583507905789284927, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169305, "value1":0.3509582334049355, "value2":4486670807718872528, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229006, "value1":0.9454115215233667, "value2":4148482506114005261, "value3":true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.108860, "value1":0.7792992725899739, "value2":6662551108461731004, "value3":false, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270948, "value1":0.22595556811111142, "value2":4858846807835096111, "value3":false, "__name__":"metric_00000808", "key_3":"e","key_6":"b","key_1":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739267, "value1":0.10553288145788361, "value2":3903915559852076766, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.771104, "value1":0.2713630569195371, "value2":9127087478037486061, "value3":true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234974, "value1":0.4512849051156243, "value2":388475201207159224, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021067, "value1":0.8525575906494762, "value2":6112677337472259317, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360331, "value1":0.5466498354493777, "value2":8416483456048971785, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.083431, "value1":0.6268377803161493, "value2":6874204752716819531, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543079, "value1":0.356809340191254, "value2":1479162719217476511, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543913, "value1":0.1823988812868871, "value2":1772497490870724163, "value3":true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697354, "value1":0.6008728382547858, "value2":4808659248566150951, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173629, "value1":0.11694918608631227, "value2":8374732016838634205, "value3":false, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.478338, "value1":0.6523910738528493, "value2":6127719310482278347, "value3":false, "__name__":"metric_00000825", "key_8":"d","key_5":"k","key_7":"j", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195226, "value1":0.11728848022547718, "value2":2692984308355874383, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472792, "value1":0.3849335854410186, "value2":5551198795362400796, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.088581, "value1":0.6479908764028293, "value2":6646669667589759935, "value3":false, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.051281, "value1":0.8063603427465644, "value2":4677280644621004022, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.421107, "value1":0.8098711885881693, "value2":781451797112683390, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702263, "value1":0.7459154166325007, "value2":8763722224980194522, "value3":false, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.105837, "value1":0.6082412929946549, "value2":9076065007808782827, "value3":false, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229179, "value1":0.5208679934438682, "value2":1614625428456495501, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452972, "value1":0.6602375068241643, "value2":3665988082745333789, "value3":true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570733, "value1":0.8539881179697112, "value2":8291534021329215591, "value3":false, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.854105, "value1":0.19867014748060782, "value2":4707537791933933479, "value3":false, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134473, "value1":0.013248508886480496, "value2":8975225813346771480, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.401924, "value1":0.8131686031327187, "value2":1570626633965770939, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462042, "value1":0.33875887576127867, "value2":1531594524235268547, "value3":true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780840, "value1":0.6167438470784343, "value2":3111876176210806342, "value3":false, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157314, "value1":0.024850360062745134, "value2":2155248434369533975, "value3":false, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.892341, "value1":0.29151476400849524, "value2":3929095109066478763, "value3":true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811920, "value1":0.3960145984061985, "value2":8079803277636061346, "value3":true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010049, "value1":0.1522516905011452, "value2":3887799589835913190, "value3":false, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.511030, "value1":0.1096312618999367, "value2":4683382419730786111, "value3":false, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799775, "value1":0.8107638537946521, "value2":1548400788132615580, "value3":true, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.034128, "value1":0.8145920212010633, "value2":4095930474475755353, "value3":false, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506330, "value1":0.8145793344024634, "value2":7248985564172971873, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.632617, "value1":0.530965707458104, "value2":6582545459218253298, "value3":false, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.427368, "value1":0.5488520970953542, "value2":7764537337432663671, "value3":true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128543, "value1":0.339288607631015, "value2":8135936826126620967, "value3":false, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.783911, "value1":0.4072868133390787, "value2":398383887746388268, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030437, "value1":0.40039567718797253, "value2":5009000449434898292, "value3":true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.301665, "value1":0.2851345919898927, "value2":196381052074952545, "value3":false, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740740, "value1":0.481454729816193, "value2":2929970498837276050, "value3":true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867428, "value1":0.8246737181844912, "value2":4477437696490734479, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.708771, "value1":0.44011710908297325, "value2":7995836793967148678, "value3":false, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663309, "value1":0.07002506566825563, "value2":3582676294388926669, "value3":false, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.628344, "value1":0.9801112824460718, "value2":196546046309844748, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330627, "value1":0.6390220685465513, "value2":5252094592319160612, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.014786, "value1":0.4666494400039896, "value2":6746944521794840076, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.736397, "value1":0.1650175517955452, "value2":4725786271252183916, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971200, "value1":0.08835643831453004, "value2":1662264677868065737, "value3":false, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.804675, "value1":0.3583297226255054, "value2":6764608743691309744, "value3":true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577888, "value1":0.7784009799351271, "value2":7389846578443042716, "value3":false, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942762, "value1":0.5281041922180323, "value2":225078423814016880, "value3":true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807869, "value1":0.12159890893033082, "value2":2007944198207870435, "value3":false, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393243, "value1":0.9174108973827193, "value2":5956246567273801920, "value3":true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842305, "value1":0.6881483813172635, "value2":51917468498479892, "value3":false, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.625958, "value1":0.7332068726661788, "value2":1595636028517738482, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.738053, "value1":0.19091798634208768, "value2":2237197021871432008, "value3":true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268129, "value1":0.8406719661444901, "value2":2842521222444292729, "value3":false, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865313, "value1":0.8513846203817129, "value2":3859037819752068385, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762176, "value1":0.0431523780543075, "value2":6794482877795635511, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.658034, "value1":0.12118411547914006, "value2":7727119685187980691, "value3":false, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497498, "value1":0.8736017830767735, "value2":5281383221003217421, "value3":false, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016157, "value1":0.45635245796699264, "value2":6588970499031475943, "value3":false, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434398, "value1":0.5982494402461963, "value2":3451343743898655419, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995659, "value1":0.41609794124038807, "value2":2577003956591436853, "value3":true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.402057, "value1":0.3168156284522602, "value2":7664962841417781702, "value3":false, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874274, "value1":0.8588869341047012, "value2":6471105672211793135, "value3":true, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.193561, "value1":0.7716291581375219, "value2":6758054142720243683, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.298108, "value1":0.8739782093698222, "value2":7600130946423390938, "value3":true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008667, "value1":0.15272944664848967, "value2":28862629936648465, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924564, "value1":0.2262144335029645, "value2":2100662153491567952, "value3":true, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.250052, "value1":0.2945787294595236, "value2":6018457540699249059, "value3":false, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395074, "value1":0.2290685251999151, "value2":4865312816831141048, "value3":false, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414183, "value1":0.2892057090700859, "value2":3284381379197405662, "value3":true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800711, "value1":0.20023612385737624, "value2":5612718418445245977, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096882, "value1":0.6190652975917784, "value2":4456707593278158335, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811596, "value1":0.1974893316007636, "value2":8350883557246751407, "value3":true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.328602, "value1":0.3577823211406552, "value2":3856849717607100839, "value3":true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165294, "value1":0.1558044782512875, "value2":2819862014599071323, "value3":true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.934329, "value1":0.6967191883920082, "value2":8317801698439570519, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826310, "value1":0.5889684966352713, "value2":8904148697557725595, "value3":false, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.544062, "value1":0.3036793591749508, "value2":1172054639314476559, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104925, "value1":0.0006382055160369547, "value2":4527189243213341516, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577496, "value1":0.8262646875704296, "value2":6857852785226312453, "value3":false, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.304163, "value1":0.23448830223910028, "value2":118297432460308257, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.201626, "value1":0.8649386204518231, "value2":2335277383532252247, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.072447, "value1":0.305504173455374, "value2":8879950191665801641, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635410, "value1":0.8646407642266409, "value2":4794888988052345760, "value3":true, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799765, "value1":0.020261599778331852, "value2":6569370350090311777, "value3":true, "__name__":"metric_00000891", "key_7":"i","key_2":"i","key_3":"d", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.643397, "value1":0.25580026537842243, "value2":8727368389050428851, "value3":true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886737, "value1":0.709498290535563, "value2":159638875141661569, "value3":false, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027896, "value1":0.4252750468577108, "value2":5985063599490557923, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659322, "value1":0.8283450566415441, "value2":3829413923759281391, "value3":true, "__name__":"metric_00000893", "key_2":"h","key_1":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.323182, "value1":0.05823156526332208, "value2":158181159624437675, "value3":true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721491, "value1":0.058149447228806335, "value2":4425192040964093188, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.715378, "value1":0.1468510162162874, "value2":957193015456701660, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759517, "value1":0.11145345073332971, "value2":9085390279667186751, "value3":false, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179380, "value1":0.22290378424729548, "value2":4452524214324184228, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018314, "value1":0.08677941054726214, "value2":6248669636718030352, "value3":false, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579864, "value1":0.1843770522783152, "value2":2395237031433429826, "value3":true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907863, "value1":0.19391713305151562, "value2":1831578948753070965, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567206, "value1":0.1200213703180807, "value2":281170079691225837, "value3":true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.908312, "value1":0.16873762365247053, "value2":3349277004368724008, "value3":true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.203346, "value1":0.3461718122656804, "value2":2053673338515188112, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.252421, "value1":0.8594831603279349, "value2":8129734681001143206, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862108, "value1":0.8625875861161819, "value2":266750263503917399, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394606, "value1":0.1284747365418792, "value2":511162186129513778, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104334, "value1":0.348212621993861, "value2":8809834773329001166, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.390817, "value1":0.056538788821091236, "value2":959306944656093342, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371946, "value1":0.0354556141213121, "value2":5315204481196619882, "value3":false, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027057, "value1":0.04051346889125732, "value2":19281603192892033, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315646, "value1":0.29470865205235175, "value2":3350853301006328333, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516736, "value1":0.32421705083072555, "value2":6409364233498955235, "value3":true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409843, "value1":0.8431693191414327, "value2":2309002455167863238, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.290066, "value1":0.769846734088458, "value2":1354615135861452763, "value3":false, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025458, "value1":0.04383062091902626, "value2":6270990659808009038, "value3":false, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422265, "value1":0.4215091845870574, "value2":903506811871753839, "value3":true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954147, "value1":0.1703656776121607, "value2":8625109899283360680, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468022, "value1":0.8656906467989927, "value2":8844353092623218329, "value3":false, "__name__":"metric_00000933", "key_6":"d","key_2":"g","key_4":"j", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.572694, "value1":0.03657142931464268, "value2":7269928716214871698, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.035238, "value1":0.2174690883774512, "value2":2760303323859671187, "value3":true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.500319, "value1":0.8848932111671852, "value2":124147493136831346, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.324572, "value1":0.32024379123233365, "value2":8082835280902735033, "value3":true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585321, "value1":0.18246288567364422, "value2":7470239488455160357, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726932, "value1":0.8209711897357084, "value2":4430579260139045397, "value3":false, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.851627, "value1":0.9988163498901884, "value2":7782482596272329873, "value3":true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126247, "value1":0.7338638289875997, "value2":2035846164975007267, "value3":true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923987, "value1":0.31023714156918913, "value2":7969614988653223555, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.797525, "value1":0.9308157523651853, "value2":2008465742081738773, "value3":true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595532, "value1":0.6306393412084162, "value2":1794355738915834222, "value3":false, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078930, "value1":0.46045211861703167, "value2":2231560481693103072, "value3":false, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.835058, "value1":0.49083826368226746, "value2":947523171682771279, "value3":false, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.757956, "value1":0.5783027668846169, "value2":7939213319454038876, "value3":false, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.431201, "value1":0.249271140930431, "value2":7943702047022612152, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.674800, "value1":0.9050148884978706, "value2":4701267498128748767, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294430, "value1":0.37838796407920183, "value2":6812226255195471591, "value3":false, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837164, "value1":0.41192774227492235, "value2":1535119829477772546, "value3":false, "__name__":"metric_00000946", "key_2":"d","key_1":"j", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464005, "value1":0.1600487895075015, "value2":1434123145157650204, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546806, "value1":0.6061159682210883, "value2":6208205100336254753, "value3":false, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096572, "value1":0.8332138223552116, "value2":7164548384493446541, "value3":true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.325386, "value1":0.6865864080335002, "value2":1198200374614070582, "value3":false, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179713, "value1":0.590117470581001, "value2":8089464430800358371, "value3":true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273193, "value1":0.6452720470474251, "value2":1981356061798226474, "value3":false, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330589, "value1":0.2830579363626061, "value2":6390191335007008364, "value3":true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468768, "value1":0.2265890199157533, "value2":4363523400851143524, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901505, "value1":0.3297489212069802, "value2":7168425838004946986, "value3":false, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850173, "value1":0.7443546478795617, "value2":1215583272150299781, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.968362, "value1":0.6932020029271382, "value2":4461729221210949302, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210225, "value1":0.36249305348248906, "value2":7617198529682822125, "value3":false, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.001816, "value1":0.46528315796295283, "value2":553457115403327004, "value3":false, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.564478, "value1":0.42774941008998635, "value2":6190959325547605268, "value3":false, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126782, "value1":0.6691920311814687, "value2":8190037233750158351, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.125382, "value1":0.3161902537409363, "value2":2607443448109856155, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.782262, "value1":0.25500427404714787, "value2":694236189793698587, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635388, "value1":0.4760006261995292, "value2":7236218797911995877, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.481920, "value1":0.5382100231194429, "value2":6783652184521772255, "value3":true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818398, "value1":0.6906166285200835, "value2":1593698347526099227, "value3":false, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283475, "value1":0.9162718353521024, "value2":131978695455153917, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434017, "value1":0.8865622180830747, "value2":5499929802234516147, "value3":true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838878, "value1":0.8831606268003614, "value2":4413611476125545911, "value3":true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451279, "value1":0.9923758573970507, "value2":7842864722613769419, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362386, "value1":0.09704282298545679, "value2":6868263842667298833, "value3":true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274837, "value1":0.5887807861238907, "value2":1280869300914331508, "value3":false, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.915666, "value1":0.6149407272649627, "value2":2848990718411372775, "value3":true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024227, "value1":0.1623145961115003, "value2":2455593750882693548, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.695944, "value1":0.6887899812661492, "value2":533109273214059474, "value3":false, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.528843, "value1":0.5075436862092302, "value2":515274582595904032, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842262, "value1":0.5447726756017138, "value2":411822948007690047, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444248, "value1":0.5454758220348956, "value2":1028316813231771868, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.604063, "value1":0.46187992572513836, "value2":8158929971855121147, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275994, "value1":0.8154001283675578, "value2":5366649133302343998, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348933, "value1":0.903584155771458, "value2":3421400845314883405, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.041062, "value1":0.1190413277495334, "value2":3602483838893891944, "value3":true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928372, "value1":0.19081223162638847, "value2":4430971211717726323, "value3":false, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.677971, "value1":0.7830672620168732, "value2":4412058601784711710, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152188, "value1":0.6024595453233776, "value2":3457200300695401373, "value3":true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838789, "value1":0.8421167420494099, "value2":690533435719409387, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.578903, "value1":0.6328751611123744, "value2":3916711663864490625, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.896839, "value1":0.9070308716530836, "value2":5545334900650073897, "value3":true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.712901, "value1":0.5273783009751816, "value2":7859089933688216880, "value3":true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884939, "value1":0.07741816375541602, "value2":2787557417309799857, "value3":true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944440, "value1":0.9501843438293957, "value2":4306730282944265532, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814029, "value1":0.12358405285849303, "value2":6299011328560473443, "value3":true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965702, "value1":0.9397419047313924, "value2":2182258508456140601, "value3":false, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045751, "value1":0.6945070980136847, "value2":6892165864647238687, "value3":false, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821781, "value1":0.27908605223075134, "value2":5742625392746876485, "value3":false, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514196, "value1":0.8720271384105083, "value2":6651696455700077053, "value3":false, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.705476, "value1":0.06250737971386458, "value2":1193722866498852150, "value3":false, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.603276, "value1":0.8453600274943766, "value2":31981742286968982, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.682527, "value1":0.2427465956249123, "value2":7462455737298824829, "value3":false, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.605338, "value1":0.7488120008677794, "value2":4273030954599275940, "value3":true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893961, "value1":0.0012093187527627421, "value2":3798553194034218364, "value3":false, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.593870, "value1":0.4389808952680249, "value2":4760750743940894178, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091153, "value1":0.8390848719802647, "value2":4663558817694651969, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.014239, "value1":0.6803598889667898, "value2":4768916054002910461, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478582, "value1":0.8497920155872134, "value2":6014547020899484560, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895949, "value1":0.2900471481778915, "value2":9139898160138156870, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.613215, "value1":0.18238778666189503, "value2":1572248498405458162, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.817510, "value1":0.7988916675957994, "value2":5551320908811649706, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881135, "value1":0.18477673938398376, "value2":5754586889701314224, "value3":false, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128674, "value1":0.2730542267310359, "value2":3723967523716120986, "value3":false, "__name__":"metric_00000015", "key_8":"k","key_0":"a","key_6":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392852, "value1":0.5342690346158169, "value2":2053063244515016190, "value3":false, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931934, "value1":0.712284281615868, "value2":5743978094943818789, "value3":false, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537147, "value1":0.023259389444222435, "value2":5526408740424742448, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.216805, "value1":0.8594083029021016, "value2":5158284428621389401, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837342, "value1":0.5180882037372726, "value2":7808493880824756093, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144558, "value1":0.054388678344141024, "value2":7174534666209181912, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599909, "value1":0.09725816948703289, "value2":4397869661367210515, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.880028, "value1":0.4672046328070093, "value2":7512677570982024303, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271509, "value1":0.8816366985657997, "value2":7431242132698500082, "value3":false, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.752197, "value1":0.6078880045612306, "value2":3606538184952837708, "value3":false, "__name__":"metric_00000025", "key_5":"b","key_0":"a","key_4":"j", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392689, "value1":0.5702352441472875, "value2":327376335260250215, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.201991, "value1":0.3880870455267555, "value2":2539019737033922024, "value3":true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.894353, "value1":0.3785370438906491, "value2":8100099805481858212, "value3":false, "__name__":"metric_00000029", "key_4":"e","key_7":"c","key_1":"f", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.193824, "value1":0.1316387900370704, "value2":2566382490326741291, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.735887, "value1":0.9017592259731758, "value2":3204066872434851855, "value3":true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891488, "value1":0.8557864633508439, "value2":7321454426994500720, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868424, "value1":0.21022389029059355, "value2":4239400393420255730, "value3":false, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.624822, "value1":0.8941824508080144, "value2":4130019486643335780, "value3":false, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.292285, "value1":0.2379306348830461, "value2":19245718469181538, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690737, "value1":0.07804300554340773, "value2":3416010458091576918, "value3":true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404288, "value1":0.7382908404371364, "value2":4673144760418942783, "value3":true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.333079, "value1":0.7119882858599182, "value2":2804563282517597579, "value3":false, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.472391, "value1":0.08143361630646322, "value2":8016722515581298003, "value3":true, "__name__":"metric_00000037", "key_3":"i","key_7":"a","key_0":"c", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.373563, "value1":0.6580401651760573, "value2":6974099035575699967, "value3":true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078367, "value1":0.04245208361261986, "value2":3309677777480755698, "value3":true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.312294, "value1":0.08781290868459149, "value2":410348162808240186, "value3":false, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768785, "value1":0.8804562203614901, "value2":3554533373089833157, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636131, "value1":0.6764401768679541, "value2":5380729408367504508, "value3":false, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041277, "value1":0.013798250982638604, "value2":7112024095753417757, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867267, "value1":0.8747601920631648, "value2":1340507463961878602, "value3":true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776242, "value1":0.08520211967435457, "value2":4488809966272125582, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634645, "value1":0.8571991096706362, "value2":4813432822336330164, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415856, "value1":0.16181998754042196, "value2":2843492259629836672, "value3":true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466761, "value1":0.6961378523412571, "value2":7745491307176492894, "value3":false, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.594554, "value1":0.04099735827317489, "value2":8913619907287518073, "value3":false, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.170785, "value1":0.5648256194335608, "value2":476929125931913132, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.184821, "value1":0.7825477180517281, "value2":4452433023228691120, "value3":false, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906243, "value1":0.48861587452050464, "value2":8430953017453652958, "value3":true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.667172, "value1":0.06605403171722925, "value2":3065000063305702983, "value3":false, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.192260, "value1":0.12827053824062132, "value2":4944252317941317234, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400652, "value1":0.728695995485893, "value2":1367272622265070542, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535047, "value1":0.4567196818143048, "value2":5331291578601537074, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466316, "value1":0.5659706744457615, "value2":5221382170536673746, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487405, "value1":0.8907853085096481, "value2":2583510249445277455, "value3":true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.375762, "value1":0.8536987085513627, "value2":1666025213570716259, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.165432, "value1":0.43405723604720065, "value2":714036789144069944, "value3":true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175401, "value1":0.9859093721286644, "value2":4617822411248224956, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.059065, "value1":0.378208510942557, "value2":8575694693763974484, "value3":true, "__name__":"metric_00000059", "key_7":"j","key_0":"d","key_6":"c", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.073374, "value1":0.7874709956095631, "value2":3297271830714214500, "value3":true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656159, "value1":0.11136055751977454, "value2":6742238162891439594, "value3":true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102008, "value1":0.8937221842663475, "value2":6378107019082012182, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.177632, "value1":0.36894176860576355, "value2":7704992361178834991, "value3":true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335734, "value1":0.6980181667187008, "value2":260944621673155935, "value3":false, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.855348, "value1":0.9199066328884838, "value2":4741986190169422412, "value3":false, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965490, "value1":0.12178040424602839, "value2":8326386976538257273, "value3":false, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090240, "value1":0.23602165280654655, "value2":9043605027949953616, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.028464, "value1":0.05960386743156977, "value2":3549887217756566974, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.956988, "value1":0.5498920550278433, "value2":6597949147890816377, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.171316, "value1":0.0904948299110727, "value2":3471748119473561940, "value3":true, "__name__":"metric_00000083", "key_6":"b","key_9":"a","key_1":"g", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.523091, "value1":0.4454856417468202, "value2":3973537432672533989, "value3":false, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453397, "value1":0.7881010562880141, "value2":448816938066732131, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384576, "value1":0.9045802469513693, "value2":4349360112470402658, "value3":true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.929660, "value1":0.1904824371420524, "value2":4775599930437109515, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642052, "value1":0.9792443542954197, "value2":6092407690974464405, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194609, "value1":0.8726282980309354, "value2":7591391705732812177, "value3":true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095352, "value1":0.384168214137503, "value2":7516656275093877057, "value3":false, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868497, "value1":0.924966560225317, "value2":4429888702948279946, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.146490, "value1":0.7150691128822396, "value2":205950019316022568, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009297, "value1":0.4940665543165739, "value2":924283179287111091, "value3":false, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904057, "value1":0.23580267125603968, "value2":531910251609622124, "value3":true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728011, "value1":0.8321936268339676, "value2":2170557262229675184, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944000, "value1":0.7690990294985828, "value2":8227626091745901, "value3":true, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.871354, "value1":0.21098039407901223, "value2":4374012870679696344, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.538323, "value1":0.681484837485245, "value2":6775701390216802286, "value3":false, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230276, "value1":0.36173491575946426, "value2":8235898214428447094, "value3":false, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552468, "value1":0.8892513220522287, "value2":612692529250917070, "value3":true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782746, "value1":0.3267438552083453, "value2":5821122346950557164, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908039, "value1":0.8750317891455761, "value2":6904198848902253780, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124465, "value1":0.27489616104124, "value2":1813939809215341473, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.739988, "value1":0.11884426917626255, "value2":6375710418065432574, "value3":false, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246847, "value1":0.065994349338974, "value2":4501641519889306838, "value3":true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.675544, "value1":0.4346958284003153, "value2":8656554552944846961, "value3":false, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907951, "value1":0.3861999534665598, "value2":7376195997991690698, "value3":false, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.601240, "value1":0.09029192218287085, "value2":4084656981993173700, "value3":false, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327311, "value1":0.46909482167142375, "value2":6484192245523357545, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453608, "value1":0.634255139121383, "value2":8880783116370612057, "value3":true, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562417, "value1":0.4260014133745824, "value2":1052849307102914712, "value3":false, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708174, "value1":0.05688677429464904, "value2":5475041244330586198, "value3":false, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.774531, "value1":0.8132424157243254, "value2":1362266175582132598, "value3":true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.106500, "value1":0.7014281430749205, "value2":7134854011640912176, "value3":false, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.486559, "value1":0.28063120575400446, "value2":3192436055979504417, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.864997, "value1":0.3477928996994262, "value2":735248433592549787, "value3":false, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.657367, "value1":0.65265411185833, "value2":2443852584375783883, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.172271, "value1":0.4417461489558258, "value2":7942690633839505457, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419664, "value1":0.16011001601248667, "value2":6549588207213967629, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416761, "value1":0.6429956000093604, "value2":8493895113638046625, "value3":false, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564282, "value1":0.4457588529183665, "value2":6725016002434781716, "value3":true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430750, "value1":0.3037956469116298, "value2":7998283396935875899, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827125, "value1":0.5250484976450595, "value2":6524724358656282389, "value3":false, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395896, "value1":0.12287086144958013, "value2":1930681199692368788, "value3":false, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851159, "value1":0.23103414004696357, "value2":4914748620478090577, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.736709, "value1":0.6408817571256187, "value2":9105122753792897591, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984808, "value1":0.2905624937515174, "value2":7851451601162680954, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487306, "value1":0.605006354442018, "value2":4276465105843409598, "value3":false, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790850, "value1":0.6268022197129945, "value2":9220780769142386741, "value3":true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185700, "value1":0.566519118535341, "value2":3705647574979636584, "value3":false, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451877, "value1":0.0460137249105004, "value2":7915985022414197030, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150718, "value1":0.35693559742029707, "value2":5014219743291793375, "value3":false, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.294260, "value1":0.9247064360015096, "value2":5127524279022159080, "value3":false, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995623, "value1":0.21178547000119013, "value2":2431494318968387519, "value3":true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.256959, "value1":0.8916548214121515, "value2":1002635579417294841, "value3":true, "__name__":"metric_00000095", "key_5":"c","key_1":"c","key_2":"g", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110758, "value1":0.9737478776363947, "value2":7247818685878473810, "value3":false, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825871, "value1":0.2917878380844755, "value2":6516114778766465158, "value3":true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910566, "value1":0.3957632746819297, "value2":7166657573372241684, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.178800, "value1":0.5649796089766207, "value2":7775035464817162126, "value3":false, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634489, "value1":0.6090054970934514, "value2":6647498876825886588, "value3":true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610499, "value1":0.5292322793488414, "value2":8830154316280741019, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574572, "value1":0.5304362659437236, "value2":3949756347058852883, "value3":false, "__name__":"metric_00000132", "key_3":"a","key_9":"i","key_0":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380651, "value1":0.956019198700713, "value2":9135146156795796096, "value3":true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778218, "value1":0.2121518375707717, "value2":8667556184145489676, "value3":true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078961, "value1":0.06325131155030916, "value2":8555129638818277403, "value3":true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.047017, "value1":0.1027203958612216, "value2":5246272405737501952, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439829, "value1":0.7417681560989317, "value2":7749536771402273968, "value3":true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203072, "value1":0.5621215132192839, "value2":6118954075279232029, "value3":true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758671, "value1":0.48389569392567205, "value2":2705225096795559234, "value3":false, "__name__":"metric_00000140", "key_2":"g","key_1":"c", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665522, "value1":0.8426429373047708, "value2":8279168363918064894, "value3":true, "__name__":"metric_00000141", "key_6":"b","key_8":"h","key_1":"a", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.345535, "value1":0.20088437097171843, "value2":4950747031440638845, "value3":false, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124655, "value1":0.41827026587025645, "value2":3430038763553236823, "value3":true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977760, "value1":0.10421019819450697, "value2":733644826366835807, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118861, "value1":0.7989242448499801, "value2":7548777360232504083, "value3":true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318842, "value1":0.009939081834750191, "value2":8375328698470876119, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590619, "value1":0.03636791737366475, "value2":7582770650474370737, "value3":true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045468, "value1":0.7763100607726895, "value2":2803435127241577664, "value3":true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.553650, "value1":0.9987121300161536, "value2":3948733587296676032, "value3":true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790022, "value1":0.793158890727088, "value2":1174928966366390949, "value3":true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.232763, "value1":0.07728571888784644, "value2":3240453509192284959, "value3":false, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074815, "value1":0.5297469977127046, "value2":2019972453697918898, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.126329, "value1":0.8018605847489219, "value2":4857597305010910280, "value3":true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.465083, "value1":0.1752053660866726, "value2":7163605506157251603, "value3":true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.714319, "value1":0.8220755414438086, "value2":876552025810846680, "value3":true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392782, "value1":0.6635667135366609, "value2":6624504880778850428, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404283, "value1":0.5416570514867133, "value2":8266175224671672272, "value3":false, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.734325, "value1":0.7575887007274229, "value2":8864761125492843547, "value3":true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.164845, "value1":0.7328611642697666, "value2":1939960544899646141, "value3":true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.087713, "value1":0.6472001702780127, "value2":7064434641814632478, "value3":false, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636375, "value1":0.263542227747472, "value2":5091288498651627633, "value3":true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439658, "value1":0.7518898990734011, "value2":6126681364245747402, "value3":true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580431, "value1":0.28444432068121084, "value2":8147884504685238694, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408258, "value1":0.5563100440073113, "value2":4573231178010278468, "value3":true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923759, "value1":0.7035415767681118, "value2":4284377617122095281, "value3":false, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069625, "value1":0.7755797987036898, "value2":7005063200108318654, "value3":true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475808, "value1":0.7602915679203158, "value2":1889690102234767528, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045338, "value1":0.4860219702134258, "value2":5422255506416156729, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.985908, "value1":0.7866996153890412, "value2":4784461751340030748, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937856, "value1":0.5239212561479751, "value2":3109818040142871590, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.376519, "value1":0.2587179526287473, "value2":1263076833301844915, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943956, "value1":0.6806333311736701, "value2":6002705815114995201, "value3":true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.653771, "value1":0.7947376720839073, "value2":2247611778314354742, "value3":true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290979, "value1":0.13605627013098143, "value2":859676226210675359, "value3":false, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166492, "value1":0.4455170530582711, "value2":3676647784290217622, "value3":false, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.625732, "value1":0.26531203043868007, "value2":8360465694110411228, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033924, "value1":0.027884795524999093, "value2":4290921120785429469, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.199865, "value1":0.6930021146351104, "value2":8033400918877373777, "value3":true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.387257, "value1":0.5315645824113636, "value2":8487597923574953049, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.385317, "value1":0.687478534046926, "value2":5788685556231917492, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.632370, "value1":0.4013559424653022, "value2":8640639439561304540, "value3":true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226297, "value1":0.2440055184343977, "value2":5133913946019355967, "value3":true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.763037, "value1":0.20026459422324144, "value2":8670140335638909516, "value3":false, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695304, "value1":0.16276770698378185, "value2":1188100569829740721, "value3":false, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595366, "value1":0.4506718132549764, "value2":5734081257378041711, "value3":true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.833543, "value1":0.5734402814385565, "value2":922425738240504967, "value3":false, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709630, "value1":0.8692041834415107, "value2":1960359817596413212, "value3":true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.732788, "value1":0.23444382672590003, "value2":2854071547092864254, "value3":true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.412896, "value1":0.19053537293020936, "value2":1082762509249072310, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988462, "value1":0.4293280866481845, "value2":4780102507614458134, "value3":true, "__name__":"metric_00000198", "key_8":"i","key_0":"h", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823600, "value1":0.01981381195792859, "value2":8743022558864727512, "value3":false, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.092886, "value1":0.9862388684646872, "value2":3622022818199205736, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.018231, "value1":0.4242288910229569, "value2":880886547836969584, "value3":true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062605, "value1":0.9179292075855371, "value2":3743117943661894467, "value3":true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.759025, "value1":0.5166884948154693, "value2":739459076946893696, "value3":true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907932, "value1":0.2882739675567565, "value2":3650155370836487214, "value3":true, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068172, "value1":0.4347373436897239, "value2":1372308133830179280, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514459, "value1":0.3578934602691126, "value2":1369826174473414234, "value3":false, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282115, "value1":0.4836091160007032, "value2":7304317831220994830, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652653, "value1":0.7715198420930932, "value2":1854799303037917400, "value3":true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.873426, "value1":0.6674125872681426, "value2":1713253808702182629, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.253882, "value1":0.35618452262868266, "value2":1451362150616905038, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058435, "value1":0.11619135407507769, "value2":729304989898004351, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711979, "value1":0.19795958240207612, "value2":5044852283450257784, "value3":true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.620015, "value1":0.7494674771784868, "value2":9072157623838715380, "value3":false, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265268, "value1":0.6332467233229876, "value2":7660280405186635297, "value3":false, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471797, "value1":0.9586561875915051, "value2":1400450692063171507, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715350, "value1":0.28992759308570154, "value2":4420882374996204645, "value3":true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447264, "value1":0.32845931032025727, "value2":3764283351468138749, "value3":true, "__name__":"metric_00000209", "key_7":"g","key_1":"f", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799817, "value1":0.8344281598160882, "value2":3194977322046448615, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225092, "value1":0.832673789719215, "value2":8242369633095170423, "value3":true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261842, "value1":0.1783779388932828, "value2":7591102613464952913, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885476, "value1":0.33420965888644627, "value2":2231088099862071194, "value3":true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015564, "value1":0.23747285224855552, "value2":2211582269309113296, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415709, "value1":0.43459757217905465, "value2":2297522408220839681, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671302, "value1":0.5521783843346304, "value2":8797575493863836447, "value3":true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809774, "value1":0.3232411464090472, "value2":689408551095099282, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391492, "value1":0.44145120690127054, "value2":741707985323247529, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593752, "value1":0.5004004561210201, "value2":7899920698831884885, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090704, "value1":0.9099972336756517, "value2":8993930752293016892, "value3":false, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.365212, "value1":0.17134030372562564, "value2":6300597784300096913, "value3":true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.458419, "value1":0.03537414872175956, "value2":4931151600790868120, "value3":true, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.266288, "value1":0.2710772740926763, "value2":5168762762336913335, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.846075, "value1":0.8530137052878358, "value2":5846881912829979057, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.441246, "value1":0.895274800378327, "value2":6555884777228809545, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181487, "value1":0.09635807954147779, "value2":5139650932097495606, "value3":true, "__name__":"metric_00000231", "key_7":"g","key_3":"a","key_4":"k", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.283526, "value1":0.6765717041541169, "value2":717191993149931255, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690428, "value1":0.5868291690096701, "value2":4142969563289744689, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461183, "value1":0.7388746066577521, "value2":3907321818804232131, "value3":true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078559, "value1":0.5509982085563974, "value2":8489285868587130769, "value3":true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.006442, "value1":0.17964435576390814, "value2":6385046429151027537, "value3":true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803653, "value1":0.007944203568736737, "value2":8630069900982836811, "value3":true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843384, "value1":0.5245784602574561, "value2":1611187771800002701, "value3":false, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.322734, "value1":0.26583004470195515, "value2":5891979565833996742, "value3":false, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019366, "value1":0.9471429425977943, "value2":3105898999363337716, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583535, "value1":0.3000091753953168, "value2":1671214988273742274, "value3":true, "__name__":"metric_00000220", "key_9":"d","key_5":"g","key_6":"h", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.528263, "value1":0.6167724953698004, "value2":3571326627931246323, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.353615, "value1":0.09339691407017821, "value2":8955717188172767409, "value3":false, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.638995, "value1":0.5498958167694153, "value2":6537844553335776020, "value3":true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327054, "value1":0.5183869093313511, "value2":4021636019817176776, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391759, "value1":0.6107482510959784, "value2":6639243589639002168, "value3":false, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.083515, "value1":0.6947157647744896, "value2":3642097726125624072, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.586405, "value1":0.683079171453159, "value2":414358593011525856, "value3":false, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847208, "value1":0.004363913601366179, "value2":6293076524070827579, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583066, "value1":0.38103391426791644, "value2":7168389465606666058, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.361829, "value1":0.0757878691848108, "value2":18451857992724418, "value3":false, "__name__":"metric_00000246", "key_5":"i","key_0":"k","key_3":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481695, "value1":0.5254185806253874, "value2":3155631786417589238, "value3":true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.072077, "value1":0.8787002152127282, "value2":5274515234654561765, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.267961, "value1":0.8648926070157235, "value2":4125897469271708547, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.245076, "value1":0.06458749862097861, "value2":3727526843973668028, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602905, "value1":0.9902842013208221, "value2":3395310217472956562, "value3":true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776585, "value1":0.7219469790990183, "value2":77676005392087147, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741224, "value1":0.9110757287517638, "value2":3235394638378837669, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.616438, "value1":0.32846413845355765, "value2":5558119133284488407, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.617955, "value1":0.12562192604901234, "value2":603576184957547937, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.651221, "value1":0.8976398132733482, "value2":7308768939039920124, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610616, "value1":0.3225584230939945, "value2":8416609441712941807, "value3":true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.688197, "value1":0.39642287994645425, "value2":7290339184191009077, "value3":true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599842, "value1":0.6105446645640619, "value2":1208112926522863577, "value3":false, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867582, "value1":0.1840371797963492, "value2":2177170750841832105, "value3":true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098316, "value1":0.966593809823329, "value2":2548324970162406510, "value3":false, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694537, "value1":0.3376604030915601, "value2":607328136959332742, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748191, "value1":0.9957584594865343, "value2":67877367036436991, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.064163, "value1":0.7248743828192261, "value2":2749527480847272451, "value3":false, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502667, "value1":0.16495830272715267, "value2":9030464128428456991, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990661, "value1":0.3949843558188797, "value2":7201429976848106189, "value3":false, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987854, "value1":0.29870559261727064, "value2":348149959926675176, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481753, "value1":0.06548607806382464, "value2":1031140396971019018, "value3":false, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.630820, "value1":0.8992403438153617, "value2":5146298573769468519, "value3":false, "__name__":"metric_00000274", "key_7":"g","key_9":"h","key_3":"k", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804082, "value1":0.2519724232639516, "value2":1298310485258981735, "value3":true, "__name__":"metric_00000278", "key_4":"h","key_5":"j","key_1":"c", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368623, "value1":0.647450377107345, "value2":1284622928379296522, "value3":true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.541781, "value1":0.3261143352667153, "value2":130140210100844559, "value3":false, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543424, "value1":0.06226681703581004, "value2":726618431953897465, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988533, "value1":0.49745860881748044, "value2":3406264037344093572, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080362, "value1":0.7895574305241833, "value2":152509341700687015, "value3":false, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150372, "value1":0.9343525151823494, "value2":6076357966504332576, "value3":true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489577, "value1":0.16062395564859527, "value2":3405065082227979672, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225570, "value1":0.28122658630857733, "value2":8458136042003760574, "value3":false, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862952, "value1":0.7649680950314343, "value2":1756495878088169871, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009186, "value1":0.7999050849241861, "value2":3718209620537893555, "value3":true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327636, "value1":0.23708513628211542, "value2":1522368463913294016, "value3":false, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.551176, "value1":0.009439114458257377, "value2":2361018304142309387, "value3":false, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.633123, "value1":0.2821310200838197, "value2":6052402151738260293, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128752, "value1":0.3359757178563009, "value2":7472471602725669166, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.456386, "value1":0.46133506790277695, "value2":2296093836300686089, "value3":true, "__name__":"metric_00000277", "key_3":"e","key_4":"d","key_1":"g", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420349, "value1":0.36664549905260974, "value2":3546467223240204480, "value3":true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.938528, "value1":0.007977829048726685, "value2":4244809297577031047, "value3":true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226579, "value1":0.6725646472034443, "value2":913735509091846801, "value3":true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.850850, "value1":0.23068766044782393, "value2":2300681752566773332, "value3":false, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285218, "value1":0.6989337075616671, "value2":2607116938388175821, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.221331, "value1":0.11523512208048978, "value2":8241016534243475750, "value3":false, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886294, "value1":0.8815954846610522, "value2":1129122380251841667, "value3":true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010095, "value1":0.4468942822052692, "value2":7305659252479204819, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007461, "value1":0.3287186449948172, "value2":8533612311279585337, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841878, "value1":0.06768912562638665, "value2":3915275733365598237, "value3":false, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239318, "value1":0.2752287175449378, "value2":6104494420782937095, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364905, "value1":0.7269412172343553, "value2":7167814392288649336, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030862, "value1":0.35386987370196926, "value2":568029767114417610, "value3":false, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945737, "value1":0.8998415941144967, "value2":2346717810373152754, "value3":false, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.093117, "value1":0.8244480275524787, "value2":3248267576826263944, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.422568, "value1":0.5719502852011219, "value2":8260911644483203383, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823166, "value1":0.2849312986012331, "value2":3755798306625123375, "value3":true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268641, "value1":0.33070121758142046, "value2":3503668812330101597, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564550, "value1":0.4481321012388454, "value2":2552310395319729245, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417743, "value1":0.047809954457076996, "value2":1081553625885385604, "value3":true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595860, "value1":0.2921995022802213, "value2":5059986207137621431, "value3":true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.505814, "value1":0.9601390700122682, "value2":1325398133306866350, "value3":true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357820, "value1":0.3304365801280413, "value2":5155828582049823863, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963899, "value1":0.8305280855606907, "value2":2952790462431625685, "value3":true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395768, "value1":0.05940274489209035, "value2":2206736622008483109, "value3":true, "__name__":"metric_00000311", "key_2":"f","key_5":"d","key_0":"b", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.994837, "value1":0.28131858851364544, "value2":6377323293251471217, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825885, "value1":0.9524553792547288, "value2":5686625986396675519, "value3":true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962844, "value1":0.7824414856459302, "value2":3176085690721005555, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525393, "value1":0.2894500361431452, "value2":9170274733300331931, "value3":true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066894, "value1":0.9750264781896856, "value2":6481943120958967565, "value3":true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118777, "value1":0.49186253332263474, "value2":196535145535309391, "value3":false, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.011455, "value1":0.25527316956832247, "value2":23216573243095801, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.516739, "value1":0.7031419417768016, "value2":2051319192678332676, "value3":false, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621518, "value1":0.7948563798954263, "value2":8394265809911073481, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.008794, "value1":0.671348308630362, "value2":8908504416251388849, "value3":false, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804142, "value1":0.34185386259040573, "value2":7569056971355045232, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971215, "value1":0.795050202357287, "value2":7291081553392495608, "value3":false, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285984, "value1":0.6233623903797153, "value2":6444347877884163801, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.436168, "value1":0.122788509000057, "value2":2542694660610691923, "value3":true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545038, "value1":0.897272884042895, "value2":3518180018317716924, "value3":true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.377957, "value1":0.26500726028503324, "value2":918021549502007270, "value3":true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186659, "value1":0.8861606285044316, "value2":7428746411605199070, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.706851, "value1":0.8539941323885553, "value2":5095874822126979293, "value3":false, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552666, "value1":0.8209937326987259, "value2":7918997962757156260, "value3":false, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708678, "value1":0.20172873469302396, "value2":473140516571282958, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.588929, "value1":0.8992364975321605, "value2":1107792164310220135, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.396543, "value1":0.9159358105409485, "value2":3240110351377657654, "value3":true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949484, "value1":0.3395753649788759, "value2":1556081794159850294, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627633, "value1":0.6064813434025144, "value2":504106831033400051, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760461, "value1":0.027052105627838302, "value2":2547821024594975963, "value3":false, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690642, "value1":0.29842739674487334, "value2":4347369482397818828, "value3":true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271622, "value1":0.9232891397814524, "value2":4902825584314369050, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678674, "value1":0.8612335750248717, "value2":2676726884600182684, "value3":false, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.789269, "value1":0.7153455595388135, "value2":8629094676547921616, "value3":true, "__name__":"metric_00000348", "key_4":"k","key_5":"e","key_3":"h", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.274447, "value1":0.04852479796186674, "value2":520491486142586388, "value3":false, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517924, "value1":0.5065900646461697, "value2":1305736770624753229, "value3":false, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440510, "value1":0.8079290707038177, "value2":745163355638334416, "value3":true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.992658, "value1":0.33022695610085456, "value2":3745647814397920317, "value3":false, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.142855, "value1":0.7836474840216356, "value2":1701655973854226818, "value3":false, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.718918, "value1":0.9106879340813463, "value2":2642697174847166539, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358697, "value1":0.6889718382775613, "value2":2396285724783610411, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782243, "value1":0.10018629733131312, "value2":3557685057170687759, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290843, "value1":0.5213600910147824, "value2":5522593432675792373, "value3":true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243393, "value1":0.9922299657128142, "value2":267617765686382025, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827775, "value1":0.17424595518809072, "value2":3247150599498567700, "value3":true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954439, "value1":0.6395234710891515, "value2":3700591746218368196, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543518, "value1":0.7123005069599985, "value2":8170425126276779400, "value3":false, "__name__":"metric_00000361", "key_9":"c","key_3":"h","key_7":"j", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862097, "value1":0.7928466963741031, "value2":1388049342872581158, "value3":false, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861104, "value1":0.44830540845614686, "value2":3638392887423846691, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996353, "value1":0.9606274415719652, "value2":6260542395914742363, "value3":false, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269620, "value1":0.27973905679528643, "value2":1556926452504115363, "value3":false, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118090, "value1":0.5428339606791369, "value2":1853095038687144617, "value3":false, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843203, "value1":0.15162943121847733, "value2":978719322151022878, "value3":true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.168392, "value1":0.30548225941021784, "value2":7137186554911304535, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.197151, "value1":0.3975385059702385, "value2":5863175014471717006, "value3":false, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565354, "value1":0.9234247531019719, "value2":812121677319261736, "value3":false, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030909, "value1":0.8792093662826526, "value2":8559138722753842843, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.484600, "value1":0.23944552081323173, "value2":7751158766892162613, "value3":true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.320620, "value1":0.12230291625693078, "value2":6192516194919614511, "value3":true, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.935691, "value1":0.28006447226692, "value2":6784015217669799303, "value3":true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.336215, "value1":0.23094104388429443, "value2":5569809866177597446, "value3":false, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988466, "value1":0.5462607441657077, "value2":8311104273733417101, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447804, "value1":0.4867937694484817, "value2":5062168010270449210, "value3":false, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.190570, "value1":0.8283372201936341, "value2":5874711067636169206, "value3":true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904281, "value1":0.8553852749887187, "value2":3311030506444553065, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.751439, "value1":0.4266991113414808, "value2":4999173498909581042, "value3":false, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.125309, "value1":0.5563950948301023, "value2":1187994145431678821, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.973142, "value1":0.11588338763957572, "value2":8318570385943791860, "value3":true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.116231, "value1":0.2526143185693312, "value2":782968109827902547, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.187535, "value1":0.27872170170772304, "value2":8766589428595489268, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945637, "value1":0.08854405995381304, "value2":3412460749402337910, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.270152, "value1":0.3638550762746485, "value2":2218032724303888987, "value3":true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051113, "value1":0.40794935925664466, "value2":7831097119693907093, "value3":true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265496, "value1":0.23954206323753308, "value2":63253225412010244, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416624, "value1":0.17710696129812786, "value2":3803821152719869396, "value3":false, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607195, "value1":0.28582291116541503, "value2":7284275038719896556, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.326362, "value1":0.17094485896565303, "value2":4663361033102261072, "value3":false, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438839, "value1":0.3486395787421312, "value2":8809206276934706557, "value3":false, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033597, "value1":0.8575270834812074, "value2":1940304743017854051, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.065029, "value1":0.9427613360719956, "value2":271108060306002877, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.488385, "value1":0.12779971800108084, "value2":3316011689829646532, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961814, "value1":0.5428137306648136, "value2":1514601534044579978, "value3":false, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263207, "value1":0.6949685012843101, "value2":895006576805426648, "value3":true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282635, "value1":0.10777955836055363, "value2":564953042266607985, "value3":false, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.001269, "value1":0.042927028237879465, "value2":1466958494603226347, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682457, "value1":0.8478178196386861, "value2":107121152123493591, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.159543, "value1":0.9604877997752348, "value2":7593923445754669936, "value3":false, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015347, "value1":0.7626139882833062, "value2":5647986907617063275, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209254, "value1":0.8450839759887067, "value2":1886374721985796728, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066244, "value1":0.07791428229072243, "value2":919090579972011287, "value3":true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.699052, "value1":0.06643124534628293, "value2":2711140899074023727, "value3":false, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.631133, "value1":0.143383362490665, "value2":971805873892273782, "value3":false, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426673, "value1":0.7738540415601494, "value2":4397130911534361286, "value3":true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.389568, "value1":0.33397631468501043, "value2":4052528315967262589, "value3":false, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.622610, "value1":0.24246003195798918, "value2":5479722481130894419, "value3":false, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904392, "value1":0.9774495314687007, "value2":8793165354063487461, "value3":true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.793581, "value1":0.4190858782174469, "value2":4926610192238705598, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110855, "value1":0.2884411940593678, "value2":5227068360933879219, "value3":false, "__name__":"metric_00000397", "key_7":"d","key_0":"k","key_2":"f", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135653, "value1":0.402267966757566, "value2":6977955000760069723, "value3":false, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.307152, "value1":0.8927574650950808, "value2":5803177489389723115, "value3":false, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.737685, "value1":0.21404400943023025, "value2":6732954103348577362, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.121533, "value1":0.18872391644762126, "value2":8989825709899786188, "value3":true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.972223, "value1":0.023522846364320095, "value2":7146680715055172496, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923388, "value1":0.8860383778342971, "value2":8303048822044107475, "value3":true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851782, "value1":0.32631560891796313, "value2":1989916996789168449, "value3":false, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391839, "value1":0.002802043190783618, "value2":8629895271323501696, "value3":false, "__name__":"metric_00000400", "key_1":"c","key_6":"c","key_0":"f", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066714, "value1":0.031995170191600183, "value2":3319906268065732292, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421535, "value1":0.635220505499552, "value2":7200157785718775016, "value3":false, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287471, "value1":0.8948051177891431, "value2":2176610475830643902, "value3":true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804659, "value1":0.28415985878229416, "value2":33052675971427371, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996360, "value1":0.6152217383986905, "value2":544588703505765788, "value3":false, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.798458, "value1":0.8665558307064753, "value2":8115268483760058936, "value3":true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448744, "value1":0.3917339620374744, "value2":9198148727913546136, "value3":false, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861092, "value1":0.9826532618696505, "value2":4765520374138069290, "value3":false, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.202532, "value1":0.0052591320469092834, "value2":336652236719043103, "value3":false, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.524950, "value1":0.8143167251197552, "value2":6529413407694528834, "value3":false, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.485366, "value1":0.5907312441951306, "value2":3337798525793799244, "value3":false, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558504, "value1":0.3641804008571667, "value2":4575793560178915745, "value3":true, "__name__":"metric_00000432", "key_3":"b","key_2":"k", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642936, "value1":0.623697886602654, "value2":7776504717328671778, "value3":true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043344, "value1":0.9623993994613507, "value2":6602176914068250275, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.241555, "value1":0.23253929846791072, "value2":685349515600255534, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926727, "value1":0.21223983767190985, "value2":3951472227334441759, "value3":false, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.131046, "value1":0.2851063666145672, "value2":6180530959986721149, "value3":true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.247299, "value1":0.5130082295270245, "value2":8139956405448602641, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758053, "value1":0.49876246690774434, "value2":3786706615107859566, "value3":false, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323322, "value1":0.4313958567662574, "value2":8854072512884634153, "value3":false, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748573, "value1":0.9590027921453678, "value2":3839028144294969465, "value3":false, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384213, "value1":0.010562139959872605, "value2":6878855550977960785, "value3":true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180644, "value1":0.6131439659014073, "value2":2706285599198878081, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593573, "value1":0.4120750238333436, "value2":1200854625465685501, "value3":false, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995840, "value1":0.297035809872462, "value2":6168541710158809840, "value3":false, "__name__":"metric_00000425", "key_9":"k","key_3":"e","key_8":"d", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.511756, "value1":0.006206978177219643, "value2":1958577823681147315, "value3":true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565859, "value1":0.4442276087687805, "value2":39986647781639917, "value3":true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715853, "value1":0.9381008994826828, "value2":3556376195988394411, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236468, "value1":0.20999691756150096, "value2":9161072742831597145, "value3":true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.161371, "value1":0.738745986035695, "value2":5138216362948975345, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445330, "value1":0.14739850543555172, "value2":8667736730671884444, "value3":true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.982188, "value1":0.40813076015564426, "value2":3022991256121544350, "value3":true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984748, "value1":0.6214792221342914, "value2":5933123653156672217, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562273, "value1":0.8568295281307874, "value2":5414869267292754213, "value3":true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476811, "value1":0.49675643916102746, "value2":5664619325942193065, "value3":false, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891490, "value1":0.8866201621688181, "value2":8214607251036840472, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931525, "value1":0.8288823187078326, "value2":3069955632009854668, "value3":false, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455511, "value1":0.8774391931652025, "value2":3046641762023967525, "value3":true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400651, "value1":0.5254465108261448, "value2":4999803900412377548, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666971, "value1":0.9993076586927376, "value2":4982431527333428358, "value3":true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.8618210720028192, "value2":761627529379782309, "value3":false, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.915374, "value1":0.37053901712348225, "value2":3944125570135845867, "value3":true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.960156, "value1":0.9374154078802892, "value2":4747973404867949779, "value3":true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210720, "value1":0.8468677886051782, "value2":4852924773263065270, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.332500, "value1":0.6465642869647131, "value2":1008458018130353532, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.788169, "value1":0.3773654642642006, "value2":4191084511110280464, "value3":false, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545303, "value1":0.25852749290398136, "value2":2783025552475099417, "value3":false, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.143623, "value1":0.36024800648413563, "value2":4932311496806548620, "value3":false, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895647, "value1":0.0053042457769262655, "value2":6923819500937313799, "value3":false, "__name__":"metric_00000447", "key_5":"g","key_8":"i","key_0":"e", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932434, "value1":0.659079228202596, "value2":4313811336647625065, "value3":true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.260771, "value1":0.403057123436948, "value2":4416521488550693116, "value3":false, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238541, "value1":0.2952177348030342, "value2":1538460010819351929, "value3":true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958540, "value1":0.5291106177059647, "value2":1249052435364791122, "value3":true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430589, "value1":0.8147197427498966, "value2":5526371695265553363, "value3":false, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431544, "value1":0.9355303369766881, "value2":4469095563720030593, "value3":true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640478, "value1":0.48586313377391904, "value2":7959068707072142404, "value3":false, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144376, "value1":0.00848981726968387, "value2":5068460906328461885, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416953, "value1":0.6734644377643653, "value2":2766510942957347250, "value3":false, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885086, "value1":0.21498485778798626, "value2":8278007933868416694, "value3":false, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537244, "value1":0.3154976316557619, "value2":2639365305979929980, "value3":true, "__name__":"metric_00000471", "key_8":"b","key_1":"b","key_3":"f", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.020210, "value1":0.453949847740753, "value2":2024272373950233485, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514022, "value1":0.3475698755294227, "value2":1372329110659095675, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806258, "value1":0.9274447352712736, "value2":3920648533163431087, "value3":false, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209427, "value1":0.014587282647209867, "value2":8695997195066251972, "value3":false, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971326, "value1":0.2797692384280631, "value2":6419079357930277367, "value3":false, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652286, "value1":0.5905911533568637, "value2":8712000656540499616, "value3":false, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878040, "value1":0.48362174266483915, "value2":2926563391771284063, "value3":true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990329, "value1":0.5049151964579438, "value2":3650817862757599534, "value3":true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049823, "value1":0.838067458662296, "value2":8854264061775325606, "value3":false, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017442, "value1":0.8431632474037919, "value2":8852347705624487512, "value3":true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.544821, "value1":0.005649842126714668, "value2":2282778264986625535, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.597518, "value1":0.2045781468759336, "value2":8575639431654539646, "value3":true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612206, "value1":0.3075892336013964, "value2":7673450384556490732, "value3":true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471060, "value1":0.40918086270080856, "value2":6747291739813622106, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.513402, "value1":0.7422854995723118, "value2":4083993911078701361, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965008, "value1":0.6372798813181509, "value2":1707224457136697946, "value3":false, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.354283, "value1":0.9108473180297934, "value2":1020266209800871174, "value3":false, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806140, "value1":0.14778437030634853, "value2":6105158525255417708, "value3":true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194962, "value1":0.3473291379662418, "value2":3795992189004973455, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.974776, "value1":0.09118542426518424, "value2":5134289787707010070, "value3":false, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.339938, "value1":0.13774510006333723, "value2":562171884148417195, "value3":false, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.550551, "value1":0.384009626532502, "value2":8609730106908869554, "value3":false, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.654960, "value1":0.2639517045663951, "value2":1862456670074602807, "value3":true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030932, "value1":0.5143312749002444, "value2":1494847764422163119, "value3":true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695169, "value1":0.2423970054677799, "value2":8509443740065644494, "value3":true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713656, "value1":0.34108742339653203, "value2":4598279459368093139, "value3":true, "__name__":"metric_00000497", "key_2":"f","key_0":"d","key_1":"e", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.794261, "value1":0.9458347874797615, "value2":7302724440323717945, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364772, "value1":0.2365974372946532, "value2":3654435595389536462, "value3":false, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395099, "value1":0.15838919107992194, "value2":1939247105684031089, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335629, "value1":0.39545025723069543, "value2":2672917998276624419, "value3":false, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074949, "value1":0.7876350399335282, "value2":625303995684394443, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536138, "value1":0.5616610785403638, "value2":8164874977498368721, "value3":false, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697782, "value1":0.23834340441765955, "value2":3951692500137347815, "value3":false, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671649, "value1":0.6501720201004972, "value2":2955880776784904551, "value3":true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.787079, "value1":0.378893964304627, "value2":3193906322405958964, "value3":false, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.031359, "value1":0.29145414339441095, "value2":5615459484207384702, "value3":false, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.306294, "value1":0.34557951146905197, "value2":2931839249030464715, "value3":true, "__name__":"metric_00000512", "key_4":"d","key_7":"e","key_1":"k", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357655, "value1":0.3923960009663459, "value2":8112783273863341075, "value3":true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961896, "value1":0.23361672067408637, "value2":432132564805318676, "value3":false, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098797, "value1":0.18420600660031175, "value2":4984332870487251645, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.071876, "value1":0.6659197757567412, "value2":1553450733606247594, "value3":false, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186866, "value1":0.7000117639690743, "value2":7329872261968800728, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406584, "value1":0.13046586013887515, "value2":1622087582581823179, "value3":false, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440301, "value1":0.11646408872647017, "value2":1011606322554570900, "value3":true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.136166, "value1":0.4328392754883894, "value2":2919499144416670917, "value3":false, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318957, "value1":0.7075576973107754, "value2":398072267285532828, "value3":false, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.012179, "value1":0.5668221134322494, "value2":6110074131405385081, "value3":true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830949, "value1":0.3334949924022553, "value2":7572722630631231675, "value3":false, "__name__":"metric_00000528", "key_7":"c","key_8":"g","key_0":"k", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145965, "value1":0.365420169950548, "value2":2535895288587113465, "value3":false, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.649060, "value1":0.16445860625759065, "value2":6921600715877525051, "value3":false, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517227, "value1":0.07192874427493294, "value2":6297114124078920008, "value3":true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832587, "value1":0.5150936291796077, "value2":181711691089923280, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291455, "value1":0.5623557100802268, "value2":8762626146094505466, "value3":true, "__name__":"metric_00000508", "key_5":"h","key_8":"a","key_2":"j", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.149447, "value1":0.5534139568709165, "value2":2510872908080100775, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476658, "value1":0.4486907399118968, "value2":8992668719445662670, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236318, "value1":0.49075032225313237, "value2":1924725427300127433, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.561765, "value1":0.9668124792198439, "value2":990391456732695590, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884668, "value1":0.0011015591734681801, "value2":5327656656973278324, "value3":false, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175200, "value1":0.9826652864577513, "value2":2876633651705447757, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379413, "value1":0.15362330796009416, "value2":7353421640256094445, "value3":false, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.319517, "value1":0.5202924305313171, "value2":7456692117889227019, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.122103, "value1":0.7187092003738956, "value2":7131563361103346240, "value3":true, "__name__":"metric_00000534", "key_3":"a","key_9":"k","key_1":"a", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678103, "value1":0.7543751799594591, "value2":8526553775473008293, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.643972, "value1":0.19798980984364278, "value2":5929329615236343098, "value3":true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.831121, "value1":0.1528427519418499, "value2":6184723596853858850, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432219, "value1":0.5515160206615604, "value2":23121424938113426, "value3":false, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058751, "value1":0.6338015087216923, "value2":7816911191202995864, "value3":true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790739, "value1":0.8762741406918414, "value2":4555556160832468314, "value3":false, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797544, "value1":0.8139645999303083, "value2":2761547225024668528, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404353, "value1":0.9860244138424634, "value2":2908826677832473108, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016900, "value1":0.053160518991546266, "value2":8380107749249883562, "value3":false, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.407973, "value1":0.7843730354485248, "value2":6824583973593836053, "value3":true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987174, "value1":0.7619687243629265, "value2":3319925462827235787, "value3":false, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577835, "value1":0.20813952139985942, "value2":6437982890007555330, "value3":false, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755320, "value1":0.3636597821442561, "value2":5470963049552133157, "value3":false, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.897987, "value1":0.07151356373469021, "value2":1477083928227057783, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388580, "value1":0.20932748154080844, "value2":3392543048891869159, "value3":false, "__name__":"metric_00000548", "key_9":"c","key_4":"i", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.025802, "value1":0.38577745314564454, "value2":868566582237921681, "value3":false, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941032, "value1":0.17111301155402217, "value2":6961085515450408974, "value3":false, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947055, "value1":0.9780343555878157, "value2":133683558419674847, "value3":true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954822, "value1":0.08800460018752372, "value2":7497584262448035095, "value3":false, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063420, "value1":0.8100121364093217, "value2":3755200469512293547, "value3":true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438294, "value1":0.3424655143965952, "value2":4316249625206240732, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809627, "value1":0.9673131880891396, "value2":4132748723398010856, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439925, "value1":0.8075664782512888, "value2":864263057207692643, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.084104, "value1":0.3818348864598292, "value2":2553923579282526899, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.647912, "value1":0.46216103925581126, "value2":4639093581287452200, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369627, "value1":0.3422458784570339, "value2":4254456412295026385, "value3":false, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.614963, "value1":0.5220645586983701, "value2":6072161543938690608, "value3":false, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.321895, "value1":0.3049038907635511, "value2":6325538472569574229, "value3":true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464290, "value1":0.22336524838033933, "value2":9076126601794953657, "value3":true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166932, "value1":0.3553203900309477, "value2":5982106259976109422, "value3":true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803898, "value1":0.5280548625824253, "value2":4063002262098608883, "value3":true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.927974, "value1":0.17895618154031318, "value2":547646730434307817, "value3":false, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.303832, "value1":0.5511007572679634, "value2":2862350870063686751, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520504, "value1":0.39724544886891017, "value2":6931266301104819935, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941095, "value1":0.14786172741452286, "value2":3205668266543533832, "value3":false, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656997, "value1":0.32075936991205234, "value2":7391478133987747835, "value3":false, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120921, "value1":0.3939864396610821, "value2":2909689659050189847, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958790, "value1":0.3269314306247572, "value2":6783238272107071770, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.386136, "value1":0.861947664801586, "value2":5805659227499250652, "value3":true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.449365, "value1":0.5799629274092946, "value2":9193070830904141987, "value3":true, "__name__":"metric_00000581", "key_3":"e","key_9":"h","key_2":"f", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.785656, "value1":0.3165816948400473, "value2":3116393083198778433, "value3":true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520711, "value1":0.6656585396817016, "value2":1017823568348313071, "value3":true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.650203, "value1":0.22215575308437716, "value2":1902945447598147226, "value3":false, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249002, "value1":0.2961338302748422, "value2":1415432970559418672, "value3":true, "__name__":"metric_00000586", "key_5":"k","key_7":"f","key_1":"e", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694015, "value1":0.7506751146797324, "value2":6142650591148786794, "value3":false, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.304466, "value1":0.37747841635497736, "value2":5375036774405769501, "value3":false, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.641004, "value1":0.46446471911344284, "value2":5955856288590241375, "value3":false, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.684973, "value1":0.907579891922439, "value2":5346005034595738366, "value3":true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.111153, "value1":0.49826719629056415, "value2":7497650141562337241, "value3":true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830722, "value1":0.28960521156096847, "value2":6154503575103623089, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392721, "value1":0.6834318849474948, "value2":2628637152649849276, "value3":true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969704, "value1":0.15429907011430974, "value2":5405475101757554186, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536252, "value1":0.7000830425153579, "value2":406966410660924195, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.807572, "value1":0.2012419938894414, "value2":4358282132524848204, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.820107, "value1":0.5076022682087846, "value2":4017461578732969062, "value3":false, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996611, "value1":0.7422015873263831, "value2":436292068764222211, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.922123, "value1":0.051072363737232486, "value2":268646986926660655, "value3":false, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.729895, "value1":0.8676962023633001, "value2":2065960180781249750, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636047, "value1":0.4996316174329004, "value2":3156670443682802457, "value3":false, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431520, "value1":0.5204212362312931, "value2":8019726224533633908, "value3":true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.903149, "value1":0.5859452724568418, "value2":2288060944551638278, "value3":false, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.070771, "value1":0.9932429756726906, "value2":6690408728176272233, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.314453, "value1":0.3472880048665214, "value2":58548757977887750, "value3":true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627243, "value1":0.7139819549630214, "value2":7080915148891558502, "value3":false, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755294, "value1":0.05408051807422248, "value2":6931097273017939069, "value3":false, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282331, "value1":0.3203397893756164, "value2":8256062300842673383, "value3":false, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.034396, "value1":0.9531256831279978, "value2":7765396458031448915, "value3":false, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.115811, "value1":0.13116244832703935, "value2":3543488806097224100, "value3":false, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666851, "value1":0.6887850450312126, "value2":5462261380989469479, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525431, "value1":0.3139759113605816, "value2":8160395548303928540, "value3":true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036488, "value1":0.7437550776292858, "value2":905546975072552710, "value3":true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246868, "value1":0.7173148070188747, "value2":7273545640281703287, "value3":true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840343, "value1":0.20321210783475557, "value2":336636579624758589, "value3":true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100735, "value1":0.8393194019944642, "value2":291716047034557387, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853981, "value1":0.5202422200638643, "value2":5204943321202939039, "value3":false, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512132, "value1":0.018314329801792806, "value2":961979132289563000, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487612, "value1":0.7033581894769818, "value2":4186345519226619201, "value3":true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778169, "value1":0.7839413953257911, "value2":7874569661704504858, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.587363, "value1":0.24066050930994098, "value2":2238921997040090404, "value3":false, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.863823, "value1":0.22725313032865457, "value2":4922946319365439720, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.462195, "value1":0.6013044688497777, "value2":137484525088955387, "value3":false, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.967535, "value1":0.21528129199568857, "value2":5907496598149934999, "value3":false, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263462, "value1":0.3398984210102334, "value2":3627872884296903263, "value3":false, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778010, "value1":0.2649025290720317, "value2":2665113842043817940, "value3":true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895866, "value1":0.8505969892692369, "value2":1773042500322059709, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475234, "value1":0.08631156102550058, "value2":519973765404949203, "value3":false, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996762, "value1":0.6378946736901862, "value2":9195196376217348512, "value3":false, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537168, "value1":0.2006763700535789, "value2":9067766312454029883, "value3":true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.585179, "value1":0.8887141585928169, "value2":1447763822787404407, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937668, "value1":0.18798426398408305, "value2":5531072459849624180, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417573, "value1":0.018969474714960195, "value2":8739232693787930523, "value3":true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432784, "value1":0.6772697684929758, "value2":3670574248400915486, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945707, "value1":0.2017640539145947, "value2":6866599884426415644, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715992, "value1":0.2770810284692845, "value2":6065196798538107388, "value3":false, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724512, "value1":0.7694937233145257, "value2":5461354547379838063, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555480, "value1":0.4116895522673444, "value2":6999095526200651536, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357696, "value1":0.9694808723879291, "value2":2308259417403012034, "value3":true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.925321, "value1":0.9804626384591183, "value2":1344780667969714366, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340972, "value1":0.8564397089342566, "value2":3335346544187347307, "value3":true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634021, "value1":0.2824134389967874, "value2":827846054793909627, "value3":true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535849, "value1":0.8383622181777869, "value2":3210193933415043724, "value3":false, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949231, "value1":0.3423558587998688, "value2":8018588983650318493, "value3":false, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.604636, "value1":0.6545573982070936, "value2":8784079580099061471, "value3":false, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041080, "value1":0.6815494814472329, "value2":2413918152267969152, "value3":true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206623, "value1":0.7407991857310414, "value2":8688412282907699410, "value3":false, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697192, "value1":0.9920334806937902, "value2":4058886921829417026, "value3":true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748394, "value1":0.3435945058384803, "value2":6002972956903494678, "value3":true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421611, "value1":0.8151473391165547, "value2":261810376504255149, "value3":true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.027674, "value1":0.32343755952478886, "value2":8467224123375485048, "value3":true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205103, "value1":0.26583420276460434, "value2":912318083515976688, "value3":false, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362886, "value1":0.6918507204150298, "value2":3042551716463426665, "value3":false, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181414, "value1":0.9588817871484018, "value2":5500996639169097947, "value3":true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607786, "value1":0.1555201931458164, "value2":3667544046344919475, "value3":false, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447991, "value1":0.6322575842993106, "value2":6581489005452708007, "value3":true, "__name__":"metric_00000656", "key_6":"k","key_7":"k","key_4":"f", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602539, "value1":0.3803193364932477, "value2":501846031619014460, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224149, "value1":0.14287189157740537, "value2":4390766248301994532, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019349, "value1":0.9275903306872896, "value2":6065489708075344215, "value3":true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.753905, "value1":0.04103970388034392, "value2":3647389822252762296, "value3":true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.808552, "value1":0.6912948158348138, "value2":7168224379303111769, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.383522, "value1":0.7213839418172744, "value2":8917533656021329315, "value3":false, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988681, "value1":0.4652567645908903, "value2":6088381044289659824, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426511, "value1":0.8194664026578082, "value2":3767723603035206676, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291449, "value1":0.11519619446576408, "value2":7042710433748403097, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.495051, "value1":0.7541019265512129, "value2":4805842413127851386, "value3":true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.498957, "value1":0.7657156651926107, "value2":4284045357936772257, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.920671, "value1":0.557277837350798, "value2":7618404853124250629, "value3":false, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.325130, "value1":0.8555228162118138, "value2":5568850179022711136, "value3":false, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619588, "value1":0.8294092067729972, "value2":1498942661428438729, "value3":true, "__name__":"metric_00000669", "key_5":"e","key_9":"d","key_4":"c", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932410, "value1":0.9716095495296341, "value2":6631794618463446087, "value3":false, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.257618, "value1":0.6931656613099204, "value2":4999381616225061323, "value3":false, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.328148, "value1":0.3114535269637382, "value2":7689977371490904120, "value3":false, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.280989, "value1":0.9111540107476489, "value2":2001844404871692409, "value3":true, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722254, "value1":0.17067710980514048, "value2":9024628709036843159, "value3":true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367790, "value1":0.7784645051385055, "value2":4706281866949049190, "value3":true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368237, "value1":0.6741959290593132, "value2":2438020324428284656, "value3":true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885688, "value1":0.4581933301264591, "value2":977306309852850973, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.856287, "value1":0.24855090559153642, "value2":5403228793051986733, "value3":false, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.305675, "value1":0.6170232598989743, "value2":274658269233518106, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908698, "value1":0.5064010761390361, "value2":5817465185742290417, "value3":true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757988, "value1":0.04843336165427696, "value2":5801707099278996481, "value3":false, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.933565, "value1":0.22461787050531695, "value2":6104462847804923186, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.720615, "value1":0.12587001891222954, "value2":7755893877648671182, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642145, "value1":0.4752800242516813, "value2":1097468219850109153, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975649, "value1":0.9950222857947861, "value2":323860279126996239, "value3":false, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.571701, "value1":0.13176848875241834, "value2":8974250013315108425, "value3":false, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841017, "value1":0.5108584547409708, "value2":1015654371149607255, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.499539, "value1":0.7323245432905996, "value2":866168553464820252, "value3":false, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.137678, "value1":0.9837633616038727, "value2":1980815750999371252, "value3":true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713500, "value1":0.6844141285754438, "value2":5215363796900492290, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564182, "value1":0.6959446753092476, "value2":2124568431454851658, "value3":true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.548476, "value1":0.8904291651425387, "value2":8960234619080614404, "value3":true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.334127, "value1":0.5777624482637982, "value2":3890652700139772457, "value3":true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.141095, "value1":0.3019838758248612, "value2":1159300889479734121, "value3":true, "__name__":"metric_00000686", "key_8":"g","key_1":"d","key_4":"a", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800239, "value1":0.4131750973298026, "value2":2344128018741531119, "value3":true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369249, "value1":0.1589896462920189, "value2":4160545898250072075, "value3":false, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.198713, "value1":0.08298578373908365, "value2":50980726393447907, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.773734, "value1":0.7496268790368541, "value2":7804146534404388324, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237426, "value1":0.27543463085841935, "value2":3694238253557813150, "value3":false, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577321, "value1":0.6586006530161179, "value2":641077213271202791, "value3":false, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368897, "value1":0.6785289242869411, "value2":2009007121053273154, "value3":true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.801586, "value1":0.7989756868311243, "value2":3470745118950983568, "value3":true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.035889, "value1":0.4089400550263047, "value2":3320771957585601823, "value3":false, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760511, "value1":0.7386595086100225, "value2":3332011099556778058, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.359596, "value1":0.8203562587923371, "value2":574506838747954548, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063436, "value1":0.2579978917934866, "value2":6993045686428025271, "value3":true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369331, "value1":0.4769488047439803, "value2":3318100445225134934, "value3":false, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795599, "value1":0.9012136284147806, "value2":5770204186306544532, "value3":false, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.219070, "value1":0.20974721576307698, "value2":2356228020734101377, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537221, "value1":0.1725299480275798, "value2":5073957380468101443, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015458, "value1":0.4025573823516011, "value2":3282394548862748845, "value3":true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580078, "value1":0.9823965911617399, "value2":8476500551084420415, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.979266, "value1":0.5236855845499949, "value2":680393934209420766, "value3":true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682327, "value1":0.844770708190905, "value2":8298068650764370173, "value3":false, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350010, "value1":0.5806359147839931, "value2":1527219564263667157, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206279, "value1":0.4978598735396454, "value2":8931241509064706089, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135137, "value1":0.7138645301142236, "value2":2633600346218814000, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.981064, "value1":0.7353726130780065, "value2":268395459435449482, "value3":false, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.526357, "value1":0.3584205029962212, "value2":7926609514114360701, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145680, "value1":0.817655724537724, "value2":2809466879616878308, "value3":false, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.877503, "value1":0.17147641272949588, "value2":1475184649319875768, "value3":true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049475, "value1":0.3904708182990629, "value2":8178189937335677971, "value3":true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419221, "value1":0.2498991121797316, "value2":2019493757980421565, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646852, "value1":0.2981893785163364, "value2":4127223489573331187, "value3":true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194796, "value1":0.9706301430727536, "value2":3907775771119307356, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135232, "value1":0.952467511486142, "value2":6824235396463011996, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392387, "value1":0.19563387556471776, "value2":6531634041022343859, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.133702, "value1":0.7965289882627238, "value2":6086508658030257940, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.691192, "value1":0.21849926406658296, "value2":2500546592704430602, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238250, "value1":0.8547288670113503, "value2":1751658935211452093, "value3":true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.259613, "value1":0.7399127039208734, "value2":3758431842842501346, "value3":false, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.959851, "value1":0.06348104065773419, "value2":5803029266892181407, "value3":false, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350885, "value1":0.07399715612950311, "value2":9184522919237034506, "value3":false, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821915, "value1":0.9143789840874811, "value2":3150076904171453287, "value3":false, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.554775, "value1":0.406185744411068, "value2":4390857727102845864, "value3":false, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.026241, "value1":0.576960195810922, "value2":864775792792522430, "value3":true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118029, "value1":0.6740072102143805, "value2":2738633295377346249, "value3":false, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.669898, "value1":0.6063573387513779, "value2":8118600251510115903, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406027, "value1":0.5408212514385392, "value2":836959541970766944, "value3":false, "__name__":"metric_00000740", "key_3":"j","key_0":"f","key_2":"b", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205752, "value1":0.9617267357194531, "value2":2302570688726041304, "value3":true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.683346, "value1":0.7764943186065553, "value2":1516613683034572821, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535753, "value1":0.8501812602308251, "value2":3781935852497505404, "value3":false, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562975, "value1":0.5762183849456854, "value2":8803839397694965730, "value3":true, "__name__":"metric_00000744", "key_9":"j","key_1":"c","key_8":"i", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224509, "value1":0.6945846689247714, "value2":8790065402136541035, "value3":true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120301, "value1":0.5693297769803048, "value2":8817337009559896419, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535111, "value1":0.7168638232585879, "value2":1459446787633025168, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590881, "value1":0.052466731276145914, "value2":6984133752001938547, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800340, "value1":0.19161367226547096, "value2":8470840993920678933, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797252, "value1":0.22382372006161758, "value2":960932677896506756, "value3":true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502545, "value1":0.4567752766501535, "value2":9200307528794857259, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.919995, "value1":0.3074462546242729, "value2":1729230620360441809, "value3":true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.273301, "value1":0.5542454650469373, "value2":2007525431527262950, "value3":false, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249396, "value1":0.30289020495328595, "value2":7647941374943306916, "value3":false, "__name__":"metric_00000754", "key_5":"i","key_8":"h","key_1":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847062, "value1":0.8617179899648977, "value2":2309423222399622901, "value3":false, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.770003, "value1":0.8185102377578417, "value2":8900535665947339098, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612076, "value1":0.2520488653737615, "value2":534199497930559417, "value3":false, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867000, "value1":0.3056461531330615, "value2":8074358922929649800, "value3":true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543352, "value1":0.1267819654500433, "value2":2282337611780682090, "value3":false, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771337, "value1":0.3860477352422603, "value2":6614019503509668230, "value3":false, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724939, "value1":0.35759761476037216, "value2":7730807476801277254, "value3":false, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.769189, "value1":0.06006911990731757, "value2":6135075757488436084, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.750837, "value1":0.22593301727381723, "value2":815096043795563480, "value3":false, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.733823, "value1":0.37434887115879495, "value2":8767359811379324612, "value3":true, "__name__":"metric_00000761", "key_7":"b","key_4":"g","key_5":"a", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110462, "value1":0.7613769242381963, "value2":679247224492133418, "value3":false, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.207493, "value1":0.26270378513285625, "value2":6127960652597929191, "value3":true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757551, "value1":0.17246010749021554, "value2":3746123411387113183, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476665, "value1":0.530111421321598, "value2":7929787788154212077, "value3":true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080232, "value1":0.24610687758058403, "value2":4382073893358318099, "value3":false, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.518795, "value1":0.13825391880428892, "value2":6023983814814871189, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.411487, "value1":0.7443212678339678, "value2":3163265105788950613, "value3":true, "__name__":"metric_00000763", "key_8":"g","key_4":"b","key_6":"k", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230729, "value1":0.6799575270403697, "value2":6569848391065499405, "value3":false, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.251807, "value1":0.23368831551353111, "value2":9121098592064787356, "value3":true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450371, "value1":0.8049793990951134, "value2":6271606339009618863, "value3":true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268506, "value1":0.7393978258409317, "value2":3084814221880218023, "value3":false, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453066, "value1":0.7167056491982889, "value2":8359177712079217325, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840677, "value1":0.3999771937764131, "value2":5907070339651841877, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853610, "value1":0.4459636741761099, "value2":2591090489482088505, "value3":true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.050602, "value1":0.9347582889921432, "value2":3586868729035531903, "value3":true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.460673, "value1":0.8573683555697056, "value2":77436103148660020, "value3":false, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.791642, "value1":0.7214035249481627, "value2":4712458830503182070, "value3":true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711339, "value1":0.7704360355595858, "value2":4818602735855589639, "value3":true, "__name__":"metric_00000774", "key_8":"e","key_2":"g","key_4":"c", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709838, "value1":0.980850892357579, "value2":5814687934592979738, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448259, "value1":0.6911929910350553, "value2":2331566896810690473, "value3":false, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069070, "value1":0.7715792084831908, "value2":1237753144790060173, "value3":false, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728905, "value1":0.6453677424318518, "value2":4827712214727464044, "value3":false, "__name__":"metric_00000783", "key_8":"e","key_9":"b","key_7":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.581880, "value1":0.2677360178055146, "value2":8844278477287710309, "value3":true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969799, "value1":0.5783967942752313, "value2":5905445599124832757, "value3":false, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741372, "value1":0.012956038256400619, "value2":914084143172531236, "value3":true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862619, "value1":0.5947373167351414, "value2":3630517863485151782, "value3":true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.139536, "value1":0.8372408825864636, "value2":6952590048486505005, "value3":false, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803976, "value1":0.43722974429572603, "value2":2176830109855416805, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.534405, "value1":0.7241541542820893, "value2":4511693404596905586, "value3":true, "__name__":"metric_00000789", "key_6":"d","key_5":"a", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574402, "value1":0.7055278131957177, "value2":1238394241461479434, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.06494326977178205, "value2":3563720913525659633, "value3":true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878052, "value1":0.8371531564279497, "value2":587508487226380908, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.672424, "value1":0.26126599404383394, "value2":3460643419757907518, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408361, "value1":0.5841391902007207, "value2":8050508010125067469, "value3":true, "__name__":"metric_00000775", "key_6":"a","key_0":"e", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.743507, "value1":0.5292956990399417, "value2":5743834222350004253, "value3":true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.024743, "value1":0.5111340297086123, "value2":4739422872896352154, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053127, "value1":0.41791773967804086, "value2":5388222563610877961, "value3":false, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489667, "value1":0.6344084078946665, "value2":6127655871996983541, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.876031, "value1":0.49328901667737785, "value2":3143559155074364531, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.278465, "value1":0.23241882776321474, "value2":805438734033729133, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380252, "value1":0.6353489769585325, "value2":1619270428062881618, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512159, "value1":0.8152126527698539, "value2":1368040827327524990, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.317649, "value1":0.9550761539015656, "value2":8914758126172814906, "value3":false, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.723807, "value1":0.7854700255052053, "value2":7095402949367847209, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.660856, "value1":0.4053530100702817, "value2":2145190666873464855, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.659680, "value1":0.12646331821035406, "value2":1576238191679234703, "value3":false, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.993554, "value1":0.8499325393155558, "value2":5900620451042065379, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091338, "value1":0.41503145937749936, "value2":2042239402577329647, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.299870, "value1":0.687218145400586, "value2":7560464390267142060, "value3":false, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.731227, "value1":0.15009967640134209, "value2":1886018024243215805, "value3":false, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.348963, "value1":0.6737669308977404, "value2":5404336119559355867, "value3":true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162168, "value1":0.813527799284827, "value2":4016392262793002099, "value3":true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086162, "value1":0.7399808700194177, "value2":8264192680187392458, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975595, "value1":0.11433087569708594, "value2":4671281305443756113, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799363, "value1":0.3067546199025452, "value2":225546722481809384, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.113667, "value1":0.6424657658728049, "value2":8718201548571703569, "value3":false, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904642, "value1":0.5465983180036486, "value2":4877073522652743329, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.467203, "value1":0.8662610389904551, "value2":399387690820391383, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082862, "value1":0.5088605240671744, "value2":1531739105872671512, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.039708, "value1":0.10189606169579954, "value2":5185895909677369244, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068495, "value1":0.2805843145287995, "value2":486469505433183294, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357082, "value1":0.75740796150096, "value2":2690075476895328471, "value3":false, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492715, "value1":0.3004718942084548, "value2":7383744941741065497, "value3":false, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.968147, "value1":0.5004304919533921, "value2":1731582924085558465, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.991776, "value1":0.7394075904771168, "value2":4141740040514370033, "value3":false, "__name__":"metric_00000829", "key_4":"k","key_6":"a","key_3":"i", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420254, "value1":0.45767543208547934, "value2":1964890506496503777, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285702, "value1":0.1366965878727093, "value2":2681549871870198885, "value3":true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492204, "value1":0.9414045318691766, "value2":1575036830396652846, "value3":true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086526, "value1":0.5271825907095232, "value2":3742001028067515529, "value3":false, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.296194, "value1":0.6916437451586811, "value2":384054583246095269, "value3":true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478834, "value1":0.5030459095566234, "value2":3918028956922111738, "value3":false, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.029769, "value1":0.2164261260808611, "value2":4102211333190644591, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102553, "value1":0.42618668443550417, "value2":7568518633836620405, "value3":true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.681434, "value1":0.7225843033115362, "value2":8160060706717748819, "value3":true, "__name__":"metric_00000838", "key_9":"b","key_5":"j","key_6":"h", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210135, "value1":0.21812369739137172, "value2":5702739804858144278, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.786060, "value1":0.33904050622912296, "value2":1017560684015420231, "value3":true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558541, "value1":0.14437656270299287, "value2":7701298697010262063, "value3":false, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.818721, "value1":0.15017675533914024, "value2":7186230665665602451, "value3":true, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910629, "value1":0.09672885609565206, "value2":6203887741849145968, "value3":true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069131, "value1":0.6075489323724108, "value2":9025102416378601029, "value3":false, "__name__":"metric_00000813", "key_8":"h","key_1":"j","key_7":"g", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362866, "value1":0.8018878313034695, "value2":8037732655719146713, "value3":false, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682018, "value1":0.5433657256031907, "value2":6040793575397278438, "value3":false, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797323, "value1":0.5032038038329079, "value2":3803498450979968327, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.829279, "value1":0.605237212493161, "value2":4367127372886022859, "value3":false, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.999162, "value1":0.5082854850214861, "value2":5031512432939201812, "value3":false, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269625, "value1":0.0007161270371379666, "value2":6545591527308584857, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180859, "value1":0.6040920298819816, "value2":8926725137631299725, "value3":false, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090602, "value1":0.5326464117623615, "value2":7752649715252147157, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.191121, "value1":0.12836456615806674, "value2":7198874487475192980, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.686863, "value1":0.708342981200594, "value2":5465607373577957408, "value3":false, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.527995, "value1":0.10447991595171255, "value2":5974156320762011232, "value3":true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.123930, "value1":0.6200031138885586, "value2":5406077264893957847, "value3":true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.356501, "value1":0.44396993176396354, "value2":6255319076363219685, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.048169, "value1":0.08153644488046811, "value2":8582968949378456970, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.112743, "value1":0.6933778275610673, "value2":4587548588265206284, "value3":false, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323148, "value1":0.20142008973704545, "value2":4527540178335323769, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.934744, "value1":0.08093337102458979, "value2":7857820814429250360, "value3":false, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962496, "value1":0.6124995000870105, "value2":6489203228482067372, "value3":true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.160517, "value1":0.5248710269553188, "value2":5026165991297404062, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932200, "value1":0.922378624964946, "value2":7930339398930216623, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.900245, "value1":0.1182228551739615, "value2":1373671224306855417, "value3":true, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790920, "value1":0.31880287032595506, "value2":6084131296995295782, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237291, "value1":0.13557228914344616, "value2":4432801334887160283, "value3":false, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.914465, "value1":0.02040606137133301, "value2":1808915457173232015, "value3":false, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941296, "value1":0.9439262294487866, "value2":6574505025862591303, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.822459, "value1":0.9942489150539108, "value2":3506354754330173943, "value3":true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239733, "value1":0.4989511485969244, "value2":7173139334014716167, "value3":false, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.444066, "value1":0.7594335589002584, "value2":3216271266780228569, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866271, "value1":0.5931978644398052, "value2":3444180839017636171, "value3":false, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421087, "value1":0.6530151828739431, "value2":19805620992839207, "value3":false, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771459, "value1":0.8073198535663951, "value2":4810887667402700825, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832482, "value1":0.734748250215754, "value2":1123934068717543450, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450110, "value1":0.15409548849312346, "value2":418445000125934927, "value3":true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.234597, "value1":0.3684594320773471, "value2":3167333011558865991, "value3":true, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.403637, "value1":0.891873955682516, "value2":6523321617886226804, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017496, "value1":0.5380231504237305, "value2":1565703384693902706, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577818, "value1":0.942234653476486, "value2":6854337928979531257, "value3":false, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886845, "value1":0.561600684880549, "value2":6700870407358100744, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.493299, "value1":0.1842060640300563, "value2":7660717134967306529, "value3":true, "__name__":"metric_00000872", "key_8":"j","key_5":"i","key_6":"b", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043840, "value1":0.2702623264568812, "value2":7040356564551372493, "value3":true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051963, "value1":0.5871637139837996, "value2":4534754735986232109, "value3":false, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.568830, "value1":0.08173891497184316, "value2":408699021018747548, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694578, "value1":0.8818619724910839, "value2":5720963057839829442, "value3":true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095000, "value1":0.9081401378248263, "value2":8656658880665915661, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977416, "value1":0.9899458292085819, "value2":665964700424941220, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.670945, "value1":0.6733933124715158, "value2":2460152217631000428, "value3":false, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636539, "value1":0.6931186653224365, "value2":7451786524160575556, "value3":true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.398486, "value1":0.8808674761773307, "value2":6001178466503449076, "value3":true, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.085292, "value1":0.9258669929119259, "value2":4480969611449139348, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451321, "value1":0.8902429562501767, "value2":2399440975743970404, "value3":false, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100110, "value1":0.4539977466656097, "value2":4906418767346623646, "value3":true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.725438, "value1":0.8779089763048636, "value2":7986118613837183355, "value3":true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.559619, "value1":0.42482197030409785, "value2":921145061767776729, "value3":true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885834, "value1":0.32962036175845005, "value2":5608652380611657042, "value3":true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619049, "value1":0.4697848554943135, "value2":6930715352479440739, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768456, "value1":0.55002846829011, "value2":6856343919804361537, "value3":true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.052488, "value1":0.9193313692023829, "value2":8075338150518594907, "value3":false, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.346748, "value1":0.7969202775153313, "value2":3854682510572365821, "value3":true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.668529, "value1":0.8901520814855493, "value2":3780299539456604836, "value3":true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.680780, "value1":0.9128377273315654, "value2":655380872616480476, "value3":false, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.298499, "value1":0.8901980009691262, "value2":4124662751920860689, "value3":true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.061822, "value1":0.922218588720657, "value2":6935461410888938736, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.220215, "value1":0.8707922044422206, "value2":6415853705476229931, "value3":false, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825872, "value1":0.6145357419122679, "value2":3712577567679334743, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287572, "value1":0.7153750220855114, "value2":6766565708182723447, "value3":true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.258723, "value1":0.8623529521546892, "value2":3906675258214843555, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417686, "value1":0.6071420867010977, "value2":8210404273898330224, "value3":false, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.939674, "value1":0.018125841559591738, "value2":2686797765813242922, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062829, "value1":0.19220806032635843, "value2":4009115073519468001, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.761409, "value1":0.282044445283748, "value2":2659364730071661456, "value3":true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340677, "value1":0.7867304449409817, "value2":4726626950116268051, "value3":true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.848210, "value1":0.7451256645145572, "value2":6540697575244319830, "value3":false, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082209, "value1":0.08477415388397415, "value2":5473839595744351618, "value3":false, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562389, "value1":0.08808192976810916, "value2":1578232710018420800, "value3":true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445878, "value1":0.7573342205320885, "value2":3400644244684606362, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.603775, "value1":0.183625034999939, "value2":7793796792648526034, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036048, "value1":0.8359424936170327, "value2":5865912253243965224, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866722, "value1":0.3949434009829113, "value2":3290170127727456353, "value3":true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697995, "value1":0.8423729542207233, "value2":4861550929219709907, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.211200, "value1":0.36512924472577757, "value2":2972205895387325421, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227557, "value1":0.22341834017020207, "value2":4806018666421767946, "value3":false, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574952, "value1":0.6007129713703638, "value2":3849253119017333239, "value3":false, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741721, "value1":0.5702349573898813, "value2":897876839681174700, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379915, "value1":0.6564162460066056, "value2":3131015984288737246, "value3":true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.229369, "value1":0.456646560158155, "value2":4778476677749084799, "value3":false, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.242697, "value1":0.4142421381714439, "value2":7101584169529051655, "value3":false, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.212755, "value1":0.39066964505760526, "value2":4253776777664568253, "value3":false, "__name__":"metric_00000922", "key_7":"h","key_9":"k","key_5":"b", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837272, "value1":0.15824518985564762, "value2":6812701504471384455, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.104360, "value1":0.9940983043766688, "value2":2203777894935310413, "value3":true, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391080, "value1":0.5579038729371453, "value2":6131596744931664143, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265096, "value1":0.7511430046090084, "value2":4283281105409640096, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881500, "value1":0.8197410181139487, "value2":2167116001911138380, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964911, "value1":0.6839484284574476, "value2":2392821775915043629, "value3":false, "__name__":"metric_00000936", "key_4":"f","key_9":"c","key_0":"j", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906639, "value1":0.12232760875324074, "value2":9162174110871211714, "value3":true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562264, "value1":0.9910780181559037, "value2":1543791401715243110, "value3":true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016345, "value1":0.3277590263870748, "value2":3851358761838506924, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963516, "value1":0.13686033235694836, "value2":2786944428195217311, "value3":false, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.446494, "value1":0.051498079678560585, "value2":1973784887996536653, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350210, "value1":0.13860925037657423, "value2":1354426546825223222, "value3":true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358378, "value1":0.28813722807183184, "value2":4138306657967174744, "value3":false, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455527, "value1":0.9787455514136848, "value2":6829598111029855692, "value3":false, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.208210, "value1":0.4291478612877578, "value2":749843861604187954, "value3":true, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.330867, "value1":0.6408478280255169, "value2":8395361043273182125, "value3":true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.176999, "value1":0.9117058417477248, "value2":8428888804090519109, "value3":false, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091618, "value1":0.8122911740955318, "value2":1478086552909570392, "value3":true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243584, "value1":0.06727976096288107, "value2":4178958373440436949, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.127002, "value1":0.8012772426876036, "value2":2149729620077994596, "value3":true, "__name__":"metric_00000923", "key_5":"j","key_6":"e","key_3":"h", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461395, "value1":0.21653382210662625, "value2":2558734709934645897, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932379, "value1":0.43744123497894677, "value2":5779172669858974850, "value3":false, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180562, "value1":0.6091002318007619, "value2":2233652952119987641, "value3":true, "__name__":"metric_00000925", "key_3":"h","key_9":"c","key_0":"e", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.738334, "value1":0.8147115963605253, "value2":1038206878786066414, "value3":true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943138, "value1":0.1805265868912326, "value2":8231024461542686979, "value3":false, "__name__":"metric_00000931", "key_7":"g","key_0":"j", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007238, "value1":0.30166077327328183, "value2":5211541986455490237, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926840, "value1":0.7197646392974351, "value2":6170887305282807365, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.976392, "value1":0.02561261869316712, "value2":8003552327575750503, "value3":false, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665296, "value1":0.38463860662667215, "value2":702575998172013860, "value3":false, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.957627, "value1":0.8365108478779814, "value2":3172439569974822855, "value3":true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.779959, "value1":0.005457784803842251, "value2":1217373846044030405, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814440, "value1":0.03586677782979768, "value2":8668868565483927092, "value3":false, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590538, "value1":0.7998407978822464, "value2":5953384113011158992, "value3":true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.381146, "value1":0.6861901618066985, "value2":806723358919392729, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.324953, "value1":0.09051509245266014, "value2":930835793453842933, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.309761, "value1":0.44326721766264904, "value2":887788347593465760, "value3":true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722069, "value1":0.08849038862609183, "value2":7352758650578649855, "value3":false, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558827, "value1":0.3337734956738152, "value2":8685182570627965041, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715649, "value1":0.23109099835442082, "value2":1083379478056182665, "value3":true, "__name__":"metric_00000961", "key_9":"h","key_7":"b","key_8":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964540, "value1":0.6330159342037405, "value2":1167652807667781397, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795126, "value1":0.5778697926785736, "value2":9140994998625427103, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236481, "value1":0.5802899488963301, "value2":2895083535971839167, "value3":true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996259, "value1":0.2785697344481934, "value2":4913023416457428502, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.477938, "value1":0.484568023326054, "value2":3495500670092878690, "value3":true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261065, "value1":0.11116858735231369, "value2":5869730879322934574, "value3":false, "__name__":"metric_00000966", "key_2":"f","key_1":"a", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185846, "value1":0.7126003929744608, "value2":3434450920636329132, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841676, "value1":0.3736127841617722, "value2":7802733756976416976, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350997, "value1":0.582318689502649, "value2":372190874054145543, "value3":true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555394, "value1":0.12831996488184216, "value2":4582628444562419674, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.155140, "value1":0.3906336986690914, "value2":8237405956495459279, "value3":true, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367910, "value1":0.741845875474543, "value2":8082126523839106878, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.950768, "value1":0.6652471104213182, "value2":8268690822588065094, "value3":true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.433269, "value1":0.7980991574588373, "value2":8513183309513339766, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.874369, "value1":0.5464494520072651, "value2":5885072115794893058, "value3":false, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537599, "value1":0.7405427933790399, "value2":6635799236257663587, "value3":true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.955763, "value1":0.8778283555530513, "value2":6592488434360400176, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.582015, "value1":0.879034635958457, "value2":7219576376735925925, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388984, "value1":0.53843340060884, "value2":2254745170299712210, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646131, "value1":0.5101339977981793, "value2":3292039546456501555, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033442, "value1":0.6456192738711094, "value2":4906576027749942202, "value3":false, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.836645, "value1":0.48635849410277077, "value2":1404791026873124424, "value3":false, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464295, "value1":0.5617144285499786, "value2":7990958033367681704, "value3":false, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010923, "value1":0.9807071857311617, "value2":5683359887238098850, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438005, "value1":0.5935141575872156, "value2":4865067817966075444, "value3":true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227310, "value1":0.6681961275740043, "value2":8551103550465421536, "value3":false, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186394, "value1":0.25048622317521113, "value2":2888181774636358576, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053227, "value1":0.10049698552895067, "value2":4557545020451680363, "value3":true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621258, "value1":0.9388516977567649, "value2":4055003743201350923, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227442, "value1":0.7022685194913532, "value2":3740668486633940756, "value3":false, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640300, "value1":0.06434066744840804, "value2":1646848528935407928, "value3":false, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162609, "value1":0.71410216780028, "value2":7370378990581242781, "value3":true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512792, "value1":0.19345496541059937, "value2":4424425935560475985, "value3":true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203787, "value1":0.9476247108925144, "value2":264480497401916863, "value3":true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.076408, "value1":0.13927961349104323, "value2":1149972870879073163, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806676, "value1":0.6172364818942355, "value2":4996985075367830737, "value3":false, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.313484, "value1":0.6715999817727264, "value2":5237271908928184120, "value3":false, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947617, "value1":0.1400224644800571, "value2":4212016629330280660, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.695969, "value1":0.13461968177274677, "value2":8209146459721882476, "value3":false, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466156, "value1":0.7851823238051198, "value2":4609596535178738402, "value3":false, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047192, "value1":0.6615735622970099, "value2":3688971736191236071, "value3":true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179244, "value1":0.5525116533205023, "value2":6958696123355987767, "value3":false, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.836535, "value1":0.8752088579621973, "value2":1772355477524999885, "value3":false, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.385661, "value1":0.6669392703121872, "value2":5026611864593335422, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.122798, "value1":0.5624470316374681, "value2":973613507611825241, "value3":false, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663323, "value1":0.2836760055083927, "value2":8274369123408107345, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988003, "value1":0.07587743066420682, "value2":4947294410032829574, "value3":false, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.240844, "value1":0.4161174869019418, "value2":4077015391953706312, "value3":false, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.945230, "value1":0.14944194749229664, "value2":7660491006096000021, "value3":true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890111, "value1":0.20296484407777726, "value2":1824051875710962915, "value3":true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522740, "value1":0.021855326277136235, "value2":3218235817889269142, "value3":true, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.679843, "value1":0.5055217891288131, "value2":4822706751533920431, "value3":false, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191340, "value1":0.9959364873278129, "value2":3932069232230299078, "value3":false, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771462, "value1":0.4557053136591391, "value2":184726250007256859, "value3":false, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581655, "value1":0.689124829253459, "value2":1817463261736430161, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.894089, "value1":0.9069576891707435, "value2":7318133204837024663, "value3":false, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.307941, "value1":0.4227283465946924, "value2":356860997794924081, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085219, "value1":0.7548908197263309, "value2":2736908468355530081, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510864, "value1":0.6045243894505045, "value2":3203823484629938988, "value3":true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194038, "value1":0.2945791412290687, "value2":8542013145729544847, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.822784, "value1":0.7046664475635507, "value2":6531211714773873879, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958034, "value1":0.6210073688384752, "value2":5171125204919187937, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245514, "value1":0.07787807249020082, "value2":4796876093300411441, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922409, "value1":0.026305095785565266, "value2":4204774996273721582, "value3":false, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618949, "value1":0.8021374519845728, "value2":7055530836155104119, "value3":true, "__name__":"metric_00000026", "key_2":"c","key_7":"d","key_1":"f", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.842155, "value1":0.9078167657864898, "value2":659406508816846003, "value3":false, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.429827, "value1":0.6861191669827441, "value2":3716781946259346478, "value3":true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.721875, "value1":0.7721427864001064, "value2":1559707715553693565, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.667085, "value1":0.41573209008012846, "value2":770484632734171477, "value3":false, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567897, "value1":0.2688265887173671, "value2":8731672977339492909, "value3":true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716882, "value1":0.20533155732399708, "value2":8158121924487423466, "value3":true, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.450278, "value1":0.45425248453866135, "value2":7472174068145252016, "value3":false, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700766, "value1":0.38900719316872584, "value2":9070338695929499459, "value3":true, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296131, "value1":0.7588298197955197, "value2":1078200536047373074, "value3":true, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644620, "value1":0.22063287558750297, "value2":1455208095653529502, "value3":true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826663, "value1":0.9059127191998474, "value2":163331002989197798, "value3":true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.375920, "value1":0.888489428778294, "value2":3478637008435710026, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795211, "value1":0.7169431218180555, "value2":5240161460250645540, "value3":false, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686886, "value1":0.1615738416501897, "value2":4522663398730909152, "value3":true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.641311, "value1":0.5919216049468233, "value2":4264524771451405531, "value3":true, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415500, "value1":0.8013193758641248, "value2":4655946330615841665, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310579, "value1":0.03464731335525151, "value2":2648033740620709930, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.135452, "value1":0.663217728481436, "value2":6656249157262608643, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036458, "value1":0.7592021704654797, "value2":4708550885998497867, "value3":true, "__name__":"metric_00000054", "key_7":"j","key_3":"k","key_6":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196106, "value1":0.9109556832409251, "value2":1182178953302502536, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897060, "value1":0.30582754840738535, "value2":5110613128180106106, "value3":false, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.386690, "value1":0.17482144522900334, "value2":208181032234291542, "value3":false, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770888, "value1":0.5775727956988672, "value2":7308314003124624706, "value3":true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710417, "value1":0.9182102368115501, "value2":8001324021984156096, "value3":false, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.133639, "value1":0.8132459830268748, "value2":6362172937464044603, "value3":true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313132, "value1":0.6425828291871425, "value2":3812023207458774027, "value3":true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816183, "value1":0.12876194798023052, "value2":353578462778793882, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.537076, "value1":0.3677093772206428, "value2":3410314947529692873, "value3":false, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.785801, "value1":0.5650444568188182, "value2":2518267372616726657, "value3":false, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793640, "value1":0.36840754337253134, "value2":6490561754060041624, "value3":false, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277029, "value1":0.5742325627742796, "value2":4576556191237422408, "value3":true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086053, "value1":0.635715694469195, "value2":3276323627400814300, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622335, "value1":0.6664883465636151, "value2":6846298203429885729, "value3":true, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784935, "value1":0.8345083778637773, "value2":2989860444276517490, "value3":false, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394440, "value1":0.3881444024090245, "value2":211938452740442471, "value3":true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.150923, "value1":0.2049623118713732, "value2":7830583450265904275, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334186, "value1":0.8877162218809062, "value2":5530254516648658024, "value3":true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919131, "value1":0.645089444938841, "value2":2752404297334193415, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.550415, "value1":0.8672407553696915, "value2":7296354502501329771, "value3":false, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.251810, "value1":0.12582294112213796, "value2":3850461942759838134, "value3":true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.768322, "value1":0.5089660327127362, "value2":4637000702345102317, "value3":true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.5790505163445078, "value2":2746113057580625956, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310041, "value1":0.7419516660136578, "value2":8573283812679768621, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567732, "value1":0.20310332375311568, "value2":1684484347666739261, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.149277, "value1":0.8537753954365711, "value2":3792414057901721222, "value3":true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019813, "value1":0.3193924681887993, "value2":4416687297621574996, "value3":false, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555536, "value1":0.2678716255116121, "value2":1978875629658477083, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.676000, "value1":0.268194488154906, "value2":9208242650214533822, "value3":false, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603232, "value1":0.7475788122275303, "value2":6003796300446172845, "value3":true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272526, "value1":0.7633068823447601, "value2":5387009257178750855, "value3":true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288629, "value1":0.0999015536587548, "value2":5230082044659723490, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198708, "value1":0.5411261720205245, "value2":1324824529248190153, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754319, "value1":0.5181119376899533, "value2":7586548988147520888, "value3":true, "__name__":"metric_00000079", "key_6":"c","key_0":"d","key_3":"d", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.974535, "value1":0.6605961675334583, "value2":4149492144268522705, "value3":false, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620393, "value1":0.0400929610839578, "value2":2611761419308325853, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489379, "value1":0.20432994152077563, "value2":4859071163254701004, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.497240, "value1":0.21587046902788842, "value2":1674028941320053082, "value3":true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227611, "value1":0.31207060679536985, "value2":3782150896328417443, "value3":false, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.813479, "value1":0.9346689305684607, "value2":4170814668135359561, "value3":false, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832568, "value1":0.40374034609384823, "value2":4890680963042405726, "value3":true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.868761, "value1":0.7456764740945415, "value2":7985095078472385264, "value3":false, "__name__":"metric_00000087", "key_5":"k","key_2":"f","key_3":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566095, "value1":0.11155632539641938, "value2":1650668369012843098, "value3":true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516140, "value1":0.2921171335805605, "value2":699561345288206492, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.009998, "value1":0.0698415110112407, "value2":2175710360877139700, "value3":true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472788, "value1":0.6894980351369026, "value2":5185256568181045963, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013027, "value1":0.47916067130802836, "value2":8339740169449446777, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947875, "value1":0.15641692100908322, "value2":1874863435098609183, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834625, "value1":0.4725529822941945, "value2":3313442976129641776, "value3":true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032027, "value1":0.11526706346113103, "value2":4474958127457336496, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.643426, "value1":0.2365209546681749, "value2":2452872778362311452, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399009, "value1":0.10587590188915917, "value2":5343568589706680846, "value3":false, "__name__":"metric_00000096", "key_2":"j","key_6":"c","key_1":"a", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464026, "value1":0.23089518212237822, "value2":6365432144590215713, "value3":false, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.573860, "value1":0.41263343273878106, "value2":7100420790565954895, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575295, "value1":0.15635829479176244, "value2":972701597090717176, "value3":false, "__name__":"metric_00000099", "key_6":"f","key_8":"j","key_5":"b", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685505, "value1":0.3500577640159365, "value2":5296267831822852754, "value3":true, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382859, "value1":0.8962437052233099, "value2":5000561412494049933, "value3":false, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793198, "value1":0.03594880833972008, "value2":5558834996866286026, "value3":false, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296341, "value1":0.6801030998229524, "value2":9104706563374398078, "value3":false, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.044529, "value1":0.43868239396683095, "value2":1119681077532090572, "value3":true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977458, "value1":0.7636375641015349, "value2":3738349085487316798, "value3":true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.769690, "value1":0.38941161495083887, "value2":4119290182206919942, "value3":false, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604117, "value1":0.06703731437988915, "value2":2776279989058320233, "value3":false, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.107061, "value1":0.9783306976372779, "value2":6724603539014036116, "value3":false, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.243947, "value1":0.3058624284281911, "value2":8607064800338196111, "value3":true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784300, "value1":0.47413357934186534, "value2":6062775812546347462, "value3":true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053929, "value1":0.8033894774680731, "value2":3798712131176287721, "value3":true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997633, "value1":0.3495480156862711, "value2":6358034341550557079, "value3":false, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399888, "value1":0.5577617871085867, "value2":7606548109917229902, "value3":true, "__name__":"metric_00000113", "key_4":"k","key_2":"k","key_3":"i", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.343972, "value1":0.37033984748970067, "value2":3823813661350967339, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741859, "value1":0.15725835002679114, "value2":7931796757818180793, "value3":false, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118790, "value1":0.8710092300018802, "value2":8432660813300591545, "value3":false, "__name__":"metric_00000117", "key_6":"k","key_8":"f","key_5":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.514050, "value1":0.3589657867466283, "value2":2807064518724813388, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105213, "value1":0.48313119234822866, "value2":9012983220702483909, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.289587, "value1":0.28297188108556603, "value2":8293007746329622374, "value3":false, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.682523, "value1":0.2601723288341452, "value2":5132563496112960923, "value3":true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.521609, "value1":0.17923659255236565, "value2":9080059564709771845, "value3":false, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.866470, "value1":0.8472875420381546, "value2":7662519070278706881, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139688, "value1":0.701824998288288, "value2":8626519461512263205, "value3":false, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693655, "value1":0.3931946923075373, "value2":4088184802361327943, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.733184, "value1":0.33562267701400733, "value2":5057125078949459009, "value3":false, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530556, "value1":0.8546701921585885, "value2":7172802034424433024, "value3":false, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890826, "value1":0.9156630138494082, "value2":1106383994017980379, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736946, "value1":0.28531565085928856, "value2":7492657224478970683, "value3":true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184271, "value1":0.66321078827276, "value2":6482907214355252145, "value3":true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771765, "value1":0.8506763394943703, "value2":2997501176379232751, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621435, "value1":0.4219907917118007, "value2":2371448704479482567, "value3":true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522551, "value1":0.4559374479237847, "value2":8705737330976472895, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579601, "value1":0.3374291561460269, "value2":6118047213487455086, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.911838, "value1":0.24595613407282738, "value2":8543292459890755038, "value3":true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.461163, "value1":0.20032563894255778, "value2":8271655059958021803, "value3":false, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480016, "value1":0.537301255549281, "value2":131039455325957900, "value3":true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264510, "value1":0.8207403430077567, "value2":448751349067313394, "value3":true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.908904, "value1":0.45150207048162533, "value2":1484572129461087565, "value3":true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279527, "value1":0.48678886025337537, "value2":4675520724179222107, "value3":true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.983786, "value1":0.5074627713755523, "value2":6761505393797376999, "value3":true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407769, "value1":0.8974462165990488, "value2":4427875231986566903, "value3":false, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906297, "value1":0.32841731679412833, "value2":4853751460801403413, "value3":true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.910866, "value1":0.22248621009204866, "value2":6153217292256934376, "value3":false, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481193, "value1":0.07256346779714405, "value2":8357563464914791440, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819751, "value1":0.7821155606881222, "value2":6350148722815418199, "value3":false, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925016, "value1":0.08017277463129999, "value2":8127038680866850197, "value3":true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380866, "value1":0.4236203787530011, "value2":6102770882422449664, "value3":false, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.687520, "value1":0.18833602677282602, "value2":2416669714696967955, "value3":false, "__name__":"metric_00000128", "key_7":"j","key_3":"j","key_4":"i", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392864, "value1":0.12162108913858492, "value2":1037740633966701101, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555618, "value1":0.5726263308688928, "value2":5340970336126619965, "value3":true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041150, "value1":0.1866697214008993, "value2":6089316958837860364, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.848709, "value1":0.5912728689790879, "value2":8374928084508664533, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.633435, "value1":0.34567925121735993, "value2":6074382604796755677, "value3":false, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615342, "value1":0.43884173283368255, "value2":2552875653507489786, "value3":true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.420473, "value1":0.024593195105800023, "value2":6998883295968298787, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.493473, "value1":0.517385578986543, "value2":7898967292563966468, "value3":false, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398409, "value1":0.9494831815326669, "value2":1043777551847385399, "value3":true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.261653, "value1":0.605095923821751, "value2":5798307460445919671, "value3":true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734130, "value1":0.17339556093076888, "value2":3888875155355529135, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.234024, "value1":0.07147953681129654, "value2":2220989894337620932, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.552276, "value1":0.46428310479897317, "value2":5429095800661644988, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.547014, "value1":0.9160534111766008, "value2":4258800689197649539, "value3":true, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084253, "value1":0.06651726425775369, "value2":8202506388672279723, "value3":false, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137868, "value1":0.6230337836484641, "value2":8788058834810483014, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.168246, "value1":0.1571304566786965, "value2":2933470821094626337, "value3":true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043250, "value1":0.3445304512204906, "value2":8908780418296865224, "value3":false, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542269, "value1":0.007638770771913262, "value2":975662304048103149, "value3":true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016306, "value1":0.44304617075801217, "value2":4700290207925657878, "value3":false, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967184, "value1":0.9943193022594318, "value2":7141028805491571993, "value3":true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522277, "value1":0.8578463560518583, "value2":6372322792910116627, "value3":false, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541009, "value1":0.5200416543540609, "value2":5687465452183940942, "value3":true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742322, "value1":0.4918651213341637, "value2":7530197332852248825, "value3":true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.478705, "value1":0.02234257118225825, "value2":381364684161431163, "value3":true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.401319, "value1":0.36716744081601327, "value2":3165286506094547515, "value3":false, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.154587, "value1":0.8570612240935895, "value2":440607090836829314, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268457, "value1":0.19708890772544582, "value2":716490672588175731, "value3":true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976461, "value1":0.37921805973698547, "value2":5187015251060220885, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.291361, "value1":0.0347133343718353, "value2":4081610772727292868, "value3":false, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.831910, "value1":0.20329085836851984, "value2":1212790782212732680, "value3":true, "__name__":"metric_00000189", "key_3":"j","key_9":"k","key_2":"g", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045270, "value1":0.742873126094037, "value2":1857680286082818546, "value3":false, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719581, "value1":0.8427487279865438, "value2":1584262554399141329, "value3":true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.878973, "value1":0.7217975560465744, "value2":6936890459625848218, "value3":true, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.786254, "value1":0.7388730574867226, "value2":3591226182562159088, "value3":true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719189, "value1":0.6138101273145392, "value2":4767024689432495577, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335758, "value1":0.6941935445321573, "value2":1310712364678635982, "value3":false, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333397, "value1":0.7084933562313812, "value2":2679977702860384198, "value3":true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816868, "value1":0.5704940444863804, "value2":3982836666436033897, "value3":true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.806625, "value1":0.7550148139027509, "value2":2819861429300922010, "value3":true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155981, "value1":0.3978060292285169, "value2":2375324886880679840, "value3":true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.172764, "value1":0.8807292451628046, "value2":1585412394695366401, "value3":false, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.127700, "value1":0.8691901922846396, "value2":4006452105841009084, "value3":false, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.937137, "value1":0.6079865671377581, "value2":9014517461395944209, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601939, "value1":0.5587486969852118, "value2":896670201927694097, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147910, "value1":0.7860716932621791, "value2":8828747393971238771, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271640, "value1":0.8634829980251869, "value2":7534202338930685599, "value3":true, "__name__":"metric_00000177", "key_7":"c","key_8":"d","key_4":"i", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510609, "value1":0.273735584071857, "value2":7995135221543970576, "value3":false, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.424702, "value1":0.6570665232819806, "value2":7130224512963405605, "value3":true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.120070, "value1":0.4366525687584962, "value2":8321790572084844233, "value3":true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.355286, "value1":0.9376756075719435, "value2":7612697762460955574, "value3":true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057269, "value1":0.9603884122588556, "value2":5675558649074747456, "value3":true, "__name__":"metric_00000182", "key_9":"b","key_3":"a","key_7":"f", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.396116, "value1":0.3241173308406254, "value2":2149479689060994101, "value3":false, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271843, "value1":0.5095285278231975, "value2":5564561021464735924, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389082, "value1":0.7189491284948987, "value2":8539354123799965742, "value3":true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780310, "value1":0.7959419420056967, "value2":791327040768245285, "value3":false, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496100, "value1":0.7137676161743793, "value2":7249423453544847575, "value3":false, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.320840, "value1":0.7088322490051288, "value2":2259284238716415015, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.588238, "value1":0.2616266339940352, "value2":7616111814012126210, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510231, "value1":0.34548313055527086, "value2":9146415948654460714, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725142, "value1":0.832268560021817, "value2":8449163586612650016, "value3":false, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722343, "value1":0.5127452250176201, "value2":8372677466999750797, "value3":true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455239, "value1":0.7274771017691084, "value2":8558710521155328448, "value3":true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070961, "value1":0.5108403289338641, "value2":6822737602377250548, "value3":true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.799042, "value1":0.6579145757389928, "value2":6367849243718047663, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997752, "value1":0.29826795257630173, "value2":7963803897292403489, "value3":false, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019959, "value1":0.5329624946388745, "value2":6166072735161874859, "value3":false, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183220, "value1":0.1192005017394279, "value2":1414525889342507585, "value3":true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.216504, "value1":0.10069780302425961, "value2":4147229267124117622, "value3":false, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184537, "value1":0.9674252449470503, "value2":7196918589322334021, "value3":true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.982196, "value1":0.5121645885657403, "value2":1331574421408029626, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.807710, "value1":0.6453473320699041, "value2":5880434997959490048, "value3":false, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139645, "value1":0.45151332154483154, "value2":5206983362834731524, "value3":true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202627, "value1":0.5085399708189182, "value2":229347926887383465, "value3":true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703675, "value1":0.1410911985436997, "value2":1995545208705825638, "value3":false, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.916538, "value1":0.9588604837943727, "value2":3014447442999476863, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174096, "value1":0.8442896590468587, "value2":1669527921851537286, "value3":true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268618, "value1":0.05460739672623512, "value2":6145595254763691216, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184210, "value1":0.08111723273128417, "value2":2941741438359406510, "value3":true, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419569, "value1":0.11197378055505244, "value2":7190302664501414682, "value3":false, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.080457, "value1":0.46189706497698463, "value2":3739595486194011632, "value3":false, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685402, "value1":0.5911751311740201, "value2":375795342652216999, "value3":false, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584152, "value1":0.842391293090085, "value2":3305999218832830126, "value3":true, "__name__":"metric_00000241", "key_9":"d","key_4":"e","key_8":"a", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719567, "value1":0.7117389665869482, "value2":4436375112558311543, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.599224, "value1":0.7507823724841362, "value2":5289861475224212810, "value3":true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298087, "value1":0.03464563720261393, "value2":5487979858306400729, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084762, "value1":0.4408096997046924, "value2":8782838705036394284, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153226, "value1":0.8396685378218853, "value2":2305238685769866566, "value3":false, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875795, "value1":0.6283427563062458, "value2":1424824369894963651, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.841870, "value1":0.2951891910717211, "value2":1229301210242052079, "value3":true, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036891, "value1":0.10272064811801765, "value2":8629352909535519019, "value3":true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.475886, "value1":0.47884350384173996, "value2":2002152937943396093, "value3":false, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286578, "value1":0.5511122934004854, "value2":6456656081230025536, "value3":false, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180709, "value1":0.591154696871619, "value2":1085648256858149575, "value3":true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.549035, "value1":0.9073632737596287, "value2":441521694946529713, "value3":false, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644644, "value1":0.42900060724414874, "value2":7252379057595536961, "value3":false, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.185531, "value1":0.28679213225919725, "value2":3666625025829964686, "value3":false, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858171, "value1":0.07899073830408582, "value2":8847544646507443860, "value3":false, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.9086649445849005, "value2":7300771702176978507, "value3":false, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.217129, "value1":0.8286760370555245, "value2":437314161092307860, "value3":true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.092457, "value1":0.3257453112147972, "value2":1241192378749271122, "value3":false, "__name__":"metric_00000225", "key_4":"a","key_2":"e", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.318459, "value1":0.6218522470963161, "value2":2601033448014127071, "value3":true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543632, "value1":0.47194572326020234, "value2":330858668685674119, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.494889, "value1":0.42021023869128465, "value2":7698787812508472546, "value3":false, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173188, "value1":0.6333586665476976, "value2":1031119021359267335, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392225, "value1":0.13669516747392368, "value2":2722787133031533732, "value3":false, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.418321, "value1":0.8852713566769926, "value2":1145580014369872247, "value3":true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499498, "value1":0.37652715283253, "value2":6023088039348052644, "value3":true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049271, "value1":0.7428393913560761, "value2":4074778390717770530, "value3":false, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344096, "value1":0.618674195595606, "value2":8432036346015765748, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.504726, "value1":0.8381168181634259, "value2":7490362504268179930, "value3":false, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130009, "value1":0.04787900547347569, "value2":8930977988943996511, "value3":false, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517353, "value1":0.9123390757414803, "value2":5406757776090744464, "value3":true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.159780, "value1":0.2711186767871672, "value2":6650537343113966916, "value3":false, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601454, "value1":0.036596921749510625, "value2":1801819188066798655, "value3":false, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740877, "value1":0.9080583731663923, "value2":6623419932438799475, "value3":true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.743094, "value1":0.9900165531318826, "value2":383393257357043043, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.939900, "value1":0.8769075090910828, "value2":487693007619851718, "value3":true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.102543, "value1":0.4399163118191967, "value2":5888144134235313914, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917706, "value1":0.6339047693941521, "value2":6760479052672151531, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108610, "value1":0.06502295112846754, "value2":8902773220268340372, "value3":true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.756394, "value1":0.7104082109611765, "value2":3854497302576556911, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914430, "value1":0.39800381262255646, "value2":1413572596816022205, "value3":true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.046273, "value1":0.9197740465700138, "value2":5994606503835738604, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.042126, "value1":0.35565383574544296, "value2":206374188969693152, "value3":true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.753926, "value1":0.14736292648599036, "value2":5720098218146494963, "value3":false, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138925, "value1":0.14592833793919333, "value2":5845534364661717380, "value3":false, "__name__":"metric_00000284", "key_7":"e","key_0":"b","key_3":"b", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966713, "value1":0.5691036450918006, "value2":5571037157834825483, "value3":false, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128296, "value1":0.522798351805377, "value2":7845314774564626095, "value3":true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.374869, "value1":0.16619436230032464, "value2":2683834913042811436, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349899, "value1":0.0806742622823487, "value2":5654997107573600799, "value3":true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621137, "value1":0.6389883746416457, "value2":6718840923740903865, "value3":false, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505989, "value1":0.5736885226925782, "value2":7660274947298694696, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245218, "value1":0.06067210631905806, "value2":7481373150924938933, "value3":true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.961402, "value1":0.40476409107709094, "value2":1656152426641438063, "value3":true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064546, "value1":0.7710994608894077, "value2":5484900997853498427, "value3":false, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686047, "value1":0.46617534097616353, "value2":1998746207850635494, "value3":true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195207, "value1":0.8036664106551048, "value2":21823515460647768, "value3":false, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890318, "value1":0.49246668518608533, "value2":4290226524505777397, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.662549, "value1":0.0648629447687713, "value2":2600958480076392029, "value3":true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867346, "value1":0.6103284095540984, "value2":959086414558873258, "value3":false, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036038, "value1":0.22211264871806694, "value2":2970074654495356013, "value3":false, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772742, "value1":0.47254264158251036, "value2":5967291460715844391, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.765254, "value1":0.59761349012731, "value2":5045361763860538939, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684477, "value1":0.3727521458297325, "value2":7120290254833886255, "value3":true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782498, "value1":0.5592140587339981, "value2":1560278611367944657, "value3":false, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472665, "value1":0.4451202433666956, "value2":7528293771469761549, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957028, "value1":0.6722101190310112, "value2":3256843532996983591, "value3":true, "__name__":"metric_00000298", "key_4":"j","key_9":"b","key_3":"g", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.558411, "value1":0.33974258843823785, "value2":9219292136379640073, "value3":true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686456, "value1":0.662252940687323, "value2":2872679473486076312, "value3":false, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.664702, "value1":0.13252769744708778, "value2":3782836160183678190, "value3":false, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340522, "value1":0.321610730663863, "value2":1993266611063130503, "value3":true, "__name__":"metric_00000301", "key_7":"i","key_9":"k","key_4":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738508, "value1":0.03455404630849952, "value2":5265512962120119813, "value3":true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722717, "value1":0.039393801449424636, "value2":8411289366884053387, "value3":false, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738985, "value1":0.4374781921483304, "value2":4532229627185520929, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.959594, "value1":0.06784833824743756, "value2":2235113542571170507, "value3":true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512001, "value1":0.45518592645372824, "value2":1219663694779720192, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699152, "value1":0.9199280757111055, "value2":920010063978439254, "value3":false, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870981, "value1":0.7153328294012858, "value2":2241596700674662533, "value3":false, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713651, "value1":0.1871397130196525, "value2":5525165593287246022, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608421, "value1":0.2007302921644417, "value2":7082345286303554347, "value3":true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.898979, "value1":0.4326799832087361, "value2":4510789740667402710, "value3":true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542323, "value1":0.5916017742256197, "value2":7802920355385118767, "value3":true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245771, "value1":0.3115277557229622, "value2":6710572841494695733, "value3":true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772755, "value1":0.17023256633885028, "value2":4693930877366777744, "value3":true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.247931, "value1":0.4740059660910414, "value2":1391972395280563550, "value3":true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394287, "value1":0.02366479635408087, "value2":7477007108410228541, "value3":false, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108172, "value1":0.971283160522281, "value2":6224196629196175206, "value3":true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932465, "value1":0.6431474969443486, "value2":6996302573266425119, "value3":false, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070979, "value1":0.4324217725560448, "value2":9202735673442875059, "value3":false, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862734, "value1":0.5425012681866345, "value2":8705521243115097026, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207036, "value1":0.7438270358882513, "value2":1482607831466689197, "value3":false, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345448, "value1":0.3875837956213489, "value2":2480267181897079015, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553288, "value1":0.3343119844132966, "value2":7890867607759034801, "value3":true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559986, "value1":0.2851751818045875, "value2":3231450807250993029, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.008285, "value1":0.11082046459395005, "value2":5670330804396842703, "value3":true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620145, "value1":0.27733408115115116, "value2":4537754734930200229, "value3":false, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.861443, "value1":0.7565577041788646, "value2":216193021440411568, "value3":true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584594, "value1":0.7717911844132257, "value2":7036437829052540741, "value3":true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.066975, "value1":0.07703464444888349, "value2":8332259918814404840, "value3":true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272133, "value1":0.2064168890828292, "value2":1598424380451243838, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725661, "value1":0.5107106855646196, "value2":2095325894892271000, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.992271, "value1":0.04606905650348332, "value2":3458383290691046043, "value3":true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988671, "value1":0.9012802494548691, "value2":3265213363984040214, "value3":true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344354, "value1":0.6315603617727772, "value2":1819854718616345426, "value3":false, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198753, "value1":0.2443944990728996, "value2":3740319967339712109, "value3":true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294599, "value1":0.7030675431685813, "value2":5821390349436487736, "value3":false, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.233177, "value1":0.5520329333372563, "value2":2208761416514593046, "value3":true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.369614, "value1":0.2829108305094591, "value2":2978362585536403835, "value3":false, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.484777, "value1":0.3351273859753424, "value2":1348038406704883991, "value3":true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511345, "value1":0.6897076794113861, "value2":3140482719657458044, "value3":true, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780083, "value1":0.056509953898858746, "value2":4983435501891849142, "value3":false, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.023345, "value1":0.08633016397064507, "value2":439606973167008387, "value3":true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013043, "value1":0.8750924151560185, "value2":2563435998029443275, "value3":true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929738, "value1":0.6942458871164995, "value2":6742189975038174280, "value3":false, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348725, "value1":0.6477096282239411, "value2":614033611645126767, "value3":false, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.148404, "value1":0.3122137082601719, "value2":5921845943066980452, "value3":false, "__name__":"metric_00000355", "key_9":"e","key_2":"d","key_3":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.892407, "value1":0.0024857877019035433, "value2":78557787569325314, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770110, "value1":0.4854945232954409, "value2":4441111957237018619, "value3":false, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969650, "value1":0.028054363655575613, "value2":6620021166465815802, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096277, "value1":0.7642864471977222, "value2":7079078098912113925, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104248, "value1":0.17068081288522707, "value2":2914828831179785715, "value3":false, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.033230, "value1":0.6691862134057828, "value2":5879037419816229961, "value3":true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442293, "value1":0.918687680755694, "value2":3522143514446827859, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340520, "value1":0.3316554635970253, "value2":3298849581857165314, "value3":true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.005701, "value1":0.22902534413133244, "value2":4040792449962896607, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.101103, "value1":0.4459003073823555, "value2":8364403783112020861, "value3":false, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.269361, "value1":0.9829136286924077, "value2":305552246857799140, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002507, "value1":0.10735284014448321, "value2":2548523454772398525, "value3":false, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516501, "value1":0.3420030652884641, "value2":449036192165412633, "value3":true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914771, "value1":0.6948526439267081, "value2":3274351745805138467, "value3":true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151382, "value1":0.9010351248418295, "value2":6225882894853282790, "value3":false, "__name__":"metric_00000323", "key_7":"d","key_2":"c", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.797287, "value1":0.450517653615541, "value2":6406142276072623664, "value3":false, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575226, "value1":0.6947969923557978, "value2":4732050542115525893, "value3":true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.256780, "value1":0.29401472322197164, "value2":8790173514967991405, "value3":true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791599, "value1":0.6204111480891676, "value2":1162220986630177027, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615289, "value1":0.17893814852804246, "value2":3187373261649831292, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738034, "value1":0.9979754782079555, "value2":6006267840987043849, "value3":true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735591, "value1":0.011768322537597629, "value2":7974963296156276842, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.751213, "value1":0.43700619365470833, "value2":6294325004534159296, "value3":true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781793, "value1":0.6112369782572161, "value2":2263959578091534198, "value3":false, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118524, "value1":0.4979095180792778, "value2":5142183829715514199, "value3":false, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.574787, "value1":0.8076799963798312, "value2":1975720601465651847, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814750, "value1":0.45891950977715895, "value2":4074714357288466936, "value3":false, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579803, "value1":0.25446438731696874, "value2":7191932871100542216, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722158, "value1":0.19082711905600802, "value2":1422914021938617784, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342798, "value1":0.5097927561705997, "value2":2281093093995561768, "value3":false, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164991, "value1":0.06555881689103157, "value2":7152427537795186892, "value3":true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.316516, "value1":0.1580165967231405, "value2":8769573796829946768, "value3":false, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548667, "value1":0.8164945933097875, "value2":1223332369807607717, "value3":false, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174435, "value1":0.5629514369055348, "value2":2359802095953741925, "value3":false, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.561139, "value1":0.9847446485189013, "value2":8748242508292199474, "value3":true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604206, "value1":0.7084164989246987, "value2":5662112818525056068, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.069641, "value1":0.9392432577384211, "value2":7977134301111651930, "value3":false, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977024, "value1":0.8770253339233329, "value2":114419076017958040, "value3":true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.981104, "value1":0.0262761349528985, "value2":3404389245833544901, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.309389, "value1":0.6700115572733264, "value2":1746800498285488694, "value3":true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907890, "value1":0.3847355476155844, "value2":1062772964889350348, "value3":true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.557849, "value1":0.8423014863985597, "value2":8693557035023467587, "value3":false, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174807, "value1":0.14044078429336324, "value2":2899359413194652810, "value3":true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907265, "value1":0.7213413890977288, "value2":711482054709798828, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730322, "value1":0.010569544047203483, "value2":2272852869521847556, "value3":true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047968, "value1":0.8749328675131659, "value2":3313417752475725526, "value3":false, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548996, "value1":0.13476734433713453, "value2":5686844894034380593, "value3":false, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090199, "value1":0.47038291207998856, "value2":6737581800965808102, "value3":true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372121, "value1":0.4982647921591242, "value2":2960213658998005236, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.732328, "value1":0.6794810773912932, "value2":5614573251611720393, "value3":true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195822, "value1":0.7817234737401128, "value2":6738900276029001186, "value3":true, "__name__":"metric_00000399", "key_9":"g","key_4":"h", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173099, "value1":0.7620323539766012, "value2":6195639668903126539, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739085, "value1":0.8805144351999263, "value2":6990390850908433, "value3":true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851839, "value1":0.19515034480983603, "value2":5022120185556407326, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.764811, "value1":0.07516380443788656, "value2":4953415333432850814, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.954192, "value1":0.6068771093686028, "value2":1921210459347583907, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.501412, "value1":0.20889031917804166, "value2":7266160086492287738, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798358, "value1":0.44429041626860627, "value2":7311639409429341020, "value3":false, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200103, "value1":0.5998566082715615, "value2":3968426630272414916, "value3":false, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443243, "value1":0.7996897987041263, "value2":2955385724626539485, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808129, "value1":0.12934958318030565, "value2":8640009016586784249, "value3":true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890136, "value1":0.395437996374284, "value2":1375026123479284573, "value3":false, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.726245, "value1":0.8383327509398333, "value2":1069495836938595505, "value3":true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464465, "value1":0.7522623057444222, "value2":3745372652823842076, "value3":false, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958198, "value1":0.6062226980863115, "value2":692634818001600691, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336639, "value1":0.3569530703633769, "value2":5114963101620697516, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153507, "value1":0.29997973310668347, "value2":35895264299031809, "value3":false, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496206, "value1":0.16796883917469935, "value2":2457448015461697922, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699773, "value1":0.1320495753882598, "value2":2684330380511330426, "value3":false, "__name__":"metric_00000416", "key_6":"i","key_9":"b","key_3":"h", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086489, "value1":0.01686874394925538, "value2":7266008503964487472, "value3":true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.755161, "value1":0.12039999679109185, "value2":7441080332932356910, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481861, "value1":0.21023924595248417, "value2":7215904118134021296, "value3":false, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141970, "value1":0.6959505249259283, "value2":5593000831636710131, "value3":false, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473262, "value1":0.20484780325876686, "value2":1515840223895186260, "value3":true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.610674, "value1":0.37794171859625975, "value2":7210398121561053631, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784669, "value1":0.8314547953224727, "value2":6889739271959092821, "value3":false, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348279, "value1":0.9806070629868892, "value2":6665537531662525499, "value3":true, "__name__":"metric_00000427", "key_7":"h","key_2":"i","key_5":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342228, "value1":0.8292222808866697, "value2":8140150785342027171, "value3":false, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049991, "value1":0.20259901011770426, "value2":860224417352173376, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442752, "value1":0.9271842959401567, "value2":8958731671751036609, "value3":false, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288552, "value1":0.19606129029896613, "value2":3803062421310719257, "value3":true, "__name__":"metric_00000436", "key_4":"c","key_5":"c","key_2":"h", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851767, "value1":0.8040030606139774, "value2":5022190100385685639, "value3":true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258187, "value1":0.06995124309795013, "value2":3290080872961653036, "value3":false, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967676, "value1":0.08273649902951419, "value2":5115999717780317436, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.972416, "value1":0.3332024948916442, "value2":3195696687672274736, "value3":true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273980, "value1":0.9801180643314533, "value2":7124826086691987600, "value3":true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962051, "value1":0.7078535280449856, "value2":2450804349449720576, "value3":true, "__name__":"metric_00000442", "key_2":"e","key_6":"f","key_1":"i", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342560, "value1":0.21100802753268208, "value2":5026066029722000095, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582068, "value1":0.9724895772631098, "value2":9045330876857967867, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716662, "value1":0.5239131397726235, "value2":6699936731993788174, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.630988, "value1":0.8757044551570439, "value2":6009726873690078427, "value3":false, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.617422, "value1":0.029531083291305976, "value2":7932542565063790762, "value3":true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389736, "value1":0.40122875818931175, "value2":7350974819717892233, "value3":true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.470743, "value1":0.286118127257554, "value2":2903264335442728018, "value3":true, "__name__":"metric_00000449", "key_6":"j","key_1":"i","key_4":"i", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118668, "value1":0.42900363662380264, "value2":2786819190490835132, "value3":false, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819243, "value1":0.7832376096990535, "value2":6768946688624968401, "value3":true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519924, "value1":0.1784273794737174, "value2":606279348173747859, "value3":false, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575499, "value1":0.8580592683181654, "value2":6065349113607731318, "value3":false, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.952434, "value1":0.8198422458926743, "value2":6520729939047239531, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180605, "value1":0.504162836348111, "value2":8107251100494910096, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.680620, "value1":0.9192990596381249, "value2":7916466116307364506, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421505, "value1":0.6014385702769156, "value2":6172295401410316312, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581350, "value1":0.2695750525680962, "value2":6289934971199391350, "value3":false, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.244031, "value1":0.7392367672457897, "value2":7438364572557990950, "value3":true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801331, "value1":0.014860008784846704, "value2":6873217646463864251, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875352, "value1":0.3975554355639885, "value2":4390005070727485119, "value3":true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929440, "value1":0.40393323123767977, "value2":1148796038957803851, "value3":false, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.817616, "value1":0.5647949954501486, "value2":1024684537861148034, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.990318, "value1":0.25287778647718107, "value2":7093365367237549470, "value3":true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741736, "value1":0.023580263258422104, "value2":1518420105714291210, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463012, "value1":0.7888469551975262, "value2":6786560784050350679, "value3":true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449551, "value1":0.2636704318808076, "value2":7884141870151198099, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419454, "value1":0.6010995928923429, "value2":224905435491328214, "value3":true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.322105, "value1":0.1331880379524159, "value2":7369022064377940075, "value3":false, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581063, "value1":0.603596835974457, "value2":4386876721702326915, "value3":false, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.857080, "value1":0.5842046650044923, "value2":1066043362420191253, "value3":true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191051, "value1":0.4639324153416964, "value2":8874234446483833192, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298751, "value1":0.20423387563647288, "value2":8774956872647827067, "value3":true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924593, "value1":0.0659156088635184, "value2":3634318708597769338, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404558, "value1":0.7799254034556268, "value2":4510574800615418821, "value3":true, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962467, "value1":0.8168896478129019, "value2":7986325005390816690, "value3":false, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622579, "value1":0.0904343638172895, "value2":8283142617455012372, "value3":false, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710038, "value1":0.6909066964841821, "value2":6203472414968382073, "value3":false, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032660, "value1":0.21417322011190834, "value2":1054486238769744890, "value3":true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801225, "value1":0.2765954328390815, "value2":442385272245544502, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116566, "value1":0.6031158754932586, "value2":3770751187916956934, "value3":true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.284086, "value1":0.055471202806315845, "value2":7266763851735180811, "value3":false, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559271, "value1":0.28967037620921054, "value2":6147439427772857051, "value3":true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224654, "value1":0.5513198770455077, "value2":7761756752100166184, "value3":false, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575799, "value1":0.07079456843554317, "value2":4847399039818291257, "value3":false, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819125, "value1":0.4806719127445899, "value2":1129446367168224920, "value3":true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730720, "value1":0.6013248235298089, "value2":1610990221268346838, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.593103, "value1":0.8196760148078414, "value2":6644540894797627889, "value3":false, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.218509, "value1":0.5227005302649587, "value2":2525326877519094680, "value3":false, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.292767, "value1":0.09698193120132978, "value2":8960471678202836654, "value3":false, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858612, "value1":0.675005592095046, "value2":6028830951911645471, "value3":true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736558, "value1":0.24026583264924886, "value2":1360237166500303266, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826970, "value1":0.3893255587955024, "value2":2292740904231475335, "value3":true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128577, "value1":0.3146179907253604, "value2":164469213437964556, "value3":true, "__name__":"metric_00000485", "key_3":"i","key_4":"a","key_0":"g", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003953, "value1":0.33225441021352176, "value2":5713576701192138110, "value3":true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352438, "value1":0.12247814806052679, "value2":976940527664803795, "value3":false, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.434520, "value1":0.6138506637398143, "value2":1920264902732481602, "value3":false, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.731843, "value1":0.8305061213621728, "value2":4607426852903927950, "value3":false, "__name__":"metric_00000489", "key_7":"h","key_0":"c","key_5":"a", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032970, "value1":0.8273657553543087, "value2":3103821801573462769, "value3":true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833737, "value1":0.3431117940438562, "value2":5529881494203381864, "value3":false, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.206561, "value1":0.03180903277463739, "value2":1534192978519074976, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598146, "value1":0.7943659660871121, "value2":2512296846803666990, "value3":false, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.830318, "value1":0.9077939825338383, "value2":1289520021940183737, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409047, "value1":0.694416059529231, "value2":520453313543014186, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.949791, "value1":0.6369046983298985, "value2":371741091848582403, "value3":true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354931, "value1":0.7633140731862355, "value2":8981576909845465787, "value3":true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.061767, "value1":0.7699262325323719, "value2":79439898840104021, "value3":false, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.941133, "value1":0.26448399729496197, "value2":6448749789711445941, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.339821, "value1":0.5246395493337294, "value2":44122001398655312, "value3":true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359654, "value1":0.5954751506379091, "value2":1668054016792098588, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.304719, "value1":0.17854657722051143, "value2":5240836642486639588, "value3":true, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533350, "value1":0.9102023121053504, "value2":5223044570382986189, "value3":true, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.495583, "value1":0.8380393694454433, "value2":2826946501048985900, "value3":true, "__name__":"metric_00000504", "key_6":"c","key_0":"f","key_2":"a", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925864, "value1":0.06057194714527472, "value2":4827880370625631466, "value3":true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200061, "value1":0.3196222334980913, "value2":5536801387533970020, "value3":true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.635607, "value1":0.5926951079597541, "value2":4317404702469315811, "value3":false, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041722, "value1":0.2038750977962134, "value2":8603762180622472, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.565080, "value1":0.5851130292640377, "value2":8612148593533709646, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090894, "value1":0.12207888531940794, "value2":435323305893709131, "value3":true, "__name__":"metric_00000510", "key_9":"e","key_4":"c","key_8":"j", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.071849, "value1":0.7914473708034223, "value2":4833048612300779383, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202326, "value1":0.6798241827808086, "value2":5321585738808934000, "value3":false, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116141, "value1":0.8646385559973755, "value2":2431723757311861306, "value3":false, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.074497, "value1":0.2771958266692314, "value2":2948732814035791763, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.453427, "value1":0.9750934919782419, "value2":7598695164797163349, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517105, "value1":0.6447380734003394, "value2":5945582855956594632, "value3":false, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969670, "value1":0.8792534113647463, "value2":9118825598855667812, "value3":true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064653, "value1":0.7641915057577142, "value2":376649068007551056, "value3":true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.299983, "value1":0.5328541897726483, "value2":7848443219053578824, "value3":false, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814787, "value1":0.5822532477986045, "value2":3517444920674306021, "value3":true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645109, "value1":0.6788316811493683, "value2":2399412759017043209, "value3":false, "__name__":"metric_00000521", "key_5":"d","key_6":"f","key_3":"h", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104970, "value1":0.5150104887448738, "value2":39162375392832614, "value3":true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124629, "value1":0.9415126690810315, "value2":2487867259401435813, "value3":true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847494, "value1":0.8236014719422879, "value2":8522949906604300533, "value3":true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.694932, "value1":0.5232173520376328, "value2":2865775556504465271, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.824405, "value1":0.33768880558190767, "value2":1449467820646971158, "value3":true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263091, "value1":0.800562879218074, "value2":4640064437824793404, "value3":true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.545676, "value1":0.6991321506153548, "value2":504604265126996830, "value3":true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685750, "value1":0.256315076419413, "value2":7805803213074768353, "value3":false, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.955572, "value1":0.12777100392825705, "value2":4697380262379050863, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.052725, "value1":0.24897441126326256, "value2":2806761823519892967, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832231, "value1":0.7123036010256201, "value2":5558652604185663810, "value3":false, "__name__":"metric_00000541", "key_7":"c","key_3":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.779475, "value1":0.42379979395746137, "value2":5877566589967590752, "value3":false, "__name__":"metric_00000542", "key_1":"a","key_2":"j","key_0":"c", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.056823, "value1":0.7557073972848881, "value2":1462009193456216531, "value3":true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179551, "value1":0.33562095061974395, "value2":2905800790209406415, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.946943, "value1":0.7895726694044407, "value2":4556871779015432465, "value3":false, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.792955, "value1":0.9478490399693646, "value2":6469796413879524879, "value3":true, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302294, "value1":0.40539394874343054, "value2":1479528610985409190, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.859086, "value1":0.8533535931566633, "value2":8328258195824345992, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.912801, "value1":0.4080605346123484, "value2":921699075685228177, "value3":false, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026911, "value1":0.5160934551500268, "value2":4192638354189389027, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.145578, "value1":0.4697942761887252, "value2":1007860316702817500, "value3":true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224036, "value1":0.21446416617045144, "value2":2612501104515894392, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.083418, "value1":0.3800566396023636, "value2":1387864613630168710, "value3":true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.483137, "value1":0.9606563644656662, "value2":2761802265245174240, "value3":false, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.855910, "value1":0.606558104786761, "value2":842530041019109994, "value3":true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332745, "value1":0.01260394609809363, "value2":2781673523626108476, "value3":false, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198691, "value1":0.0952542911286906, "value2":4639530496495690336, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207605, "value1":0.29598976404510996, "value2":149815254996856780, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.219100, "value1":0.7173110134058144, "value2":207519709019999249, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.065034, "value1":0.7683750910989782, "value2":6262446867113607884, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362737, "value1":0.5652820007764047, "value2":4138085582551770816, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.790498, "value1":0.45469936210178996, "value2":3874789041764013688, "value3":false, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581214, "value1":0.8566672554079884, "value2":5701861631669426000, "value3":false, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741360, "value1":0.7962411718913905, "value2":3357161583638308077, "value3":true, "__name__":"metric_00000534", "key_9":"k","key_1":"a","key_3":"a", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.934536, "value1":0.07212743130410183, "value2":6319820377917318485, "value3":true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258971, "value1":0.09357358589212993, "value2":5081900353713330302, "value3":false, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.436226, "value1":0.5553015007542973, "value2":6124312980620622172, "value3":false, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566508, "value1":0.7132604990750537, "value2":8485961090508027563, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298396, "value1":0.4460940570451377, "value2":5709626152776289179, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596697, "value1":0.3774598732286396, "value2":1819820653928002535, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.758549, "value1":0.49680425119422045, "value2":1371518397741898898, "value3":false, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151965, "value1":0.9897517149153644, "value2":3974511699577597667, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.223787, "value1":0.9859897177824236, "value2":6368178524733411270, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699805, "value1":0.9240609624872184, "value2":4091684677817576336, "value3":true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.640771, "value1":0.3605713604653968, "value2":2196974552050802581, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917234, "value1":0.3769625716544593, "value2":7094294232648752669, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541154, "value1":0.07721056009394252, "value2":2450341303960944649, "value3":false, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188174, "value1":0.511305577711157, "value2":4677738772360990938, "value3":false, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277439, "value1":0.006756936522771483, "value2":2736398221812059244, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847832, "value1":0.23362336838695744, "value2":6292362019324286334, "value3":false, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.840677, "value1":0.6137390254843741, "value2":6506004284907452986, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.397331, "value1":0.5291020730916409, "value2":766558029100390419, "value3":false, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.668711, "value1":0.6677953904066025, "value2":2813624786174976455, "value3":true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.728127, "value1":0.9455687108064199, "value2":2598623203001035108, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.260429, "value1":0.5485107722176485, "value2":8238880717060307339, "value3":true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407939, "value1":0.7503222721986553, "value2":8088756690439902034, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016860, "value1":0.6281421773683825, "value2":4463756144226416929, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.592662, "value1":0.7215969122220701, "value2":2957728488172560734, "value3":true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596916, "value1":0.6365196560861767, "value2":3939747841173152393, "value3":true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.650813, "value1":0.33343709461482135, "value2":8822031185214434530, "value3":true, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543088, "value1":0.5660730417813042, "value2":6651394201039088708, "value3":false, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.416422, "value1":0.2907903630149851, "value2":5811650096165674198, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505086, "value1":0.44660978158387593, "value2":6270587282710629926, "value3":false, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.055558, "value1":0.394254430161103, "value2":9153429600073084016, "value3":false, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.578079, "value1":0.6160395423413576, "value2":7388869261196091596, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.347142, "value1":0.28357580684654937, "value2":7651284402887473938, "value3":false, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089369, "value1":0.24785364470389606, "value2":2827626134616355884, "value3":true, "__name__":"metric_00000563", "key_2":"f","key_5":"a","key_0":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922968, "value1":0.9908293125287131, "value2":5006327563319926460, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489701, "value1":0.9764768678976257, "value2":3780590525108581330, "value3":false, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700790, "value1":0.8524762188330061, "value2":2298426148247437198, "value3":false, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.884155, "value1":0.17838161460811935, "value2":707461345329966226, "value3":false, "__name__":"metric_00000592", "key_1":"e","key_7":"b","key_0":"k", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.877319, "value1":0.597182710013635, "value2":66780576193296735, "value3":false, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906533, "value1":0.44570088765160515, "value2":1159099942400157753, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555879, "value1":0.6225620185948739, "value2":7381136800707660104, "value3":false, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.498993, "value1":0.9016213008618695, "value2":8432726482609273880, "value3":false, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349195, "value1":0.16850895992044404, "value2":6332393888509943818, "value3":true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530337, "value1":0.8558953203814225, "value2":76710422944574508, "value3":true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345065, "value1":0.8230787557550705, "value2":4201898391845439571, "value3":true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947222, "value1":0.9688974381514411, "value2":2441269560109987314, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.287425, "value1":0.20362983597374928, "value2":5349125652009205097, "value3":true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519644, "value1":0.3548258293385043, "value2":1594647307470342039, "value3":true, "__name__":"metric_00000602", "key_5":"i","key_0":"j","key_1":"f", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.027626, "value1":0.9120926865897546, "value2":356446699135295995, "value3":true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.054561, "value1":0.22396147232559485, "value2":3930608769790828524, "value3":false, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.388931, "value1":0.5101134733621842, "value2":6568513065035568570, "value3":false, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567065, "value1":0.79529931520039, "value2":6002480040724485306, "value3":false, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449050, "value1":0.9291008790205846, "value2":741995070657009132, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798788, "value1":0.9907520697579564, "value2":2343010721096789225, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.927945, "value1":0.8814555609090619, "value2":6729196069806990782, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917877, "value1":0.15366203128840322, "value2":2435285873402812217, "value3":false, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.301622, "value1":0.20876721045801763, "value2":1403445590939959710, "value3":false, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.745719, "value1":0.6060177757203955, "value2":2370675136825151538, "value3":true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156169, "value1":0.40315211694606023, "value2":5606845635294528121, "value3":false, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.651978, "value1":0.02572219217737101, "value2":6216284509197748039, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.395139, "value1":0.4352070266196555, "value2":2437710317158637331, "value3":false, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.132661, "value1":0.3845242564807311, "value2":1179550652058134811, "value3":true, "__name__":"metric_00000616", "key_7":"j","key_1":"k","key_5":"d", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.088075, "value1":0.27124710684397163, "value2":5954367646517477044, "value3":false, "__name__":"metric_00000618", "key_3":"g","key_0":"e","key_1":"j", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213664, "value1":0.9844242730115798, "value2":1430390936289238362, "value3":false, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070775, "value1":0.6998833636894136, "value2":8373393772820367148, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213733, "value1":0.44329246485521806, "value2":6509431828213171835, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608249, "value1":0.15493446331886743, "value2":4384868943457502322, "value3":true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.956693, "value1":0.9523530599130996, "value2":365504939691815078, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.432058, "value1":0.16692183631193722, "value2":4864357898977097624, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.303592, "value1":0.934106512625592, "value2":7515104000227729499, "value3":false, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333787, "value1":0.6736717480454248, "value2":5830960931165083327, "value3":true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359385, "value1":0.922695683809537, "value2":6703804740319670530, "value3":true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.211845, "value1":0.13745320161077518, "value2":8688931964841266452, "value3":false, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600304, "value1":0.365028686112319, "value2":4172510140835977605, "value3":true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045527, "value1":0.1293058694163136, "value2":4396801118892738860, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124929, "value1":0.2927998769649342, "value2":7840410733128704268, "value3":true, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.708707, "value1":0.42917010198000677, "value2":190549078586280269, "value3":false, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543921, "value1":0.45395915446447055, "value2":4038857743678185292, "value3":true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001958, "value1":0.8799882402246043, "value2":8959405737069039234, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834357, "value1":0.9787241750393116, "value2":4713255732772978232, "value3":true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398402, "value1":0.3356690698861162, "value2":8600435717396946203, "value3":true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.532477, "value1":0.5790086501753019, "value2":585720405314800565, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995284, "value1":0.019519382747907175, "value2":561302586535494029, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.492003, "value1":0.22729751026433825, "value2":2059445045999328988, "value3":false, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.357595, "value1":0.2653305071097244, "value2":7325567281409701416, "value3":false, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423155, "value1":0.8492553718976581, "value2":5551377313853459727, "value3":true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582438, "value1":0.9616765075659243, "value2":1026983801153932760, "value3":false, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409214, "value1":0.1250667617190597, "value2":5921255804215527024, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744368, "value1":0.8513042177093529, "value2":6603153879156409175, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970658, "value1":0.2306199750722779, "value2":2906238010168281574, "value3":false, "__name__":"metric_00000643", "key_6":"c","key_1":"e","key_4":"d", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520199, "value1":0.9665587748785213, "value2":2458054282507819610, "value3":true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.098618, "value1":0.12617188969368395, "value2":2230070381051915192, "value3":false, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279517, "value1":0.4244894688924968, "value2":7931154210918628962, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394850, "value1":0.011258876506077731, "value2":3335326787820059838, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.672924, "value1":0.7327755721792956, "value2":3880440053684337972, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.431109, "value1":0.6741923566632704, "value2":2327920765233612965, "value3":true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.659350, "value1":0.11708731624685002, "value2":276841347975235584, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.115617, "value1":0.43754612855223973, "value2":3365515217954889202, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533611, "value1":0.8399047779975791, "value2":2365721152492621420, "value3":false, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265333, "value1":0.7911288127976842, "value2":5955361860868770714, "value3":true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966474, "value1":0.3536325699896996, "value2":7032196306810796442, "value3":true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684964, "value1":0.8626527282208771, "value2":7197488766581698381, "value3":true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559000, "value1":0.010659679933132453, "value2":4115322864019466158, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455110, "value1":0.6204915686658947, "value2":6428809732964624602, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625937, "value1":0.9727096027334052, "value2":6229477409536583105, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.113032, "value1":0.5903167177404812, "value2":4156394072465021166, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567689, "value1":0.10316863931938577, "value2":8418749122094896416, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.987774, "value1":0.7873883846984865, "value2":2030919041886568670, "value3":false, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.811061, "value1":0.4992724231992665, "value2":3986295112752729017, "value3":false, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194651, "value1":0.2743659705657923, "value2":583251715501918289, "value3":false, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950390, "value1":0.5427606927000886, "value2":2986321212639280881, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598120, "value1":0.20329762636615106, "value2":5612645284463886443, "value3":true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950214, "value1":0.4515603514606786, "value2":259527570321604761, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645723, "value1":0.13300625888317788, "value2":9108039921133288290, "value3":false, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001569, "value1":0.01777574006312956, "value2":7180543411595530153, "value3":true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036902, "value1":0.19075245150411826, "value2":5889621655868191557, "value3":false, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929644, "value1":0.8218270153578531, "value2":6338216490369302155, "value3":false, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036799, "value1":0.8121072864023562, "value2":4644698086978403677, "value3":false, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099804, "value1":0.41836860978181317, "value2":2663944418369780756, "value3":true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.607018, "value1":0.04603283781595935, "value2":7178332091114199020, "value3":false, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075733, "value1":0.9728379954214806, "value2":4310309256078899557, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795422, "value1":0.6466237769859894, "value2":344630441945165516, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924050, "value1":0.5005809462467858, "value2":4504593684309524057, "value3":false, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352892, "value1":0.9539405805618134, "value2":996952878654521350, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141322, "value1":0.01789552453746491, "value2":5938050187519951412, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.783153, "value1":0.1436758434821978, "value2":6441131001960903605, "value3":false, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473891, "value1":0.3495022243472323, "value2":4306013569645375103, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814977, "value1":0.38372259399528474, "value2":436380091488564183, "value3":true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716628, "value1":0.6699570981955947, "value2":3523603923463352771, "value3":true, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735953, "value1":0.6776358805955892, "value2":5856380565120662945, "value3":false, "__name__":"metric_00000693", "key_5":"e","key_8":"b","key_0":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265185, "value1":0.11607380521583967, "value2":5582686822797693437, "value3":false, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516162, "value1":0.2994616004522488, "value2":2261467086953311820, "value3":false, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334873, "value1":0.14715552224004888, "value2":8821110853892469057, "value3":false, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.400930, "value1":0.8143295386608986, "value2":1312967392198736440, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196057, "value1":0.5352481987346313, "value2":3859881997560184862, "value3":false, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925763, "value1":0.823842231642209, "value2":7076092787875459652, "value3":true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.012976, "value1":0.08033889071311322, "value2":2194666423113168625, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656711, "value1":0.979250223824098, "value2":295513105593707440, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542943, "value1":0.34216024465522854, "value2":8482137672426222632, "value3":false, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738294, "value1":0.6164314293888586, "value2":7863948057381466211, "value3":false, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957221, "value1":0.4133406835193543, "value2":3905154346081616319, "value3":true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047765, "value1":0.9114235810126055, "value2":8926222646986865918, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423884, "value1":0.30572384015704446, "value2":2339988500116190675, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209371, "value1":0.07849561730590998, "value2":3882974907897625962, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310209, "value1":0.2535587414218041, "value2":7470349035360557369, "value3":false, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988088, "value1":0.2879292667105676, "value2":5363951038000132107, "value3":true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947720, "value1":0.958399617505033, "value2":5218176225849015970, "value3":false, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.674130, "value1":0.9617762098247892, "value2":5270132257081817867, "value3":true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279827, "value1":0.8686925044217091, "value2":7240549587345744089, "value3":true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.257178, "value1":0.28758053646406956, "value2":7938343410176527229, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870350, "value1":0.16917076304978226, "value2":9111760754875505635, "value3":true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.763299, "value1":0.8896924750809221, "value2":6658856857804173307, "value3":true, "__name__":"metric_00000690", "key_8":"k","key_1":"g","key_5":"h", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734200, "value1":0.44507702947189576, "value2":4594583173060659683, "value3":true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155510, "value1":0.8897408774019222, "value2":3124062956444214018, "value3":true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.363722, "value1":0.07362703906856644, "value2":6124606823090260320, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336581, "value1":0.7586482338064128, "value2":2691495825421343596, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075863, "value1":0.7294045132994561, "value2":8264827508243049447, "value3":false, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108381, "value1":0.45788844677640256, "value2":2616045646790504337, "value3":false, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901873, "value1":0.08770240902249962, "value2":6640573405482547401, "value3":true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.438350, "value1":0.25439050102218247, "value2":7286046839968613612, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.314379, "value1":0.33872622756746645, "value2":2585280362910555462, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736079, "value1":0.12615475607591273, "value2":8133018761204018534, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130003, "value1":0.7963354396181769, "value2":6450859077912673686, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.485449, "value1":0.39497609142163287, "value2":170013329412671465, "value3":true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.06350907823991539, "value2":18163724507539892, "value3":false, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.6759106102023759, "value2":8613373636446320004, "value3":true, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522262, "value1":0.9415564313481927, "value2":183076459384497822, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.456571, "value1":0.3755600914749928, "value2":1317951286539480179, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713142, "value1":0.3684027597862527, "value2":6666246661014977211, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332747, "value1":0.2712525974797622, "value2":6685862283475642897, "value3":false, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622289, "value1":0.8099425380336615, "value2":6696254947892698280, "value3":true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099077, "value1":0.4218807485485893, "value2":8151022600373825319, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833739, "value1":0.471092674976307, "value2":4241908968763096086, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512079, "value1":0.6891384899122228, "value2":2796503546861546091, "value3":true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.405406, "value1":0.9060026283921703, "value2":4773855974745798571, "value3":true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742427, "value1":0.6129480541080172, "value2":3917404693410650609, "value3":true, "__name__":"metric_00000739", "key_5":"k","key_2":"d","key_3":"e", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.613376, "value1":0.3584257069399756, "value2":6068105032813229930, "value3":true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.759402, "value1":0.29210317562134064, "value2":3082388413784210214, "value3":true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090526, "value1":0.4678159387104084, "value2":1605658432958093758, "value3":true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380672, "value1":0.5086572266239265, "value2":2924449619880684423, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.249203, "value1":0.6593022058411281, "value2":727243123088870599, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.142921, "value1":0.5973096622193752, "value2":2056855353114653381, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041929, "value1":0.03439471294074195, "value2":8729944365498889284, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970035, "value1":0.629094069595545, "value2":954705184774295714, "value3":false, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656521, "value1":0.4935396027418025, "value2":1513112375773530412, "value3":false, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286143, "value1":0.590796213295746, "value2":3085171514913296735, "value3":false, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.611048, "value1":0.3503905353305475, "value2":5805006250107454516, "value3":false, "__name__":"metric_00000748", "key_6":"f","key_3":"e","key_4":"h", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265940, "value1":0.8719050400063297, "value2":2334709836967925044, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.231945, "value1":0.9482208594424564, "value2":7605532380274260726, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099422, "value1":0.6387093062831974, "value2":6416034503475989540, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862301, "value1":0.0844793706976894, "value2":1724212296853776891, "value3":false, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.882958, "value1":0.45080094639121554, "value2":6596820877226155907, "value3":true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851802, "value1":0.784896015737414, "value2":2267862818127250105, "value3":false, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.871761, "value1":0.9219727566018483, "value2":9044274636085577036, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704109, "value1":0.6618033501935818, "value2":8585487696070011938, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.863670, "value1":0.6882846261541673, "value2":5478374167595454238, "value3":true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183651, "value1":0.21762989261712523, "value2":2054870959404109040, "value3":false, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157246, "value1":0.1072057187986972, "value2":2474773268716838947, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263088, "value1":0.6424740570966222, "value2":4311694979316615362, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227492, "value1":0.6060012408935248, "value2":8262744174339677603, "value3":true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919259, "value1":0.26635737492611994, "value2":146832998544852566, "value3":true, "__name__":"metric_00000762", "key_2":"e","key_0":"d", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.810229, "value1":0.6433893313883111, "value2":6020303671131449049, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.021406, "value1":0.5663729785218883, "value2":6847532330100114835, "value3":true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.331141, "value1":0.03123867674281506, "value2":2370510435982901478, "value3":true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313836, "value1":0.7842831796528092, "value2":1464971588517507800, "value3":false, "__name__":"metric_00000711", "key_6":"i","key_0":"j","key_4":"h", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147234, "value1":0.232833406436683, "value2":5302769737190544079, "value3":true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423280, "value1":0.5646235246180139, "value2":3807301011411244203, "value3":false, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.872535, "value1":0.3508318181055984, "value2":8035632162257252295, "value3":true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516267, "value1":0.7376190499237748, "value2":1779371132519351560, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.020480, "value1":0.5129193623789473, "value2":6378565762108475442, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421514, "value1":0.72712434334131, "value2":3908844968753938444, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026297, "value1":0.8255820903801332, "value2":5180017205009891488, "value3":false, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138610, "value1":0.5467990020300735, "value2":8134249053330675938, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754861, "value1":0.5832263769294902, "value2":8216858329154999597, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655676, "value1":0.45804041306603793, "value2":8403704610668639637, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.091295, "value1":0.7313452902241809, "value2":6274232113944917429, "value3":false, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370397, "value1":0.7082796943852117, "value2":16646752364177844, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.072185, "value1":0.8009986319191017, "value2":6123195610574092125, "value3":true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068380, "value1":0.3124676060355689, "value2":2569282749677030904, "value3":true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013897, "value1":0.9321729707819904, "value2":2281258981052686264, "value3":true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.963470, "value1":0.9927608945020866, "value2":6050801587231871156, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481229, "value1":0.3689051456876514, "value2":1924733693360792633, "value3":true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.295891, "value1":0.44913357839758283, "value2":8069037361024844952, "value3":false, "__name__":"metric_00000779", "key_7":"i","key_1":"h", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409485, "value1":0.18423629567656025, "value2":5066355149165531813, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744404, "value1":0.9737949013567385, "value2":5174295290317701561, "value3":false, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739701, "value1":0.4976173273573324, "value2":5658855721642744182, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.186283, "value1":0.419361633943128, "value2":2053614790636840527, "value3":true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566841, "value1":0.7094596520198607, "value2":2148895338889314160, "value3":false, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.215921, "value1":0.34276587375112927, "value2":1645566772308625552, "value3":false, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.701909, "value1":0.22008909623955855, "value2":2295629303145372991, "value3":true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002697, "value1":0.1765088617877308, "value2":4201797060324467420, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354799, "value1":0.6538892310533279, "value2":4577607391337663337, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345385, "value1":0.3843950816957309, "value2":8800458811056296695, "value3":true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988457, "value1":0.5588868714021769, "value2":8433496452370061074, "value3":false, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415009, "value1":0.1746188760147135, "value2":6912110645446824396, "value3":false, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655363, "value1":0.9949350156565988, "value2":8110347995943103660, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.242069, "value1":0.48932951486152193, "value2":8286237027514157124, "value3":true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085368, "value1":0.0689091306398061, "value2":5859648999386501040, "value3":true, "__name__":"metric_00000801", "key_6":"e","key_3":"j","key_4":"d", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.661208, "value1":0.7864155291083853, "value2":7826276349217880531, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.803222, "value1":0.6015539667825235, "value2":1102772758151131198, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.458180, "value1":0.3809696927779604, "value2":4439664400270353254, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334077, "value1":0.4097523501895673, "value2":7454062761319263955, "value3":true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.885572, "value1":0.548188359160327, "value2":3020054439494862697, "value3":true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489119, "value1":0.6763563343582067, "value2":4242764599158368849, "value3":true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.187849, "value1":0.7091709802238197, "value2":6867639316908364367, "value3":true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663732, "value1":0.42631474518610635, "value2":9001795643794270349, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.585010, "value1":0.6237338911978378, "value2":4365340422086338410, "value3":false, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.379644, "value1":0.15387674044669916, "value2":818900165755081001, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791337, "value1":0.32588514831554516, "value2":1691735796263334884, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.082385, "value1":0.5745615587827129, "value2":772277205802820637, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086473, "value1":0.8230348349086334, "value2":8156169851471463456, "value3":false, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808463, "value1":0.47620124507643474, "value2":2357967788477712632, "value3":true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380686, "value1":0.011323528011005758, "value2":2028993392634762324, "value3":false, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.371579, "value1":0.5028955605922396, "value2":7910467950096619453, "value3":false, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.883127, "value1":0.20067026957438594, "value2":3625887507362062386, "value3":false, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.305224, "value1":0.7123715236402623, "value2":7522375891893442954, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443461, "value1":0.7555458976714571, "value2":655917858760911221, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901204, "value1":0.7387538487064921, "value2":7525092307824794219, "value3":true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.376917, "value1":0.7993175855480273, "value2":2807412127532185201, "value3":true, "__name__":"metric_00000822", "key_6":"h","key_5":"c", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740075, "value1":0.3001865006906782, "value2":7190874681083752315, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209665, "value1":0.89276344594564, "value2":4778320938543530902, "value3":true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.341730, "value1":0.20200750378616714, "value2":1664606777941171586, "value3":false, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.629099, "value1":0.017583528638071864, "value2":2707650519412422348, "value3":false, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057519, "value1":0.04492925365315839, "value2":2578667952703739482, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.723798, "value1":0.8454916194785576, "value2":1835422305385826626, "value3":true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.034767, "value1":0.201322613886975, "value2":7356183312830760350, "value3":false, "__name__":"metric_00000794", "key_5":"h","key_8":"a","key_1":"b", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520001, "value1":0.5251237247182675, "value2":134900877343320532, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489083, "value1":0.8513994038065534, "value2":2159556869883772494, "value3":false, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344496, "value1":0.16316310680106225, "value2":5518614284082110727, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.724230, "value1":0.6875155435793379, "value2":35737216326475935, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568987, "value1":0.5138501785175705, "value2":4338888431721237426, "value3":false, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598387, "value1":0.4390074014236514, "value2":7041842749721887181, "value3":false, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.626270, "value1":0.2391917526821926, "value2":3091537573368363672, "value3":false, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.297931, "value1":0.11690748126426959, "value2":4039852836857346610, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.175994, "value1":0.6789300486469183, "value2":1246840956922367253, "value3":false, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976992, "value1":0.43367432666147043, "value2":564959136229688173, "value3":false, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703741, "value1":0.47707265609489413, "value2":4068705543834170412, "value3":false, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.235944, "value1":0.8012892224294691, "value2":1988300147401692461, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499512, "value1":0.9990473134151273, "value2":8394845441054007311, "value3":false, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404130, "value1":0.6068400138809874, "value2":1935348003856793271, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542560, "value1":0.12776285353397152, "value2":6795264619125251375, "value3":true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693012, "value1":0.2569652780369043, "value2":1475809083419006811, "value3":false, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068970, "value1":0.3881338979751516, "value2":1218503134871756052, "value3":false, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264926, "value1":0.19791850227382649, "value2":5513222623463327397, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.408769, "value1":0.39494174800117104, "value2":1576673230492191692, "value3":false, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.081892, "value1":0.9402614771700961, "value2":1589947956431908818, "value3":true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.058425, "value1":0.5327215264224379, "value2":445308327885517649, "value3":false, "__name__":"metric_00000850", "key_8":"d","key_3":"b","key_4":"f", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.838442, "value1":0.9519819634366921, "value2":745087205152711718, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064037, "value1":0.23777264651361862, "value2":4675095342249511184, "value3":true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.518857, "value1":0.8856046396694535, "value2":1109051615753148759, "value3":true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077463, "value1":0.8624901705258308, "value2":3793672263940517136, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.921342, "value1":0.9049145350717305, "value2":3570805638374912588, "value3":false, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157723, "value1":0.6223760989748452, "value2":2996869515277202402, "value3":false, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155659, "value1":0.13854963250628805, "value2":6512544911629959850, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977987, "value1":0.036771554508911904, "value2":5994549505435027834, "value3":true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265064, "value1":0.3403489462496419, "value2":2814929460502496771, "value3":true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404153, "value1":0.3208642154389825, "value2":5256172233350798960, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.312852, "value1":0.011902750850203012, "value2":5526100237606282605, "value3":false, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.491995, "value1":0.46992302986974577, "value2":157557322951942167, "value3":false, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704212, "value1":0.9425835794283676, "value2":915577115687321604, "value3":false, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.576745, "value1":0.7854877804515764, "value2":6382074091209031167, "value3":true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070556, "value1":0.18030779510661127, "value2":6308540602135606131, "value3":true, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542532, "value1":0.3384327861708296, "value2":1613795669513593265, "value3":false, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511484, "value1":0.8996232864271185, "value2":7405365074827637694, "value3":true, "__name__":"metric_00000840", "key_3":"i","key_7":"c","key_2":"b", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.562131, "value1":0.10596222820549657, "value2":3306812514398053557, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.391786, "value1":0.20647295311593183, "value2":5486022828508870343, "value3":false, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466143, "value1":0.6224930113197604, "value2":8535178573473477072, "value3":true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423074, "value1":0.3088852292542975, "value2":3166186037825297943, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.762488, "value1":0.33490363470863166, "value2":2912130613463378266, "value3":true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713117, "value1":0.2993000272696047, "value2":494302122350368825, "value3":false, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028636, "value1":0.6503466890929638, "value2":4759183611197798725, "value3":false, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404461, "value1":0.043678508683084524, "value2":6408629579798949638, "value3":true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077439, "value1":0.2418209601880771, "value2":6820734383602297695, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.624831, "value1":0.4718269105814766, "value2":4622812711240963076, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053083, "value1":0.6875169297990352, "value2":6119677537083039891, "value3":true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345042, "value1":0.7324917812164005, "value2":5511749504782560024, "value3":true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.623531, "value1":0.9320016848414698, "value2":744369719732623498, "value3":false, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.182404, "value1":0.9674131847138503, "value2":3838480249361897923, "value3":false, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.696213, "value1":0.7881956638078709, "value2":4371203118864420550, "value3":true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.628037, "value1":0.17572736986644139, "value2":3461303046815504013, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273728, "value1":0.6916742591847842, "value2":2382468702478522273, "value3":false, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335163, "value1":0.9758093684084915, "value2":1680436716344950339, "value3":true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782020, "value1":0.1537134689497552, "value2":4414043374214499577, "value3":true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.134363, "value1":0.5083067111414208, "value2":4982306711291417821, "value3":true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.865817, "value1":0.9406393737656967, "value2":5857776137176686892, "value3":true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716601, "value1":0.1916346431757554, "value2":5910562859970195422, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.427366, "value1":0.03546691845658599, "value2":7315776376647279090, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950000, "value1":0.5369396780416313, "value2":7524727304557507728, "value3":true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.325387, "value1":0.06313877213741768, "value2":7980198837673375086, "value3":true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950927, "value1":0.32484655923390965, "value2":8297079571694626407, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832701, "value1":0.8101399444767259, "value2":6277528961017233439, "value3":true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969582, "value1":0.5893386387714608, "value2":8824793605581596113, "value3":false, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302334, "value1":0.03653755659557745, "value2":2394591637437372554, "value3":true, "__name__":"metric_00000896", "key_5":"i","key_1":"b", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463188, "value1":0.2645292258357236, "value2":7750296583775050063, "value3":false, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.774073, "value1":0.7854839946508317, "value2":4045742754979149401, "value3":false, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.729676, "value1":0.4706720879040935, "value2":4750321291295032700, "value3":false, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618814, "value1":0.2230204325484985, "value2":9003120111243218251, "value3":true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603138, "value1":0.22960374502253963, "value2":6338685193000832935, "value3":false, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.412324, "value1":0.4957126042457222, "value2":8735002444024394358, "value3":true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657244, "value1":0.08184942404758783, "value2":7470578294751016176, "value3":true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442850, "value1":0.7229538622773333, "value2":4724836800934702763, "value3":false, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156841, "value1":0.7233638365572691, "value2":7858912978560139100, "value3":true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734329, "value1":0.3404208114361277, "value2":3050306358924697552, "value3":false, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.465656, "value1":0.38391986849101767, "value2":307361261718394449, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.445647, "value1":0.990731352209279, "value2":1207071782335047705, "value3":false, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298620, "value1":0.11180836767709064, "value2":3870338901819904571, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.095445, "value1":0.2870001965302557, "value2":2415945267551238695, "value3":true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.918448, "value1":0.7277791066957175, "value2":2821193899111524763, "value3":true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541594, "value1":0.09005789780178877, "value2":2925035824598662472, "value3":true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.631726, "value1":0.9959155652844405, "value2":3833701945431571315, "value3":true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049800, "value1":0.8110556334200572, "value2":2218497283809170432, "value3":false, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070530, "value1":0.65401432282683, "value2":498838541277181702, "value3":false, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736247, "value1":0.7394025858411815, "value2":7148599556072452144, "value3":false, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736608, "value1":0.08218047603491815, "value2":3776524585186258439, "value3":true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003840, "value1":0.6757338653122574, "value2":379207741924258587, "value3":true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362539, "value1":0.11651117191245357, "value2":3906526956584011608, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.539295, "value1":0.44782322310256395, "value2":2595894391836664475, "value3":false, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392973, "value1":0.8520294340743481, "value2":7384112654505361476, "value3":false, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.078027, "value1":0.3547792528217613, "value2":8155526662947982478, "value3":false, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.776543, "value1":0.6604220360650918, "value2":4011373177447838470, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.691748, "value1":0.4862664301051333, "value2":7696312329542980810, "value3":true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964567, "value1":0.11169294328511947, "value2":4727405940623359973, "value3":true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722155, "value1":0.3780569227165093, "value2":3316392130699376882, "value3":true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457388, "value1":0.9243183084724984, "value2":2432925818286345186, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.040558, "value1":0.7743604477646876, "value2":627834147679968207, "value3":true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272066, "value1":0.5917633264655242, "value2":5918933193525430349, "value3":false, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200729, "value1":0.7327561219912169, "value2":1393602674371677220, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.905147, "value1":0.8568183299448732, "value2":9034660617242414820, "value3":false, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310139, "value1":0.802088771431449, "value2":3471155770712582770, "value3":true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.821462, "value1":0.5273591260032361, "value2":5140674777677119228, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.384953, "value1":0.4025027387592999, "value2":5097429126997347384, "value3":true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.829568, "value1":0.8045993326905381, "value2":2809186620050229002, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294923, "value1":0.5644401567133956, "value2":3718732987675906525, "value3":true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.556732, "value1":0.48820899714237304, "value2":4078290062854715892, "value3":true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.176213, "value1":0.2510405463103239, "value2":635371208287508106, "value3":false, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.267004, "value1":0.22999772540025595, "value2":1871342923142731692, "value3":false, "__name__":"metric_00000919", "key_9":"g","key_6":"i", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.506694, "value1":0.4581021751822573, "value2":8794558811813763792, "value3":false, "__name__":"metric_00000924", "key_3":"k","key_4":"a","key_1":"c", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354208, "value1":0.6088901271960526, "value2":132806552362853695, "value3":false, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516024, "value1":0.7664701528892452, "value2":107006237498756274, "value3":false, "__name__":"metric_00000923", "key_6":"e","key_3":"h","key_5":"j", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.750598, "value1":0.3217472443950159, "value2":8378919480372645762, "value3":false, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419841, "value1":0.6663125192971521, "value2":2423610833256438065, "value3":true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164890, "value1":0.2006090031725153, "value2":1844725035830445264, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264867, "value1":0.42671643117928926, "value2":3438912836085300466, "value3":true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.430421, "value1":0.07257562390395779, "value2":6088518193551139346, "value3":false, "__name__":"metric_00000939", "key_8":"c","key_1":"k","key_6":"g", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.673454, "value1":0.6691702132206785, "value2":8625848517577296022, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995936, "value1":0.702118704788376, "value2":4086471613439852656, "value3":true, "__name__":"metric_00000941", "key_5":"i","key_9":"g","key_4":"i", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964584, "value1":0.7747761523528802, "value2":1036051996047955870, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.948289, "value1":0.25140012327548283, "value2":788619002191666922, "value3":false, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372555, "value1":0.9999769483157629, "value2":7184076213289982875, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.920761, "value1":0.868179497430529, "value2":1697965828697918238, "value3":true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372531, "value1":0.5396817815353727, "value2":7506498166097940651, "value3":false, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.278206, "value1":0.17061135686969672, "value2":713580384695452010, "value3":false, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600654, "value1":0.6616469498902554, "value2":774519242775197957, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188862, "value1":0.5531871959368098, "value2":7422109554403592667, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457643, "value1":0.7840548969774126, "value2":7827846462514659006, "value3":false, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655234, "value1":0.76526166835666, "value2":2375166258510616618, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370401, "value1":0.021438154414782817, "value2":2771089523399378961, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137411, "value1":0.5646878830454839, "value2":5968018682136017050, "value3":false, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423235, "value1":0.36529823070220574, "value2":7162022842889925076, "value3":true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272788, "value1":0.219359460529438, "value2":181241653539382719, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.707284, "value1":0.3923075865949308, "value2":2782827649523891195, "value3":true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858145, "value1":0.688162185571026, "value2":5086226934464427172, "value3":true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867650, "value1":0.6841047492247676, "value2":8098586901309655486, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.902390, "value1":0.706932118893774, "value2":9086709640294623772, "value3":false, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131998, "value1":0.8578147943558058, "value2":41651666626493230, "value3":false, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932017, "value1":0.24496475938485004, "value2":7307212968529901562, "value3":false, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.469902, "value1":0.3291978179419431, "value2":6674355534843744608, "value3":false, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.715852, "value1":0.5949781040197312, "value2":6585582445453110248, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.439874, "value1":0.25098256408304576, "value2":9009893614730547675, "value3":false, "__name__":"metric_00000972", "key_4":"d","key_6":"k","key_2":"f", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.775831, "value1":0.888090262607404, "value2":7565201070136116477, "value3":true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480681, "value1":0.47062171981528383, "value2":4853013107516729574, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.654578, "value1":0.4096044979675714, "value2":1606198625715352901, "value3":false, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.881884, "value1":0.31666628836121824, "value2":5391861566395998267, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028180, "value1":0.2934267405568024, "value2":1598172822680744133, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245149, "value1":0.09374074237151239, "value2":6863637452711576117, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105508, "value1":0.7783021910779061, "value2":2669034341302440615, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474623, "value1":0.5494532131881231, "value2":6109855504680847451, "value3":true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625728, "value1":0.471409564376847, "value2":5293699949847690348, "value3":true, "__name__":"metric_00000957", "key_5":"g","key_0":"c","key_1":"i", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382751, "value1":0.7835774962833667, "value2":5926458098948297057, "value3":false, "__name__":"metric_00000960", "key_3":"e","key_2":"c", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.527286, "value1":0.6322552498331545, "value2":437257708157331646, "value3":false, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.166237, "value1":0.6386670258285629, "value2":2053216930342060527, "value3":false, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.942234, "value1":0.7212756328576085, "value2":8527112750674030055, "value3":true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.880290, "value1":0.19477642153523786, "value2":8136028599409004140, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.477164, "value1":0.7162729706892131, "value2":3933534960467417574, "value3":true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089992, "value1":0.47761006384043037, "value2":4843140910052787407, "value3":false, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.204970, "value1":0.4917659716382411, "value2":8639172605312608175, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.428786, "value1":0.5707960425927678, "value2":757157967003444236, "value3":false, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131753, "value1":0.32321264423119905, "value2":1903644540800881525, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.540623, "value1":0.528958932538034, "value2":8999603816652982043, "value3":true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.991648, "value1":0.7098945581193735, "value2":8146175572944228409, "value3":false, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.523699, "value1":0.499825459180699, "value2":1724724639775860908, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116240, "value1":0.6561999665129753, "value2":5656111967421346554, "value3":false, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618968, "value1":0.2848979660832576, "value2":2049018652807530472, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116765, "value1":0.8252075257277172, "value2":4775243296364010632, "value3":false, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.410327, "value1":0.06668100857135961, "value2":2183721339943212726, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.688645, "value1":0.41653256081146506, "value2":5969120448550176776, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657571, "value1":0.9568582636724965, "value2":730216901987394123, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043883, "value1":0.40826500145612177, "value2":6407138186851676502, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474102, "value1":0.6914723313082367, "value2":469628036375832328, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596154, "value1":0.050104900538538846, "value2":1354999585856659679, "value3":true, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096028, "value1":0.6282483124845963, "value2":116602313358973711, "value3":true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568083, "value1":0.07619222667132636, "value2":6984709326920626754, "value3":false, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.984466, "value1":0.3323790308005344, "value2":675504136493878123, "value3":true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553260, "value1":0.45304427452622936, "value2":8153841508229333988, "value3":true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781038, "value1":0.2512350719527022, "value2":1697664403294613401, "value3":false, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230875, "value1":0.8575694561008984, "value2":1340681872774498770, "value3":true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180011, "value1":0.7159258322890252, "value2":7263307563948982502, "value3":false, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318368, "value1":0.7865256606198752, "value2":7176709659967207104, "value3":false, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.594668, "value1":0.9818689490949409, "value2":4816922436179228865, "value3":false, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.178515, "value1":0.9620215494327841, "value2":7544435408144975228, "value3":true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596427, "value1":0.7405089300246521, "value2":6613451887211746808, "value3":true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317317, "value1":0.8954816734384349, "value2":4023989770168105300, "value3":true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.624103, "value1":0.1565943994591959, "value2":5829676556068380878, "value3":false, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.259139, "value1":0.08662232425939413, "value2":2137874058817765285, "value3":true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.495345, "value1":0.7444165320555235, "value2":1512339128303545319, "value3":true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.873798, "value1":0.10364343021646215, "value2":497864294044982023, "value3":false, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077492, "value1":0.7825627514339433, "value2":497480348120915725, "value3":false, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.887058, "value1":0.6684575895211121, "value2":8461019606631895989, "value3":false, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374572, "value1":0.30885911828548585, "value2":6810974248136318973, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.381097, "value1":0.062395135046400096, "value2":1505084432772472894, "value3":true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796962, "value1":0.4796896579785329, "value2":5095295117314039911, "value3":true, "__name__":"metric_00000015", "key_6":"k","key_8":"k","key_0":"a", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.625725, "value1":0.5385278812414747, "value2":7715803409892624904, "value3":true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430762, "value1":0.42060637147418184, "value2":7474778452563228226, "value3":true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886443, "value1":0.6738972816294144, "value2":582629852137142515, "value3":false, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467512, "value1":0.5597247611878908, "value2":8349056677712047690, "value3":false, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301942, "value1":0.5324107668792922, "value2":2601407711369777027, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.959260, "value1":0.3409466272260335, "value2":495480232494613355, "value3":true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824942, "value1":0.690980654161434, "value2":73040149987568706, "value3":false, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647608, "value1":0.9936863999178829, "value2":2595521887523034584, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206529, "value1":0.7042253871626942, "value2":6786989253298041563, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654900, "value1":0.04874287974341856, "value2":6080516591012621533, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.192783, "value1":0.636728117953383, "value2":7620028626322794241, "value3":true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.757400, "value1":0.3932217938599448, "value2":4695382142084123513, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578308, "value1":0.7154241327471093, "value2":8875337832548314386, "value3":false, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025767, "value1":0.4896117284329862, "value2":7827833960384552450, "value3":false, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695043, "value1":0.4735636518198123, "value2":5776645863491585306, "value3":false, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067392, "value1":0.2568532613491557, "value2":4347058259737260198, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145851, "value1":0.867279626694445, "value2":7767138234344286903, "value3":false, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.491824, "value1":0.43935171533716344, "value2":2117420540525504913, "value3":true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413343, "value1":0.7815374349474034, "value2":7674175325619501370, "value3":false, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408821, "value1":0.6468318207569199, "value2":3382854605353241895, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864749, "value1":0.5018653748831062, "value2":3163263403613237522, "value3":true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811002, "value1":0.1264068238414509, "value2":8051527093894333669, "value3":true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922550, "value1":0.797889997967527, "value2":4244570533853409024, "value3":true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045039, "value1":0.6136304375492249, "value2":7079529098273717788, "value3":true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914685, "value1":0.6884426743061036, "value2":3608727831529868858, "value3":true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.429556, "value1":0.37532682108769366, "value2":1177251454911069616, "value3":false, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779878, "value1":0.5585578516583662, "value2":5769602877819666427, "value3":false, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.711503, "value1":0.3057843361151979, "value2":2044462261279390237, "value3":true, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142192, "value1":0.0045630377024844515, "value2":4953964644129769622, "value3":true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.542755, "value1":0.236125279127913, "value2":5065525208816588827, "value3":false, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578196, "value1":0.8156211595242715, "value2":1914083289002989297, "value3":false, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110649, "value1":0.26711648319193754, "value2":2012127151944954874, "value3":true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041387, "value1":0.3776450082432283, "value2":6801275370589593726, "value3":false, "__name__":"metric_00000047", "key_7":"h","key_2":"g","key_5":"i", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808716, "value1":0.04169212753399746, "value2":9054979711039988827, "value3":false, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062171, "value1":0.7218616273097948, "value2":740018633881444642, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146630, "value1":0.5003843971030013, "value2":1655397867016219461, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.309989, "value1":0.11293107623655986, "value2":3732853355411608797, "value3":false, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.333206, "value1":0.17591880468368, "value2":8123233873446258834, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145427, "value1":0.43545559910718556, "value2":4862315899801656213, "value3":false, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.644901, "value1":0.9950957928656415, "value2":4691155465042626084, "value3":false, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568884, "value1":0.6163869833398097, "value2":9217755948904745777, "value3":false, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930812, "value1":0.3180893474971966, "value2":8238800974641951308, "value3":false, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.148564, "value1":0.7643083780125881, "value2":8977913426229482254, "value3":true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369460, "value1":0.37462703037399614, "value2":2532444456287017290, "value3":true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134410, "value1":0.7407330348790686, "value2":2290478450455253969, "value3":false, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.623624, "value1":0.23159512407081487, "value2":39715061290046297, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497315, "value1":0.9059175925838082, "value2":9042312906596816530, "value3":true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.591581, "value1":0.9196881162354394, "value2":3577426841534493512, "value3":true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.229214, "value1":0.677998841646216, "value2":4155628140480575006, "value3":true, "__name__":"metric_00000063", "key_8":"f","key_1":"c","key_6":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419533, "value1":0.14115394562090194, "value2":4482838294646117015, "value3":true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352495, "value1":0.614805207196906, "value2":6313920398652894644, "value3":true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158009, "value1":0.13136879968484017, "value2":1444276749086955965, "value3":true, "__name__":"metric_00000067", "key_4":"d","key_9":"a","key_0":"c", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394523, "value1":0.390863430258994, "value2":1479817062186057883, "value3":true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084220, "value1":0.8798339682629527, "value2":5374208931949760111, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.604039, "value1":0.6734519068343852, "value2":6936119518718549917, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.407184, "value1":0.883448918218706, "value2":2783824499389725952, "value3":false, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.987554, "value1":0.04159873743578623, "value2":6347778463017159524, "value3":true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015322, "value1":0.9633747370317121, "value2":886249751055268990, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903730, "value1":0.2538648955540415, "value2":5257877330086190982, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109241, "value1":0.3895615184638186, "value2":2398924749380096990, "value3":true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946726, "value1":0.8408201754901692, "value2":5956645672957756887, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.126734, "value1":0.8247534822394186, "value2":8919709316239049787, "value3":true, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419044, "value1":0.20899742823905534, "value2":9208355419350776806, "value3":true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354221, "value1":0.38324817137377604, "value2":7243408993217101630, "value3":true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.620584, "value1":0.25505110416085736, "value2":2000013463174956675, "value3":true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306280, "value1":0.24053747539237172, "value2":104074303789974504, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.694994, "value1":0.14347374003976163, "value2":2424092953659426107, "value3":true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399682, "value1":0.5974894049526064, "value2":6226843809112311375, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.244602, "value1":0.4472550716634916, "value2":6661706815157471152, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154170, "value1":0.15894047878423925, "value2":1910278084951118791, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.338581, "value1":0.8000650933715514, "value2":932928130299819996, "value3":false, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385782, "value1":0.2346060605687393, "value2":1779773261165382974, "value3":true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.925102, "value1":0.31969179251919677, "value2":2628705870105871940, "value3":true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783196, "value1":0.47412332370344556, "value2":770542511728920904, "value3":false, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779626, "value1":0.6092963005254489, "value2":916209608160555146, "value3":false, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194805, "value1":0.5004447430389768, "value2":4252088119108151132, "value3":false, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.758680, "value1":0.486921957595336, "value2":989531111026827083, "value3":true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477327, "value1":0.1584089004594205, "value2":8696347531856094472, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162006, "value1":0.537289345346911, "value2":2077753746787895762, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613845, "value1":0.3033370157767779, "value2":7793623951886359033, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.373236, "value1":0.3796012074348545, "value2":6115758866481998084, "value3":false, "__name__":"metric_00000082", "key_9":"b","key_4":"h","key_5":"e", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080535, "value1":0.47198866208030016, "value2":6615627174770526167, "value3":true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.401469, "value1":0.5986326083333331, "value2":2667038551831095428, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605284, "value1":0.7390739400172244, "value2":361713221830612946, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.543913, "value1":0.568574574470533, "value2":8152708069238595670, "value3":false, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262409, "value1":0.058861415000961634, "value2":5954875590777239919, "value3":false, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056599, "value1":0.20694947175492623, "value2":3664736464652868067, "value3":false, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.557868, "value1":0.04995052697654973, "value2":2073326631524402431, "value3":true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613501, "value1":0.31045156583158495, "value2":8926511362033688194, "value3":true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.047167, "value1":0.7509456855802353, "value2":846763672875326934, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362486, "value1":0.30051878680197913, "value2":8317722150065418470, "value3":true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.669592, "value1":0.6148907714481391, "value2":3070072928838789844, "value3":true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061749, "value1":0.4026807862923137, "value2":2054834671958607537, "value3":false, "__name__":"metric_00000110", "key_7":"c","key_8":"e","key_1":"h", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077647, "value1":0.02456732222252374, "value2":4163774252593358275, "value3":false, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439333, "value1":0.49417199138729123, "value2":8947661323810810369, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.592816, "value1":0.25731799708888187, "value2":5220216189538554324, "value3":true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.344893, "value1":0.0022390888763635323, "value2":6178831795736016910, "value3":false, "__name__":"metric_00000112", "key_5":"i","key_8":"h","key_4":"e", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350091, "value1":0.39159828262162255, "value2":3395049172404740462, "value3":false, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417133, "value1":0.19779968591074149, "value2":3982459725160448681, "value3":false, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313282, "value1":0.5954778061697951, "value2":2609201495243589975, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416054, "value1":0.6479875646988947, "value2":6379744126737729174, "value3":false, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416195, "value1":0.9737347842892922, "value2":3437414765355850053, "value3":true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.351132, "value1":0.5274165691272555, "value2":5569886633354264071, "value3":true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273952, "value1":0.4616579953521536, "value2":4410692243343844144, "value3":false, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948309, "value1":0.21495634970202532, "value2":1337568920394438603, "value3":false, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.120015, "value1":0.6827231094645871, "value2":4910087454372528211, "value3":true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.220281, "value1":0.5451953602332654, "value2":5295885271016141901, "value3":false, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222991, "value1":0.38652141757520153, "value2":611291833871236608, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432311, "value1":0.4952506069055437, "value2":6235358954173265246, "value3":true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.305907, "value1":0.7399971630668104, "value2":7853724941674753946, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.701437, "value1":0.07448132935373361, "value2":1676692481709109964, "value3":true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233939, "value1":0.9922833997352541, "value2":8485630674384935881, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.196307, "value1":0.3811609484351306, "value2":6762437704329339708, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.598701, "value1":0.7218662020742757, "value2":5746405219362751126, "value3":true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535319, "value1":0.470769974287709, "value2":5160211896753607127, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455357, "value1":0.41061237500976755, "value2":3731616592511902245, "value3":true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682037, "value1":0.02412728786773881, "value2":8461839080730070447, "value3":false, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618426, "value1":0.6208202835196076, "value2":3667062185468357188, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094952, "value1":0.9317381860791135, "value2":8865888160428380876, "value3":false, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295576, "value1":0.034878221959591026, "value2":4553331116235737600, "value3":true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605252, "value1":0.5687954362360221, "value2":4628895813773640059, "value3":true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.167973, "value1":0.8144348491443895, "value2":3884298273747196831, "value3":true, "__name__":"metric_00000139", "key_4":"a","key_1":"k","key_2":"h", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803115, "value1":0.14234455011711838, "value2":7333273581911274613, "value3":false, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809534, "value1":0.31251923098668316, "value2":5000940696241059800, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.504196, "value1":0.37041029395255115, "value2":7518926478801994928, "value3":true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886399, "value1":0.2510698129904212, "value2":1686947485208902024, "value3":true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322861, "value1":0.6398715658295313, "value2":5260107554586637670, "value3":true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.575248, "value1":0.8821568194363127, "value2":2676120924049029771, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.924938, "value1":0.6642600755588765, "value2":9154692137110805453, "value3":false, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534144, "value1":0.2220263150896839, "value2":7296067315122241408, "value3":true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680757, "value1":0.36579467004129973, "value2":5336171210655722369, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.361476, "value1":0.42049076121425283, "value2":3694543289414649508, "value3":true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.423620, "value1":0.9226817255397154, "value2":5312418306728579961, "value3":true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.398746, "value1":0.4448382205324241, "value2":5199222544329766041, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.492975, "value1":0.6490408315820692, "value2":3620354939805403786, "value3":false, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.732787, "value1":0.26280435119702705, "value2":7186318610805553949, "value3":true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385151, "value1":0.7991171293820531, "value2":1271579952216532684, "value3":false, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067778, "value1":0.2653370821341721, "value2":8977665444967590904, "value3":false, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941069, "value1":0.7995772580327314, "value2":7272267767125923064, "value3":false, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.547209, "value1":0.7653567065104109, "value2":6328078604409399184, "value3":true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649757, "value1":0.49061605503159605, "value2":5991812000082844933, "value3":false, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.017745, "value1":0.8857176370602151, "value2":8890483656354164816, "value3":false, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478915, "value1":0.49344955581785127, "value2":6898479319738793663, "value3":false, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.600610, "value1":0.1145004603393322, "value2":7934326719175266154, "value3":false, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752045, "value1":0.7917363851798865, "value2":6655960268552880542, "value3":true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.198446, "value1":0.8915785759457425, "value2":6424163196395537238, "value3":false, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415881, "value1":0.6309493378355412, "value2":623919229201632317, "value3":false, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.652344, "value1":0.6344286572245156, "value2":9189348363675107233, "value3":false, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774566, "value1":0.042498471702267726, "value2":6073051427209338110, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.918626, "value1":0.4530903546971259, "value2":5667043007795158116, "value3":false, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.340057, "value1":0.13032389853235377, "value2":2621616876191250318, "value3":false, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363370, "value1":0.25548611447808117, "value2":7869413584380375849, "value3":false, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316114, "value1":0.7913570973054609, "value2":714341173131110534, "value3":false, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.726210, "value1":0.6340822577426605, "value2":7078070670524096081, "value3":false, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809728, "value1":0.31189335942918994, "value2":2593764393145326151, "value3":false, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828156, "value1":0.2513167249865656, "value2":4199156936630630581, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074063, "value1":0.6299866633999338, "value2":7375082593010468605, "value3":true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551341, "value1":0.5634049844701874, "value2":553811382543796103, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395240, "value1":0.9163098471877911, "value2":2102676899677978041, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812285, "value1":0.5658853008127968, "value2":860920050673825541, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.699745, "value1":0.6686081106918065, "value2":4484995258456828647, "value3":true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862811, "value1":0.006750603964471356, "value2":8966268587906792618, "value3":false, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649656, "value1":0.33455087075476775, "value2":3638358249838833627, "value3":true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978094, "value1":0.9765011127569251, "value2":5363905042672157366, "value3":false, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824326, "value1":0.5852292268881885, "value2":8525521016560267029, "value3":true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972834, "value1":0.2713220476194647, "value2":1730194033904802699, "value3":true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.513347, "value1":0.20240183224033417, "value2":3586323241046235606, "value3":false, "__name__":"metric_00000182", "key_7":"f","key_9":"b","key_3":"a", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165801, "value1":0.25754092027876374, "value2":3897618512711030520, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245034, "value1":0.276430555426299, "value2":7235576154009687664, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209920, "value1":0.33218423022426385, "value2":6956800146565179953, "value3":false, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802797, "value1":0.9498551679797749, "value2":4204819498986608868, "value3":true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.002994, "value1":0.33304452518131644, "value2":299998551436331074, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.136076, "value1":0.58569890261748, "value2":8821512503402703105, "value3":false, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605724, "value1":0.3779582962124763, "value2":4381826070482821493, "value3":false, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.664393, "value1":0.7508643438704989, "value2":9133405472026903918, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236332, "value1":0.4456796524317141, "value2":3982698031313790805, "value3":true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320127, "value1":0.5073225371595068, "value2":2378617268397591284, "value3":false, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865511, "value1":0.36097655561074876, "value2":2650611720712784810, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777541, "value1":0.27613383935735863, "value2":3133546266043557517, "value3":false, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647086, "value1":0.09971158325173911, "value2":7192486223907690883, "value3":false, "__name__":"metric_00000199", "key_9":"j","key_2":"f","key_5":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.172420, "value1":0.6050664492556429, "value2":3816871624777837698, "value3":true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639560, "value1":0.40402358999541305, "value2":7105433557941999346, "value3":true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.945205, "value1":0.38392410498955126, "value2":1178256579293465746, "value3":false, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277675, "value1":0.3660884472168037, "value2":6046887408166693464, "value3":false, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219346, "value1":0.5871741254566942, "value2":2616399527910270141, "value3":false, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116739, "value1":0.7713481330592996, "value2":7718152730102741547, "value3":true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.951685, "value1":0.6692969608681443, "value2":6198709686728043657, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.880602, "value1":0.3938915734742536, "value2":5600053552458054507, "value3":false, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.831337, "value1":0.47065388482990184, "value2":5918647530397871720, "value3":false, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865810, "value1":0.9557189505427555, "value2":1533596579704888254, "value3":false, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578483, "value1":0.3164962822621824, "value2":4784731554506893709, "value3":false, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158191, "value1":0.36054084773482775, "value2":6343389237742355198, "value3":true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761167, "value1":0.9149570761492709, "value2":6140964050711145447, "value3":false, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.955044, "value1":0.34237203961558577, "value2":7321019711278117639, "value3":true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137258, "value1":0.2734770926499478, "value2":5312006546786938240, "value3":false, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876289, "value1":0.584796985987568, "value2":2146521016660183394, "value3":true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.345388, "value1":0.7891673161299368, "value2":9211813724293662180, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.291358, "value1":0.5443723430932246, "value2":1305196521853642686, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107992, "value1":0.33317636728859645, "value2":4303995107486054548, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.065010, "value1":0.792101797297302, "value2":5624493167148885444, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.512622, "value1":0.026747806334027496, "value2":31336873860845918, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796113, "value1":0.3713464074931213, "value2":4041884710949108537, "value3":true, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.794333, "value1":0.5998279188033826, "value2":2263989663686243457, "value3":false, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930716, "value1":0.7150630462147562, "value2":5925307983835648806, "value3":true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.631145, "value1":0.12175047903656237, "value2":2424100894887965716, "value3":true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889832, "value1":0.33399829473514187, "value2":3091557798470852457, "value3":false, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.036666, "value1":0.8679594907099448, "value2":6691229273816200596, "value3":true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.853608, "value1":0.5774160801209688, "value2":980449146620020072, "value3":false, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.356874, "value1":0.6962074391730027, "value2":2178872746047471215, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162362, "value1":0.6573318971529195, "value2":8364158949095374627, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110201, "value1":0.41287549892040376, "value2":7444688864434046520, "value3":true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.441462, "value1":0.960708146788197, "value2":7272057802651917640, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.020362, "value1":0.14970758198915565, "value2":5476649529178658344, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434430, "value1":0.7114147585195696, "value2":6784565480587871276, "value3":true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795723, "value1":0.9222061229277446, "value2":5415396163282444341, "value3":false, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.092199, "value1":0.0594949540077623, "value2":733783646287392653, "value3":false, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277448, "value1":0.36348546035264123, "value2":2527022975440439707, "value3":true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607130, "value1":0.21203009019347513, "value2":3167160934083348057, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424117, "value1":0.42561548619630274, "value2":439532732667459958, "value3":false, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365073, "value1":0.8492201961187499, "value2":1130712017318335183, "value3":true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248310, "value1":0.6476029521632828, "value2":5228653953193583715, "value3":false, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744619, "value1":0.5713572042259071, "value2":1575714640192742698, "value3":false, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.026559, "value1":0.543673726494076, "value2":3134877395946459112, "value3":true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972610, "value1":0.09611280545479016, "value2":4984229756632609213, "value3":false, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763906, "value1":0.8517215040951974, "value2":6804317567810698432, "value3":true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105838, "value1":0.8805288612544102, "value2":7653928454276555905, "value3":true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383435, "value1":0.3683543847634455, "value2":2147384927542025505, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027377, "value1":0.4869984514514779, "value2":6620303412327477269, "value3":true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417188, "value1":0.24989866283446016, "value2":7011787964949226310, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084529, "value1":0.9844535394576189, "value2":6876673068454117053, "value3":true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032097, "value1":0.6712678103298776, "value2":3320740930339209104, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394244, "value1":0.7796841432655915, "value2":4446257418849643255, "value3":false, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828297, "value1":0.2648442239286722, "value2":25756618593200917, "value3":true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731687, "value1":0.07632667820503707, "value2":5775744844960229233, "value3":false, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.348738, "value1":0.4462284223326741, "value2":7761375931867531240, "value3":true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817431, "value1":0.9846258074003919, "value2":5377182091927671364, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603703, "value1":0.1579124346010748, "value2":7829825462713051842, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743604, "value1":0.012350717590309025, "value2":3299133246704731924, "value3":true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941914, "value1":0.37406772316207776, "value2":3581449411399106092, "value3":false, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.323291, "value1":0.2629524147159168, "value2":2560363092546342475, "value3":true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206164, "value1":0.1605470780394023, "value2":1476326866755963525, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376246, "value1":0.8410819090377242, "value2":6697199584394109087, "value3":false, "__name__":"metric_00000249", "key_6":"c","key_0":"d", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.438458, "value1":0.05246539756995679, "value2":2689405601803288451, "value3":true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.807203, "value1":0.730536792123947, "value2":7996980666084265973, "value3":false, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.842861, "value1":0.2085319390948787, "value2":750997239048285659, "value3":true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.097539, "value1":0.943951867959508, "value2":8362598506758468383, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363032, "value1":0.5870242519277287, "value2":6264967362494680685, "value3":true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212907, "value1":0.4603451527715893, "value2":4171196630516497508, "value3":false, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177573, "value1":0.8948006671577682, "value2":5720852449593066754, "value3":false, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286908, "value1":0.7373750433446898, "value2":2889582543857330611, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.780354, "value1":0.11954278924179859, "value2":206386290438050068, "value3":false, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744582, "value1":0.5414546611545269, "value2":1573274748881736123, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262127, "value1":0.8114751808169933, "value2":1156734310814584450, "value3":false, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.627688, "value1":0.05870166177597653, "value2":8802667963431750791, "value3":false, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432936, "value1":0.5733149806402271, "value2":8971566642401975801, "value3":true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252007, "value1":0.6150552921263627, "value2":6524226679068869341, "value3":true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230350, "value1":0.8727510444388746, "value2":5913218581367894399, "value3":true, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.989663, "value1":0.8376111097555229, "value2":2874987014333615633, "value3":true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.435356, "value1":0.7022017770410863, "value2":6778124246365180945, "value3":false, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673069, "value1":0.2749380629969126, "value2":1514980079784151790, "value3":false, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328789, "value1":0.865778242569301, "value2":384106145681984426, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.986292, "value1":0.9273192971072537, "value2":6526516407617182264, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.240749, "value1":0.7234109649728769, "value2":6790075567204476237, "value3":false, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774780, "value1":0.5871689076780541, "value2":5434961575452625626, "value3":false, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.406076, "value1":0.7420142270194664, "value2":4969819996381131142, "value3":false, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146123, "value1":0.2540849065850774, "value2":4942775185551508816, "value3":true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.928078, "value1":0.7980654446229606, "value2":2117433885577041648, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933008, "value1":0.05348899498806063, "value2":7864335047299894108, "value3":false, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655622, "value1":0.7668514308351416, "value2":5108933691323452489, "value3":false, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799422, "value1":0.8658511508098131, "value2":1584220631636831527, "value3":false, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153780, "value1":0.5100425920946189, "value2":6860613724470124155, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.832729, "value1":0.6352815444925497, "value2":6914920278709539881, "value3":false, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862121, "value1":0.40978666741053005, "value2":8297372411728279444, "value3":false, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.046814, "value1":0.6860769857954689, "value2":7207194751565200226, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.106175, "value1":0.9141202615431735, "value2":8093659818506693875, "value3":true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568924, "value1":0.2818708416167068, "value2":4682897128638978687, "value3":false, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743387, "value1":0.19055511730870822, "value2":6005851706828394010, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.131634, "value1":0.786038669470468, "value2":2052265664401672629, "value3":false, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864417, "value1":0.6509022777920431, "value2":6405639625996257390, "value3":false, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769553, "value1":0.3306779516460208, "value2":63644576329067046, "value3":true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705994, "value1":0.39211177275150355, "value2":6019747875631249901, "value3":false, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.085415, "value1":0.989053918126363, "value2":8753375624734757247, "value3":false, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.966642, "value1":0.146948164341443, "value2":7640844254824954510, "value3":true, "__name__":"metric_00000285", "key_9":"k","key_5":"b","key_8":"b", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.548525, "value1":0.7240074879750936, "value2":8956195213201860326, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073937, "value1":0.7314322966939647, "value2":254693443211564548, "value3":false, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347726, "value1":0.8209138159763902, "value2":6620059103742183058, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.378046, "value1":0.5808416882480918, "value2":6139629334504131344, "value3":true, "__name__":"metric_00000302", "key_6":"a","key_1":"d","key_4":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797601, "value1":0.7784388478906132, "value2":354987022131004184, "value3":true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411013, "value1":0.45158233474379805, "value2":6507365193047028596, "value3":false, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.152084, "value1":0.34791281429560394, "value2":2152822969747742943, "value3":false, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808545, "value1":0.6451074006535574, "value2":4535693696526329424, "value3":true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735815, "value1":0.5617099301887051, "value2":4545722791278282166, "value3":true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170237, "value1":0.7594425432841077, "value2":2121502996639945525, "value3":true, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839462, "value1":0.42088416942799856, "value2":4668152333347445855, "value3":true, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977985, "value1":0.9874529650666419, "value2":7279887982544559299, "value3":true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777993, "value1":0.32638782808208466, "value2":7945394269696237457, "value3":true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931446, "value1":0.5421003602914947, "value2":7796969745729042644, "value3":false, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746112, "value1":0.6059100687669938, "value2":8599287190529748569, "value3":false, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.950726, "value1":0.8607618900249634, "value2":2432626342683975270, "value3":false, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647237, "value1":0.7468406151868747, "value2":7200054131729412480, "value3":false, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.460709, "value1":0.4549015015070196, "value2":1628296475817521290, "value3":false, "__name__":"metric_00000311", "key_5":"d","key_0":"b","key_2":"f", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007652, "value1":0.3132830085813196, "value2":1943034905431272670, "value3":false, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.103043, "value1":0.029190956951399328, "value2":5218539584338803865, "value3":false, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506730, "value1":0.4547431011729936, "value2":3644260335220928016, "value3":true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564064, "value1":0.2217922532972611, "value2":3866244575842378014, "value3":false, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.308096, "value1":0.4735939830858083, "value2":6374663223385153398, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415080, "value1":0.9760882060049622, "value2":1970837347852222146, "value3":true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909084, "value1":0.6294752667120725, "value2":9199477369979291397, "value3":true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.969935, "value1":0.10788341374154452, "value2":7002111391558217482, "value3":false, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.622182, "value1":0.21938176298467404, "value2":4706131254461339381, "value3":false, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.722928, "value1":0.21266609523973992, "value2":5957309419507426661, "value3":true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062556, "value1":0.1304962392242022, "value2":4535950458577175850, "value3":false, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056693, "value1":0.6559280448597679, "value2":6798153078557023165, "value3":false, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.275009, "value1":0.8708722343930831, "value2":2992233127991751043, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027324, "value1":0.8089539697327325, "value2":6582915861289307476, "value3":false, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769519, "value1":0.6866136235615642, "value2":3188730142010270506, "value3":true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471216, "value1":0.036922660768241786, "value2":69160799499838231, "value3":false, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.289160, "value1":0.9707556984503585, "value2":3226749743394249266, "value3":true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719377, "value1":0.6150053305666618, "value2":8057451250881285379, "value3":false, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884188, "value1":0.32970677783871427, "value2":6299179245210453660, "value3":false, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.128583, "value1":0.46621614630474323, "value2":3810272679777334373, "value3":false, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.016304, "value1":0.07070759725845735, "value2":3860287779798372481, "value3":false, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556253, "value1":0.3162226930220112, "value2":811964996210240316, "value3":false, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.662220, "value1":0.04332778395279835, "value2":1557538607744070859, "value3":false, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713709, "value1":0.6625325269051697, "value2":4541028658322459718, "value3":true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297119, "value1":0.15013885945438676, "value2":2149756203418913335, "value3":false, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685110, "value1":0.9036182355788681, "value2":4377728436719035164, "value3":false, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846262, "value1":0.34678741602681534, "value2":2162478118726349508, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410862, "value1":0.11378608528656459, "value2":614293064254453793, "value3":false, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.060980, "value1":0.22871224178742908, "value2":4099107042293178536, "value3":false, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960058, "value1":0.5820948808821428, "value2":1201783870513585432, "value3":true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.214885, "value1":0.03550184669484978, "value2":2051901639161172531, "value3":false, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696549, "value1":0.44298333983599963, "value2":8432193022117819833, "value3":true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368813, "value1":0.6629289022292644, "value2":6617784174875177478, "value3":false, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313857, "value1":0.15526487291119703, "value2":95708758682513207, "value3":true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744683, "value1":0.6423949710674116, "value2":3611912809247643043, "value3":false, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.786335, "value1":0.589184376420666, "value2":5802753188533704826, "value3":true, "__name__":"metric_00000353", "key_9":"a","key_0":"a","key_5":"b", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930246, "value1":0.20872893267238513, "value2":2224049910905425698, "value3":true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515204, "value1":0.20960234541456155, "value2":7647445186122035759, "value3":false, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.800072, "value1":0.20249832362898804, "value2":1073544363493262212, "value3":false, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.420652, "value1":0.651081573013809, "value2":355187443325178523, "value3":false, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760554, "value1":0.28915917130829083, "value2":5965361551796616223, "value3":true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.852984, "value1":0.45938574348035566, "value2":2528322122553170113, "value3":false, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352738, "value1":0.900978009547613, "value2":3975764535045467018, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066294, "value1":0.08305245567845902, "value2":2394249365413597097, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654208, "value1":0.08608535610142205, "value2":2036163695122831466, "value3":true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.988436, "value1":0.04790022290628863, "value2":6418792057544120186, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884316, "value1":0.5495267159618948, "value2":4438533595454630352, "value3":true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154168, "value1":0.4348957861418331, "value2":8838135202914157189, "value3":true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874389, "value1":0.42584041610286777, "value2":2273157634255144393, "value3":true, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431882, "value1":0.06486477270973905, "value2":3345519389285150964, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685508, "value1":0.5499323474722744, "value2":5054596449098633988, "value3":true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.660995, "value1":0.17858690508054326, "value2":8567568075077444285, "value3":false, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271767, "value1":0.8266242576408339, "value2":8856688759394626086, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755581, "value1":0.7494066282765741, "value2":4893716534306029427, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171573, "value1":0.6765014228252123, "value2":5524113092747902135, "value3":false, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269557, "value1":0.24446627014298006, "value2":2739808324008179094, "value3":true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.967939, "value1":0.5539434802495194, "value2":3700007071044991493, "value3":false, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.656891, "value1":0.9116275072171186, "value2":7153356187047849001, "value3":true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248386, "value1":0.09058562459659583, "value2":7526211020899452764, "value3":false, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948164, "value1":0.6887494090196015, "value2":4600657675678245799, "value3":true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228675, "value1":0.8494478798473957, "value2":317720425357796205, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101265, "value1":0.8190567291465818, "value2":4663927178372352248, "value3":false, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820988, "value1":0.775820207166787, "value2":6038722642101305434, "value3":false, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922087, "value1":0.22553959289224842, "value2":779713635671076382, "value3":false, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.888330, "value1":0.020416312330820526, "value2":7204412129555533981, "value3":true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.290446, "value1":0.9852816320447644, "value2":4592385145364652482, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.679824, "value1":0.17818186899277277, "value2":252209780074899647, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686778, "value1":0.8339304122124636, "value2":3658007614464097080, "value3":false, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257632, "value1":0.614764442927346, "value2":4642146906724062657, "value3":false, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482850, "value1":0.18190770107834264, "value2":8954437725061891309, "value3":false, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998993, "value1":0.7044002667070637, "value2":431084763192513731, "value3":false, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.875898, "value1":0.7812743007979214, "value2":2705148521595588507, "value3":false, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084211, "value1":0.8180424469178956, "value2":7499444983471620842, "value3":false, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053317, "value1":0.3379873516583276, "value2":5874640652872975301, "value3":false, "__name__":"metric_00000390", "key_8":"i","key_1":"k","key_2":"b", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746275, "value1":0.11387537552973875, "value2":4778407492588259899, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744142, "value1":0.945713466310749, "value2":3761848793498692794, "value3":false, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.274112, "value1":0.14967892390238677, "value2":3777191830426872225, "value3":false, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680316, "value1":0.5765506588965501, "value2":3913164426928614622, "value3":false, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.582474, "value1":0.13820731268553305, "value2":1359306396719437577, "value3":true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.572068, "value1":0.9236046727786195, "value2":731327351641596838, "value3":false, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.838138, "value1":0.5915185036534663, "value2":4354316161210605047, "value3":true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755088, "value1":0.18889070665371863, "value2":4180101489297988878, "value3":false, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362663, "value1":0.4708250834755644, "value2":8678758888644206492, "value3":false, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.464079, "value1":0.13171451384673036, "value2":8187780706682520691, "value3":true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839674, "value1":0.16354088623961133, "value2":8461213608449952033, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777782, "value1":0.005150900343416017, "value2":9058250336565790520, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.520149, "value1":0.6583211185838453, "value2":179671992454725443, "value3":true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354725, "value1":0.26618356833133044, "value2":4502556297298851018, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.588275, "value1":0.40556503705171876, "value2":1831243556070013492, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.991768, "value1":0.5964588343272011, "value2":7216754835822865383, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.829272, "value1":0.8185283045838991, "value2":7054142095046490626, "value3":true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028671, "value1":0.9570557166393036, "value2":5667948933240210255, "value3":true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171455, "value1":0.3984068015975904, "value2":6709012624962450094, "value3":true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112456, "value1":0.8897199020551853, "value2":619757189037583586, "value3":false, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.366328, "value1":0.9348857036272434, "value2":4207327317390530443, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.787818, "value1":0.06046424642679486, "value2":3800391541062933624, "value3":false, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431436, "value1":0.5742437522880309, "value2":3338797017937456724, "value3":false, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.130697, "value1":0.8564226707318178, "value2":2563408398646331614, "value3":false, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531522, "value1":0.21916844982697795, "value2":6377428321672110838, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799330, "value1":0.2117471404567077, "value2":6841854064863394582, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415853, "value1":0.7458907419184496, "value2":3858225758706811596, "value3":false, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769460, "value1":0.9292446985209156, "value2":4294237114969210404, "value3":true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053422, "value1":0.19156586881852822, "value2":8401149164501095807, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.634910, "value1":0.13000039932274055, "value2":4131562742009700028, "value3":true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703579, "value1":0.6862332441253757, "value2":7454437023717068913, "value3":true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327549, "value1":0.40704854485658054, "value2":4932202555006075846, "value3":false, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.314755, "value1":0.5681370276361754, "value2":3506285901156371358, "value3":true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.466686, "value1":0.42167473519309434, "value2":3558723778128133952, "value3":true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160258, "value1":0.4575847781643395, "value2":6023651621402119686, "value3":false, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228508, "value1":0.5111004373311687, "value2":8895687587282710909, "value3":true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.111511, "value1":0.09506770522467269, "value2":2655655566098996088, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.784900, "value1":0.5435091084418109, "value2":5760921829736802814, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210512, "value1":0.25861533775417506, "value2":6914577108338158111, "value3":false, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.663100, "value1":0.8688427076237359, "value2":9173486087944898117, "value3":false, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066304, "value1":0.7818969735770589, "value2":4913317894269750379, "value3":false, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824798, "value1":0.9535971366603737, "value2":8366422359718269509, "value3":true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.090470, "value1":0.7644445079507416, "value2":6086882240277966349, "value3":false, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994994, "value1":0.046461773505700306, "value2":1146509415122743778, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409080, "value1":0.028382316546416403, "value2":2068020782874554558, "value3":false, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.815596, "value1":0.08796047569888332, "value2":8378736063122737266, "value3":false, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683447, "value1":0.24029660357741323, "value2":8797505888002523812, "value3":false, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.611937, "value1":0.2489739097238809, "value2":7210056341279458271, "value3":true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347662, "value1":0.7553826852514858, "value2":8429002700088766063, "value3":true, "__name__":"metric_00000447", "key_8":"i","key_0":"e","key_5":"g", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318289, "value1":0.5406931125323163, "value2":1745967906139345488, "value3":true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.900516, "value1":0.7025144762707813, "value2":456260840471493385, "value3":true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153699, "value1":0.20283897976623247, "value2":7209402260011396171, "value3":true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.813264, "value1":0.23005364054850463, "value2":4244429874559011543, "value3":false, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.174420, "value1":0.6527990870752745, "value2":5344843801392876512, "value3":false, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142219, "value1":0.1627182050289727, "value2":114175501496781641, "value3":false, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.481454, "value1":0.37374721367966557, "value2":1638215468267960931, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.523380, "value1":0.48736316265440816, "value2":4581537275255860768, "value3":false, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317190, "value1":0.0039195876566355085, "value2":5341281001441055802, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619251, "value1":0.5258022392764804, "value2":4434264026251453750, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.641615, "value1":0.33039615993236077, "value2":6928858155661176736, "value3":true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961004, "value1":0.5399790911878092, "value2":7046629202789685841, "value3":true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025798, "value1":0.5984426914038641, "value2":6528145482673170360, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.901084, "value1":0.06259887743500082, "value2":3236298745887395724, "value3":false, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.489095, "value1":0.21376661515997028, "value2":7928954356959433352, "value3":true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655274, "value1":0.5385351363841719, "value2":7272624008760795422, "value3":true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.082369, "value1":0.4014980361714972, "value2":7005111860489403789, "value3":true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519418, "value1":0.19683342336444604, "value2":1435942937723913885, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226945, "value1":0.9922423305002035, "value2":867760431487143669, "value3":false, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752765, "value1":0.9916422768378336, "value2":3430696359309587884, "value3":true, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.078412, "value1":0.6964630260101603, "value2":2102364069319122633, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328609, "value1":0.19123451863699675, "value2":8170554763374290069, "value3":false, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.470499, "value1":0.03919799070813115, "value2":3366292364134765186, "value3":true, "__name__":"metric_00000462", "key_8":"g","key_9":"d","key_6":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740670, "value1":0.08849655325518448, "value2":1691579140130982717, "value3":true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.393865, "value1":0.061408201218711954, "value2":174448511358385380, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.200654, "value1":0.2836024427807901, "value2":4326318289877991961, "value3":false, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970400, "value1":0.7730199010023195, "value2":8574178501642024588, "value3":false, "__name__":"metric_00000465", "key_6":"f","key_1":"f","key_3":"h", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.465169, "value1":0.542725080263732, "value2":9196833379545093375, "value3":false, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.563410, "value1":0.7958042531287333, "value2":7393645101828557391, "value3":false, "__name__":"metric_00000468", "key_5":"i","key_8":"k","key_1":"j", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213281, "value1":0.7928629505338592, "value2":1752466947098105566, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954474, "value1":0.3032546869220648, "value2":1748556721646671927, "value3":true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.847068, "value1":0.7688745080204612, "value2":5187070635183292150, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251152, "value1":0.5682862581346041, "value2":2363792865069284658, "value3":true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360666, "value1":0.08726468762579924, "value2":643126609584411238, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619231, "value1":0.7434795567208692, "value2":4822313794436821661, "value3":false, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603123, "value1":0.10532993201555661, "value2":2544972440464847934, "value3":false, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607459, "value1":0.20036031311934685, "value2":1168760905394403690, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447800, "value1":0.7195092335832372, "value2":5642489879930119928, "value3":true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.850819, "value1":0.6761002637886042, "value2":833307298139301862, "value3":true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539724, "value1":0.5294892272320822, "value2":2667359803805326763, "value3":true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.208291, "value1":0.8393697010611292, "value2":683397536019101988, "value3":false, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550311, "value1":0.9963724696480404, "value2":5468577449184757808, "value3":true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.023114, "value1":0.32353069050613953, "value2":9187169003922087415, "value3":true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286063, "value1":0.9137524095402076, "value2":6445901279543298486, "value3":true, "__name__":"metric_00000492", "key_9":"k","key_0":"k","key_3":"j", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.204901, "value1":0.4076696093660646, "value2":6677345639081163017, "value3":false, "__name__":"metric_00000477", "key_6":"k","key_0":"b","key_3":"f", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107007, "value1":0.5776778447129313, "value2":8463235586423448216, "value3":false, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271841, "value1":0.6930651904253268, "value2":6787444131360802180, "value3":false, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.069717, "value1":0.7690475248125641, "value2":1837461702894857026, "value3":false, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368696, "value1":0.5454486705451115, "value2":3041446083114186683, "value3":true, "__name__":"metric_00000488", "key_5":"c","key_3":"g", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395274, "value1":0.24368851515268314, "value2":4392956019271269002, "value3":true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352289, "value1":0.22988542503215345, "value2":6550852329237611667, "value3":false, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703725, "value1":0.026480349904897907, "value2":1885153320482283414, "value3":true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.256528, "value1":0.26111767999722507, "value2":6808025827523544315, "value3":true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252909, "value1":0.20711336158884097, "value2":3024039988582180695, "value3":true, "__name__":"metric_00000493", "key_9":"h","key_1":"d","key_8":"d", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.633970, "value1":0.07845906332760666, "value2":6511716740509653564, "value3":false, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605465, "value1":0.5950957840062573, "value2":2806262404335314036, "value3":false, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.081867, "value1":0.08398189819376621, "value2":5728413114044088113, "value3":false, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074111, "value1":0.6218602698989528, "value2":1011556855791832544, "value3":true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675019, "value1":0.11095025598530854, "value2":5207952003333828816, "value3":true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.738057, "value1":0.5410628771237361, "value2":2540645323761255412, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317014, "value1":0.5259626957217648, "value2":3505614768801064603, "value3":false, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776152, "value1":0.302147585405568, "value2":7518755412283013617, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094787, "value1":0.5814132866660779, "value2":4822671923080781538, "value3":false, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.517582, "value1":0.10766355589362117, "value2":8453589798026915731, "value3":false, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882088, "value1":0.8040313368583433, "value2":8495228225039478737, "value3":false, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803773, "value1":0.22925549140141077, "value2":8938270200397652857, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403205, "value1":0.07946210415899989, "value2":7443253750975595381, "value3":true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211183, "value1":0.27303249192461276, "value2":3758490705819764753, "value3":false, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684989, "value1":0.19635657331096867, "value2":3768344870136296803, "value3":true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596986, "value1":0.9569957010370201, "value2":5843413270558977752, "value3":false, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796083, "value1":0.4870882675033077, "value2":8040326882339558395, "value3":true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.674664, "value1":0.05017490820454691, "value2":1512965384940722080, "value3":false, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682218, "value1":0.05038298218752238, "value2":3666135732590503511, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871616, "value1":0.007362621326077148, "value2":3382155619299232500, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.479139, "value1":0.48744354938142825, "value2":2536302313422526956, "value3":false, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408860, "value1":0.5030243670352939, "value2":3709214658604138842, "value3":false, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.079849, "value1":0.4036712273225728, "value2":2906136993668691733, "value3":true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735107, "value1":0.11572752191146224, "value2":9135232014249483896, "value3":false, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.579000, "value1":0.832018905121829, "value2":4291485121105439680, "value3":false, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297326, "value1":0.3053519839954546, "value2":7762295205550727215, "value3":true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077473, "value1":0.5244918634463549, "value2":3651586678818959294, "value3":true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211495, "value1":0.3162790532742153, "value2":8307153509301118839, "value3":false, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.018730, "value1":0.4578847558537525, "value2":3121380712594178301, "value3":false, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.254109, "value1":0.0022335788974661955, "value2":1113680598327977122, "value3":false, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327840, "value1":0.38572838732788006, "value2":2539441632284031099, "value3":false, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802473, "value1":0.6047951359580281, "value2":1001558191118170380, "value3":true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299508, "value1":0.05862512584912177, "value2":5514618695776223541, "value3":true, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.011661, "value1":0.31240922806678983, "value2":6137561733651595636, "value3":true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728655, "value1":0.1278483939203575, "value2":9035381570106713199, "value3":true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.667349, "value1":0.5613783650549519, "value2":596862476082819858, "value3":true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.253799, "value1":0.021910250550882106, "value2":6533597878636060832, "value3":false, "__name__":"metric_00000527", "key_9":"k","key_3":"a","key_5":"b", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.288444, "value1":0.30057474196214407, "value2":3169010106041213703, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.798940, "value1":0.9681683301198919, "value2":3021363800209965251, "value3":true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.981962, "value1":0.9612929543752854, "value2":9171352851679704895, "value3":true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149939, "value1":0.09981385175599812, "value2":2281824057904455016, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943557, "value1":0.6955975747592801, "value2":2634327756448196264, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459593, "value1":0.6446822058280443, "value2":6342926358496707008, "value3":false, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.734204, "value1":0.6639025089026711, "value2":2476897145958348918, "value3":false, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506327, "value1":0.10053599414522371, "value2":2221394931414407943, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410931, "value1":0.8059701840141592, "value2":3838253097677401347, "value3":false, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.964398, "value1":0.603279381255061, "value2":5782827613318848456, "value3":false, "__name__":"metric_00000549", "key_3":"f","key_0":"f","key_2":"d", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.278202, "value1":0.7115230775467353, "value2":8371900549612489852, "value3":true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408547, "value1":0.21860759782537603, "value2":8039563012338877282, "value3":true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.261349, "value1":0.11227782989658946, "value2":7783215654484926349, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059288, "value1":0.07895743947800146, "value2":5667050368393356509, "value3":true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.601079, "value1":0.6482240475140798, "value2":4251195673123293243, "value3":true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.225112, "value1":0.7216063470142905, "value2":148937314023613788, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977412, "value1":0.6598247780794319, "value2":5965401930753401354, "value3":false, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194225, "value1":0.6141698639690517, "value2":8817766861356540801, "value3":false, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.597081, "value1":0.07693962991598453, "value2":5029342104400232103, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.335617, "value1":0.5583831809146028, "value2":1406686773640908295, "value3":true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916403, "value1":0.7018505752435668, "value2":664450605422502237, "value3":true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497447, "value1":0.8277122605282176, "value2":1219079943087879547, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716326, "value1":0.7509816698301687, "value2":3921665603218422131, "value3":true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716585, "value1":0.2920975208652007, "value2":3756232698934542100, "value3":false, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301567, "value1":0.8942632920542656, "value2":2291395595568091863, "value3":true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.599725, "value1":0.5659814539312558, "value2":7320359369690410226, "value3":true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.614192, "value1":0.7911920709201787, "value2":8781166815844312813, "value3":false, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.195843, "value1":0.3963219582946417, "value2":4716889546078718843, "value3":true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555996, "value1":0.602486402959913, "value2":4687524095847830496, "value3":false, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.917870, "value1":0.15392369008366974, "value2":7045583703847585198, "value3":true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.616039, "value1":0.6919323182152138, "value2":511455339796298853, "value3":false, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930107, "value1":0.12610440100742193, "value2":6997765950733061577, "value3":false, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194345, "value1":0.7209468362854391, "value2":556025580881809360, "value3":true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797831, "value1":0.2266151348199214, "value2":2004107184327193155, "value3":false, "__name__":"metric_00000572", "key_7":"i","key_1":"d","key_6":"b", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949681, "value1":0.8884059641939672, "value2":7011478951165849176, "value3":true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213425, "value1":0.9929181204525178, "value2":8631476778841936239, "value3":true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821834, "value1":0.3090901919909019, "value2":5522454281472043559, "value3":true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.181027, "value1":0.06720359916226379, "value2":4456725170730825432, "value3":true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.762490, "value1":0.474581541255306, "value2":7005342387987070026, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.759790, "value1":0.5398913013405661, "value2":2663889184009973941, "value3":false, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112196, "value1":0.8025326007392437, "value2":883355344681130779, "value3":true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374886, "value1":0.9718695344446878, "value2":1646239120227544644, "value3":true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.729457, "value1":0.9682037233981026, "value2":6151398746046248850, "value3":true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048424, "value1":0.7273914683080029, "value2":8000884246159354431, "value3":false, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.937043, "value1":0.1526646293780592, "value2":266299265780041426, "value3":true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752328, "value1":0.9655606802627454, "value2":5596664612933771240, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666200, "value1":0.10048257972248233, "value2":4289061240642605106, "value3":true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922103, "value1":0.8963130727289086, "value2":8836663020903305631, "value3":true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105202, "value1":0.8314839649252564, "value2":3551543011617183084, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886891, "value1":0.66836051888972, "value2":72366475317978094, "value3":true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432056, "value1":0.745064543848614, "value2":2189880642993257536, "value3":false, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042716, "value1":0.3956017159748215, "value2":3605211356176936944, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.984359, "value1":0.2911243881226883, "value2":7856628067522273320, "value3":false, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295731, "value1":0.4520766897124352, "value2":610878216891620607, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.650753, "value1":0.03928359938969399, "value2":2072987167295070336, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211564, "value1":0.7625006233645502, "value2":4401190374987004409, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792725, "value1":0.47390601969195995, "value2":4329714295913254034, "value3":true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.444277, "value1":0.5857721636217157, "value2":3580156964556605157, "value3":true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535990, "value1":0.9433719155892893, "value2":5643233427116703402, "value3":true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.688127, "value1":0.7720856573472716, "value2":5631377472070167195, "value3":true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.697966, "value1":0.9263837993798216, "value2":4093734777340860370, "value3":true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.996439, "value1":0.3910900735196092, "value2":8375564545525062485, "value3":false, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872727, "value1":0.8324218421388521, "value2":7152928233109153205, "value3":false, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.246110, "value1":0.2551941160691682, "value2":5602869232123697435, "value3":false, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931727, "value1":0.5795265646922866, "value2":7403691202626500132, "value3":true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673141, "value1":0.06896155961831285, "value2":4415828287435091542, "value3":true, "__name__":"metric_00000605", "key_8":"a","key_9":"h","key_7":"d", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684610, "value1":0.16872221976263158, "value2":505317720854796636, "value3":false, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559417, "value1":0.12924994486885147, "value2":4787886921303451969, "value3":false, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094359, "value1":0.5164211297919008, "value2":1315713014945297109, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007976, "value1":0.7735634457421625, "value2":8579409036573615608, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286892, "value1":0.43624877878480095, "value2":6924175474906299981, "value3":true, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.902926, "value1":0.6780220008663965, "value2":4822375571733927097, "value3":true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.486540, "value1":0.23333524037004374, "value2":6314520721238807162, "value3":true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007170, "value1":0.9857337079596743, "value2":4448133028853613455, "value3":false, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.030552, "value1":0.24355749578071725, "value2":8327413108187969392, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.379606, "value1":0.4729923914146341, "value2":914941226083135930, "value3":false, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835232, "value1":0.9602105867588695, "value2":3810448989361248064, "value3":false, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433684, "value1":0.9595084872985944, "value2":455874222103117607, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411653, "value1":0.037665634780994134, "value2":6833101830522402640, "value3":true, "__name__":"metric_00000587", "key_4":"a","key_5":"j","key_2":"e", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713632, "value1":0.5664925972399699, "value2":7794307235376671149, "value3":true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391266, "value1":0.6608939962095463, "value2":5860330325168511608, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212303, "value1":0.2089060448996, "value2":5651881484326487830, "value3":true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180078, "value1":0.7876093968011537, "value2":2862206598597477486, "value3":true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.175199, "value1":0.14395244434800714, "value2":2376804168343245120, "value3":true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809965, "value1":0.12003398504952464, "value2":5187708593154594566, "value3":false, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.851381, "value1":0.607956049195976, "value2":7288813225458184427, "value3":false, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933118, "value1":0.7246239590339857, "value2":1234716149527350625, "value3":true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.939251, "value1":0.34949617152726564, "value2":1847804623713516854, "value3":true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665651, "value1":0.7331985679295692, "value2":1370393562180863722, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.827386, "value1":0.6896205618409067, "value2":1887650763331073033, "value3":true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.637520, "value1":0.13783942741851515, "value2":7482561145284406008, "value3":true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657534, "value1":0.26275939644072077, "value2":6263897972267140208, "value3":false, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740684, "value1":0.9117133919549965, "value2":6149129956357538684, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032295, "value1":0.9865289627211871, "value2":5555291464918052387, "value3":true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545212, "value1":0.5648289415381519, "value2":6789070914120574869, "value3":true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.321143, "value1":0.6917810996904441, "value2":4171532082462131046, "value3":true, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.693006, "value1":0.9813132387635279, "value2":6633038595296515364, "value3":true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.895453, "value1":0.7852256444188942, "value2":143518625251882031, "value3":true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165192, "value1":0.9962691534924663, "value2":1006093794801043666, "value3":false, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.353290, "value1":0.5341309097782416, "value2":1258165174425707067, "value3":true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257167, "value1":0.7039937676297155, "value2":1145181626509784790, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413647, "value1":0.8270957619940229, "value2":2929580620701623843, "value3":true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817697, "value1":0.5617132216229666, "value2":9160744958657565723, "value3":false, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.117851, "value1":0.20816644440142626, "value2":5357495560720255737, "value3":false, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545126, "value1":0.5346084869679877, "value2":7401215357009964293, "value3":true, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824766, "value1":0.44946104541199977, "value2":4136655445147421363, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.292920, "value1":0.8837887841085235, "value2":7386066379061583540, "value3":true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949095, "value1":0.3445432872175305, "value2":7354448880530731294, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705243, "value1":0.4595023646368128, "value2":1229720604223933971, "value3":false, "__name__":"metric_00000641", "key_5":"c","key_6":"a","key_1":"j", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610093, "value1":0.4977300016550588, "value2":6055289293785959372, "value3":true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.710138, "value1":0.8494303412007083, "value2":3284734735098008783, "value3":false, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.503055, "value1":0.7306081415151926, "value2":7335475436979640441, "value3":false, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194198, "value1":0.4080577435044893, "value2":1817328694183447312, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409059, "value1":0.8174198346097175, "value2":1028006350387911539, "value3":true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696870, "value1":0.3114167302109578, "value2":5761149591452007692, "value3":true, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483258, "value1":0.6191704897662682, "value2":6062885989254649767, "value3":true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534924, "value1":0.6117230725465559, "value2":193279975656637243, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639609, "value1":0.47495483196675986, "value2":5738372947955229915, "value3":true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532964, "value1":0.01280767557324011, "value2":3610414855244726673, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.741210, "value1":0.5736887790720044, "value2":9118221200383527859, "value3":true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459701, "value1":0.09043619123087489, "value2":4263857141830808661, "value3":false, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.064891, "value1":0.8303048678510601, "value2":1417745368253260336, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.193982, "value1":0.5954818624712542, "value2":1866813575466479927, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728941, "value1":0.42773413448912906, "value2":5011276511494346486, "value3":true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978643, "value1":0.9720431088123873, "value2":5835438838603942392, "value3":true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376007, "value1":0.5894070491287171, "value2":1536141728264317947, "value3":false, "__name__":"metric_00000665", "key_8":"g","key_3":"k","key_6":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226693, "value1":0.26901914665508003, "value2":1625748998624229948, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.963696, "value1":0.5769249508695262, "value2":9054438071997375931, "value3":false, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.418644, "value1":0.766497146060167, "value2":5911142909752098984, "value3":true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564856, "value1":0.11883244261524029, "value2":1743904390192603929, "value3":true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055140, "value1":0.52087787431266, "value2":402388317326273396, "value3":false, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165833, "value1":0.8230062512281381, "value2":3737986001509812244, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.537380, "value1":0.3719641116241002, "value2":7620785809655703628, "value3":false, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059138, "value1":0.3359364898323379, "value2":2258599254367397283, "value3":false, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532103, "value1":0.07168825170242699, "value2":8419082906568750462, "value3":false, "__name__":"metric_00000674", "key_7":"f","key_1":"a","key_6":"c", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286202, "value1":0.8607616531331588, "value2":5301488987302943339, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477954, "value1":0.7462715469038064, "value2":8695685405375237694, "value3":false, "__name__":"metric_00000648", "key_2":"e","key_9":"h","key_1":"e", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.377516, "value1":0.1251480000451169, "value2":5693123483571141635, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799701, "value1":0.377835354984664, "value2":7635656423964308029, "value3":true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947251, "value1":0.7193521006934637, "value2":3495998112139060094, "value3":false, "__name__":"metric_00000654", "key_9":"c","key_3":"g","key_8":"h", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.630158, "value1":0.27928250023508844, "value2":8779396817967517012, "value3":true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886093, "value1":0.6334013505397469, "value2":5098375435747464833, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.942245, "value1":0.3390011070632797, "value2":1737381722392040741, "value3":false, "__name__":"metric_00000656", "key_7":"k","key_4":"f","key_6":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.009762, "value1":0.2515798026660567, "value2":2465310044177783178, "value3":false, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.896699, "value1":0.584030830613032, "value2":5977614913522628306, "value3":false, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820378, "value1":0.7811580321417821, "value2":1709536028386531684, "value3":true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653683, "value1":0.7005991085562125, "value2":7295655368561133786, "value3":false, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522361, "value1":0.7296163959268779, "value2":5916302487026661508, "value3":false, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.127496, "value1":0.24876472584091772, "value2":8637191483262110011, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.091531, "value1":0.9069922237355105, "value2":3364698751790785415, "value3":false, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137633, "value1":0.9475238029065713, "value2":6717128691639834645, "value3":false, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550582, "value1":0.1420769771765079, "value2":1644264737205720610, "value3":true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059092, "value1":0.38378121050522995, "value2":4775737880122110580, "value3":true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236099, "value1":0.8215350110894001, "value2":1784075564711617189, "value3":false, "__name__":"metric_00000693", "key_8":"b","key_0":"b","key_5":"e", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.636343, "value1":0.4453072911799391, "value2":3315454218709810887, "value3":true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799575, "value1":0.17197278191202653, "value2":5903905346180673218, "value3":true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.584133, "value1":0.4235206584790561, "value2":1691406943949519298, "value3":true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015622, "value1":0.34252290670938007, "value2":5505899012400247300, "value3":false, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.021661, "value1":0.11232931832514273, "value2":6468715321661687664, "value3":true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.593330, "value1":0.8735006589361365, "value2":831831146444791684, "value3":false, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273986, "value1":0.25953904800502825, "value2":5320773233674225264, "value3":true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357026, "value1":0.11876193765893824, "value2":8703386441096799470, "value3":true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903715, "value1":0.0317054368098292, "value2":2551886074115870652, "value3":false, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.905359, "value1":0.9780528237638186, "value2":3462176768443711504, "value3":false, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.272907, "value1":0.5077414966128044, "value2":5351891928699763264, "value3":false, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610476, "value1":0.5253243304839273, "value2":7118905075072261164, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684508, "value1":0.3558646139061106, "value2":5157018975048237545, "value3":true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748383, "value1":0.7510255655970447, "value2":4180265211771950319, "value3":true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.801038, "value1":0.2649030050080282, "value2":509475464752529120, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657950, "value1":0.2544231826829063, "value2":4768343360725290987, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149149, "value1":0.24278734421658493, "value2":2310223808628977449, "value3":true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.558331, "value1":0.3866811766896264, "value2":3120828999446479419, "value3":false, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.076776, "value1":0.8191629913767823, "value2":531265332159947696, "value3":false, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881067, "value1":0.16630603149873838, "value2":8244214645789653431, "value3":true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385842, "value1":0.7521248304618617, "value2":4350867045350111094, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914596, "value1":0.12306660567858013, "value2":5899167982504356408, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.159683, "value1":0.24808395371130693, "value2":3321017992196518857, "value3":false, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.058378, "value1":0.4115012836332614, "value2":5551372136464658518, "value3":false, "__name__":"metric_00000708", "key_6":"g","key_0":"a","key_2":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819794, "value1":0.9548946315713374, "value2":2661379806705881657, "value3":true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.293628, "value1":0.30579616916280866, "value2":2771189581765698489, "value3":true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.904159, "value1":0.33983467932998995, "value2":3076540259059103682, "value3":true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946959, "value1":0.14422457278803882, "value2":619548712042120909, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383295, "value1":0.23130372874617672, "value2":6179693520548172628, "value3":false, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482434, "value1":0.8720643520387502, "value2":6593209094509926896, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613596, "value1":0.9574764532675973, "value2":8706916487148773835, "value3":false, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551130, "value1":0.9901941419895447, "value2":5559435899121181473, "value3":true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954673, "value1":0.7022122624865335, "value2":6926608561209244320, "value3":true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781834, "value1":0.1129731019718482, "value2":3356276176028256450, "value3":true, "__name__":"metric_00000717", "key_4":"f","key_9":"e","key_0":"b", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746610, "value1":0.5148787087784333, "value2":2343142454537691934, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409647, "value1":0.3828843970677094, "value2":4346661084777988652, "value3":true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.427543, "value1":0.6220594320308941, "value2":653819026690948218, "value3":false, "__name__":"metric_00000721", "key_9":"e","key_1":"d", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467056, "value1":0.23591956688182053, "value2":5924417417665977435, "value3":false, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245070, "value1":0.32465613049106207, "value2":8745282484552741587, "value3":true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719344, "value1":0.6149579128373653, "value2":7721284234183306230, "value3":false, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095318, "value1":0.7783448314627689, "value2":7496490910707147022, "value3":false, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696761, "value1":0.23959350451136963, "value2":7667698021180616606, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170976, "value1":0.2107634553446009, "value2":1067019224393043111, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177955, "value1":0.15559883601716398, "value2":570151235008619291, "value3":true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.334876, "value1":0.05250568880425774, "value2":313709473500400875, "value3":true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.645622, "value1":0.5874495193049836, "value2":4998753194711581798, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403716, "value1":0.681377437500311, "value2":6083653541111639303, "value3":true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.511374, "value1":0.3173656336054262, "value2":3280525464397494846, "value3":false, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.218153, "value1":0.6649425163171951, "value2":984695368325234886, "value3":false, "__name__":"metric_00000742", "key_5":"g","key_6":"j","key_4":"a", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424332, "value1":0.13915015029096536, "value2":177058255857848790, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882802, "value1":0.9604830357087591, "value2":3462432728417263080, "value3":false, "__name__":"metric_00000724", "key_9":"f","key_0":"a","key_5":"d", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933587, "value1":0.05218287981155221, "value2":9126870231879692354, "value3":false, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426076, "value1":0.7165203610998703, "value2":4392854854028036091, "value3":false, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.302645, "value1":0.34553849779237905, "value2":2907279110591723700, "value3":true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269877, "value1":0.8711177705575849, "value2":8366161311313542515, "value3":false, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.995800, "value1":0.6906708270307872, "value2":2179499360020827214, "value3":false, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433217, "value1":0.612549221250184, "value2":3062667501076675896, "value3":true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550193, "value1":0.8112010714019721, "value2":3511775054758615153, "value3":true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947917, "value1":0.4309103807271251, "value2":5927887337207634099, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874180, "value1":0.8528260534422161, "value2":6693253358548032140, "value3":false, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.714464, "value1":0.7660112509524851, "value2":4667923922359952270, "value3":false, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.088061, "value1":0.2662708805339307, "value2":2956847029720911185, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960166, "value1":0.6603171866146907, "value2":4062360360972662646, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.771991, "value1":0.06535928533875911, "value2":3275575695469134507, "value3":false, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.907738, "value1":0.4729789504578976, "value2":6665898542309531788, "value3":false, "__name__":"metric_00000749", "key_6":"c","key_9":"e","key_5":"f", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.446527, "value1":0.054360638009933175, "value2":8447197342747029062, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871013, "value1":0.9325815658063547, "value2":3295816637987069156, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322468, "value1":0.7896334184172146, "value2":2062716273940484317, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145073, "value1":0.15908044288327283, "value2":31202777072956548, "value3":true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.450280, "value1":0.20707514852594208, "value2":3916360901630366923, "value3":true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.834638, "value1":0.6993008706531769, "value2":5937180712489291188, "value3":true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116654, "value1":0.23637601643034672, "value2":4626933419743360623, "value3":true, "__name__":"metric_00000765", "key_5":"h","key_9":"b","key_3":"a", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.265735, "value1":0.37825344249371357, "value2":1259576346841819002, "value3":true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683748, "value1":0.8969615470937745, "value2":3878280364099631128, "value3":true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219470, "value1":0.2735521439606725, "value2":2004029025316087157, "value3":false, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476407, "value1":0.3060298444596037, "value2":2752192683000499763, "value3":false, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973916, "value1":0.6236548692215601, "value2":1218902262886276264, "value3":true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777866, "value1":0.4605467848114755, "value2":1483419366258255115, "value3":false, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045114, "value1":0.6713933571025698, "value2":7389349750224761078, "value3":true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.570321, "value1":0.6477885368549638, "value2":952734344221236, "value3":true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821043, "value1":0.6987507461356852, "value2":7961454653125032378, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.586904, "value1":0.6114733442241732, "value2":3313794499633786815, "value3":false, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705481, "value1":0.05021439091843144, "value2":1692908847970065889, "value3":false, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497544, "value1":0.7911397038857368, "value2":7192760038857907146, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.932189, "value1":0.9148929522533059, "value2":9058585624863979355, "value3":true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769986, "value1":0.08834899410804685, "value2":1734900992084257549, "value3":false, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695397, "value1":0.9679316547671841, "value2":7641677535001923086, "value3":false, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084900, "value1":0.7534718430023655, "value2":5595569809726413737, "value3":true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.562250, "value1":0.19691330948497934, "value2":5079334807276275801, "value3":false, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.843384, "value1":0.13037385236459426, "value2":1781212766913480310, "value3":false, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.102533, "value1":0.8945673902737089, "value2":4413715911002881, "value3":false, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350432, "value1":0.9789410844857529, "value2":8061203484333522616, "value3":false, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857648, "value1":0.17405749140176002, "value2":9069075917121269047, "value3":false, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559262, "value1":0.2097671258596821, "value2":613585525667388168, "value3":true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.712886, "value1":0.7017574550139256, "value2":807895944720896464, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.936984, "value1":0.8849159113766228, "value2":6360061394425352910, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545107, "value1":0.7492436378397768, "value2":1930556756703562417, "value3":true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.681593, "value1":0.044080654493611975, "value2":4672445330328029132, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388330, "value1":0.4856178458787539, "value2":6153421365310799825, "value3":true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.879197, "value1":0.16996161009540478, "value2":260583885651182711, "value3":true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.770981, "value1":0.7283458398178451, "value2":6187029331538785539, "value3":true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101721, "value1":0.5814924092013966, "value2":3246021123474652609, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665903, "value1":0.2698824852319885, "value2":1593984595070302417, "value3":true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.014945, "value1":0.41108393803888965, "value2":849125531814643716, "value3":false, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455276, "value1":0.875114673636132, "value2":5155846223745306265, "value3":false, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.044985, "value1":0.9997830302808556, "value2":2517744318699317082, "value3":false, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236657, "value1":0.10301472150509888, "value2":3916175947622962913, "value3":false, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.595845, "value1":0.681705831836613, "value2":4919968063926505445, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909129, "value1":0.10658700130911442, "value2":9017518189462648176, "value3":false, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994913, "value1":0.5742718119176038, "value2":2926652004213539521, "value3":false, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350277, "value1":0.15279738754509417, "value2":5229335911692726266, "value3":false, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.022399, "value1":0.3330863087425972, "value2":4953311412687090451, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268930, "value1":0.7193122390898454, "value2":4256538916269794836, "value3":false, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.983431, "value1":0.0510995090351011, "value2":1768238660724776680, "value3":false, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672570, "value1":0.255109482234144, "value2":5911656502600207006, "value3":true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478340, "value1":0.7629467314333804, "value2":2248461028173940598, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954691, "value1":0.780819131964203, "value2":4988575109325316188, "value3":false, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.854956, "value1":0.888056149862958, "value2":8089385147769653445, "value3":true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028845, "value1":0.6170371731466705, "value2":4427914831964658823, "value3":true, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639652, "value1":0.8230090544258113, "value2":6386454055832759048, "value3":false, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.810891, "value1":0.8133492204735011, "value2":6725307713283048525, "value3":false, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761452, "value1":0.9287463566978789, "value2":1984332858107282238, "value3":false, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318552, "value1":0.12867977830402738, "value2":4258557511687812956, "value3":true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316280, "value1":0.5022437385431537, "value2":995773241737628619, "value3":false, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954122, "value1":0.5240772572203263, "value2":6037304727626529988, "value3":true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177671, "value1":0.76376619698564, "value2":2077146903364533171, "value3":false, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731650, "value1":0.7327573768170443, "value2":3115299633570203304, "value3":true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.050578, "value1":0.7589539042984136, "value2":8472839195693038957, "value3":true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.606514, "value1":0.9364993124842877, "value2":5612839525396767856, "value3":false, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369368, "value1":0.048054238834199556, "value2":5282267936079713586, "value3":false, "__name__":"metric_00000823", "key_9":"i","key_8":"d", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619618, "value1":0.3228482622950357, "value2":8295983975137448078, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783482, "value1":0.20802671279789772, "value2":3029896725457460111, "value3":true, "__name__":"metric_00000825", "key_7":"j","key_8":"d","key_5":"k", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332994, "value1":0.9302305981560293, "value2":1560834164036895150, "value3":false, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320192, "value1":0.22351800921367507, "value2":367222968940142358, "value3":false, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881266, "value1":0.08414682717481595, "value2":4156359981061160049, "value3":false, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.708929, "value1":0.5859616740437592, "value2":4218658472873761504, "value3":true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.974600, "value1":0.18236565991254491, "value2":7158153525091699583, "value3":false, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760005, "value1":0.2499852109211892, "value2":3103407843415565026, "value3":false, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994395, "value1":0.6346134433025367, "value2":8895656809884955572, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482016, "value1":0.09096777794371665, "value2":6086112719790576068, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442570, "value1":0.8266959930861106, "value2":2275327626493666884, "value3":false, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416700, "value1":0.36518175488705273, "value2":5793835688591222111, "value3":true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.488107, "value1":0.8111992750792233, "value2":3432067066553462639, "value3":true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.296157, "value1":0.748182736526456, "value2":5908492840734386304, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655322, "value1":0.77012733378921, "value2":2328983933411827460, "value3":false, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299380, "value1":0.8617587030482643, "value2":4249851149905507488, "value3":true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.569550, "value1":0.8799767655587674, "value2":114484019272599760, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.737420, "value1":0.660133828675834, "value2":47092260178100634, "value3":true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961538, "value1":0.5805975021285331, "value2":3098519932913684476, "value3":false, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197731, "value1":0.7355297940768138, "value2":5347385484676578771, "value3":true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653672, "value1":0.9639932888139914, "value2":4154150822821006863, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209735, "value1":0.25867188901232296, "value2":7573003393945357593, "value3":false, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.243923, "value1":0.34698853560785725, "value2":4046809595104442791, "value3":false, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055125, "value1":0.9118136199691569, "value2":6132859986145115351, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.129146, "value1":0.737958001899574, "value2":3639000020359385593, "value3":true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439824, "value1":0.7612995291197047, "value2":9120669622233440416, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222157, "value1":0.2453738047252645, "value2":5040757825498916672, "value3":false, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916336, "value1":0.7380620101187605, "value2":4223451018219910420, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365543, "value1":0.035239810434389264, "value2":7558213648465294836, "value3":false, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.860762, "value1":0.4102850563256095, "value2":1429493420542448489, "value3":false, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388872, "value1":0.5076851349389522, "value2":5984142931688899418, "value3":true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.324767, "value1":0.32798077886519317, "value2":7010471358073816774, "value3":true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605843, "value1":0.9054516940429083, "value2":5196636487460887513, "value3":true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653438, "value1":0.9546349180436571, "value2":1964265110886490351, "value3":false, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576178, "value1":0.9072096436167739, "value2":4650061523788539671, "value3":true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.304613, "value1":0.07126080366571305, "value2":2739761425628315393, "value3":false, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399413, "value1":0.9339282528280148, "value2":2738842204765000811, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943959, "value1":0.08784682744168541, "value2":7785212390244201422, "value3":false, "__name__":"metric_00000813", "key_7":"g","key_8":"h","key_1":"j", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.155440, "value1":0.06596281114416563, "value2":5614935012679054029, "value3":true, "__name__":"metric_00000814", "key_4":"d","key_2":"f", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.585232, "value1":0.38735104883356297, "value2":5357846878220935711, "value3":true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.231739, "value1":0.7284804092280743, "value2":1167665308911410443, "value3":true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555892, "value1":0.5915854017921878, "value2":1567147347614329809, "value3":true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.855303, "value1":0.09176207020704319, "value2":3808984845296778232, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.938710, "value1":0.4819531123010938, "value2":4697805519732493178, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476902, "value1":0.22082616880251751, "value2":2129509252592428060, "value3":false, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744743, "value1":0.33257150382979367, "value2":7648865469799769588, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306532, "value1":0.04481922646422544, "value2":5772762433855832524, "value3":false, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364825, "value1":0.3587026951251621, "value2":5278057080002614026, "value3":true, "__name__":"metric_00000863", "key_4":"c","key_3":"j", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.484995, "value1":0.39478883780380347, "value2":3136752353967205980, "value3":true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.359118, "value1":0.08414989767840077, "value2":1942845542819450866, "value3":true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095178, "value1":0.16106032134150491, "value2":2585752892335863488, "value3":false, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268896, "value1":0.613902523132248, "value2":687319106737333189, "value3":true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716029, "value1":0.4960997765937663, "value2":2263474829486803965, "value3":false, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748556, "value1":0.30380613001187684, "value2":569759032424044386, "value3":false, "__name__":"metric_00000869", "key_3":"c","key_5":"b","key_1":"c", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819011, "value1":0.9766133829878352, "value2":4041486474077430360, "value3":false, "__name__":"metric_00000868", "key_4":"i","key_8":"j","key_3":"i", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109307, "value1":0.7831795468260019, "value2":6028647505036494731, "value3":false, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.179692, "value1":0.36834450950873376, "value2":5143099232657374567, "value3":false, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408811, "value1":0.37776607558956526, "value2":6856480231263656565, "value3":true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.006253, "value1":0.5270930124398272, "value2":1411653061914296136, "value3":true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776912, "value1":0.4551811310760191, "value2":1536148212309064628, "value3":false, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447453, "value1":0.8174960750892899, "value2":4331073111578365688, "value3":false, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696722, "value1":0.2434463944121451, "value2":3006348927793308525, "value3":true, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.747552, "value1":0.46652870663413926, "value2":8487775958608976630, "value3":false, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618043, "value1":0.5667967701695303, "value2":445864738400581228, "value3":false, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162568, "value1":0.6628301293329222, "value2":8197909955604355013, "value3":false, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.861425, "value1":0.3485000903134516, "value2":8816338849324478138, "value3":false, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.806516, "value1":0.4953223448201175, "value2":7711648028791565560, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970539, "value1":0.2083396416590356, "value2":5815736098137722214, "value3":false, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108983, "value1":0.42614208367663026, "value2":5403929578330914973, "value3":false, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095108, "value1":0.4675226715836234, "value2":4103322823365161263, "value3":true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.279008, "value1":0.5189838609534998, "value2":5583248834742899795, "value3":true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080151, "value1":0.04025953736590447, "value2":1709967085975216083, "value3":false, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960603, "value1":0.48558007294067956, "value2":147087377780565755, "value3":true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522699, "value1":0.8996419465199031, "value2":8383086671138869204, "value3":true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812906, "value1":0.42145746065428685, "value2":3807545219581503501, "value3":false, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973610, "value1":0.9578107981118065, "value2":4941505183486571577, "value3":false, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972491, "value1":0.7644375474791567, "value2":5826757011862504756, "value3":true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048527, "value1":0.2281920875592009, "value2":4718173727183162630, "value3":false, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515627, "value1":0.3746899214010938, "value2":5127946555951269169, "value3":false, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.346004, "value1":0.5308363801498025, "value2":1141799226947758562, "value3":true, "__name__":"metric_00000894", "key_7":"d","key_3":"k","key_5":"c", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391384, "value1":0.5665502681985652, "value2":5138410786658734048, "value3":true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857410, "value1":0.1451852744317182, "value2":6603434642077187944, "value3":false, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.414061, "value1":0.265028330167258, "value2":8861513360159934632, "value3":false, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653333, "value1":0.28437623266560746, "value2":5007130114350069495, "value3":false, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.123846, "value1":0.29659698191393213, "value2":8540905938700351347, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477802, "value1":0.6141130483574686, "value2":4145514544028354410, "value3":false, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531351, "value1":0.3974291747311908, "value2":7891249160701823099, "value3":false, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197949, "value1":0.9490013407410278, "value2":8218574898456721251, "value3":false, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411857, "value1":0.8458181580037749, "value2":3063835093954499296, "value3":false, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.877510, "value1":0.828501312919876, "value2":5054318282608028643, "value3":true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665063, "value1":0.5885292293078199, "value2":7952781418541268636, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041521, "value1":0.9878113457449454, "value2":7450867973872472368, "value3":true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073061, "value1":0.939649958460976, "value2":7875175737198226822, "value3":false, "__name__":"metric_00000898", "key_5":"k","key_9":"d","key_4":"i", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781888, "value1":0.5031750456506159, "value2":7116386269513023773, "value3":false, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160017, "value1":0.23130907810117704, "value2":2438724480357189127, "value3":false, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835359, "value1":0.4415459106308202, "value2":8077656610946018544, "value3":false, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251312, "value1":0.10148089002006272, "value2":1896400908090076616, "value3":false, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.632483, "value1":0.7757884759608441, "value2":4168124167987668028, "value3":true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.772638, "value1":0.14296917471420942, "value2":5892426599914094550, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426554, "value1":0.4152913794309927, "value2":3491478566670263783, "value3":true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672520, "value1":0.5730477814369939, "value2":7734177692857714726, "value3":false, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025475, "value1":0.8422765595744146, "value2":7729206509934976065, "value3":true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384117, "value1":0.370521152586741, "value2":5964592513649221172, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666556, "value1":0.5944594406776452, "value2":6664609919274167291, "value3":true, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424484, "value1":0.6898557357450742, "value2":7620342586495006501, "value3":true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313298, "value1":0.7769705418438511, "value2":5904168920374633694, "value3":false, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556524, "value1":0.7197157815883978, "value2":1785992040018522529, "value3":true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360907, "value1":0.5368763819006307, "value2":1221035619431068900, "value3":false, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576297, "value1":0.2976581980063287, "value2":6069726578064867551, "value3":true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.425007, "value1":0.05760812531545847, "value2":3218040486200104849, "value3":false, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442385, "value1":0.7140138290502642, "value2":5982710133241839409, "value3":false, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364950, "value1":0.7133219094796532, "value2":2621884575274392083, "value3":false, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211537, "value1":0.9662389413186122, "value2":6837048628615994370, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.161776, "value1":0.9162528731861198, "value2":98225928051258831, "value3":false, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811493, "value1":0.5668121931771126, "value2":4899385784647025957, "value3":false, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822383, "value1":0.016988952685027904, "value2":7343988970036816255, "value3":false, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865454, "value1":0.48779466867997384, "value2":3996429126028951858, "value3":false, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751902, "value1":0.517057218253837, "value2":4621244619940358234, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.525579, "value1":0.7270638229553396, "value2":4556294068287353316, "value3":false, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822049, "value1":0.8487037862810487, "value2":8759327482669918657, "value3":false, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483767, "value1":0.795243973695518, "value2":5137269638112922032, "value3":false, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042738, "value1":0.7385935337493647, "value2":1091563145939490509, "value3":true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.521929, "value1":0.10598411546030329, "value2":1018022188446431653, "value3":true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792757, "value1":0.4495950673323956, "value2":1856520078086460155, "value3":true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695775, "value1":0.8122734246000456, "value2":2520061166650080735, "value3":false, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105722, "value1":0.9820835252391706, "value2":7455782937968387088, "value3":true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820568, "value1":0.22530315350938107, "value2":3276960573086645000, "value3":false, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.140320, "value1":0.5664165038340241, "value2":7444501896936670459, "value3":false, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.501828, "value1":0.6731262889242471, "value2":1425113709817927232, "value3":true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930172, "value1":0.48625040372483846, "value2":5327468289350467201, "value3":false, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.706858, "value1":0.07914368053314678, "value2":1800402440657604521, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134725, "value1":0.4678852928796053, "value2":2325913710025246340, "value3":true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876578, "value1":0.9424035369996965, "value2":8055626132574707503, "value3":true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108826, "value1":0.29002692913973105, "value2":774202804060462158, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.031892, "value1":0.9251224968475468, "value2":6954662396929356191, "value3":false, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539689, "value1":0.24245379006948578, "value2":460788187782305876, "value3":false, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.858085, "value1":0.02056945040393724, "value2":4590009984548317252, "value3":false, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.202540, "value1":0.37546125893661825, "value2":6399435061817101457, "value3":false, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384494, "value1":0.010928897877400335, "value2":2783102565309398205, "value3":true, "__name__":"metric_00000953", "key_1":"f","key_9":"g","key_0":"k", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061578, "value1":0.259570974455371, "value2":8569411528487848312, "value3":false, "__name__":"metric_00000954", "key_8":"h","key_9":"e","key_2":"h", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027239, "value1":0.09519589010929196, "value2":6611488629400003141, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210858, "value1":0.6912816460992459, "value2":7725980859343417825, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.825726, "value1":0.48050045655332785, "value2":6843673580007675724, "value3":false, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430241, "value1":0.8940246106870715, "value2":7204593499636831454, "value3":false, "__name__":"metric_00000967", "key_5":"d","key_9":"f","key_1":"h", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332710, "value1":0.05615399939144247, "value2":4573382128185893311, "value3":true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374418, "value1":0.5152317418659397, "value2":7908425193534726276, "value3":false, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519252, "value1":0.2299372870124918, "value2":6913359063794068428, "value3":false, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.405529, "value1":0.49571648923268996, "value2":1675726536905469427, "value3":true, "__name__":"metric_00000971", "key_8":"i","key_2":"j", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605031, "value1":0.09917638686306547, "value2":3890367351649717445, "value3":true, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194280, "value1":0.5756913899097403, "value2":6546196099308660172, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763643, "value1":0.049961431289849915, "value2":2156338695320389229, "value3":true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755741, "value1":0.4853393193247385, "value2":5380709479032006182, "value3":true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434015, "value1":0.6730243170500183, "value2":4072172223941247851, "value3":true, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.188499, "value1":0.8036991963254426, "value2":2042780350133201142, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.788778, "value1":0.6590319721348468, "value2":6545712378323399560, "value3":false, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431899, "value1":0.10111044541197162, "value2":3033625496942345343, "value3":false, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846633, "value1":0.4624813135357754, "value2":5961232902625455903, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649911, "value1":0.42335939078880974, "value2":9206873087714105578, "value3":true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532557, "value1":0.8374513257646443, "value2":4063679098902368460, "value3":false, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.993435, "value1":0.026158890681344544, "value2":8561365511354827010, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808220, "value1":0.6771449672369052, "value2":2392568978099066809, "value3":false, "__name__":"metric_00000959", "key_2":"h","key_9":"g","key_0":"b", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357037, "value1":0.7840791560680648, "value2":1293280054604430799, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686467, "value1":0.009361168431443715, "value2":1158105653011230048, "value3":false, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.971166, "value1":0.5749373357687312, "value2":5056280036307903399, "value3":true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.544825, "value1":0.6259231852657728, "value2":1572064409805832355, "value3":false, "__name__":"metric_00000965", "key_9":"c","key_5":"i", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889816, "value1":0.6691815472587526, "value2":5159786074932669495, "value3":false, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751169, "value1":0.5117344773320296, "value2":2709445168494389044, "value3":true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.358316, "value1":0.5060937970199147, "value2":4487218678538317276, "value3":false, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.577717, "value1":0.8145066136721544, "value2":3597604992290980548, "value3":true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.990084, "value1":0.031203549542102884, "value2":6033709675731250939, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613856, "value1":0.5443798055071071, "value2":1023967738912955990, "value3":false, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795434, "value1":0.2935457393386216, "value2":1392092134182260244, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233100, "value1":0.7087138744541079, "value2":6901046651435105011, "value3":false, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.215643, "value1":0.301381153698089, "value2":5724617386824483093, "value3":true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.536387, "value1":0.050992650300612716, "value2":1926309233688447762, "value3":true, "__name__":"metric_00000998", "key_9":"c","key_3":"e", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.661031, "value1":0.5259350399353443, "value2":3757516621940142781, "value3":false, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.883217, "value1":0.9897587697648693, "value2":166453122026887271, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482631, "value1":0.7903296358928328, "value2":6917975765021057279, "value3":true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301142, "value1":0.03214802664084894, "value2":1209824808667363268, "value3":false, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618630, "value1":0.34174203368906997, "value2":687273686120110296, "value3":true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872422, "value1":0.7312694201106051, "value2":1954246364933112681, "value3":false, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675094, "value1":0.02386805295420225, "value2":2582799143372120508, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.397537, "value1":0.7375439234748727, "value2":3761195316124557282, "value3":false, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998621, "value1":0.44001824362826686, "value2":2298817484668315029, "value3":false, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471460, "value1":0.5040752524973563, "value2":7440124910918057262, "value3":true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303559, "value1":0.34760281561681783, "value2":3856193801537489239, "value3":true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477238, "value1":0.10358452410298057, "value2":3433455763442399890, "value3":true, "__name__":"metric_00000001", "key_5":"g","key_6":"a","key_4":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974082, "value1":0.9936842430072798, "value2":7647779573135010712, "value3":false, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.679212, "value1":0.33597414933520936, "value2":7871330604163867562, "value3":true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.420730, "value1":0.4119033435452023, "value2":7398827678901559124, "value3":false, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.546764, "value1":0.0476713858704466, "value2":112659874965942961, "value3":false, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730784, "value1":0.2868023177018098, "value2":7913444002207179200, "value3":true, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862440, "value1":0.5312714100525405, "value2":6175515648240114931, "value3":true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769163, "value1":0.9602531490116414, "value2":5345461780217485343, "value3":true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855124, "value1":0.553411156209368, "value2":4062088865979243439, "value3":false, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.210418, "value1":0.8711908622408605, "value2":8107060141858943082, "value3":false, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436733, "value1":0.7746479416891845, "value2":3021240871572817126, "value3":false, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.692065, "value1":0.594498398968098, "value2":988266694346712016, "value3":true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.441926, "value1":0.8776750344471822, "value2":6190746113032965957, "value3":true, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013512, "value1":0.14091811490255743, "value2":4960449076483043020, "value3":true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548236, "value1":0.32376195419487824, "value2":4889372139161861463, "value3":true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.828595, "value1":0.6520731819283484, "value2":508152344585097791, "value3":false, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759392, "value1":0.11332264235829892, "value2":8149117049560006848, "value3":true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901588, "value1":0.1665732927247099, "value2":1166517989622232910, "value3":true, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035913, "value1":0.07988414283718974, "value2":7894355898795371390, "value3":false, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.583982, "value1":0.25757663417932825, "value2":6007061793954380382, "value3":true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636209, "value1":0.38383005316111324, "value2":4618851528086053316, "value3":true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672395, "value1":0.002191687957779578, "value2":1207882083575373188, "value3":true, "__name__":"metric_00000031", "key_5":"c","key_7":"f","key_0":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462535, "value1":0.3192805953827191, "value2":365124979437384544, "value3":false, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508318, "value1":0.6798889100739353, "value2":255815407107212152, "value3":true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905933, "value1":0.882876839434679, "value2":1335222091221270902, "value3":false, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.721902, "value1":0.3457876641727297, "value2":7448197074268866329, "value3":true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.837057, "value1":0.03599562374649498, "value2":1792244431983896309, "value3":true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056476, "value1":0.22903239483804314, "value2":5638633766102759583, "value3":true, "__name__":"metric_00000012", "key_7":"j","key_2":"e", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.457531, "value1":0.05016525338775747, "value2":2976936137139451754, "value3":true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617621, "value1":0.16600638886403743, "value2":3903308221116403976, "value3":false, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415798, "value1":0.5206207545088288, "value2":7426460172854544634, "value3":true, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.470076, "value1":0.02758169975135837, "value2":3279193513719457677, "value3":false, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.847049, "value1":0.818759672974311, "value2":5946222659493106047, "value3":true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367937, "value1":0.4074258090017152, "value2":6858775210252472573, "value3":false, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973712, "value1":0.13887537149474657, "value2":2800343207358775074, "value3":false, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403585, "value1":0.2978164609006992, "value2":6234312472150253679, "value3":false, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904490, "value1":0.7280336451114688, "value2":2026079147622689137, "value3":false, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.541237, "value1":0.1368259441713387, "value2":6424894922861577254, "value3":false, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.783065, "value1":0.5704544299311838, "value2":4180282487041693175, "value3":false, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.890800, "value1":0.7757728483786852, "value2":8132981856885227577, "value3":true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901025, "value1":0.22908718180210688, "value2":4747104448031391939, "value3":false, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180212, "value1":0.5476402281995001, "value2":8223100966387528196, "value3":true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626159, "value1":0.7996650121856338, "value2":64003761107808118, "value3":false, "__name__":"metric_00000041", "key_9":"g","key_3":"k","key_4":"f", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545550, "value1":0.1740742638651991, "value2":5824563041193046282, "value3":true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026911, "value1":0.9915609719154322, "value2":3839917984332114189, "value3":false, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970605, "value1":0.049651840615534626, "value2":3663818871674092196, "value3":false, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.795861, "value1":0.6046374060545386, "value2":6467693625848837489, "value3":false, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741291, "value1":0.4689122935380642, "value2":8701125505618349182, "value3":true, "__name__":"metric_00000046", "key_6":"h","key_2":"k","key_4":"e", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.877913, "value1":0.596765963129067, "value2":8094571743010403516, "value3":true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.790394, "value1":0.06093826578205693, "value2":733010313976904838, "value3":true, "__name__":"metric_00000048", "key_9":"b","key_1":"e", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603208, "value1":0.8364659181351286, "value2":1088146971265159206, "value3":true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823621, "value1":0.47775163306986124, "value2":7344279199603749996, "value3":false, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171666, "value1":0.02782471041039711, "value2":1659046247294927834, "value3":true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.899246, "value1":0.23664326650032677, "value2":764233009249486091, "value3":true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658885, "value1":0.13203154011115373, "value2":5571262153472749697, "value3":false, "__name__":"metric_00000064", "key_4":"a","key_7":"k","key_1":"j", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.711786, "value1":0.6314799048378197, "value2":8955079539774261393, "value3":true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.358690, "value1":0.4626190815088963, "value2":5349880367436474865, "value3":false, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436669, "value1":0.7604788504004301, "value2":2830736413446294733, "value3":true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283630, "value1":0.11435547311453685, "value2":5764151651661369748, "value3":false, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550602, "value1":0.9394300496620045, "value2":3385128630739961936, "value3":false, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004479, "value1":0.08103496493647538, "value2":1096104398248692524, "value3":false, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272179, "value1":0.007258870887977933, "value2":2565647835692463774, "value3":false, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.207218, "value1":0.8139650845703748, "value2":905077995346686402, "value3":true, "__name__":"metric_00000072", "key_3":"g","key_9":"k","key_0":"g", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217842, "value1":0.48125835395143424, "value2":6789722720604854599, "value3":true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.155093, "value1":0.7189778774843847, "value2":6292044296017930854, "value3":false, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723501, "value1":0.3703438091023322, "value2":3955334986917106318, "value3":true, "__name__":"metric_00000075", "key_9":"f","key_3":"j","key_5":"i", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216869, "value1":0.12053628422917362, "value2":636444633907082482, "value3":true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.568591, "value1":0.23735656022841886, "value2":4679225743620568386, "value3":false, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.792289, "value1":0.8769118352474913, "value2":8705903369099981004, "value3":true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.377367, "value1":0.9028525772471804, "value2":8394233771809162273, "value3":true, "__name__":"metric_00000079", "key_3":"d","key_6":"c","key_0":"d", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511370, "value1":0.06603888812647277, "value2":1587071294143636433, "value3":true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060860, "value1":0.3291125358805135, "value2":7312301032222805401, "value3":true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816308, "value1":0.6388914782369252, "value2":1128303197769336870, "value3":true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.068805, "value1":0.319277617656504, "value2":6681946261878539245, "value3":true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.805301, "value1":0.009280749666995351, "value2":544578594920738627, "value3":false, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493659, "value1":0.6717590147189912, "value2":2097325929268646339, "value3":false, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.253857, "value1":0.2354585485783823, "value2":5182832084934966477, "value3":false, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454134, "value1":0.7059786460183258, "value2":8734869026681606253, "value3":false, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769480, "value1":0.16571806998598893, "value2":1451771174019239273, "value3":false, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.043302, "value1":0.0805122859716286, "value2":8615098369512434691, "value3":true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493432, "value1":0.37562446579716846, "value2":447590635610718249, "value3":true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.340933, "value1":0.476702778910786, "value2":2962733058848207480, "value3":false, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.351270, "value1":0.04893100925940487, "value2":3216639061913163460, "value3":false, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819062, "value1":0.43774247002290106, "value2":6810500585509381104, "value3":true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.789988, "value1":0.13044652854726224, "value2":6447819044361282673, "value3":false, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.137973, "value1":0.9280703615023543, "value2":2854967575973809701, "value3":true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359816, "value1":0.6661196608352099, "value2":7585154254761447569, "value3":false, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.374877, "value1":0.2719104405916207, "value2":1097487272974828210, "value3":true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766465, "value1":0.8656426220974218, "value2":2886501368125017740, "value3":false, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267362, "value1":0.594401465949902, "value2":1715419869280476267, "value3":true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.689698, "value1":0.09904556283985433, "value2":361858677456228074, "value3":true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244000, "value1":0.4739214533638735, "value2":6722075717885814318, "value3":true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957522, "value1":0.4793380026682715, "value2":8094832594089347058, "value3":true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578816, "value1":0.32497702056237604, "value2":8938729328568281395, "value3":false, "__name__":"metric_00000103", "key_6":"h","key_1":"f","key_5":"e", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999599, "value1":0.703313156867648, "value2":5755903281901537381, "value3":false, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829223, "value1":0.9601134377422823, "value2":4062518676536818057, "value3":true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.145200, "value1":0.7216961923243395, "value2":9217282407920967916, "value3":false, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820803, "value1":0.3093137747462916, "value2":7468155061834676810, "value3":true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682889, "value1":0.04485861766843415, "value2":4270651219964958841, "value3":true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549256, "value1":0.7924629761166294, "value2":5462651594092502924, "value3":true, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515774, "value1":0.1753099089980214, "value2":4245273970497284960, "value3":true, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280472, "value1":0.19807313433421816, "value2":6932983129449398217, "value3":true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.183908, "value1":0.8803355871960044, "value2":3795185008893112660, "value3":false, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.250576, "value1":0.8766744958241532, "value2":8316217539045644702, "value3":true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811589, "value1":0.7673136679205708, "value2":1656000207332411296, "value3":false, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617752, "value1":0.33750626542384204, "value2":2818737959409200195, "value3":true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.402465, "value1":0.5975891290632979, "value2":8117852232376404604, "value3":false, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.779395, "value1":0.6126614135133973, "value2":1972761209604651335, "value3":false, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171014, "value1":0.3147679970132471, "value2":3872394099215939892, "value3":false, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.602514, "value1":0.9979964274456362, "value2":8280861801559743509, "value3":true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.870286, "value1":0.1346288239954595, "value2":2167488486464723184, "value3":true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.902696, "value1":0.53464205049288, "value2":8019370943564861440, "value3":true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050828, "value1":0.7038559591596831, "value2":6864265904757939564, "value3":true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.465826, "value1":0.7686595552419602, "value2":1927886873036321447, "value3":true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149509, "value1":0.9153544198103084, "value2":7814774915075726621, "value3":true, "__name__":"metric_00000116", "key_9":"f","key_0":"k","key_1":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.485222, "value1":0.9188620757546552, "value2":5192187368195488529, "value3":true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520165, "value1":0.1037499454739713, "value2":2861179515410191072, "value3":true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894795, "value1":0.6670064141722539, "value2":8570373623610344103, "value3":true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858375, "value1":0.9690125335533832, "value2":6598305524856472910, "value3":false, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639161, "value1":0.07029962179108588, "value2":4198012464202851461, "value3":true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.422856, "value1":0.243592288273798, "value2":7060563137853890653, "value3":true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356376, "value1":0.12506395613411664, "value2":3055799774165215720, "value3":false, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296059, "value1":0.7972570100142576, "value2":2611996617533897408, "value3":true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.461343, "value1":0.024502184861611945, "value2":1123093231220631621, "value3":false, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529133, "value1":0.12083989402126777, "value2":7219868115171352433, "value3":true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110681, "value1":0.8049351220640796, "value2":4305443631669757518, "value3":true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.981566, "value1":0.19931635115078358, "value2":4376781646759017505, "value3":true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631835, "value1":0.4428488032269089, "value2":2962840158667509272, "value3":true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596817, "value1":0.01388179851872705, "value2":5526059113621970146, "value3":true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988044, "value1":0.08966434896908937, "value2":845094337456207269, "value3":false, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032366, "value1":0.4632294657773622, "value2":2896178627081634846, "value3":true, "__name__":"metric_00000132", "key_9":"i","key_0":"i","key_3":"a", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.195349, "value1":0.3544490738754114, "value2":5485050023862879453, "value3":false, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.488827, "value1":0.05815655955779658, "value2":5446494658153199948, "value3":false, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723246, "value1":0.6834006388190103, "value2":4274252704044712919, "value3":true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141449, "value1":0.3967110267075465, "value2":2849522299224594762, "value3":true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.712863, "value1":0.2643526520952433, "value2":6055035031034459665, "value3":false, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974542, "value1":0.07826746989883603, "value2":8013938940173093770, "value3":false, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476496, "value1":0.08282820679282239, "value2":8740534065670450787, "value3":true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.265568, "value1":0.7141009753996177, "value2":8015024495118709084, "value3":false, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163000, "value1":0.9787912408769939, "value2":1407010919457389562, "value3":false, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560997, "value1":0.22142486556739396, "value2":2439658579639630159, "value3":true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325355, "value1":0.2775049821475937, "value2":2346830518660166745, "value3":false, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491491, "value1":0.3135314864967855, "value2":1129771089143019455, "value3":true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.459317, "value1":0.2812010467625179, "value2":7830625013744386941, "value3":false, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.684694, "value1":0.4935589110227349, "value2":2540823529765104488, "value3":false, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.237886, "value1":0.07014579422912577, "value2":7022680572815993337, "value3":false, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.414046, "value1":0.06496797245021041, "value2":2711277679569543532, "value3":false, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799639, "value1":0.8106870375099371, "value2":1007449970411192794, "value3":false, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.264752, "value1":0.0960343886163872, "value2":3312684645465619564, "value3":true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035469, "value1":0.0008360965935936516, "value2":5539611781321403864, "value3":true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.306352, "value1":0.07245144250837937, "value2":3686038376310116060, "value3":false, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.048629, "value1":0.3482697350949442, "value2":2011496763457379089, "value3":true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978673, "value1":0.9982728323868918, "value2":702860377347588565, "value3":false, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987564, "value1":0.21515438859453273, "value2":2583082073184523816, "value3":true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.842602, "value1":0.3444834642501587, "value2":4908475207072135082, "value3":false, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966769, "value1":0.7858094998520652, "value2":6077992589658011123, "value3":true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.165450, "value1":0.6383111602434741, "value2":7226409376141032655, "value3":true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.575923, "value1":0.8562665414193692, "value2":2173652645149705334, "value3":true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010707, "value1":0.5955696347147411, "value2":5814682382951778842, "value3":true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271135, "value1":0.1229263700863702, "value2":6350918748538162755, "value3":false, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143782, "value1":0.975371469047403, "value2":8279280670157614673, "value3":false, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536001, "value1":0.2818838543932868, "value2":5548964524114515649, "value3":false, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.274203, "value1":0.525661345793623, "value2":7632566212944122955, "value3":true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394070, "value1":0.7913688238824117, "value2":2991067692464176886, "value3":false, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243187, "value1":0.9989819654882474, "value2":7814314371764134982, "value3":true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490869, "value1":0.30039262696070124, "value2":7927229014613646504, "value3":false, "__name__":"metric_00000166", "key_6":"i","key_0":"f","key_5":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.850916, "value1":0.8481086988896461, "value2":3502125731621593449, "value3":false, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113680, "value1":0.32930706694057965, "value2":6445034393695608409, "value3":true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192297, "value1":0.3973423811708676, "value2":8167271156640822516, "value3":false, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646981, "value1":0.6421917320177064, "value2":2441775949474583008, "value3":true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.482724, "value1":0.3152543309412001, "value2":8846147135578854122, "value3":true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777161, "value1":0.26611205622992573, "value2":4707069095348547526, "value3":false, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529669, "value1":0.25143936257925226, "value2":9113227763618672958, "value3":false, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708875, "value1":0.10515643642729285, "value2":7582828897771165151, "value3":false, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041864, "value1":0.7204022074713844, "value2":4505351249313209540, "value3":true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.517118, "value1":0.2778141202519239, "value2":7041089660646534661, "value3":true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255695, "value1":0.5252045968012302, "value2":1609597628350217466, "value3":true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.001450, "value1":0.15419422555422022, "value2":6306687575854728640, "value3":false, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.230403, "value1":0.7229181740004053, "value2":1306679799234664681, "value3":false, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035841, "value1":0.6895973487797646, "value2":2559143873675894370, "value3":false, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.449904, "value1":0.7368166767257398, "value2":7052097343090100521, "value3":false, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139346, "value1":0.807172748525886, "value2":4308240895194146306, "value3":false, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977592, "value1":0.7560595694496225, "value2":3367349773561100464, "value3":false, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.831129, "value1":0.43661553653957313, "value2":1514573148689105923, "value3":true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804508, "value1":0.7956852333376209, "value2":8820887809742814806, "value3":true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858280, "value1":0.8493727617212654, "value2":8684654170930913793, "value3":false, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640988, "value1":0.2223414127619492, "value2":7250214226981174458, "value3":true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153838, "value1":0.9864893077863718, "value2":1366599113373589549, "value3":false, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144317, "value1":0.034179582463187676, "value2":4811674883286124820, "value3":true, "__name__":"metric_00000186", "key_4":"g","key_7":"h","key_1":"k", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.670072, "value1":0.3811265659697653, "value2":4090252794464650298, "value3":false, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486186, "value1":0.9084897972797951, "value2":2292875585752622652, "value3":false, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724703, "value1":0.04582892017411178, "value2":5556549536478579022, "value3":true, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012322, "value1":0.6305358760062275, "value2":1288975268472701530, "value3":false, "__name__":"metric_00000192", "key_6":"a","key_8":"f","key_0":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.218849, "value1":0.8926861107470235, "value2":7051558642534889749, "value3":false, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.577242, "value1":0.96150058991849, "value2":5205402300881087071, "value3":true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525938, "value1":0.010643579624627687, "value2":9179800346032814438, "value3":false, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393624, "value1":0.5282994953423363, "value2":6629726373755372175, "value3":false, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697100, "value1":0.328819156896584, "value2":7890032700245067940, "value3":false, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.911016, "value1":0.23451404605523937, "value2":1559532623894065416, "value3":false, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128593, "value1":0.958219222491235, "value2":5238496228333042095, "value3":false, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704394, "value1":0.5513948920447201, "value2":9062299692444161349, "value3":true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255075, "value1":0.35898456496590414, "value2":5087989108776558277, "value3":true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788201, "value1":0.7466905262312481, "value2":479894845243154201, "value3":true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565236, "value1":0.15707319443827472, "value2":93010754190809493, "value3":false, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462851, "value1":0.1393959893683005, "value2":1383438862980713878, "value3":false, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.587928, "value1":0.9257613742750218, "value2":461452136435423636, "value3":false, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.204910, "value1":0.45361150342145223, "value2":8114896084179399798, "value3":true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970393, "value1":0.8262379198959926, "value2":1410864297783479414, "value3":false, "__name__":"metric_00000207", "key_1":"e","key_3":"g","key_0":"k", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571697, "value1":0.05523474900350812, "value2":6912470816433257487, "value3":false, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900965, "value1":0.3994455375010903, "value2":5910800813383852824, "value3":false, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533983, "value1":0.8439578978089138, "value2":6520871065252687028, "value3":false, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732970, "value1":0.4096847894313233, "value2":4961560961953820016, "value3":false, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.512990, "value1":0.9925070463343235, "value2":4031040144184448394, "value3":false, "__name__":"metric_00000212", "key_9":"j","key_3":"b","key_5":"a", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.875107, "value1":0.8529856732344743, "value2":8171410486624791720, "value3":true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.662542, "value1":0.7969197657043129, "value2":4707105172837538302, "value3":false, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.132562, "value1":0.5095481874741737, "value2":8469586257322312509, "value3":false, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171675, "value1":0.7140266335782444, "value2":8998618655550288053, "value3":false, "__name__":"metric_00000217", "key_9":"h","key_1":"h","key_4":"a", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829529, "value1":0.8556707112048024, "value2":84360751498479327, "value3":true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.383509, "value1":0.6267421890569228, "value2":3548620263321255935, "value3":true, "__name__":"metric_00000219", "key_5":"k","key_7":"h","key_1":"a", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725412, "value1":0.6792289410958199, "value2":8743030074795737682, "value3":true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217960, "value1":0.06523873388277363, "value2":4237433473740985929, "value3":true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.052828, "value1":0.39596522726411565, "value2":5268673916985408378, "value3":false, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243583, "value1":0.24855400987123744, "value2":7145331513033105803, "value3":true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.643757, "value1":0.4621121441928861, "value2":4607779943623171423, "value3":false, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777085, "value1":0.9713754435345697, "value2":3872286492266737542, "value3":false, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741256, "value1":0.33772530129863026, "value2":2720260557973633476, "value3":true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040901, "value1":0.5366678442503715, "value2":5043767454211598671, "value3":true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.566517, "value1":0.3199753043754913, "value2":1787736737657465265, "value3":false, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871703, "value1":0.40150570806322944, "value2":7426113550614212915, "value3":true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.063390, "value1":0.8093881999536079, "value2":6062591517984503571, "value3":false, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630695, "value1":0.5987756911109672, "value2":4564890959914284940, "value3":false, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089316, "value1":0.9756718556017356, "value2":4422554680156925050, "value3":true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433679, "value1":0.6504367128725771, "value2":7179863028410040853, "value3":false, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418340, "value1":0.5167928405286337, "value2":796797443554380087, "value3":false, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148063, "value1":0.2982285613686749, "value2":3455828331199929793, "value3":false, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.562273, "value1":0.1744043124258891, "value2":7056453619952557866, "value3":false, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554680, "value1":0.9985428971035419, "value2":3134607000357482375, "value3":true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.498692, "value1":0.34351042666983833, "value2":9219919937586266788, "value3":false, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983651, "value1":0.20667520837153053, "value2":8261113453188452260, "value3":false, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900233, "value1":0.49430245377625404, "value2":1887699130768202184, "value3":false, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.269041, "value1":0.24848195612656065, "value2":27297122896884729, "value3":true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.716521, "value1":0.703324321123364, "value2":5138993531610984864, "value3":true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.331287, "value1":0.6585119514499647, "value2":7805239120031598554, "value3":true, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.898740, "value1":0.7897434254535518, "value2":8577413153173117857, "value3":false, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366667, "value1":0.4615000106211479, "value2":4478388423093675456, "value3":false, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060808, "value1":0.3611416577570406, "value2":1763611736817673526, "value3":false, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987608, "value1":0.6511729153448047, "value2":7895118250604250511, "value3":true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.920802, "value1":0.8966340709602193, "value2":1619385988446426949, "value3":false, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759468, "value1":0.9772199770736059, "value2":486666067155730811, "value3":false, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697569, "value1":0.2215623962484913, "value2":998498599369301173, "value3":false, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550984, "value1":0.23501917809435333, "value2":203018852406902455, "value3":false, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865253, "value1":0.24390812385740993, "value2":2606627451901201346, "value3":false, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050134, "value1":0.6480270130439445, "value2":6247279314207653751, "value3":false, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614221, "value1":0.7050465104295126, "value2":3377246595104905199, "value3":false, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166533, "value1":0.3877660847452995, "value2":7536153185490054501, "value3":true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644807, "value1":0.7190092729952982, "value2":5813482232408839023, "value3":true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739661, "value1":0.7257627180887302, "value2":3214351184373245408, "value3":false, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552516, "value1":0.4939513402753451, "value2":465350226895561432, "value3":false, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503358, "value1":0.14929982411048387, "value2":2610344885697195594, "value3":false, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.765487, "value1":0.45537690586006585, "value2":4288868163982148480, "value3":false, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.838240, "value1":0.7605183341936481, "value2":7774940282079152902, "value3":true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646978, "value1":0.988755613370451, "value2":4713140451733570872, "value3":true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746410, "value1":0.8174040579458727, "value2":7395122770602669759, "value3":true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823505, "value1":0.758453176804426, "value2":7518087779323040688, "value3":false, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020233, "value1":0.29899993635819666, "value2":5085917800846866623, "value3":false, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561926, "value1":0.8862149028157266, "value2":3614075635724672575, "value3":true, "__name__":"metric_00000275", "key_2":"i","key_6":"f","key_0":"j", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.437110, "value1":0.32845862992503655, "value2":2977427326936422575, "value3":false, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706546, "value1":0.15047181179887134, "value2":2307845458700050829, "value3":false, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.619350, "value1":0.1117344044076149, "value2":4988357694886705006, "value3":true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.808461, "value1":0.715214325361221, "value2":7172177171113874947, "value3":false, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154987, "value1":0.3436009437038216, "value2":35294647035701645, "value3":true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.293185, "value1":0.1982820741163276, "value2":4721034174581841393, "value3":true, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.177501, "value1":0.24102187665043326, "value2":5183952097737049563, "value3":true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434009, "value1":0.4074440055222358, "value2":2740366314909969021, "value3":true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.962158, "value1":0.6945642879060417, "value2":8488893912024069218, "value3":false, "__name__":"metric_00000284", "key_3":"b","key_7":"e","key_0":"b", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037976, "value1":0.05977827983958813, "value2":8714018046983615411, "value3":true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491879, "value1":0.5875339799333666, "value2":3507531774420107282, "value3":true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708852, "value1":0.941564068848663, "value2":7441568589430319196, "value3":true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665427, "value1":0.15650940720497003, "value2":2725018303601038315, "value3":true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728790, "value1":0.16909603056430217, "value2":7913664106090409634, "value3":false, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748451, "value1":0.9309585511638759, "value2":6423846277129286464, "value3":false, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701551, "value1":0.2886585284950581, "value2":1577558950874330311, "value3":false, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356768, "value1":0.42594704925771354, "value2":3150589955894640969, "value3":true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056847, "value1":0.8050681337140542, "value2":3504760629437797890, "value3":false, "__name__":"metric_00000293", "key_9":"k","key_4":"b","key_7":"e", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.797591, "value1":0.9381497767913094, "value2":8069233160965701213, "value3":true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.653188, "value1":0.21115616320062328, "value2":2091006864800018643, "value3":false, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489289, "value1":0.6249865113250967, "value2":3636072468848176197, "value3":true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.426149, "value1":0.7525465264386705, "value2":2554715790387615481, "value3":true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629922, "value1":0.244780100518832, "value2":2525567513505106874, "value3":true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.652167, "value1":0.44363110594266053, "value2":2074305704102402281, "value3":true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111575, "value1":0.7142448949298847, "value2":2599619943612027614, "value3":true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.720934, "value1":0.46314177190410183, "value2":5473312180675819943, "value3":false, "__name__":"metric_00000240", "key_8":"d","key_9":"c","key_4":"d", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.542023, "value1":0.17074635857117051, "value2":576994586783200158, "value3":false, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862606, "value1":0.849600128173387, "value2":4044011490168844428, "value3":false, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.581818, "value1":0.4717710471390932, "value2":2021524622009984217, "value3":false, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.634138, "value1":0.3045108371861026, "value2":769588972141174181, "value3":false, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554282, "value1":0.9061849342045651, "value2":5246859609213477778, "value3":false, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.275353, "value1":0.12209379665466558, "value2":3903264969659125855, "value3":false, "__name__":"metric_00000306", "key_8":"f","key_9":"i","key_5":"d", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931382, "value1":0.38822648110972485, "value2":4321311553224557998, "value3":true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.703867, "value1":0.6637597201672076, "value2":5683430536850722079, "value3":false, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958497, "value1":0.011335239730124937, "value2":2491874770397540377, "value3":true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569325, "value1":0.8472934144463635, "value2":6850134665353105473, "value3":false, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564173, "value1":0.28056996036761495, "value2":1661710999114510072, "value3":false, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.046264, "value1":0.7988719698656381, "value2":80936298017494610, "value3":false, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.086050, "value1":0.7269357269059649, "value2":3703081511929708827, "value3":true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.691310, "value1":0.968367674569065, "value2":8685309070712113009, "value3":true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.734580, "value1":0.16065685872806734, "value2":8341255006788851479, "value3":false, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110471, "value1":0.2857648075220212, "value2":7895547866112316696, "value3":false, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.594791, "value1":0.6011580461407761, "value2":8667655502460513802, "value3":false, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.074613, "value1":0.7518175733891237, "value2":6808912328433269494, "value3":false, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827510, "value1":0.4635989764886008, "value2":8535571864109924880, "value3":false, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020243, "value1":0.9625575743602495, "value2":3255772979573003294, "value3":true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508001, "value1":0.9246513785407203, "value2":6789019308045406067, "value3":false, "__name__":"metric_00000315", "key_3":"e","key_2":"b", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.175770, "value1":0.6231844141584031, "value2":5604032128383909386, "value3":false, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338187, "value1":0.7085089535016765, "value2":7296894563489025180, "value3":true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924177, "value1":0.9072759724228442, "value2":2056681116966727911, "value3":false, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474372, "value1":0.09374734312421011, "value2":8184327150731414451, "value3":true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970300, "value1":0.9441105421259884, "value2":6689514820107172641, "value3":false, "__name__":"metric_00000317", "key_4":"f","key_9":"f","key_0":"e", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035348, "value1":0.14347791413195674, "value2":2423771496925044659, "value3":true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.807051, "value1":0.1533684521364213, "value2":8351770123785179919, "value3":true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814282, "value1":0.48165153274600025, "value2":2348575893675627866, "value3":false, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438370, "value1":0.4065426353146516, "value2":5566206066472585427, "value3":false, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.321909, "value1":0.057863583124824335, "value2":2888660799506273189, "value3":false, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.191383, "value1":0.5857228209038977, "value2":343682983842502232, "value3":false, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.030645, "value1":0.054007147421613756, "value2":7408821068820948342, "value3":false, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.022320, "value1":0.5082171149433462, "value2":708896583672613900, "value3":true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359147, "value1":0.10928795884420436, "value2":5509040341747881439, "value3":true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245107, "value1":0.08990020039433881, "value2":4415685661085482000, "value3":false, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.685276, "value1":0.9327230470765273, "value2":7932626821430255186, "value3":false, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160518, "value1":0.9697823127450904, "value2":9176665002489756811, "value3":true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851145, "value1":0.3835892105704814, "value2":5122340871731266104, "value3":false, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390684, "value1":0.8638095835380812, "value2":2196309567306303445, "value3":false, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197750, "value1":0.5321904528249127, "value2":7059427737086849843, "value3":true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.606374, "value1":0.1712913717696049, "value2":6671507460576496016, "value3":false, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814189, "value1":0.21162810005419255, "value2":6089699189069603789, "value3":true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565029, "value1":0.651039299686122, "value2":2488735588911638142, "value3":false, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310043, "value1":0.3341800696393593, "value2":1886591498798226069, "value3":true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646773, "value1":0.587523277238801, "value2":3984875805092838666, "value3":true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487672, "value1":0.4085632878496485, "value2":4006763171436697564, "value3":true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180539, "value1":0.614724172390471, "value2":3474361726777387017, "value3":false, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413423, "value1":0.8015313942257714, "value2":3699130733479555941, "value3":false, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375412, "value1":0.6194400337512462, "value2":89560215401334488, "value3":true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739736, "value1":0.01565923183582112, "value2":2040823465506133480, "value3":true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164236, "value1":0.05609752284300993, "value2":459144572970603788, "value3":true, "__name__":"metric_00000343", "key_5":"a","key_8":"g","key_3":"b", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857605, "value1":0.3918518578057805, "value2":2660076743726482223, "value3":false, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182324, "value1":0.19737269797519194, "value2":4090312209234045823, "value3":true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413638, "value1":0.23128603835680528, "value2":2432247666638343859, "value3":true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029664, "value1":0.7948810691175306, "value2":1090049786919167701, "value3":true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953471, "value1":0.7020043042253941, "value2":4064796540960008955, "value3":false, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925306, "value1":0.8273248639036314, "value2":7383533353614933778, "value3":true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.353560, "value1":0.7154582581173291, "value2":7672433526090470392, "value3":false, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.440133, "value1":0.8443212771497766, "value2":4871633339564000823, "value3":true, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079309, "value1":0.9841787259191465, "value2":5158435517537975833, "value3":false, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687146, "value1":0.547850782348478, "value2":107890618456621161, "value3":true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013887, "value1":0.9829126596506367, "value2":2398801344797864345, "value3":true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142865, "value1":0.17685569327146947, "value2":2003576222991068603, "value3":true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503089, "value1":0.2089707954081848, "value2":5329263674364077685, "value3":false, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055628, "value1":0.9427569700789281, "value2":8376349363379609336, "value3":false, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663522, "value1":0.5447970553906242, "value2":7034169322760923664, "value3":true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569193, "value1":0.4748210869235274, "value2":5009934188959179459, "value3":false, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.964851, "value1":0.1480433642451733, "value2":5780759776802899813, "value3":false, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.674266, "value1":0.01926887357642196, "value2":5539462413096535, "value3":false, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397018, "value1":0.3343175727423989, "value2":8363367362679630761, "value3":true, "__name__":"metric_00000364", "key_8":"c","key_2":"e","key_7":"k", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665761, "value1":0.9909509158985629, "value2":598831903509406706, "value3":true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225685, "value1":0.9692539977538356, "value2":6753560478082272468, "value3":true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857888, "value1":0.3089073345023022, "value2":469238271712070571, "value3":false, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296144, "value1":0.8953425439665634, "value2":5808544938153527415, "value3":false, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641028, "value1":0.26770927578250153, "value2":4804718717106479393, "value3":true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865055, "value1":0.21277992187222125, "value2":7652322888419294762, "value3":true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.147216, "value1":0.4361812114988848, "value2":5453489060842625813, "value3":false, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931652, "value1":0.9338772117034952, "value2":7926554101044226499, "value3":false, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451647, "value1":0.49187863740688065, "value2":626417169671954650, "value3":false, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552836, "value1":0.08117641553234622, "value2":6479037192065714806, "value3":true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644667, "value1":0.8782871355247011, "value2":7618066153253340565, "value3":true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486533, "value1":0.34858818937194813, "value2":8450118240549444684, "value3":false, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080371, "value1":0.1720187423008763, "value2":7683320479544120606, "value3":true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527916, "value1":0.09000309934383972, "value2":5784629514237788639, "value3":true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682327, "value1":0.9194439212395367, "value2":4269189023261883692, "value3":true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.961106, "value1":0.4405062734385544, "value2":1799260749356167755, "value3":true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640959, "value1":0.04934157301568955, "value2":4101326969656042416, "value3":true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827544, "value1":0.6093692966542871, "value2":5408051744545184279, "value3":true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.916148, "value1":0.37486434605250213, "value2":8005097965237384093, "value3":true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477104, "value1":0.7823899457952176, "value2":2425125801220437344, "value3":true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631094, "value1":0.38182215652170554, "value2":6885679048992226816, "value3":true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242126, "value1":0.897478447775271, "value2":5819274494025981203, "value3":false, "__name__":"metric_00000386", "key_4":"g","key_8":"b","key_2":"c", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134175, "value1":0.8364024700095963, "value2":2669419474556636937, "value3":true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.785456, "value1":0.47900969825766393, "value2":427990087950056586, "value3":false, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.702759, "value1":0.4509768057551493, "value2":6899018348994248896, "value3":true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.844804, "value1":0.4506687041971474, "value2":2704465019491437568, "value3":true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841097, "value1":0.08259262088200636, "value2":3254311166592288697, "value3":false, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.579894, "value1":0.634370403808335, "value2":8239103463468673989, "value3":true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.671756, "value1":0.3978266397618443, "value2":2011463990612690211, "value3":true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639712, "value1":0.5843126119887666, "value2":8514657269434627024, "value3":true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.090535, "value1":0.47166906237349404, "value2":2986627777164864344, "value3":true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079733, "value1":0.49607189078610653, "value2":7935174201590062011, "value3":false, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484442, "value1":0.10087228739324149, "value2":5758223217302730640, "value3":true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.881297, "value1":0.743802063178793, "value2":7666746241262123009, "value3":true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189244, "value1":0.8471771982112483, "value2":8717600770763450338, "value3":false, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958729, "value1":0.24046081337833752, "value2":2429080107883352722, "value3":true, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904544, "value1":0.5967867401359257, "value2":5915623307556355066, "value3":false, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.016356, "value1":0.2664341494092446, "value2":6082384923182522956, "value3":false, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906737, "value1":0.8646356549293164, "value2":6093331013624350146, "value3":true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.650026, "value1":0.17820070872685148, "value2":2506978860590163255, "value3":false, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.380897, "value1":0.559051486368018, "value2":959051280528283072, "value3":false, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.645954, "value1":0.44113348983757733, "value2":3640682441395107897, "value3":false, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.705006, "value1":0.7603563234560595, "value2":185556188647790850, "value3":false, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.946294, "value1":0.6649942159160177, "value2":8061815176820463265, "value3":true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.007871, "value1":0.472486274082402, "value2":8110170220979486982, "value3":false, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.817193, "value1":0.01062083501470273, "value2":8652411971395108438, "value3":false, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.203058, "value1":0.2545105965864827, "value2":3692372485982923495, "value3":false, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724358, "value1":0.49614729928160556, "value2":794673815448737267, "value3":true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029363, "value1":0.8128873282396352, "value2":5575910506626889012, "value3":false, "__name__":"metric_00000412", "key_3":"b","key_6":"b","key_1":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.388594, "value1":0.37922666609989497, "value2":5857319265944344227, "value3":true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111761, "value1":0.07280095456744823, "value2":7190870932028618354, "value3":false, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556559, "value1":0.011144718316818122, "value2":5637958779343170502, "value3":true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900852, "value1":0.2436048429102491, "value2":3178328441413468775, "value3":false, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746269, "value1":0.44159443384096786, "value2":1064726188435771697, "value3":false, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089137, "value1":0.5802300524767475, "value2":1179475447484216893, "value3":false, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045149, "value1":0.8412834417368987, "value2":2843566571645769718, "value3":true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032148, "value1":0.9106893921082526, "value2":2375027858901255873, "value3":false, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180832, "value1":0.8856494080629095, "value2":2259505020949178568, "value3":false, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571194, "value1":0.2087472777529694, "value2":3787711915274839208, "value3":true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113569, "value1":0.6253289147788459, "value2":2290822587909135561, "value3":false, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708878, "value1":0.3051231954056551, "value2":536614295748659218, "value3":false, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397834, "value1":0.004032765819560939, "value2":2143090210108317054, "value3":true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.279982, "value1":0.6698352738967139, "value2":8265876412027179825, "value3":false, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.117595, "value1":0.36898116171165335, "value2":7917937119372643007, "value3":true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490128, "value1":0.7119026496334794, "value2":4217226709077888871, "value3":false, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.337373, "value1":0.24700927278867024, "value2":8633933010561287594, "value3":false, "__name__":"metric_00000430", "key_4":"c","key_9":"j","key_1":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393403, "value1":0.3187858750570018, "value2":7155530342572901954, "value3":true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787897, "value1":0.731243787879042, "value2":3162978630167698275, "value3":true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394636, "value1":0.15744716640466827, "value2":6804804340000012886, "value3":false, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589125, "value1":0.9206991440936233, "value2":3240974047558242833, "value3":true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122199, "value1":0.9027140609185327, "value2":4793574504420000662, "value3":false, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739368, "value1":0.0296490298815606, "value2":7450276243203379932, "value3":true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839046, "value1":0.885358696687765, "value2":2580404634255699486, "value3":true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432366, "value1":0.6379661863831838, "value2":4996786498907777190, "value3":true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129751, "value1":0.5622916100360914, "value2":8341777138936467511, "value3":false, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202469, "value1":0.24574613351572672, "value2":7146206121672605367, "value3":true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.832142, "value1":0.5178470928944944, "value2":4133801221430187756, "value3":true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080311, "value1":0.486175008187209, "value2":5039371713095786937, "value3":true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723875, "value1":0.552001350718302, "value2":7101322585254858336, "value3":true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434143, "value1":0.6525700542145968, "value2":2081065923746015157, "value3":true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.410329, "value1":0.15418313782350743, "value2":2121964379701341165, "value3":false, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423463, "value1":0.984744888870281, "value2":7148485292923472325, "value3":false, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658931, "value1":0.9086990564903296, "value2":4792038160363904869, "value3":true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.891060, "value1":0.6990632896148243, "value2":8537384774472959390, "value3":true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154122, "value1":0.10983855840529697, "value2":3710113544658117053, "value3":true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.543143, "value1":0.3045282199429521, "value2":6184622652371484263, "value3":true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174103, "value1":0.1622180223685842, "value2":1046120514820502820, "value3":false, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393987, "value1":0.12110279784132964, "value2":5830532384578071588, "value3":false, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907289, "value1":0.9509521630571898, "value2":4933206466498891083, "value3":true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987070, "value1":0.5242278192300958, "value2":7374919878503376521, "value3":true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641701, "value1":0.3760479049716257, "value2":4296425671360614419, "value3":false, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.460082, "value1":0.7715932085121491, "value2":4216486260610422297, "value3":false, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.308698, "value1":0.6733032497636853, "value2":8314310095556752952, "value3":false, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356337, "value1":0.9180371571261539, "value2":7442497884658250592, "value3":true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407046, "value1":0.8826200541831951, "value2":6272206334208867752, "value3":false, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263547, "value1":0.5493018603625203, "value2":5332141292811821371, "value3":false, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.690903, "value1":0.6510082736025081, "value2":2282336418703477702, "value3":false, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599129, "value1":0.21529129751322154, "value2":5758745763053757888, "value3":true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.294727, "value1":0.17537508292575937, "value2":1562987372702839556, "value3":false, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905165, "value1":0.5372817599107224, "value2":5788710138851546437, "value3":true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714618, "value1":0.5147887871360798, "value2":2734992135513585397, "value3":false, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.845879, "value1":0.03659856815217442, "value2":717088561981717268, "value3":true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.172178, "value1":0.5465356547159791, "value2":6019732615424773946, "value3":true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263201, "value1":0.7472793901983326, "value2":6735352007287694901, "value3":false, "__name__":"metric_00000467", "key_4":"i","key_8":"a","key_2":"d", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418982, "value1":0.029853021112705932, "value2":7962917578850181628, "value3":true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.866946, "value1":0.10874512532228968, "value2":4523576734362004451, "value3":true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556178, "value1":0.3725090999387309, "value2":8591394434164967395, "value3":false, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827178, "value1":0.564605338181153, "value2":1565791376780985293, "value3":true, "__name__":"metric_00000472", "key_9":"d","key_5":"b","key_8":"j", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.427240, "value1":0.5937818761029834, "value2":725738161862814755, "value3":true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153851, "value1":0.6812022447305227, "value2":2627803719401441193, "value3":false, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982516, "value1":0.5197201488713766, "value2":5245967664424169262, "value3":true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.922654, "value1":0.8797142069534722, "value2":2538441830233727435, "value3":true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036732, "value1":0.8882312994752156, "value2":2860925793915692434, "value3":false, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.848915, "value1":0.5650391916990911, "value2":8021099293155564160, "value3":false, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189564, "value1":0.5152547802767392, "value2":1569067174314315770, "value3":false, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.802112, "value1":0.9536154419324024, "value2":6565733445920939612, "value3":true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983922, "value1":0.4632188030903474, "value2":3429972812413807249, "value3":true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477177, "value1":0.333324845660299, "value2":4203450320031165809, "value3":true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097938, "value1":0.9171714255895188, "value2":2925569760522108797, "value3":true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.551984, "value1":0.23531625096597034, "value2":8336676757418711666, "value3":true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.774461, "value1":0.26483886652245114, "value2":1317358716847210463, "value3":true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354793, "value1":0.850445247905183, "value2":3123908181356206943, "value3":false, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.409594, "value1":0.7075083567078372, "value2":7315484295756660732, "value3":false, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407267, "value1":0.6525751933510557, "value2":5480727731388217494, "value3":false, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.572642, "value1":0.5899578957097297, "value2":5245828360681206097, "value3":false, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.731119, "value1":0.6986058457616005, "value2":4059549341372221062, "value3":true, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.627725, "value1":0.5600488168022775, "value2":2564410170230629806, "value3":true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050520, "value1":0.8920464631997315, "value2":9043850999622300098, "value3":false, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.500683, "value1":0.004883654125166552, "value2":8338418914599161440, "value3":false, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970722, "value1":0.01152412783847713, "value2":4977793912981679064, "value3":true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.860222, "value1":0.796875300130385, "value2":126152372728632215, "value3":false, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629467, "value1":0.658754492873807, "value2":2189248151613602825, "value3":false, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.098126, "value1":0.3838755732008427, "value2":8710340059607681652, "value3":true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812187, "value1":0.7527055571509035, "value2":4522503161830090202, "value3":true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303237, "value1":0.4170647409134182, "value2":1359570910802075414, "value3":true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794250, "value1":0.581379594095589, "value2":6842995986442641402, "value3":true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247703, "value1":0.7617287060418184, "value2":8955434883327208405, "value3":false, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.815768, "value1":0.5618395871174356, "value2":6646583475463132751, "value3":true, "__name__":"metric_00000501", "key_2":"i","key_5":"b","key_0":"d", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.248823, "value1":0.1954800995898102, "value2":1363474452253194433, "value3":false, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.531845, "value1":0.018792519031917622, "value2":2287281788624224406, "value3":false, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966028, "value1":0.08281088225013297, "value2":1000202598795297845, "value3":false, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.935610, "value1":0.3880941622449169, "value2":3231927844022227730, "value3":true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.268000, "value1":0.37669283923759284, "value2":1820767283408305591, "value3":true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.229141, "value1":0.6757864654636687, "value2":850049194951914043, "value3":true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.889660, "value1":0.14317979698283573, "value2":4562037574783866851, "value3":false, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836795, "value1":0.7422552273700601, "value2":1164639099163690154, "value3":true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.311980, "value1":0.6163196714525997, "value2":5427592556050607568, "value3":true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584578, "value1":0.38212773421861185, "value2":1414397003551807223, "value3":true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065172, "value1":0.5125177091208547, "value2":2833074963007555228, "value3":true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564271, "value1":0.5135791893267198, "value2":3074560175179976918, "value3":false, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894567, "value1":0.8429539964906779, "value2":3097821505618288757, "value3":true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.733587, "value1":0.9459948105676399, "value2":3812465080486684733, "value3":true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.053881, "value1":0.6018467371683773, "value2":5589588667277003729, "value3":false, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599521, "value1":0.5951991237753406, "value2":4485462521384926953, "value3":true, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354099, "value1":0.15573795731250434, "value2":8080731261812523696, "value3":true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969724, "value1":0.21276604577267808, "value2":8483165896229109105, "value3":true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263088, "value1":0.8705529546951637, "value2":1868660470516930885, "value3":true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.212304, "value1":0.28383507730046376, "value2":3678145555670791199, "value3":true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965823, "value1":0.23336598309503237, "value2":1387918772122086546, "value3":false, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536800, "value1":0.6576906678976834, "value2":6988018932226580717, "value3":false, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.884180, "value1":0.9109732193854059, "value2":2950864149550098491, "value3":true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448663, "value1":0.37035460819883087, "value2":6933572641209900634, "value3":false, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965580, "value1":0.9403308475710539, "value2":5776171572361995550, "value3":false, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.349421, "value1":0.19847751056585392, "value2":3418305989652553355, "value3":true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.469447, "value1":0.13572479625794354, "value2":4687020001075041801, "value3":true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393705, "value1":0.5764820211787193, "value2":296609812894901072, "value3":false, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862549, "value1":0.3023964248018198, "value2":7566431551834336763, "value3":false, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415447, "value1":0.07086082925570908, "value2":6961625726898421777, "value3":true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495354, "value1":0.5778296524778606, "value2":2860533724845138756, "value3":true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.510139, "value1":0.9905369504866598, "value2":7400245650080116394, "value3":false, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129326, "value1":0.3322893702950888, "value2":273820673011485924, "value3":false, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.793922, "value1":0.5795215776373468, "value2":7565118610448753571, "value3":false, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114305, "value1":0.4965968551084071, "value2":3076343444745895229, "value3":true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539515, "value1":0.7424425979048557, "value2":3570552565123324839, "value3":false, "__name__":"metric_00000546", "key_8":"b","key_9":"b","key_0":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714983, "value1":0.4711244304607692, "value2":5134628196697329789, "value3":true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182520, "value1":0.11684538125954054, "value2":4642866567682796714, "value3":true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701117, "value1":0.46816724142010296, "value2":8359014347252866849, "value3":true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135967, "value1":0.6405455196141, "value2":6186468831622665956, "value3":false, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.424454, "value1":0.5144728145894553, "value2":1160618526301999790, "value3":false, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476344, "value1":0.8109846569048221, "value2":1049565304185759462, "value3":false, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571700, "value1":0.5504531727903846, "value2":133387170146495310, "value3":false, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324604, "value1":0.16411596790106633, "value2":4350527762007764055, "value3":false, "__name__":"metric_00000554", "key_9":"g","key_1":"e","key_5":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272811, "value1":0.06793234743561845, "value2":3491914082935836765, "value3":false, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496778, "value1":0.2139795243463974, "value2":9108500124467919738, "value3":true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855922, "value1":0.8034347088304469, "value2":1547578413909373466, "value3":true, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.516953, "value1":0.46095603855138756, "value2":5334901216429463877, "value3":false, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.941610, "value1":0.9546476007247299, "value2":1561805534534904078, "value3":false, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157907, "value1":0.9790803369585773, "value2":7490355100154440772, "value3":false, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.667071, "value1":0.23582538943103912, "value2":6462516448359960159, "value3":true, "__name__":"metric_00000524", "key_3":"d","key_9":"f","key_2":"b", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.447846, "value1":0.08703603490291748, "value2":6137779903742056640, "value3":false, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.201754, "value1":0.703586761350304, "value2":5895955356127986300, "value3":false, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.214135, "value1":0.28977641931297293, "value2":2670270804816119209, "value3":true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171493, "value1":0.09189861336902101, "value2":3835094797774579940, "value3":false, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.507098, "value1":0.8469316808229229, "value2":4394084036001197157, "value3":true, "__name__":"metric_00000559", "key_2":"j","key_4":"d","key_1":"f", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.567706, "value1":0.4315878020065951, "value2":3026063068553888025, "value3":false, "__name__":"metric_00000530", "key_9":"f","key_3":"i","key_5":"c", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.369862, "value1":0.25873173058422444, "value2":2694491932093728275, "value3":false, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393535, "value1":0.2941316364368656, "value2":3384752395363156914, "value3":false, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354927, "value1":0.13513861393265603, "value2":8481431502383347556, "value3":true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.468119, "value1":0.5679212228606271, "value2":2713405380262941377, "value3":false, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163006, "value1":0.4227063637930681, "value2":6321042013913967311, "value3":true, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.656131, "value1":0.5736248371417144, "value2":2472629476600653129, "value3":true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511635, "value1":0.7728428788355186, "value2":7276523252477639153, "value3":false, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080037, "value1":0.9949587251210614, "value2":1148201472548354758, "value3":true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725064, "value1":0.05030431554866517, "value2":4579601693593819193, "value3":true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.342713, "value1":0.6797941637549237, "value2":2282393859292368923, "value3":true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012227, "value1":0.25217885012179175, "value2":2820047039844714200, "value3":true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.121409, "value1":0.08353369182495488, "value2":6648793005050735517, "value3":true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.944843, "value1":0.7935143781883287, "value2":1141746305838051122, "value3":false, "__name__":"metric_00000571", "key_7":"b","key_9":"b","key_1":"k", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216784, "value1":0.687358900553268, "value2":5413083464946109711, "value3":false, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.744704, "value1":0.26762670212162987, "value2":4557617436876999047, "value3":false, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.105367, "value1":0.7098778066529884, "value2":2505389065681570054, "value3":false, "__name__":"metric_00000574", "key_3":"k","key_2":"f", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164496, "value1":0.375746421410976, "value2":874151975168015904, "value3":false, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648692, "value1":0.26898807878557085, "value2":8174372151408578377, "value3":true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160502, "value1":0.8995428170685682, "value2":3027568952484727683, "value3":true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149122, "value1":0.0907853096527507, "value2":4677811628653537246, "value3":true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296780, "value1":0.3995862207774202, "value2":3598801053628990992, "value3":true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.464060, "value1":0.9713793366221031, "value2":4865123999270835737, "value3":false, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288334, "value1":0.6512177032146146, "value2":1792245924965210603, "value3":false, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.869907, "value1":0.007513793036623767, "value2":7995353402339020473, "value3":false, "__name__":"metric_00000582", "key_8":"i","key_1":"i","key_5":"e", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527381, "value1":0.5920004855495694, "value2":1386169961861528891, "value3":false, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.928125, "value1":0.7418903709677874, "value2":5060751330275548305, "value3":false, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131800, "value1":0.9351703924333601, "value2":827562310794270736, "value3":false, "__name__":"metric_00000585", "key_8":"g","key_0":"b","key_2":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367161, "value1":0.9314616031794654, "value2":8545717112597189310, "value3":false, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704807, "value1":0.7062626357522712, "value2":1882734585108600742, "value3":false, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270485, "value1":0.942646876486616, "value2":4665280302852454779, "value3":true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.521647, "value1":0.7127769947534454, "value2":7002036018179718342, "value3":true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.011614, "value1":0.2150115012001749, "value2":2368106908969391947, "value3":true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160161, "value1":0.5544338078106065, "value2":6068552071959300958, "value3":true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545350, "value1":0.5865458699587862, "value2":1122097202466314910, "value3":true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.948757, "value1":0.9353310253197936, "value2":9083948862293457994, "value3":false, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.003970, "value1":0.13057333820161732, "value2":4887307186667923394, "value3":true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811016, "value1":0.0597992472783255, "value2":7198325554706277943, "value3":false, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687715, "value1":0.029450254550187717, "value2":7347375517848816816, "value3":true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892901, "value1":0.5081883055870532, "value2":3289845879859033841, "value3":false, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006320, "value1":0.35956731353734483, "value2":6174658822769262086, "value3":true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.088057, "value1":0.5893474112602632, "value2":2264301146979603153, "value3":false, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366406, "value1":0.09531376481437555, "value2":8460504532640850958, "value3":true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.742879, "value1":0.21104814940031102, "value2":8239563761654323619, "value3":false, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014364, "value1":0.27433866046434513, "value2":1123249940897490575, "value3":true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442350, "value1":0.5779165272183289, "value2":7076967122005445282, "value3":false, "__name__":"metric_00000603", "key_6":"e","key_2":"c","key_3":"k", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.363242, "value1":0.7223710776768617, "value2":4947532477450255339, "value3":false, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794231, "value1":0.9595692700611697, "value2":7086029131344930712, "value3":false, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.696096, "value1":0.2902282025796308, "value2":2826023125040762860, "value3":false, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366795, "value1":0.8338186933189231, "value2":7469909471986494672, "value3":true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.642061, "value1":0.025310231945638022, "value2":5571346287819468860, "value3":true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673449, "value1":0.9385869454324781, "value2":1722013371242991173, "value3":true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375120, "value1":0.09609963357456859, "value2":5591579449982830960, "value3":false, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391308, "value1":0.5308398826498623, "value2":2744980541531859716, "value3":true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658264, "value1":0.08305585159828374, "value2":3868051792086152227, "value3":false, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.540598, "value1":0.9729141879466757, "value2":2886891615332464723, "value3":true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.077420, "value1":0.756799108978319, "value2":3709648144634270824, "value3":true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.762914, "value1":0.7697479099606849, "value2":3475167187400734782, "value3":true, "__name__":"metric_00000614", "key_5":"f","key_6":"g","key_0":"c", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907718, "value1":0.44276263620911915, "value2":8007229889762742191, "value3":false, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.677735, "value1":0.7114610811603074, "value2":420499484804274745, "value3":false, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097065, "value1":0.9614980550005483, "value2":401687894704730769, "value3":false, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107273, "value1":0.7588043489806351, "value2":5258772379410637950, "value3":false, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907641, "value1":0.16046492238486323, "value2":5755077471601284208, "value3":true, "__name__":"metric_00000620", "key_7":"f","key_9":"b","key_2":"j", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407605, "value1":0.5733175508809538, "value2":2858689191661453591, "value3":false, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412001, "value1":0.6163004444656547, "value2":3938161718625625501, "value3":false, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174438, "value1":0.7741718765670191, "value2":8151281186076102637, "value3":false, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433895, "value1":0.6190720762613371, "value2":4753613664907841729, "value3":false, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.754894, "value1":0.6861936508507025, "value2":4982660479879807579, "value3":true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135440, "value1":0.8484346139801648, "value2":4916410761830427120, "value3":false, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901526, "value1":0.7029579471688302, "value2":7091187814115860335, "value3":true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969298, "value1":0.9971487141151381, "value2":4678178370126159004, "value3":false, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.302462, "value1":0.5451450337065363, "value2":4647862155949132966, "value3":false, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174504, "value1":0.8364678736703143, "value2":4662336943284880044, "value3":false, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724429, "value1":0.7605319213856574, "value2":5775369825299529065, "value3":false, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196992, "value1":0.485221307178674, "value2":8161727305469743810, "value3":false, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673298, "value1":0.12884478288346873, "value2":3433606731713363675, "value3":true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.008175, "value1":0.6854875474834051, "value2":7177608068823530117, "value3":true, "__name__":"metric_00000634", "key_8":"b","key_1":"j","key_5":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.152906, "value1":0.937989165728203, "value2":2388297772896467042, "value3":true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354518, "value1":0.5251867146486633, "value2":2890885067302311396, "value3":true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.872640, "value1":0.7546063425064781, "value2":8461706694506205274, "value3":false, "__name__":"metric_00000636", "key_8":"f","key_3":"h","key_4":"j", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403113, "value1":0.7711761108505557, "value2":8776285961045082566, "value3":false, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.713712, "value1":0.3686652392717231, "value2":3561820735837135599, "value3":false, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.115073, "value1":0.21496854825964173, "value2":1666306538045744009, "value3":true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.127362, "value1":0.0348593634697097, "value2":3277092601539134615, "value3":false, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.042887, "value1":0.9391085748107487, "value2":3037900421018347064, "value3":false, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957170, "value1":0.2742109117364031, "value2":7538881467268645993, "value3":false, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.284083, "value1":0.6552095129564431, "value2":8324596282478983070, "value3":false, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139776, "value1":0.32060963733799414, "value2":4387891090637798442, "value3":true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.446847, "value1":0.7514268853453937, "value2":1042496829752491158, "value3":false, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.429327, "value1":0.7027895753887083, "value2":4302894157835600757, "value3":true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496272, "value1":0.2459555820489922, "value2":4743527397302686896, "value3":false, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599745, "value1":0.3085143386003063, "value2":8510884550594616394, "value3":false, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233545, "value1":0.4081732430516885, "value2":4780887317944713264, "value3":true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.396869, "value1":0.3834820243913789, "value2":2867737400343386587, "value3":false, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.000965, "value1":0.7645272489414043, "value2":8292218247137907442, "value3":false, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390310, "value1":0.9922190988066435, "value2":6520801004207332848, "value3":true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.408746, "value1":0.11967416242419873, "value2":1289404280671317844, "value3":false, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.513154, "value1":0.43853228030860203, "value2":3816263914269322442, "value3":true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.425564, "value1":0.9721541652281699, "value2":1199225910826614784, "value3":true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.896976, "value1":0.5841175850919846, "value2":6380661791453585128, "value3":false, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.385017, "value1":0.3642021068005677, "value2":7460310647019127617, "value3":false, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.861063, "value1":0.18830600982566692, "value2":6167598179189147828, "value3":false, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.942353, "value1":0.4661402168565476, "value2":8018642185562026949, "value3":true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.750347, "value1":0.3217686952651615, "value2":830941682724984711, "value3":true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.926420, "value1":0.0451768298368594, "value2":3858422103614938711, "value3":false, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451255, "value1":0.012250475428480134, "value2":468924453611892408, "value3":true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708013, "value1":0.5360411826543746, "value2":7399971881816481719, "value3":false, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.661279, "value1":0.5535815660872848, "value2":4401269288089200510, "value3":false, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.710860, "value1":0.6074402471823324, "value2":7073880640711856002, "value3":false, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245087, "value1":0.7738521153135302, "value2":5302937747460405991, "value3":true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724822, "value1":0.06719994230964553, "value2":1984993846520154478, "value3":true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841255, "value1":0.7199212526961583, "value2":3580548125031735289, "value3":false, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130289, "value1":0.14169983508615547, "value2":7980800511434824689, "value3":false, "__name__":"metric_00000670", "key_6":"i","key_8":"d","key_5":"k", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.215185, "value1":0.05456621441388223, "value2":3313875019669857059, "value3":true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730585, "value1":0.1938148933748891, "value2":2133259010192691032, "value3":true, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603048, "value1":0.27374938601691284, "value2":6401354620294279338, "value3":true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148949, "value1":0.5947828484485513, "value2":8147515290794947255, "value3":false, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.610152, "value1":0.4085609201069522, "value2":9002523177156729302, "value3":true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.607808, "value1":0.9266705539974761, "value2":4953852228607795114, "value3":false, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197811, "value1":0.7637421466978226, "value2":7727001922124797631, "value3":true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672455, "value1":0.2140320622344972, "value2":8504587633123200194, "value3":true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781600, "value1":0.18861407062595717, "value2":2041192855707568703, "value3":true, "__name__":"metric_00000678", "key_4":"i","key_5":"e","key_0":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648320, "value1":0.9980339865005591, "value2":6740770170971470746, "value3":true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481193, "value1":0.5203406695297639, "value2":6425306083284764527, "value3":true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954444, "value1":0.5949050635321682, "value2":2690790583476351571, "value3":false, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971678, "value1":0.3007304152123297, "value2":4562717443975915242, "value3":true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026021, "value1":0.27575882598297025, "value2":978861683515042668, "value3":true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.840073, "value1":0.15154871188435406, "value2":7561212279807822666, "value3":true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393063, "value1":0.5006737130316841, "value2":835200053720218776, "value3":false, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589637, "value1":0.29002140408035565, "value2":5208166554872726624, "value3":true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525315, "value1":0.7084919237025886, "value2":4888712288784480920, "value3":true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026489, "value1":0.9459638977891762, "value2":6799511668473867834, "value3":true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356460, "value1":0.8482487483215856, "value2":4902836930964609800, "value3":false, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871117, "value1":0.35037006883754834, "value2":6550768382747712274, "value3":true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.062992, "value1":0.3329675649141359, "value2":8563631384946292163, "value3":false, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.751101, "value1":0.9811341531679872, "value2":621345745999214417, "value3":false, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.535295, "value1":0.922840944798324, "value2":586063777073635225, "value3":false, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.258523, "value1":0.7865906481221376, "value2":1141638202189461929, "value3":true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.10833711533264617, "value2":5739011556983431812, "value3":false, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407251, "value1":0.4266309513517482, "value2":4708724256538796193, "value3":false, "__name__":"metric_00000697", "key_6":"h","key_2":"e","key_4":"d", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931023, "value1":0.6949078276162761, "value2":6914579995867144592, "value3":false, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.882968, "value1":0.27938573922269233, "value2":2134628658651324357, "value3":true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599828, "value1":0.31593636777024753, "value2":2026576607314660982, "value3":false, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166118, "value1":0.30221394298164866, "value2":7042763501275182896, "value3":true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114195, "value1":0.16993347985800208, "value2":356239511173356550, "value3":false, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.676940, "value1":0.8142744662164688, "value2":1615963097558644501, "value3":false, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525949, "value1":0.9287689609270591, "value2":8144174091016595622, "value3":true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852923, "value1":0.6682421979821775, "value2":1660916359233146706, "value3":true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714742, "value1":0.6940562364820247, "value2":8279633113622088778, "value3":false, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.069728, "value1":0.023054188899827854, "value2":7432273276574908949, "value3":true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397060, "value1":0.6029753734289958, "value2":5859678381429923175, "value3":false, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788868, "value1":0.4080940744104717, "value2":2775616148615876742, "value3":false, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.497778, "value1":0.0933297131976096, "value2":2710674316640074263, "value3":false, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.251386, "value1":0.5835060543515784, "value2":1694819851220806994, "value3":false, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144280, "value1":0.7762012742074322, "value2":6551654579563921110, "value3":true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778177, "value1":0.8529872059895475, "value2":4843965544345174856, "value3":true, "__name__":"metric_00000712", "key_7":"c","key_6":"k", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495383, "value1":0.8821308188431486, "value2":7181474101563747015, "value3":true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421122, "value1":0.04021723515347381, "value2":8758814764030728391, "value3":false, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055252, "value1":0.5894200751471586, "value2":3745371455267104958, "value3":false, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343067, "value1":0.2157486959489704, "value2":3077439566978720910, "value3":false, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489306, "value1":0.4542119549735926, "value2":5846254702440735178, "value3":false, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233627, "value1":0.2946889464915457, "value2":2465181057584653557, "value3":false, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.523076, "value1":0.579630183513931, "value2":4633358909899504280, "value3":false, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.317577, "value1":0.49633251300853154, "value2":7243885459219565500, "value3":false, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142016, "value1":0.41458124596836643, "value2":87263970443989795, "value3":true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014459, "value1":0.3889314073670658, "value2":6642542110052864260, "value3":false, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.249899, "value1":0.5981564529561014, "value2":4725189280029367657, "value3":true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.087078, "value1":0.48311123248467286, "value2":7915582091365577059, "value3":true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.681336, "value1":0.3569069050336655, "value2":7513851204641167627, "value3":true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266570, "value1":0.6724875939535222, "value2":4615308903415712327, "value3":false, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474729, "value1":0.3876423090885996, "value2":319900338975917805, "value3":false, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.328471, "value1":0.061053039369872907, "value2":9085903408798527191, "value3":false, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010016, "value1":0.7472091713600674, "value2":8976037153524824381, "value3":false, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412206, "value1":0.7796477061197584, "value2":6842085213034340664, "value3":false, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288276, "value1":0.26992787569132787, "value2":1806846857113271486, "value3":false, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.501201, "value1":0.781398815935773, "value2":4251812051340782659, "value3":true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270221, "value1":0.051986823543989535, "value2":4993776637838984787, "value3":true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.986643, "value1":0.7021309080168643, "value2":703842817955022555, "value3":true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.453898, "value1":0.4599169268098947, "value2":7963554429950594113, "value3":true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202118, "value1":0.94451324780601, "value2":7780362140982384262, "value3":true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.801389, "value1":0.7867483731478175, "value2":7192213512007631986, "value3":false, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.228749, "value1":0.4182819773434862, "value2":3027632485243175966, "value3":true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.184080, "value1":0.9567901938339193, "value2":5313717148514876661, "value3":false, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.707330, "value1":0.826196713820751, "value2":8078395604816162581, "value3":false, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130983, "value1":0.31932740597678144, "value2":8325619009408095997, "value3":false, "__name__":"metric_00000745", "key_6":"j","key_7":"j","key_5":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477577, "value1":0.8037112800341328, "value2":4641052188020943712, "value3":false, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.835784, "value1":0.981119547136869, "value2":5830864768532244942, "value3":true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.951325, "value1":0.9017984159519533, "value2":7172321278356171509, "value3":true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949416, "value1":0.9735936936041008, "value2":2397210877296618753, "value3":true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.227491, "value1":0.056278467666299646, "value2":3619861078542842990, "value3":true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614325, "value1":0.9338179733958348, "value2":1856588427373456649, "value3":true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.167780, "value1":0.6264999203061654, "value2":1887390160426943194, "value3":false, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421682, "value1":0.3034767704367279, "value2":1117719742198046331, "value3":false, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131869, "value1":0.7226359930389739, "value2":6009067442340606148, "value3":false, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.576451, "value1":0.4077319058968275, "value2":5262129379842658951, "value3":true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487400, "value1":0.49878540971656044, "value2":378267425259112843, "value3":true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417401, "value1":0.7695532802557139, "value2":6353721086899765325, "value3":false, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.597343, "value1":0.9334570369057625, "value2":8106460523980946423, "value3":false, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947755, "value1":0.599677921450711, "value2":1518293404194401856, "value3":true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533432, "value1":0.4134611723516883, "value2":4789921078575497056, "value3":true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271038, "value1":0.7693055378733823, "value2":2557684070989427251, "value3":false, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.209928, "value1":0.5501502486470365, "value2":6523836811607831530, "value3":false, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.514479, "value1":0.742552198141705, "value2":5448311945921729044, "value3":false, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097323, "value1":0.342716151747015, "value2":5333351730397550620, "value3":true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036400, "value1":0.9364465322438232, "value2":3515155010826544091, "value3":false, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.161709, "value1":0.4801914817306215, "value2":4167056653387877843, "value3":true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128367, "value1":0.42861309279829746, "value2":6367124429829008192, "value3":false, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.104233, "value1":0.5604661779053219, "value2":8908744031459681115, "value3":false, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814224, "value1":0.8505352643130756, "value2":3611537515395296790, "value3":true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.257817, "value1":0.9534356262275627, "value2":6358735114691060805, "value3":false, "__name__":"metric_00000769", "key_3":"i","key_5":"j","key_2":"e", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590971, "value1":0.8532839122834444, "value2":4683754442672481085, "value3":false, "__name__":"metric_00000767", "key_9":"f","key_7":"b", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110492, "value1":0.563891691575366, "value2":4631985245890497320, "value3":true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953652, "value1":0.20700443841383587, "value2":1925175802270628257, "value3":true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012273, "value1":0.5960529408710756, "value2":6611884946690949474, "value3":false, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107781, "value1":0.5528403788424425, "value2":2332669698469083563, "value3":true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289974, "value1":0.4420135407607212, "value2":3690892585669795676, "value3":true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.193764, "value1":0.5159643736419933, "value2":6218735166584063092, "value3":false, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.386095, "value1":0.777297131904327, "value2":636830504959980122, "value3":false, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977928, "value1":0.01676315692753733, "value2":4060533084263704194, "value3":true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.066993, "value1":0.001865435996499948, "value2":4739969679441502931, "value3":false, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.910616, "value1":0.057836079222670464, "value2":5276870038639917478, "value3":true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971825, "value1":0.18153964718314727, "value2":2807508624852997267, "value3":true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.782626, "value1":0.41404427259634824, "value2":6643422113997639050, "value3":true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892943, "value1":0.5830967595104455, "value2":499806168567753593, "value3":true, "__name__":"metric_00000783", "key_9":"b","key_7":"b","key_8":"e", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423402, "value1":0.9103545774481498, "value2":911868702803131336, "value3":true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006069, "value1":0.7748758341609289, "value2":6539224747903805609, "value3":true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143311, "value1":0.3741630815676959, "value2":375500557355639144, "value3":false, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267012, "value1":0.06234671061027635, "value2":5595306219658860658, "value3":false, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548719, "value1":0.7992113362076428, "value2":7214856807865350002, "value3":false, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.919970, "value1":0.973745629473067, "value2":4169123266267227326, "value3":false, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.071680, "value1":0.08778281877161515, "value2":7668684104220358875, "value3":false, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.222513, "value1":0.4425030874471083, "value2":5614432156299415760, "value3":false, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.031556, "value1":0.6971180191067721, "value2":6128893201277772027, "value3":true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839957, "value1":0.03151274581462018, "value2":8360563655204493448, "value3":true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924623, "value1":0.5790628849747803, "value2":6190166511034220622, "value3":true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.647381, "value1":0.18478113887851288, "value2":2971279323948578904, "value3":true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.818067, "value1":0.36190113937740354, "value2":1714448127957842853, "value3":false, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.715974, "value1":0.23843892303460118, "value2":2439937704889068215, "value3":true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192672, "value1":0.6080907905788115, "value2":976012584384444918, "value3":false, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157561, "value1":0.6894676876528992, "value2":7135259052981755980, "value3":false, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134773, "value1":0.09678670230466922, "value2":8180183833096777286, "value3":true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.616057, "value1":0.8520049469696344, "value2":2950033021270268791, "value3":true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255317, "value1":0.7477685612049465, "value2":179441442795111864, "value3":false, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906502, "value1":0.8256982980631076, "value2":4367074024600682775, "value3":true, "__name__":"metric_00000801", "key_4":"d","key_6":"e","key_3":"j", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.116793, "value1":0.8891425230837835, "value2":1719946264236570624, "value3":false, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324866, "value1":0.5386590615308351, "value2":8011171897264590756, "value3":true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.352403, "value1":0.7153316706866707, "value2":4379546221964794474, "value3":true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836874, "value1":0.14443484962198383, "value2":1482104610634689373, "value3":false, "__name__":"metric_00000805", "key_9":"b","key_5":"c","key_8":"a", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403735, "value1":0.8924124150178864, "value2":9156970764441343023, "value3":false, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141142, "value1":0.434266569307909, "value2":4978801873133265222, "value3":true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.234292, "value1":0.7164352385365066, "value2":8038096300931135389, "value3":false, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973287, "value1":0.6600392290118359, "value2":2638933878817864633, "value3":false, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728882, "value1":0.476991229582368, "value2":6510855349407866452, "value3":true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787371, "value1":0.7330600795491425, "value2":6570294519251889382, "value3":true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903661, "value1":0.5944117402324142, "value2":6524268081747181442, "value3":false, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.659983, "value1":0.26592399871964695, "value2":1212972365053449050, "value3":false, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723204, "value1":0.31589384939280846, "value2":2041435063818693781, "value3":true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701487, "value1":0.5974401671491981, "value2":625551200860992484, "value3":true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.893866, "value1":0.38643143057076507, "value2":6815713871810191176, "value3":true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561678, "value1":0.4057036812724311, "value2":4919580315129580518, "value3":true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004391, "value1":0.4801368737089002, "value2":7547608457928538573, "value3":true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.923757, "value1":0.7739736243035474, "value2":6832671633674661606, "value3":true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819284, "value1":0.7572793995611461, "value2":68290908865890434, "value3":false, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.524308, "value1":0.44777317605800043, "value2":4272485517844220240, "value3":true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804762, "value1":0.20222965946470728, "value2":2164767398150388054, "value3":true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060732, "value1":0.060291087703427844, "value2":8373529279962935628, "value3":true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325849, "value1":0.2785168531780935, "value2":8895250356037639155, "value3":false, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.378439, "value1":0.43808690997464833, "value2":7750734163790876517, "value3":true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.341809, "value1":0.21201598129686366, "value2":5439024219774015639, "value3":true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.027383, "value1":0.519384455440142, "value2":2830105304694072087, "value3":true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578370, "value1":0.4103102602537048, "value2":3491530677431875174, "value3":true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.588914, "value1":0.6688147984910581, "value2":7129525479125432993, "value3":false, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781407, "value1":0.7254015801455519, "value2":2135519716242173777, "value3":true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852755, "value1":0.6592390034228756, "value2":5378675889994915559, "value3":true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.996821, "value1":0.5491613669334785, "value2":3858296464786926451, "value3":false, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.364027, "value1":0.6056949341817615, "value2":2956076236179631886, "value3":true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.605233, "value1":0.2433469771258824, "value2":8502972839119365612, "value3":false, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.092828, "value1":0.29453399849858286, "value2":5192582279143186491, "value3":false, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.179799, "value1":0.19121582692459024, "value2":9103159579935481171, "value3":true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.054748, "value1":0.7255577017707996, "value2":8476394128272498701, "value3":true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.959235, "value1":0.5345272240165994, "value2":8621793756396494146, "value3":true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.735109, "value1":0.7362204691534435, "value2":942332422397834734, "value3":false, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778088, "value1":0.8485308761635748, "value2":8355286652172671018, "value3":true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026529, "value1":0.058403258262061046, "value2":3881865567315099972, "value3":true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.405396, "value1":0.78515016074467, "value2":3124806945169635921, "value3":false, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325780, "value1":0.8484383369933163, "value2":5639577297430380059, "value3":true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.125526, "value1":0.9437452957692816, "value2":1600583134494091930, "value3":false, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040638, "value1":0.8412837985326597, "value2":1891897645252751874, "value3":true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225539, "value1":0.7607755326390468, "value2":4494740517836964298, "value3":false, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417989, "value1":0.8351179906534547, "value2":6640751019261660918, "value3":true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.930391, "value1":0.7797566384384467, "value2":5344540873017186839, "value3":true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130784, "value1":0.9081520679550602, "value2":2673655563019876645, "value3":false, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065288, "value1":0.37878147094860193, "value2":40939122026803562, "value3":true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.023954, "value1":0.7421706325870447, "value2":3936838989809781841, "value3":true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639378, "value1":0.04809332390342072, "value2":138564553177528595, "value3":false, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.968036, "value1":0.4229941922357839, "value2":3252691734036720501, "value3":true, "__name__":"metric_00000853", "key_6":"k","key_9":"j","key_4":"e", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925601, "value1":0.5862382194484029, "value2":8809171247244862983, "value3":false, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.615163, "value1":0.41538444706998723, "value2":7806072407872661917, "value3":false, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432657, "value1":0.2054174608330251, "value2":523721991362366859, "value3":false, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584157, "value1":0.38115353295991117, "value2":5695252506949414616, "value3":true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969537, "value1":0.31115941644555445, "value2":3813252754578465682, "value3":false, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.755713, "value1":0.6078558755538082, "value2":4745658402471707078, "value3":true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.299770, "value1":0.39494561763395397, "value2":7454472376009363943, "value3":false, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549446, "value1":0.9276674467108077, "value2":7450051253648573468, "value3":true, "__name__":"metric_00000870", "key_4":"c","key_6":"c","key_3":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.339276, "value1":0.16417640235809516, "value2":376449946500069421, "value3":true, "__name__":"metric_00000871", "key_7":"j","key_4":"j","key_6":"c", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.044679, "value1":0.022401336029968563, "value2":7584563374980649526, "value3":false, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978082, "value1":0.634805138120068, "value2":2509593268797839298, "value3":false, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020832, "value1":0.2523976489068679, "value2":5330781529709694724, "value3":true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.874340, "value1":0.5447943369952332, "value2":753638192956179932, "value3":false, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.532731, "value1":0.3415364500896806, "value2":4617908501249248943, "value3":true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.888667, "value1":0.03025871907918381, "value2":389467035709739331, "value3":true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630653, "value1":0.3815158272162314, "value2":1365697000438851836, "value3":false, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947344, "value1":0.018503516339994433, "value2":6964047457370572921, "value3":true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.635335, "value1":0.7735503313212675, "value2":7863214250826862536, "value3":false, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.323808, "value1":0.9277969228346735, "value2":7104799245414939756, "value3":true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527571, "value1":0.19552679967510578, "value2":2150832405165980506, "value3":false, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.076891, "value1":0.45024886897327476, "value2":7461894110864047862, "value3":true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433797, "value1":0.03730319472343055, "value2":8041320080812623904, "value3":true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745634, "value1":0.6889162665973937, "value2":7455824590286680395, "value3":true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310508, "value1":0.4992657909321023, "value2":5571385578668134720, "value3":true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343053, "value1":0.21251046051646363, "value2":6470523853548900348, "value3":true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903927, "value1":0.9720310949900105, "value2":7380705280316406577, "value3":false, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.200927, "value1":0.42664784751514406, "value2":5730113255528003277, "value3":false, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.563845, "value1":0.8055339411145567, "value2":7403476714487686904, "value3":true, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.329254, "value1":0.8021973639133464, "value2":3080745955617166362, "value3":false, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.879352, "value1":0.4277546544111605, "value2":1478811657161573091, "value3":true, "__name__":"metric_00000884", "key_3":"a","key_9":"h","key_0":"i", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442846, "value1":0.006346529763383217, "value2":5543697338259725446, "value3":false, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.094781, "value1":0.40728088001797547, "value2":6613387129832293245, "value3":false, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.975150, "value1":0.3661351435962595, "value2":830819320558121686, "value3":false, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245141, "value1":0.02569157141735858, "value2":2603499185616374516, "value3":false, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949515, "value1":0.3643114110362523, "value2":832829152625075749, "value3":true, "__name__":"metric_00000888", "key_6":"k","key_2":"f","key_4":"i", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741852, "value1":0.28794172682449687, "value2":453442634675086185, "value3":false, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.738181, "value1":0.6087693219046798, "value2":7047508567290984350, "value3":true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.492510, "value1":0.5382269802655012, "value2":2689541564459522343, "value3":false, "__name__":"metric_00000891", "key_3":"d","key_7":"i","key_2":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988702, "value1":0.0728816330863758, "value2":5556532985784164765, "value3":true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.655057, "value1":0.7734197867503856, "value2":5244006855203800595, "value3":true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520001, "value1":0.4376795320695461, "value2":1470774311764721388, "value3":false, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164473, "value1":0.6572062696131487, "value2":1949710197196336093, "value3":false, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.190159, "value1":0.9525604598487357, "value2":5478966602860717950, "value3":true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829372, "value1":0.6923157559412111, "value2":5023489111088251295, "value3":false, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391391, "value1":0.6340217327197775, "value2":8000248111604879168, "value3":false, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883302, "value1":0.8070620002939071, "value2":653446263691796732, "value3":true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745426, "value1":0.45672801954182946, "value2":6893619651580083925, "value3":false, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982512, "value1":0.5060051247468136, "value2":7419982333827237040, "value3":true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827729, "value1":0.15692262245568137, "value2":3804375123078218665, "value3":false, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244618, "value1":0.6655062510702947, "value2":1607640639377335930, "value3":false, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.945643, "value1":0.3793044572114196, "value2":8429298239346478761, "value3":true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626649, "value1":0.22068614209697351, "value2":8899816776188911022, "value3":false, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045981, "value1":0.34313373362766186, "value2":1556434949234629007, "value3":false, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.381111, "value1":0.4930245395823471, "value2":3486502897378084847, "value3":true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596053, "value1":0.23068160326995202, "value2":1094706285002463668, "value3":false, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454071, "value1":0.4016206555788349, "value2":7766949472696777624, "value3":false, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122082, "value1":0.12814082114630934, "value2":6860595913103836152, "value3":false, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267563, "value1":0.915043661363235, "value2":7117774485449523197, "value3":false, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883324, "value1":0.48609131055046695, "value2":168158041910482971, "value3":false, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.798512, "value1":0.9160093333280128, "value2":5957101416277255647, "value3":false, "__name__":"metric_00000912", "key_6":"c","key_9":"a","key_0":"k", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.956973, "value1":0.08410692933360829, "value2":3670111299929079308, "value3":true, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799611, "value1":0.006983554187354543, "value2":560914434528784616, "value3":false, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539372, "value1":0.1527069613506525, "value2":5396674816694339651, "value3":true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.538750, "value1":0.21348159925068225, "value2":4409731296615206554, "value3":false, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.854320, "value1":0.921941847389472, "value2":4199458567691684987, "value3":false, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451148, "value1":0.5527973855585238, "value2":5090625521103681352, "value3":true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.585722, "value1":0.08217397453744285, "value2":5891174882356923897, "value3":true, "__name__":"metric_00000920", "key_8":"k","key_9":"g","key_2":"c", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280675, "value1":0.17159417706244065, "value2":1155748852941674950, "value3":true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673546, "value1":0.9064641866629127, "value2":5826431744512876414, "value3":false, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748734, "value1":0.7184282675490825, "value2":465504921239629350, "value3":true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448890, "value1":0.21947167931430242, "value2":97944850538911159, "value3":false, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.034158, "value1":0.6471204512621925, "value2":6508843210808028097, "value3":true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495253, "value1":0.7273452290068572, "value2":2698451345131648958, "value3":true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.362458, "value1":0.6109728168729301, "value2":8601029350565666815, "value3":true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263467, "value1":0.5089131780165215, "value2":7125458167541328714, "value3":false, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373946, "value1":0.4318385725650386, "value2":2714678475704530055, "value3":true, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714267, "value1":0.9575832102733836, "value2":6392945512202004757, "value3":true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448342, "value1":0.22459353529927165, "value2":3260297440141888471, "value3":true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636333, "value1":0.5831279192445404, "value2":4643961286576988019, "value3":false, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.544083, "value1":0.5292455018046274, "value2":2947716990317042035, "value3":false, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.834093, "value1":0.03492084160563978, "value2":317252294366234894, "value3":true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266819, "value1":0.3363934052516993, "value2":7398703592096323308, "value3":true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006504, "value1":0.7543858008870775, "value2":1775651744055633759, "value3":true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114763, "value1":0.5343311798406293, "value2":3440557676505536535, "value3":true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338231, "value1":0.46269924741830853, "value2":2414529329564577894, "value3":false, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.613570, "value1":0.5208853551038555, "value2":1476820686457140000, "value3":true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954830, "value1":0.41362302765893333, "value2":5510842372132552038, "value3":true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.009033, "value1":0.39228566619392624, "value2":4427054644662596059, "value3":true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182689, "value1":0.6549868089884497, "value2":4035004692027824929, "value3":false, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037729, "value1":0.7486872272646202, "value2":4847504507990849152, "value3":true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.025112, "value1":0.34907292952301494, "value2":6027470898165077200, "value3":false, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438904, "value1":0.6318755547819239, "value2":7111135668532014691, "value3":true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812145, "value1":0.3675412312936663, "value2":5692745323903444622, "value3":true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.445020, "value1":0.3643115537301237, "value2":1216854539448528229, "value3":false, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149138, "value1":0.5909969143971484, "value2":7044155605397820069, "value3":false, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841137, "value1":0.5031802947436413, "value2":4756712241095891395, "value3":true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766360, "value1":0.42396402951055434, "value2":5375665261877407507, "value3":true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663263, "value1":0.21109789310310256, "value2":7791325581861318665, "value3":true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.623457, "value1":0.10997466706442727, "value2":4022405196702857232, "value3":false, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687642, "value1":0.5890580006230834, "value2":4798397386267850627, "value3":true, "__name__":"metric_00000952", "key_2":"b","key_0":"i", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041534, "value1":0.7259891148801754, "value2":7530272514921895889, "value3":true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.8243889839817137, "value2":2465746906164654846, "value3":false, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.282429, "value1":0.6519583818560468, "value2":6092402468810251742, "value3":true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816484, "value1":0.18010854645368016, "value2":8191892113724148654, "value3":false, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851506, "value1":0.5899554257608193, "value2":4617440579117228711, "value3":true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.382630, "value1":0.5412707148385829, "value2":3548532358946210519, "value3":true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.473179, "value1":0.6228095400098033, "value2":5572845251799335628, "value3":true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266666, "value1":0.5868701622069099, "value2":6991916315373855609, "value3":true, "__name__":"metric_00000962", "key_9":"d","key_3":"d","key_7":"f", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.952001, "value1":0.6189910604042005, "value2":6928874617046934238, "value3":true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107532, "value1":0.28068061790544596, "value2":7672880301382817972, "value3":true, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.688840, "value1":0.6035662636797529, "value2":5891970996329552554, "value3":true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283889, "value1":0.9151251791608558, "value2":2952612587882160016, "value3":true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629939, "value1":0.5930273861647344, "value2":7387132779647308781, "value3":false, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418325, "value1":0.9507448442764749, "value2":167010402424657095, "value3":false, "__name__":"metric_00000967", "key_9":"f","key_1":"h","key_5":"d", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373327, "value1":0.6218852126400313, "value2":8371256546452205404, "value3":true, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515481, "value1":0.0854866845303678, "value2":6987828829468502321, "value3":true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820817, "value1":0.698882043745769, "value2":8235587889784697376, "value3":false, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.365626, "value1":0.4531560215251981, "value2":4722270617283077560, "value3":false, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484276, "value1":0.6659326817404172, "value2":8721021931100680532, "value3":false, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.528708, "value1":0.03487880924213162, "value2":6041728758379789938, "value3":true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590377, "value1":0.34239911489608105, "value2":7295038165138773264, "value3":true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481162, "value1":0.8346535177821227, "value2":4835378003160835691, "value3":false, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.664721, "value1":0.9670197543347829, "value2":7077761062907993448, "value3":true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242914, "value1":0.7732131934363016, "value2":6013582492758534765, "value3":true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.185221, "value1":0.10448347281111905, "value2":4626218721873500711, "value3":true, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.126356, "value1":0.6396318028070207, "value2":7845764314900334021, "value3":true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673777, "value1":0.7918565155988029, "value2":7808431007480445571, "value3":true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706238, "value1":0.49404656958089044, "value2":8867399074074176905, "value3":false, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216638, "value1":0.2610211982854062, "value2":5851177857276860153, "value3":false, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247760, "value1":0.720377370743505, "value2":284915355108242656, "value3":false, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484098, "value1":0.20488798005491424, "value2":8924950640215344491, "value3":true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.345583, "value1":0.26657891861913446, "value2":1138178968371346598, "value3":true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.315052, "value1":0.7472751608991773, "value2":5324337837076333029, "value3":true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247789, "value1":0.35415032152422526, "value2":3640906326280302592, "value3":false, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787688, "value1":0.6162646188121896, "value2":8883014095767101323, "value3":true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477237, "value1":0.04016123924628, "value2":9093318721099578903, "value3":true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560140, "value1":0.30318020971980014, "value2":2686556711005804654, "value3":false, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.002924, "value1":0.5022585667052755, "value2":3145515076422963133, "value3":true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732614, "value1":0.3545238139152862, "value2":1059232874175794196, "value3":true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.332613, "value1":0.32230993437491456, "value2":2724091454501972237, "value3":false, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012206, "value1":0.931677320101252, "value2":5370261932149351692, "value3":true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196655, "value1":0.8224901135235602, "value2":9208484728253262649, "value3":true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.326865, "value1":0.5922171068495821, "value2":7224402625385076257, "value3":true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289542, "value1":0.9512116686992638, "value2":7523857296190024264, "value3":true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.170535, "value1":0.5872728280225836, "value2":6413447856713598093, "value3":true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434673, "value1":0.18109149352889556, "value2":1218373397895276791, "value3":false, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.860439, "value1":0.30809143156043617, "value2":8296859231789889769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697105, "value1":0.07464704753781262, "value2":3626726022956376302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499281, "value1":0.2598747988733079, "value2":1472024199460334836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389559, "value1":0.9035230182410444, "value2":5735152112214129185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259346, "value1":0.6088348511469825, "value2":2118449310673513905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_6":"k","key_7":"e","key_3":"f", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265140, "value1":0.41315907782222017, "value2":6967606456828959585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273243, "value1":0.3670231186776885, "value2":3094287292036347916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697145, "value1":0.4712183861515544, "value2":233543654937300193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.570507, "value1":0.11967792858372435, "value2":1467009477255473639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421572, "value1":0.25352243267763397, "value2":5433840270731074230, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423291, "value1":0.6724498851050211, "value2":652727500980374465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072653, "value1":0.4078722430828867, "value2":6479204450678692857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.576237, "value1":0.03506340983128008, "value2":2409142807432889102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928873, "value1":0.9749671198417696, "value2":806127466997750314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.121356, "value1":0.3710118368988569, "value2":2961209029330787894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.527082, "value1":0.427919527676362, "value2":4168209888386865559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971852, "value1":0.4558638608790214, "value2":7271190379082523197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758405, "value1":0.642984416191772, "value2":635944986021178390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.191870, "value1":0.4028374291432409, "value2":4925274338231144549, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.177741, "value1":0.112412024356175, "value2":1799812085131373089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.562889, "value1":0.9463319128677472, "value2":8604221624040619668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.145145, "value1":0.6344301595600267, "value2":5934019334637844764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512765, "value1":0.12698238608047413, "value2":3179145500529665851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.947023, "value1":0.9362933587076374, "value2":6552003615608175727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479188, "value1":0.8368835012768996, "value2":1048667356432629681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.886895, "value1":0.147063588220359, "value2":2554258082886382046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495557, "value1":0.4897837285320758, "value2":3386687451644953805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267510, "value1":0.7137029646630675, "value2":4723657657521756601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662037, "value1":0.39308757655823606, "value2":5160142240464547787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.001360, "value1":0.2816582812804884, "value2":6076665375380746353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.501969, "value1":0.4236689406205709, "value2":725918256085862567, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.956392, "value1":0.4674704766085337, "value2":4478503621053198593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442861, "value1":0.15012494378573155, "value2":6132679974455498265, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394657, "value1":0.8655032404449484, "value2":2033181385746149676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.574670, "value1":0.8578324023084722, "value2":9030512731073145161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844866, "value1":0.401882448563829, "value2":6461587433040802039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.677301, "value1":0.17574043431028194, "value2":13200916467489149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643407, "value1":0.09478762453322248, "value2":9040728601657069686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.461584, "value1":0.2313038745196538, "value2":8542291140491717734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480373, "value1":0.33926139409918454, "value2":1470854458100424231, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.387467, "value1":0.9027179517679272, "value2":2152676801390448497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.081729, "value1":0.5930629975633843, "value2":3441892662554339501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.523164, "value1":0.27807377593445504, "value2":8726571622906981746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643880, "value1":0.8856942439946653, "value2":6532657219501338712, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.233730, "value1":0.12117132337083063, "value2":5262200042073716921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113242, "value1":0.15754750121773875, "value2":3512597140842356375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.592540, "value1":0.8501957441234611, "value2":1120617963379990831, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.754300, "value1":0.2516365689787487, "value2":1009529273537079157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.738781, "value1":0.9115797570978498, "value2":654040880693718238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_5":"j","key_2":"a","key_3":"b", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730945, "value1":0.567968215036453, "value2":3597506204077270120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.940173, "value1":0.9712119929757523, "value2":5983635236065602039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326670, "value1":0.07295490390000714, "value2":6022184335907772623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.571799, "value1":0.5813154467308365, "value2":9191368645337346592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.547202, "value1":0.12691857025568046, "value2":3820781159866636536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097732, "value1":0.6481650315571341, "value2":6773475876286839908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.184127, "value1":0.5736971101036478, "value2":7284609149743260027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.925677, "value1":0.3771049289873205, "value2":1181794630742224031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.157660, "value1":0.5233716379944181, "value2":4014079834827187874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.413654, "value1":0.21255841055237895, "value2":4422822169208563976, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_9":"a","key_0":"c","key_4":"d", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282429, "value1":0.5134755380331291, "value2":2875914720894216239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510799, "value1":0.4811695837424015, "value2":2131765002626290228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855059, "value1":0.1803859944439004, "value2":4143870194342227353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841548, "value1":0.8921162909498762, "value2":2995639098842929032, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904367, "value1":0.6827455696983789, "value2":4486024070965990492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903308, "value1":0.5275068815139481, "value2":4431489892412543662, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369269, "value1":0.3088848570530923, "value2":2711948390039375929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.246425, "value1":0.22469514297321577, "value2":3311794715258074199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.627147, "value1":0.5085084020761419, "value2":7449844520407226749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757065, "value1":0.7340080407469496, "value2":1399142812283896331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404001, "value1":0.6927872548099725, "value2":4611922647039691390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.548980, "value1":0.9504303368975154, "value2":784663243740669037, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088930, "value1":0.5112126366657939, "value2":1061303860942908201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808273, "value1":0.07227725639854525, "value2":705291637800185980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.195134, "value1":0.9915791690400784, "value2":1380206536675804157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588271, "value1":0.9410252962292832, "value2":2976150472594021014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830440, "value1":0.5990964794781901, "value2":6208539757039787288, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_9":"a","key_1":"g","key_6":"b", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.988764, "value1":0.12421784563285446, "value2":600116094525731026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793643, "value1":0.013993638720735705, "value2":249013181149620349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_5":"j","key_1":"d", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537335, "value1":0.9710360883032237, "value2":3990188937402642001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.783911, "value1":0.5202417888338783, "value2":9207852713084863631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_4":"b","key_6":"f","key_2":"k", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000369, "value1":0.4757654945131251, "value2":617662898409566699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720049, "value1":0.11616063522827627, "value2":1134111314294224704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850537, "value1":0.856287241545387, "value2":3915015074142268096, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117449, "value1":0.23938708551093754, "value2":6182563875693526929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720636, "value1":0.6929840624902686, "value2":7184719611552077354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890463, "value1":0.39990668829147297, "value2":5236439635142539127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598510, "value1":0.794840854439141, "value2":7790247245768394828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_2":"e","key_7":"j","key_1":"c", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.103618, "value1":0.837788909134762, "value2":1746463551678400384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890891, "value1":0.701693361054133, "value2":750546117784279688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243694, "value1":0.04252945224095913, "value2":7022047191936829657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698720, "value1":0.8318935119459053, "value2":2403728383571794535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.362462, "value1":0.5425805070018154, "value2":7722798226519636439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447582, "value1":0.6901117678727204, "value2":1225566659535077888, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884009, "value1":0.9848633762091424, "value2":1508601246687431755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.895176, "value1":0.1370326687436836, "value2":7673037864112111033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.521633, "value1":0.14853540424497083, "value2":6875647668629772843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388849, "value1":0.856681841687182, "value2":289008565390108021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.747443, "value1":0.735786478790032, "value2":7309545501096360932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528790, "value1":0.010059663849976508, "value2":3366632872144662566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.586068, "value1":0.8316103940598538, "value2":6216090017784726937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154422, "value1":0.6449454723419513, "value2":5588370277273470937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.476987, "value1":0.688075774177057, "value2":3232344568636281729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.176248, "value1":0.8630722552735859, "value2":7054623290814757387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.553861, "value1":0.6275284095417549, "value2":8413581575795957099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482699, "value1":0.30152766631318734, "value2":2728742455813076667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228157, "value1":0.09912287337858451, "value2":1368124455579266783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.363726, "value1":0.2556344439654489, "value2":8083466247113334458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598631, "value1":0.00654705967457182, "value2":678136325524679028, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188097, "value1":0.33693220746708563, "value2":5349169986765993063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.591197, "value1":0.3158077067804786, "value2":8408038257495371649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.723537, "value1":0.8935982855510853, "value2":6781320910854884909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.714830, "value1":0.8483752317059017, "value2":4114947449469032042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510153, "value1":0.8569271663104114, "value2":2163617609666100779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.556403, "value1":0.46846499185707535, "value2":4543935722642344774, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115027, "value1":0.13242237154578534, "value2":4001391755557856082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700402, "value1":0.31919253163578853, "value2":9168039730828764814, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473591, "value1":0.914966327309789, "value2":7574079944969934140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657212, "value1":0.7123671399711553, "value2":5845893001941155972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389666, "value1":0.46892477810031413, "value2":3773448585226589360, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365529, "value1":0.10038955567825718, "value2":7677417085782135970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.181780, "value1":0.3807188874415628, "value2":213730861941923164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632990, "value1":0.6352610882615881, "value2":7695215886438259449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.801703, "value1":0.5435516239407236, "value2":1197124892718412788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943146, "value1":0.6459645738185148, "value2":257718117926443886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047331, "value1":0.2599891672558978, "value2":1859555081827581052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.329003, "value1":0.7536967883211322, "value2":8944365307464980699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750531, "value1":0.8871777984871712, "value2":4584326435451561707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820378, "value1":0.6275908409813838, "value2":2808244151477882254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.664852, "value1":0.971622999773127, "value2":3565786429109692783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502157, "value1":0.19081539446623427, "value2":1479393056820182163, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396316, "value1":0.5123967312660013, "value2":8943627767884961054, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934631, "value1":0.19466544110858217, "value2":3956006751549658946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206549, "value1":0.017226775576345523, "value2":7020366329672792405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118256, "value1":0.5188472954068934, "value2":8138153858856845378, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584343, "value1":0.14448963894972008, "value2":4790553180218846981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.877457, "value1":0.6880296047171478, "value2":1215785358113912003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409285, "value1":0.05411761237341633, "value2":4709001633389005280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364854, "value1":0.8998190824029622, "value2":2511578657661549153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.973548, "value1":0.8069354470874932, "value2":2595652771451514899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.073905, "value1":0.2517811560702055, "value2":6140023292659546232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.262163, "value1":0.5630653619265122, "value2":3018188000144695876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641275, "value1":0.6900145697913901, "value2":5216098013140123128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392564, "value1":0.018717055152633055, "value2":8077083634713789634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932220, "value1":0.8388125579510224, "value2":2685186121830778256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206042, "value1":0.49376606624615665, "value2":5850726108082863170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215184, "value1":0.356531340492791, "value2":4642776908341780996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.558758, "value1":0.575409651955595, "value2":2207683625879983306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953416, "value1":0.9073618953615663, "value2":5651749773058007750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.648335, "value1":0.46358450476088653, "value2":1290620516019440255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.487910, "value1":0.012547280333283235, "value2":4919259233823782968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.378926, "value1":0.7085830860476057, "value2":2171649428344764858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_8":"j","key_3":"b","key_4":"k", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802501, "value1":0.31728201505130527, "value2":5419079224591774860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.656795, "value1":0.24875018847328167, "value2":485829881375178402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938831, "value1":0.08769582144908596, "value2":6951540802412020838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231956, "value1":0.7863053106916462, "value2":1030265256144215264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.631246, "value1":0.46917511334330375, "value2":3774063736313175623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305077, "value1":0.834556547518894, "value2":636203159633468096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808070, "value1":0.5245784354956413, "value2":5259003801435693215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641827, "value1":0.7755759714857714, "value2":3467599768461222251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.306769, "value1":0.47958408429359173, "value2":5365936635633468542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294040, "value1":0.5158994633208911, "value2":7429034992856357163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.961943, "value1":0.7435043392340425, "value2":8500250963468273773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935561, "value1":0.5801624578827164, "value2":8754050195411098007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444846, "value1":0.030970547839451835, "value2":8387858736748608077, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358418, "value1":0.5401368375372775, "value2":7107146838213157067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.546704, "value1":0.46330699877542014, "value2":8769213962266908815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.270889, "value1":0.46793759900681503, "value2":6364231395353410490, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549392, "value1":0.3430929972372329, "value2":4966477192488000261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428697, "value1":0.7243138275027047, "value2":2836857643122176656, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993399, "value1":0.40084653235121936, "value2":6794009333725376121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.506725, "value1":0.07262914457616802, "value2":3459356114730929347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202924, "value1":0.07381206265687722, "value2":2649194281139606297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.411238, "value1":0.9684314554036981, "value2":3705584143921682379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_5":"h","key_6":"k","key_4":"c", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358603, "value1":0.534711919539449, "value2":4138168909279457429, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989681, "value1":0.19207319251633845, "value2":1593591737448226138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.052714, "value1":0.27756026777792103, "value2":6377557735209128817, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614036, "value1":0.018601161874125947, "value2":151552447676967905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305181, "value1":0.031288435638661224, "value2":3937949086874523036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715232, "value1":0.6397334976781403, "value2":7614941574913315366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705549, "value1":0.6996034412212425, "value2":6200772172174019130, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.302276, "value1":0.5239925847096263, "value2":1057710529698426724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200111, "value1":0.2567985665426193, "value2":4529602646404884783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_2":"g","key_8":"f","key_1":"a", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.727436, "value1":0.43685005741153693, "value2":3304580792456252079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.351586, "value1":0.3164739225094333, "value2":3897998541684002875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.665954, "value1":0.5906227199646251, "value2":6746215572356057637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361921, "value1":0.3364933451115498, "value2":761972198573021391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.889779, "value1":0.36794570756599093, "value2":2090465055639313788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.699184, "value1":0.5483254879513706, "value2":7720905658039555449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388162, "value1":0.5287855210410441, "value2":7195732574645022274, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154496, "value1":0.6641476496489667, "value2":9137939751947784172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327473, "value1":0.5518373975194654, "value2":3971910670147599886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063454, "value1":0.43820340286329656, "value2":5405166066848368254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.781841, "value1":0.031101766659472223, "value2":4189381224837071716, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171755, "value1":0.46498479978119844, "value2":4931395791650935239, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.559617, "value1":0.8752912652134259, "value2":8586908304160851241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.323852, "value1":0.35821728605182146, "value2":5163275989952284916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030550, "value1":0.8225486647874358, "value2":2481647473431927932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.712533, "value1":0.368964069168292, "value2":8062115249497297834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298843, "value1":0.30262512534550123, "value2":5999722310995378983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778244, "value1":0.3716152000047593, "value2":6978129792906951032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.144352, "value1":0.42812039567105226, "value2":6591666290600795514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616112, "value1":0.5501694032291233, "value2":5953070646927834342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843613, "value1":0.7957093053513518, "value2":779447045934204434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.021516, "value1":0.3651263217338076, "value2":1101615937937937836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.707183, "value1":0.9474415697823648, "value2":3163832075607398652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807066, "value1":0.328067579350897, "value2":1404186285892115543, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327059, "value1":0.4531450671817094, "value2":3186214527240993583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.629874, "value1":0.555595498018525, "value2":7512625623217629070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203064, "value1":0.24518269440439186, "value2":7974348531674940058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865398, "value1":0.7125714321236861, "value2":1185618794516530658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.098668, "value1":0.6050958429061423, "value2":7737776979675298527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659514, "value1":0.8901996322311656, "value2":7752156196236457080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_6":"h","key_9":"d","key_5":"g", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650309, "value1":0.7869508663362883, "value2":6010716207202506080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.779127, "value1":0.6366846028665012, "value2":7797452629031711950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.281652, "value1":0.9118975722819298, "value2":6103834325448620626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868834, "value1":0.18117722257021981, "value2":6098726761586315616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865255, "value1":0.5168875940701739, "value2":7616170573520868343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000667, "value1":0.8879098452429443, "value2":2716291319460579102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938952, "value1":0.27931352835868933, "value2":2351410106131072829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524371, "value1":0.5375950258397566, "value2":7234763394924546015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718980, "value1":0.7918266548241415, "value2":1156714261082582562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532951, "value1":0.4063578415664877, "value2":2401980297367889095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780599, "value1":0.15947463153475608, "value2":6812388220933080605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078621, "value1":0.6070073793267291, "value2":1012316510571499763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404621, "value1":0.9613908610430107, "value2":6888867545087821595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.130043, "value1":0.26321351304547996, "value2":4446851417687808950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602508, "value1":0.38864606344069585, "value2":2797998328851090673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244218, "value1":0.5263437247014857, "value2":884821963010111580, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415267, "value1":0.3515494200462383, "value2":3356463166369801537, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303765, "value1":0.5270577861362743, "value2":2138283533246685033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.254002, "value1":0.303619701206094, "value2":7629281847530948891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588498, "value1":0.5481446854335168, "value2":7641330013258723790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.620575, "value1":0.6420150180085478, "value2":7965173856733830205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243174, "value1":0.400523011031962, "value2":4931302022232098402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_9":"c","key_4":"d","key_8":"d", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.104291, "value1":0.4501470045022247, "value2":6419830717700946282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605242, "value1":0.5489672412234975, "value2":25272414308366080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705962, "value1":0.2696978321817019, "value2":7266661553560831399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.204594, "value1":0.050667752810758936, "value2":4939383554296719335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.694791, "value1":0.8451677349011486, "value2":1862202581082766068, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178458, "value1":0.47634419936555994, "value2":3993368962808231698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879184, "value1":0.05969390345398635, "value2":5274917008847651276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854470, "value1":0.8497662902937875, "value2":3035062077693412013, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.397422, "value1":0.08213093074253511, "value2":5967759139282299209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584032, "value1":0.08273000380656788, "value2":2735658568798834919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822339, "value1":0.581737952515166, "value2":2604615641240906232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.308039, "value1":0.34242828701171635, "value2":3092061640651599674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.913723, "value1":0.005597195694116873, "value2":2620704762797681315, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_3":"b","key_8":"c","key_1":"f", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618957, "value1":0.6413425640197833, "value2":5287503767189177120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454987, "value1":0.32512559877918956, "value2":2506424876460932044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447066, "value1":0.20850990987663237, "value2":7977478851800531911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.683345, "value1":0.838649315370365, "value2":4999282712485449007, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935535, "value1":0.4089194818619663, "value2":722917113975878160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439326, "value1":0.8812716905619982, "value2":6658838831662104981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069099, "value1":0.15024751394280958, "value2":1328305014122533446, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_7":"f","key_3":"a","key_5":"e", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473312, "value1":0.9280459649583414, "value2":3438360323166932806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215711, "value1":0.8566516621694267, "value2":1663749581830575806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312134, "value1":0.566681231011358, "value2":6911167899525793851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862090, "value1":0.5364584771183928, "value2":7729755763890728353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456397, "value1":0.7096399907515605, "value2":5531147306124199678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.587843, "value1":0.8985325214047336, "value2":4030305760543278105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710467, "value1":0.5099257391896752, "value2":1179017586099298270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188204, "value1":0.8723809422062396, "value2":5093258983701866401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.681846, "value1":0.23032248565211208, "value2":3357194399728955391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_2":"j","key_4":"c","key_1":"i", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614220, "value1":0.9249072784237294, "value2":4254302260697988757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.028000, "value1":0.2236758406407766, "value2":5499863050282572666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923027, "value1":0.4209047854675161, "value2":4430077989918325993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203873, "value1":0.8404470170534011, "value2":532376833912389989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_2":"f","key_6":"b","key_1":"d", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.541847, "value1":0.24084743744285772, "value2":2314457094095648600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932124, "value1":0.7743913731102018, "value2":7842905515694643998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.986636, "value1":0.18488491173653657, "value2":1233277417792576747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_6":"f","key_0":"j","key_2":"i", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.491189, "value1":0.5404267488261955, "value2":5193038540757554939, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.449064, "value1":0.9915502184839048, "value2":3914891481887672833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203773, "value1":0.11544040430435766, "value2":8720914144754355340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.637865, "value1":0.5397959742632489, "value2":5016935922539098494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.164476, "value1":0.40957356383804516, "value2":8101990321123394900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212208, "value1":0.31170589199654203, "value2":8172096864950686883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427578, "value1":0.9500075990560398, "value2":4448206698192420471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375888, "value1":0.08043137384857349, "value2":5428459057669693793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.253419, "value1":0.30426981771564987, "value2":3701960397375147641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.064111, "value1":0.6613366419136151, "value2":9214004723321204161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618634, "value1":0.28919063506417325, "value2":2549339895850480456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.581311, "value1":0.5011828194867549, "value2":2394702011956086199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.619872, "value1":0.3579296101205142, "value2":3114339742928723546, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_6":"f","key_0":"c", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928952, "value1":0.7843473952520837, "value2":4007100053019753408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.678699460362967, "value2":1449401258062210262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.791597, "value1":0.5225036343492986, "value2":7759663471842896645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794616, "value1":0.3395042657376233, "value2":1662502503940889595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.466754, "value1":0.9041538990976105, "value2":535273624649157699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693734, "value1":0.009440601901378784, "value2":2148867264362048197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841713, "value1":0.059732250656449104, "value2":6677903169635737819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.746032, "value1":0.16006431951166478, "value2":5022444125278904149, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.158292, "value1":0.8619758377742992, "value2":3793514584183151927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364688, "value1":0.7050682785197309, "value2":8826236482901828470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_9":"f","key_2":"e","key_7":"e", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273970, "value1":0.40265473643498934, "value2":8303926135399734280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030530, "value1":0.7286415975665108, "value2":160039363895729267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_5":"j","key_3":"g", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266449, "value1":0.033547786099323434, "value2":7620487585623102750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025977, "value1":0.17793807237193235, "value2":8744525786084307091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.806031, "value1":0.5052650450730324, "value2":4060064708931174932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035732, "value1":0.9743453198182447, "value2":4099915557434576704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.646092, "value1":0.21956311768718226, "value2":8589888788620337480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078229, "value1":0.4079449893422866, "value2":6173402600986499343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666881, "value1":0.9863097557928853, "value2":8279527785132450943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.741108, "value1":0.31651275509392585, "value2":1547895388613496809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866620, "value1":0.6883437972608208, "value2":8802986125706842330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718942, "value1":0.8847629636653906, "value2":800711854768452001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375188, "value1":0.9532398566953418, "value2":2394341312296044798, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624088, "value1":0.6934451210618476, "value2":8474880456434193538, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680602, "value1":0.2209467711169388, "value2":8621094656072865370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.522533, "value1":0.5439997834907557, "value2":8198708215843245045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439975, "value1":0.8101860664171935, "value2":8929518033480490042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602144, "value1":0.17084337908018365, "value2":6000683736434217499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594415, "value1":0.3959186882607224, "value2":1454959879954934650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917433, "value1":0.849867816493911, "value2":8615713829914363729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439894, "value1":0.7578484758847723, "value2":3887712481108512291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.965697, "value1":0.8251003697840577, "value2":836422853680285494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259251, "value1":0.4596388195214864, "value2":1597474370288095053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512513, "value1":0.783528381108141, "value2":8045517537006666296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_4":"i","key_6":"g","key_3":"c", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.452258, "value1":0.4477592871421156, "value2":1459300159451076858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735408, "value1":0.5067192602284456, "value2":7587131980242978873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.108601, "value1":0.3076343047473968, "value2":5066182910326911353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419278, "value1":0.12962316194187043, "value2":8864033357034639760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336677, "value1":0.24448588819210298, "value2":4567827434979480913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131541, "value1":0.02538083388681256, "value2":6449819714583284688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235312, "value1":0.7537710944451972, "value2":3840033782203811299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713440, "value1":0.6660557019192793, "value2":6317879942694845725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250100, "value1":0.2226451425042414, "value2":6756354166360377412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.545039, "value1":0.7203310314493744, "value2":6637075765032095898, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846538, "value1":0.5133638544522705, "value2":6146673076285104795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.099134, "value1":0.26713136173061386, "value2":5047631075889612988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.999599, "value1":0.19120338269199497, "value2":3009559225203387009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456369, "value1":0.6496125050532283, "value2":8358687825156497010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.882684, "value1":0.4767076839685737, "value2":2058131066392704343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326891, "value1":0.0072166654150041835, "value2":3877874561619455924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.753206, "value1":0.9891987223468547, "value2":2500770621881849387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.505150, "value1":0.33485751726586077, "value2":3075226382158740800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_6":"k","key_3":"h", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.337847, "value1":0.7587725777413157, "value2":4030061892629350122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.061555, "value1":0.5715183894256888, "value2":1380477982282562120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102225, "value1":0.18128048387136908, "value2":1107305884228635610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.663571, "value1":0.2082433172088735, "value2":2331488443653185612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400352, "value1":0.36344640764280667, "value2":4156784209596494480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567282, "value1":0.6853752986060114, "value2":1738184775840508143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580625, "value1":0.21556187277712435, "value2":1716362597563830669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.068750, "value1":0.5617303119904301, "value2":215978150703712746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948465, "value1":0.4015798445122638, "value2":4918723285934260509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818268, "value1":0.4923649301244579, "value2":2640505566674328479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344093, "value1":0.24900402040054898, "value2":3729697701643357874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096571, "value1":0.84402994415599, "value2":5393515911153006286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880117, "value1":0.7901487854013907, "value2":2074107282217049907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923332, "value1":0.6838866899470619, "value2":1947909482655854334, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.432183, "value1":0.35953677139878504, "value2":3191972662401496477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251030, "value1":0.9314169082560206, "value2":2792551904040732916, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.315849, "value1":0.9521659578315848, "value2":278261838735054052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.790453, "value1":0.9152891069156645, "value2":1668356628128321724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.260477, "value1":0.2126605025888137, "value2":6877799862560969541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.213669, "value1":0.023418304769384207, "value2":4056268380917249508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.647354, "value1":0.1608657659472805, "value2":7997008030753632127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364473, "value1":0.2924502770902306, "value2":2291771922128424424, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.399561, "value1":0.6211538305973102, "value2":6848308885300872697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135252, "value1":0.6565821688014496, "value2":4331814359581194434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365542, "value1":0.9118204132722055, "value2":2417655238291803790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758482, "value1":0.17112508145844177, "value2":1991118257939966294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.032543, "value1":0.08125554957598005, "value2":6803892048654362563, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.520072, "value1":0.4275632442269591, "value2":2250122493336577938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.900940, "value1":0.14035051591842881, "value2":3790466116523762354, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794030, "value1":0.6000173402567552, "value2":1696544400981072969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.760746, "value1":0.10139199442682838, "value2":2726344565529290448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687331, "value1":0.770768299751441, "value2":8765361349894188719, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035711, "value1":0.48562696854381604, "value2":95585559412343711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739784, "value1":0.5825982451536238, "value2":2923240563522303723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454300, "value1":0.4006888000990195, "value2":2468975659398879205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884626, "value1":0.8026957862944495, "value2":3605855909002945057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395577, "value1":0.28352828399222396, "value2":7602348181385211161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250646, "value1":0.2316228525268274, "value2":29961024893090406, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421024, "value1":0.7990468794285002, "value2":2264750479744979673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380413, "value1":0.677589393211687, "value2":1235396941989449609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_4":"f","key_1":"j", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680912, "value1":0.25625503996995874, "value2":2396418552817135385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624528, "value1":0.5836651399092099, "value2":8089438271359233374, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.578517, "value1":0.7858613429590018, "value2":3691936139324370664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_7":"h","key_0":"b","key_2":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.824035, "value1":0.3471915666356168, "value2":7523600740099466883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069777, "value1":0.6471590850539041, "value2":545376892289254611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390523, "value1":0.8681181997676135, "value2":5603080382596898622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.590994, "value1":0.6390330221088167, "value2":6898933669096943734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222958, "value1":0.7361676398339201, "value2":7798894502817396812, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.313614, "value1":0.9956617360294369, "value2":8114343594051219927, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.471497, "value1":0.2028990027253043, "value2":8672046519486478878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299387, "value1":0.90821231206369, "value2":7139029616253188226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_7":"c","key_9":"g","key_2":"j", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822834, "value1":0.7462012785016429, "value2":2497140784779218477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.958689, "value1":0.6493610280581492, "value2":7375518635495424751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755680, "value1":0.3173769236575059, "value2":1038885732982663879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.906259, "value1":0.7803156579997583, "value2":1671762930600330124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320586, "value1":0.047928946477745184, "value2":2364817790122909266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.237873, "value1":0.5919698389786022, "value2":2456361237893496498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_9":"c","key_2":"j","key_4":"d", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.183324, "value1":0.25157377599570596, "value2":3302857039571325572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427134, "value1":0.9570675751476113, "value2":241686111761765133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.353599, "value1":0.19102611162010216, "value2":7635824178108933655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933576, "value1":0.5497618436271264, "value2":178483151772454986, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.888923, "value1":0.6996148110998511, "value2":7261028738310365771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414681, "value1":0.45596913295291375, "value2":8486467978432271730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354962, "value1":0.6882059355313528, "value2":4219371503140919650, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149786, "value1":0.048507930319514725, "value2":3361043402215780354, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320396, "value1":0.48978604231367256, "value2":8391350623429868941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.189903, "value1":0.49045511147932325, "value2":4568889448781542889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.531173, "value1":0.015565420626107091, "value2":1532700275127700107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.911622, "value1":0.9315116645649003, "value2":845964128004551720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447558, "value1":0.6852687200103666, "value2":1997580526601582597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389703, "value1":0.7679907754096043, "value2":8650394236218808766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221750, "value1":0.8409665471906171, "value2":8068148192803213421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.679200, "value1":0.5889377996015627, "value2":6603283664639278646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.658212, "value1":0.20335938525562192, "value2":6814120282733950138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_8":"c","key_4":"d","key_7":"g", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917850, "value1":0.07209516605018491, "value2":3465054040297242627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.463048, "value1":0.8121431937413348, "value2":7711657955494481167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280683, "value1":0.9792224347770722, "value2":6637142200642308638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938341, "value1":0.6616785812007566, "value2":4430772053877820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778057, "value1":0.21301644528770988, "value2":5550954213897210258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.435560, "value1":0.25143911919093503, "value2":8602791844656832235, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980296, "value1":0.4483217784998845, "value2":8061950324304064559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.845189, "value1":0.23223688199088727, "value2":2019205569808444797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030298, "value1":0.009150371898544056, "value2":2179401507551835981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_9":"e","key_4":"j","key_7":"k", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439657, "value1":0.5569344336018628, "value2":3896413214679997959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037278, "value1":0.0752646633945245, "value2":2986036073412577705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005641, "value1":0.470809080359862, "value2":7672259019023724818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169543, "value1":0.6908284999086921, "value2":6236799599986001574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.168435, "value1":0.1656190822947285, "value2":2938449310867540367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901700, "value1":0.6243405512256929, "value2":2234990933462726180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282060, "value1":0.14169460807404005, "value2":820280722029319872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868372, "value1":0.8809198418653182, "value2":7786878634034667094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700989, "value1":0.028109982788470232, "value2":1201575202891520734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.304734, "value1":0.8385104418360362, "value2":6703304420616815596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901530, "value1":0.8011320750502264, "value2":7539936763030777880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862233, "value1":0.47963788532278256, "value2":3960941336665870754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.511307, "value1":0.1832580868434161, "value2":7188980001326450399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700199, "value1":0.6958679254162045, "value2":2878046054907323632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299563, "value1":0.31054488059495927, "value2":8075467755079023785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383092, "value1":0.290554447303189, "value2":6694033035935161393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.696053, "value1":0.16066281765668294, "value2":3961201674767505870, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392128, "value1":0.5736001777474582, "value2":168331305812241403, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_9":"h","key_3":"j","key_5":"a", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752731, "value1":0.4825725339245043, "value2":7800665573914476062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293547, "value1":0.12126767569308682, "value2":1612512565132344690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_8":"a","key_0":"b","key_3":"d", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874204, "value1":0.7014202076829791, "value2":476647849984184674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.132714, "value1":0.5122323816164429, "value2":646780496771729886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358264, "value1":0.8238092496263985, "value2":9061776775816645023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358876, "value1":0.15679289393208987, "value2":8163846432367416312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921374, "value1":0.8339787140810333, "value2":3649804286907707134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395687, "value1":0.875997280853855, "value2":9212367459114589124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.459642, "value1":0.7528040980492385, "value2":4150547104368392230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782674, "value1":0.7121212215364499, "value2":6531310437181949577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.229939, "value1":0.32814366772056375, "value2":6374161097327904245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594347, "value1":0.6163078141302316, "value2":1538685760517154018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.009874, "value1":0.36102842691855586, "value2":1131193318052688252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.606094, "value1":0.7074086026345966, "value2":7018588286889757726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.777708, "value1":0.9795566345359867, "value2":7358687237436509918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_4":"a","key_2":"i","key_3":"k", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.963990, "value1":0.9872552891680104, "value2":6171290753066239119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772158, "value1":0.4830977012506835, "value2":6211566808050254036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.952768, "value1":0.43798623227092814, "value2":3513514828808633618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566018, "value1":0.7713759201698291, "value2":5320413786417127410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.563469, "value1":0.6599121888373632, "value2":1404388254364767252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786109, "value1":0.9075953036601083, "value2":8494075902382221765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812207, "value1":0.5042296550006388, "value2":2362200937900291190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093099, "value1":0.3386496883604208, "value2":7071312823587957868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.381119, "value1":0.031714519630475106, "value2":2849328345417435639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.815827, "value1":0.14737457839891155, "value2":199255383994378266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050041, "value1":0.5503190199883978, "value2":7356958752349024534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.472083, "value1":0.12465887530301682, "value2":6392360821928490144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713056, "value1":0.12067504962982098, "value2":4673070422231359809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.034034, "value1":0.25231429703394476, "value2":5489447259456982836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502818, "value1":0.4156967719918407, "value2":4612443309724781532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427071, "value1":0.24921718574451088, "value2":7646229328406586652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643652, "value1":0.07902787804812644, "value2":34188484750195348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.488097, "value1":0.7204444807131233, "value2":4412925367441450672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.180195, "value1":0.3056776312063749, "value2":7687653658691030932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.873382, "value1":0.6827451433784463, "value2":1751876103177277887, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529999, "value1":0.39651784939201806, "value2":1747188854631655971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971674, "value1":0.8695844176281972, "value2":2878591399245613156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702173, "value1":0.9786793419693177, "value2":4388723377141224465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844886, "value1":0.690355517305138, "value2":6283981589400857877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298362, "value1":0.6666814629745373, "value2":6187669811817639042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266804, "value1":0.7091063369563013, "value2":4664946045775147969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400948, "value1":0.8630878707275965, "value2":7287257182404940937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_3":"f","key_6":"k","key_0":"b", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135133, "value1":0.6122563921627874, "value2":2139209888725866722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.370672, "value1":0.01650778323939298, "value2":3489547069770540399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.279256, "value1":0.7619200195662909, "value2":5915401631721082002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170496, "value1":0.01005373661086586, "value2":1482576688135431000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499284, "value1":0.36435815211219275, "value2":1326048436273736829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185099, "value1":0.9391170841095738, "value2":8924546898546218998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772826, "value1":0.7574432732083424, "value2":4488845007322620734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_6":"h","key_5":"f", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169744, "value1":0.5739689347042248, "value2":741420931729893242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.998044, "value1":0.8965817632976586, "value2":6239062200473429860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594662, "value1":0.6301979977894112, "value2":2054875786295626784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755433, "value1":0.06261106040596412, "value2":5455645386351916850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.682584, "value1":0.03167274357401835, "value2":310883965859994658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072455, "value1":0.9148162667555992, "value2":382836579698407350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.725171, "value1":0.3647728943926306, "value2":2421892347307790025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798022, "value1":0.22510760744817873, "value2":2442091377551743699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688933, "value1":0.07828818881955478, "value2":6735636563745227652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.649962, "value1":0.9679958182791478, "value2":2607996809419409341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_8":"d","key_9":"h","key_1":"d", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.382199, "value1":0.14064207963814382, "value2":1623159006724541520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293647, "value1":0.919909760697381, "value2":58019425907871235, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715683, "value1":0.40584027014114893, "value2":7884356555342436928, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.496280, "value1":0.6458580424511011, "value2":1385234809290785609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037411, "value1":0.5356364996267606, "value2":4485181030850732806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.092167, "value1":0.08471879259284998, "value2":1047090205307301338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203791, "value1":0.9626197309889007, "value2":458130535579789756, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_5":"b","key_0":"d","key_2":"i", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395018, "value1":0.7684680824595476, "value2":787781374740198173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408536, "value1":0.147741569501137, "value2":3967017244497727033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693578, "value1":0.011896444619318294, "value2":4942230313540034093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780418, "value1":0.8750655230631833, "value2":8170450018971591774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812004, "value1":0.011539724469496575, "value2":1314191368195668298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.090381, "value1":0.870578773268634, "value2":5701169500016143920, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512721, "value1":0.9238819426049193, "value2":5759090244464429103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938235, "value1":0.5991043580028484, "value2":7404773139120927140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.751490, "value1":0.1795090853269717, "value2":6360470713237738478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267071, "value1":0.625707646354563, "value2":6040902311152523365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.002110, "value1":0.08011219998212876, "value2":5850595301611238867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257610, "value1":0.8187562858163207, "value2":8061987864802598079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.446622, "value1":0.7315172185892279, "value2":2428635533374338014, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730403, "value1":0.7209673234694193, "value2":8416924883569369777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.814732, "value1":0.8815237283497432, "value2":1852270652578106849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.318594, "value1":0.610086630044871, "value2":4524314099691048466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.852591, "value1":0.9383715352044596, "value2":453270928675336465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989585, "value1":0.8337913033606761, "value2":6440744488245743508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666424, "value1":0.9182270518817748, "value2":6618322333494361668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.010788, "value1":0.6687390259133706, "value2":1979105606239962267, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_9":"a","key_0":"c","key_6":"j", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752852, "value1":0.4586423067764851, "value2":3289626005749005579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005307, "value1":0.4780784157090063, "value2":2142574953538675780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752795, "value1":0.7908677214518265, "value2":5326267086837385392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202377, "value1":0.001998326079086547, "value2":2359915256333417411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216786, "value1":0.03377151611828389, "value2":5536807322346137933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243069, "value1":0.48298272047333335, "value2":6984955197084383580, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199895, "value1":0.6668507451944505, "value2":1160751712622009388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605559, "value1":0.014501073348709297, "value2":2976544013604671560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046312, "value1":0.45283346882557124, "value2":1106313893608419094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841418, "value1":0.9439193659449161, "value2":1585819181629654484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.816329, "value1":0.8314432076191595, "value2":671388396305183300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.672039, "value1":0.19428152515853786, "value2":4678503251263728412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.652936, "value1":0.05999646842980615, "value2":6549432288618958292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212442, "value1":0.6061066332871502, "value2":6044859751282824016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200351, "value1":0.9858548441265623, "value2":6095603519503403992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047144, "value1":0.8823748802173365, "value2":2346527148275584633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272669, "value1":0.49903571079222264, "value2":7606961266627055929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244016, "value1":0.3505008339279857, "value2":9165454668662797150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_6":"e","key_0":"a","key_4":"f", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595436, "value1":0.1769134432972796, "value2":4183298646463696655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_7":"e","key_6":"f", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.832887, "value1":0.05781334979440461, "value2":2976605700126921242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.417486, "value1":0.03855593830020744, "value2":6499287743834404350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850592, "value1":0.8819483685824858, "value2":3365942495366951256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904339, "value1":0.4096492831256871, "value2":7870603688751891889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849740, "value1":0.5985549217817763, "value2":5972692186135936344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080962, "value1":0.6508975263449264, "value2":620195756611450814, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.172604, "value1":0.36758717062599394, "value2":3093546059237948200, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.611964, "value1":0.09212647541898711, "value2":600064894791793802, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.136731, "value1":0.3518781010346145, "value2":4172668838781409561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221150, "value1":0.7696791785774878, "value2":4882972115027784491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118097, "value1":0.3822037091545315, "value2":5547505099823892976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.209184, "value1":0.3119528694472212, "value2":8126435007719933970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.708227, "value1":0.8414289260599137, "value2":1887259462865725736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127263, "value1":0.417919506626244, "value2":3560935891426387222, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.500871, "value1":0.16110435350355187, "value2":2428171488019731436, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.848583, "value1":0.4961159478803282, "value2":5456304085389531044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.468311, "value1":0.9289954682212651, "value2":4285742227173179225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.160825, "value1":0.59740490878922, "value2":9140238758107732393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_4":"d","key_1":"f","key_2":"j", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423818, "value1":0.2325925821959117, "value2":7321787707684172397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_7":"k","key_5":"i","key_6":"g", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.689012, "value1":0.5753370946686959, "value2":5125074341961139213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941993, "value1":0.009508850201920292, "value2":2477284877979206282, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344874, "value1":0.012483207365877868, "value2":4898429158465049577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625728, "value1":0.9281594076368007, "value2":771428325581269474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701217, "value1":0.8504319614540372, "value2":3030161014659152421, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.288648, "value1":0.7225335235603462, "value2":4184194222447924377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735765, "value1":0.7338027307218647, "value2":8053319255149398119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853312, "value1":0.31226366748735784, "value2":48160047989763460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289371, "value1":0.5473984505709404, "value2":2530747852470429293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825683, "value1":0.2907108545003152, "value2":3061253498575467483, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532231, "value1":0.5859058541023265, "value2":4975638816250812874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512440, "value1":0.511793469522306, "value2":5419688896766823090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709116, "value1":0.0431590704274129, "value2":565903028384253279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709410, "value1":0.45910825737421945, "value2":425558835807211279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.970542, "value1":0.1219905172153899, "value2":24331626097709718, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.043601, "value1":0.49284843085319935, "value2":7672646678928068810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739151, "value1":0.4246916831482226, "value2":4312260845153215448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489280, "value1":0.8582555742505251, "value2":3110601136234274420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935791, "value1":0.4664631517475295, "value2":4539773222290649379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.654384, "value1":0.10389342147634097, "value2":5731036292461383708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193783, "value1":0.5459140576966053, "value2":6006070322533917783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409382, "value1":0.472091611755319, "value2":4277330658526282078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423822, "value1":0.4751601960954781, "value2":2652715507916534737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.660521, "value1":0.38225985421870495, "value2":9011573276516869930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975891, "value1":0.5238616298909866, "value2":472916590695431743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843227, "value1":0.6679925899402425, "value2":906046342124593465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829312, "value1":0.16298135421846682, "value2":1838361763539535864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480674, "value1":0.259388407981778, "value2":6440770726430652573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690879, "value1":0.0735354737983211, "value2":9181104589648553044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126782, "value1":0.4486397942700849, "value2":7462094819783188022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948901, "value1":0.9959329498625173, "value2":5479458805805092746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127088, "value1":0.9575385572151783, "value2":2275563983258531267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_7":"b","key_0":"k","key_1":"e", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259685, "value1":0.3372857504409974, "value2":6098187454917492671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336884, "value1":0.24553452230487985, "value2":8241989909296706238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_3":"a","key_9":"e","key_1":"g", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169076, "value1":0.17392408086361133, "value2":7622066852267723117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977238, "value1":0.5433155948072138, "value2":6898148363520327391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.278455, "value1":0.6005033834895888, "value2":8001662694808432224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993722, "value1":0.9597867721155696, "value2":1299805439881399559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.530999, "value1":0.2209888826520892, "value2":4066778144109849949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782378, "value1":0.7523792160016672, "value2":5556510651542494721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.219804, "value1":0.1717195689538366, "value2":7048403972015561437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.946856, "value1":0.5985084131744225, "value2":4885828087902675075, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759489, "value1":0.27172565847552677, "value2":7664668067438271399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687774, "value1":0.42250850460802286, "value2":7088707511295378411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560049, "value1":0.2616853019800064, "value2":2950184265013984605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097585, "value1":0.2684028422029654, "value2":675122184817942746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205813, "value1":0.44731824636014533, "value2":1428863070750239410, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.543640, "value1":0.544977219392789, "value2":2157919960028175094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566856, "value1":0.8008181036581771, "value2":7402387212023409312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.765221, "value1":0.8596387394697663, "value2":1232636922510789742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.386997, "value1":0.10171781350903462, "value2":2933191729226405023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.674282, "value1":0.05504930364585503, "value2":724010655267136473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820827, "value1":0.1512145830665697, "value2":6612187124412654634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537712, "value1":0.6818284933003707, "value2":833546983771380703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.020134, "value1":0.7293182995525076, "value2":2126763229656176078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.919742, "value1":0.47707314485124014, "value2":8035884626808128837, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_5":"d","key_7":"j","key_1":"k", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941486, "value1":0.13322259126479358, "value2":2182305210138540441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.089091, "value1":0.6999475934755898, "value2":5942262186593070970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.922220, "value1":0.01137462679977003, "value2":4337636877649564119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846010, "value1":0.10326336153396976, "value2":8556623018979218867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_9":"b","key_2":"j","key_7":"f", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361361, "value1":0.9603555432714893, "value2":7276258796427191775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303375, "value1":0.9198009606494528, "value2":3654960055261327174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354372, "value1":0.9154736221447883, "value2":6754725607301294498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836769, "value1":0.17500751807732254, "value2":5225830062891206724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.514481, "value1":0.712612132062798, "value2":5405795909104498480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.857662, "value1":0.4771648896771537, "value2":9197955420056459031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218504, "value1":0.43388279897052595, "value2":4876293374213376800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854318, "value1":0.29673697161398044, "value2":2557767286012732771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.621120, "value1":0.07462842813102197, "value2":5246869953360521284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482880, "value1":0.4261787117422624, "value2":7122235599178548659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880280, "value1":0.38443671400954443, "value2":5167341510277945257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849192, "value1":0.5161558417921461, "value2":4578699148249247151, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_8":"g","key_9":"d","key_1":"i", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826648, "value1":0.873997512950574, "value2":7985090519609883665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_3":"i","key_7":"j","key_2":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199619, "value1":0.8346974218021976, "value2":8292656933621980920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.959402, "value1":0.6444072126847691, "value2":4320840087483118841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117065, "value1":0.9272062952128088, "value2":4910297155014939874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.551716, "value1":0.13475804603313432, "value2":8222696695222433380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643768, "value1":0.8311693472811903, "value2":7965400634034447655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.484767, "value1":0.5064114690168263, "value2":5755735513825830154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377345, "value1":0.1504036301883094, "value2":2988203951245132390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991676, "value1":0.972772556086247, "value2":1470890803841584803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361353, "value1":0.6540839592057779, "value2":985314732060254627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585584, "value1":0.7709261757498476, "value2":2565995410162492585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.704212, "value1":0.3118041755703027, "value2":6201623885191421475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.518173, "value1":0.9814365163985729, "value2":8388580742116222397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.187366, "value1":0.6732011290892043, "value2":5408814647988876882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.296035, "value1":0.5028422803953023, "value2":7236554080452628944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.768145, "value1":0.8394583660519598, "value2":5325173567069752762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549064, "value1":0.12911593374108088, "value2":8305911548302819241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_9":"h","key_1":"e","key_2":"e", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758468, "value1":0.0009372001034959489, "value2":5525854549400493996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.525822, "value1":0.9778487221852099, "value2":2741333418274183362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396148, "value1":0.9074357631670523, "value2":8809499083604789974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414129, "value1":0.8132493023109502, "value2":4942425104780689529, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.398880, "value1":0.39814811004109957, "value2":6997600198610651439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.667851, "value1":0.7812827159518755, "value2":111949400321343617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.065047, "value1":0.5608722991855313, "value2":128957686943303349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.494481, "value1":0.17816439204883128, "value2":8141342025149658982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.515335, "value1":0.9138017166447161, "value2":4317898835838327168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713454, "value1":0.43899440111826654, "value2":143487884063527116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185777, "value1":0.5460630190657308, "value2":3180840112779032107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450264, "value1":0.9313181115462055, "value2":8328870888335802008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_5":"k","key_6":"e","key_1":"d", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567912, "value1":0.6046426149595721, "value2":3206194822374359841, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.896555, "value1":0.6026393505223315, "value2":4823045987798108744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710017, "value1":0.5880756405721221, "value2":4714155846612246340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265002, "value1":0.824988737224255, "value2":8146626468221994382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029140, "value1":0.9928569418325714, "value2":1115616498765210900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.994788, "value1":0.6191499887981317, "value2":1491651902318479820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310266, "value1":0.6540537390689438, "value2":8869916203173244502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786610, "value1":0.5759332726822003, "value2":7232889220252179533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036697, "value1":0.31309276046690226, "value2":4197456548662367361, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.684262, "value1":0.39030765212524027, "value2":8590977643860948235, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.295632, "value1":0.4747190839423866, "value2":2819114746608115923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.613481, "value1":0.02792670630263574, "value2":4669805487355644126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.462933, "value1":0.7475159252861773, "value2":2383279322222812533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179395, "value1":0.19102052131729996, "value2":1856517930791738382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991666, "value1":0.3546374487300389, "value2":6354333338863606534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.568174, "value1":0.4332276746738033, "value2":8645759887904070759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.321899, "value1":0.2992200951220634, "value2":3116962887997170979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390308, "value1":0.02747272568305014, "value2":926243844801181597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.805520, "value1":0.5096841522295752, "value2":7414378226567615667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230674, "value1":0.03960399937089453, "value2":1710851993634315848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.425150, "value1":0.4167707239792382, "value2":5244023674899540768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879335, "value1":0.4867774224489378, "value2":553896137255768087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.021415180040758975, "value2":6576886716131511876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.875807, "value1":0.22236040526017317, "value2":8247944924957719281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.596786, "value1":0.5382707900600514, "value2":6504957777564642511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853544, "value1":0.6591705273657046, "value2":6314593273328284879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.792273, "value1":0.8726534877662263, "value2":8304088035724056780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_8":"a","key_3":"i","key_5":"h", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050549, "value1":0.962790936441039, "value2":4045656520780865827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493970, "value1":0.39340114596435655, "value2":5294670971014916935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251929, "value1":0.5462996764305353, "value2":5897296033220091874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115726, "value1":0.8575474248088841, "value2":2146148594913460918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.787443, "value1":0.8185914401069716, "value2":6891346883169646879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.982163, "value1":0.8404209142239473, "value2":2046890513913645569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529940, "value1":0.15216580562912718, "value2":235763707139823565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036141, "value1":0.21797894391861414, "value2":2037043718914992589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489425, "value1":0.578085671307559, "value2":1968373430108550733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377536, "value1":0.5230650685409198, "value2":1948377520185967051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.644803, "value1":0.09118740593062584, "value2":3209698888069603256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149640, "value1":0.2149828010685946, "value2":324803347557649497, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.788006, "value1":0.35151805877228876, "value2":5736730574020425869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528326, "value1":0.12892234054630594, "value2":40722644873633005, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.517453, "value1":0.12096018419440699, "value2":8591818996293676301, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580879, "value1":0.18567989692889683, "value2":2424108579790648867, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.418474, "value1":0.10109548853357979, "value2":6805501278544557691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192558, "value1":0.5389855440079544, "value2":2468356112053849635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_6":"h","key_9":"d","key_5":"e", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700652, "value1":0.8426222872981881, "value2":5586434719045643994, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836161, "value1":0.6261463649204317, "value2":7998455739612119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.410987, "value1":0.19322483253914569, "value2":7602154106800870019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920472, "value1":0.07895320792744571, "value2":3621399116338923589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_8":"a","key_9":"g","key_1":"c", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058885, "value1":0.038506638786952846, "value2":8443519094492233492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.357953, "value1":0.49873265104684444, "value2":6689514258543734175, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444144, "value1":0.7257783571311162, "value2":2132381976912359864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.003034, "value1":0.21025100795379004, "value2":4437597220292182437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794651, "value1":0.0987214108191527, "value2":4856891238525466286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.101167, "value1":0.5530900889416458, "value2":5869924593453820746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813524, "value1":0.7032391307910569, "value2":2962680547459287448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102641, "value1":0.05280577735508823, "value2":6197989994765662479, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.042807, "value1":0.46787236774210145, "value2":6987091087352317601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023412, "value1":0.4117202092054757, "value2":5734736277249168119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.582529, "value1":0.14733901739015626, "value2":2008347713878225513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843555, "value1":0.45648522904759514, "value2":8748231721848476899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.609920, "value1":0.5332106841490003, "value2":3789122902542055079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_9":"d","key_0":"h","key_1":"b", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.004873, "value1":0.9250068088247592, "value2":4060601419917043509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598410, "value1":0.20935422794575423, "value2":3458524447228408749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327673, "value1":0.6622534351078718, "value2":1975526610989314262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701834, "value1":0.018453031281240855, "value2":7208203361477507972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.544888, "value1":0.022192838541852087, "value2":420807956823837942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_8":"e","key_2":"i", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.333657, "value1":0.6116791297905804, "value2":6780782056382077493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.224370, "value1":0.58595743970275, "value2":2273745727787058306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_5":"h","key_0":"f","key_1":"a", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874813, "value1":0.4160482798834957, "value2":5130896603060858656, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_8":"k","key_4":"i", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800557, "value1":0.6485257630686503, "value2":8645194213531066428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_6":"b","key_2":"g","key_3":"e", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798578, "value1":0.3562881109226897, "value2":1936635714706401113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222829, "value1":0.18800753190841593, "value2":3868467029118261102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394596, "value1":0.9715849939407708, "value2":2892398559190644533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088141, "value1":0.19677712178571452, "value2":1290002108232651066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046604, "value1":0.7961651460552367, "value2":6142675575156594148, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755275, "value1":0.26382736011465824, "value2":4790349544446362608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.300482, "value1":0.6458439650096045, "value2":5076464481756841749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294901, "value1":0.3937755673758644, "value2":4837753175838197566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717849, "value1":0.8465941273216573, "value2":4538488428913350745, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018027, "value1":0.7099513420503992, "value2":6589927070191288064, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686148, "value1":0.31341443389871104, "value2":4542163293900247497, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554562, "value1":0.6021518819649633, "value2":657935532838913237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549050, "value1":0.9913051319754902, "value2":9133655827436281017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.564021, "value1":0.3933144754852601, "value2":8691343872756358848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.555563, "value1":0.12216755310514293, "value2":8181517965767093469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991822, "value1":0.9046701031504584, "value2":7169764745600183537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536610, "value1":0.6245723927615995, "value2":5410550698568344254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.640481, "value1":0.7953791600529995, "value2":8896491255145975602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.983650, "value1":0.39162313489970674, "value2":5865250699451794074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.748358, "value1":0.3770029381067136, "value2":7671767291617145847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632308, "value1":0.9672641688276767, "value2":3043416827366033954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829101, "value1":0.63693380465596, "value2":3688916592463553468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.271187, "value1":0.7822523952945989, "value2":7538651950678502209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757851, "value1":0.4993335289484615, "value2":8433879559771522559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.022503, "value1":0.5685717542133112, "value2":5232283843486040657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046586, "value1":0.6991953129630171, "value2":767301512539468704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170088, "value1":0.9099698273235788, "value2":6375281583140134087, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.819909, "value1":0.9395198166293325, "value2":9208855013717569494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078675, "value1":0.35105170816316494, "value2":7441300028994205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977562, "value1":0.27747207540421476, "value2":2610474267165011454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.577027, "value1":0.4084658901530552, "value2":7348713799224820014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705040, "value1":0.13989635719875274, "value2":8770545555481284876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.821460, "value1":0.5135499070770553, "value2":1707896495831766919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369139, "value1":0.07309187404043328, "value2":5262455623289086241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536485, "value1":0.3546193597483191, "value2":5532328840062432345, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632890, "value1":0.87913605477188, "value2":4206324763882989783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.352893, "value1":0.22467668553540646, "value2":8020391351501404610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058110, "value1":0.08476564402932071, "value2":2402316898103266567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312920, "value1":0.14715981410041812, "value2":515215788552749041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335941, "value1":0.5607746607254104, "value2":4029056354863603408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493922, "value1":0.5228229815154212, "value2":5917395680452547526, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.721451, "value1":0.8180794492453682, "value2":1670905717947422538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.347574, "value1":0.7935601761833552, "value2":6084751528698562987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.722657, "value1":0.4075231573944958, "value2":7748408088453297294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.451863, "value1":0.3657057400324225, "value2":7302526748126423806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495468, "value1":0.10049524830706323, "value2":7515132691589300738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193570, "value1":0.5847911145927462, "value2":4656707981929867290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866643, "value1":0.9003805975682342, "value2":3827225598428319501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.535740, "value1":0.5365482035917958, "value2":3695801733453249426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.197799, "value1":0.7228560209917014, "value2":5697296448849523746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536756, "value1":0.2916665533141402, "value2":8616484406626615233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231668, "value1":0.5491980830875538, "value2":5401247188111922849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836905, "value1":0.6668911346325062, "value2":5956397501314084543, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.060078, "value1":0.41433534941101835, "value2":259739043603601632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933921, "value1":0.15068665469322526, "value2":4615815953640952732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.147080, "value1":0.17748824888193676, "value2":838395034976203435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662077, "value1":0.7882764714348716, "value2":4457961488907810285, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023154, "value1":0.24361462379316554, "value2":7839025824541823553, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826915, "value1":0.569504859021508, "value2":6251982817444532308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585595, "value1":0.8340020457470065, "value2":2938024413765410558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146190, "value1":0.9581727102338353, "value2":4413141265483838870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.245440, "value1":0.6417183759653234, "value2":5364021531028647444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493343, "value1":0.4938700840392069, "value2":3091997225543832896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807379, "value1":0.4690524091058998, "value2":3061093742738985330, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.208752, "value1":0.8639618600039569, "value2":3187072739982271504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.011216, "value1":0.18367276511432976, "value2":2360627840986057827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_8":"d","key_6":"i", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.440539, "value1":0.6977700817568234, "value2":3581694346818534680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489990, "value1":0.309872070478555, "value2":7457465639523954616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113130, "value1":0.6897957784827682, "value2":90019976280058795, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.604031, "value1":0.797243453747431, "value2":1234756804789352079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.881184, "value1":0.28155440991949554, "value2":4070028978929644169, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.367426, "value1":0.016167903292426008, "value2":4863636678376461059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.769132, "value1":0.967909876972054, "value2":9131005500727231455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.905490, "value1":0.6194024275206055, "value2":210353074026913884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.915138, "value1":0.6781979759382127, "value2":6372020977221093753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093709, "value1":0.6187727010105798, "value2":312866041452100946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758312, "value1":0.8273322264897901, "value2":7512565884920091651, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.284383, "value1":0.39739645167776044, "value2":8773985992873355471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216803, "value1":0.8653155039130678, "value2":8937229562905505386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.770263, "value1":0.9293790766191896, "value2":4560937013728510181, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096191, "value1":0.22323546625261562, "value2":8010469458396680860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.984434, "value1":0.9454942818827864, "value2":660593493740706018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222194, "value1":0.02420800916517378, "value2":5602553742039024765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752900, "value1":0.3672193480709247, "value2":6716506478528347816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.967703, "value1":0.2944363320939628, "value2":7783913193752729609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.600560, "value1":0.1440348243160143, "value2":9110640288768831559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400353, "value1":0.2233756240085349, "value2":2151981935371948568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.269597, "value1":0.9641297731318997, "value2":8169933506866910786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.486730, "value1":0.1006988853266946, "value2":3116477845424070312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953298, "value1":0.4756644907363518, "value2":2464865227545117596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705992, "value1":0.8173093209310923, "value2":8219739418749111053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.851844, "value1":0.28729444422491424, "value2":191979366247033806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616037, "value1":0.08828307120467993, "value2":1006916952645293364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.007265, "value1":0.43891779173017603, "value2":4642029911324023153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408095, "value1":0.9633558663387003, "value2":2764041604557827140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825923, "value1":0.12824733189500837, "value2":7952369571604255477, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853643, "value1":0.8488628783599163, "value2":3110877858489702411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686891, "value1":0.5538740126823626, "value2":7618257543247002890, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408459, "value1":0.2427343835864084, "value2":1637171870696137413, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.635245, "value1":0.42768890444095087, "value2":4013428109133869754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793205, "value1":0.23512439386454564, "value2":5799487535459689842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920141, "value1":0.9415332414772868, "value2":1520706400052294853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.930133, "value1":0.47818008921632604, "value2":4567447002026438924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.599550, "value1":0.9293362893760267, "value2":1195481394235809552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903387, "value1":0.11210859774886604, "value2":3335652489364502188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_4":"h","key_0":"f", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698033, "value1":0.3148016773642106, "value2":4939787017916634516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813706, "value1":0.3325590668831616, "value2":3252287249574729793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_2":"d","key_6":"b","key_1":"i", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.048837, "value1":0.053013772957975554, "value2":4819883401481045104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310950, "value1":0.41839244392518726, "value2":8733225969530074129, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.062225, "value1":0.858129811136947, "value2":7961256500615532109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428880, "value1":0.2502872252911392, "value2":7064927828432353118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_6":"c","key_0":"c","key_3":"b", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560002, "value1":0.11724155985130476, "value2":3115965127965032390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657940, "value1":0.5473444163031204, "value2":8727302484888278411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_9":"d","key_0":"k","key_5":"g", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554826, "value1":0.376684927630788, "value2":2585922874708871992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079791, "value1":0.9982500603963644, "value2":7391513557121794219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_8":"e","key_1":"h","key_4":"k", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.174380, "value1":0.13350980294424206, "value2":9115933928609206333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690448, "value1":0.8022085172421284, "value2":5503579136981394270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178959, "value1":0.5153762777175099, "value2":4750624720468516504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880368, "value1":0.3826318223193023, "value2":3994162284701586149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594211, "value1":0.0032667252772956594, "value2":211951125332337865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080419, "value1":0.3371270895888467, "value2":970359359029436118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354035, "value1":0.6830560620508377, "value2":351440888758257733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686006, "value1":0.15832224781392848, "value2":7973904723234304705, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688086, "value1":0.14780349920952468, "value2":2027463870413046281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.407959, "value1":0.04437078472668187, "value2":1566333717176214387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_6":"d","key_3":"i","key_5":"a", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943264, "value1":0.6495072511389037, "value2":5853361075489349645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400773, "value1":0.2676114612948212, "value2":6666522398737719346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228255, "value1":0.29124903807175784, "value2":128611338987653451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702835, "value1":0.39306652609769904, "value2":4553551121428133233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346580, "value1":0.9157209666515556, "value2":4177607716262676740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000990, "value1":0.3155726084083431, "value2":7473596942971857359, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_8":"d","key_1":"b","key_4":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380807, "value1":0.5730835803736063, "value2":6788452783424900655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018359, "value1":0.11046900836774676, "value2":61241953976306769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097897, "value1":0.6529098432421965, "value2":9123131154054179256, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118936, "value1":0.312336727346538, "value2":7294156215339952794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.256520, "value1":0.817765585396917, "value2":4276314621602341409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.239028, "value1":0.27988068073239764, "value2":8048493989378214976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.359463, "value1":0.33204054027814056, "value2":4286372625461580954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625120, "value1":0.8053374337323631, "value2":1700890355040268131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.519453, "value1":0.2764225180730322, "value2":3111696426903897415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.019685, "value1":0.5511793734194003, "value2":3991968620227079942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.040695, "value1":0.6967938051735891, "value2":8919927137208366483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750443, "value1":0.9319592750217813, "value2":7402732360820495311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.497932, "value1":0.6545667018250457, "value2":7616613351817353333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.773399, "value1":0.5580363664821956, "value2":314222158348617253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.869323, "value1":0.7418088662526952, "value2":8657709877874943583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812016, "value1":0.4480192862830272, "value2":8470890763908250393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079326, "value1":0.3872756599905638, "value2":4349661970334556580, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.186686, "value1":0.8815811840060216, "value2":938888750792010987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235117, "value1":0.8673030373216688, "value2":4951632884410168399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335964, "value1":0.6073261439232968, "value2":8008517076279782152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063573, "value1":0.12394184604832796, "value2":1362755555847887698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.685355, "value1":0.2361653248224142, "value2":2210259628223947558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029806, "value1":0.3517864573309168, "value2":6754043647766872898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.593026, "value1":0.38937941917051955, "value2":1945974862147158833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595711, "value1":0.9836966964700219, "value2":4159340747167974849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.143026, "value1":0.5945202518873367, "value2":3646754986945085167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733493, "value1":0.18903944802082667, "value2":8837255893708962102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.120646, "value1":0.4261710458329441, "value2":6946544380857137617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.163665, "value1":0.24245216879890383, "value2":555667735507028882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934782, "value1":0.31976487170355755, "value2":6301272802564976177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975795, "value1":0.2007851651150272, "value2":866319078855783895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630757, "value1":0.2788228743238552, "value2":2540783851122363232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.661317, "value1":0.5218066875354199, "value2":8869741999431102494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980967, "value1":0.7624092409640837, "value2":628203903335846730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380622, "value1":0.4837921884626896, "value2":3628530218991586661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.248953, "value1":0.02097807162306582, "value2":4598542592315882026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.384743, "value1":0.7608954854738191, "value2":1886663813765371215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.950140, "value1":0.2550158514673341, "value2":4065397223823067237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.516999, "value1":0.6292445006161491, "value2":4125156478992786356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733841, "value1":0.37206344867268365, "value2":3082112298619357468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086119, "value1":0.5738504787109479, "value2":5573707094695082413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800475, "value1":0.7767745931253813, "value2":6702226199646136395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.175876, "value1":0.8346328055447434, "value2":1066680083349750545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752359, "value1":0.7257723037237084, "value2":7568317247183343047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.165816, "value1":0.19937439179339156, "value2":2068495750767911677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.597980, "value1":0.6555890535602861, "value2":2883065227864053972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.138569, "value1":0.22532513024599726, "value2":8297154458719010882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.668715, "value1":0.11645984529741488, "value2":5555518408390232617, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063265, "value1":0.001988016117254865, "value2":8483308279719955566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086644, "value1":0.4644931461203292, "value2":6366519140385299368, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.140571, "value1":0.5416650487590775, "value2":1665979086436164947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.045805, "value1":0.9888897436719353, "value2":7697021265197143502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272912, "value1":0.2088625700639475, "value2":8643238667192348189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_9":"k","key_3":"e","key_7":"b", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289278, "value1":0.005139072821858179, "value2":1003021960883868455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.242063, "value1":0.32725526486053264, "value2":2911166424297597473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.393077, "value1":0.6314586962746932, "value2":2631691711252794732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.274319, "value1":0.9496413905130506, "value2":3462918152034782288, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659954, "value1":0.9955163986622194, "value2":2213038727032863641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657842, "value1":0.11877952023160425, "value2":2754577958762836530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146197, "value1":0.5152391159350134, "value2":4396760573611697803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759786, "value1":0.051119901574673096, "value2":965796598734594927, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.891935, "value1":0.6695409149263635, "value2":7204593893035944941, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096828, "value1":0.7785299263381299, "value2":843269019498255824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822107, "value1":0.06887356745350987, "value2":3890683236062212546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818726, "value1":0.3529788280342954, "value2":2023962428787621327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.356164, "value1":0.8626419946204176, "value2":3137373073213134707, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855692, "value1":0.3727202998903748, "value2":1868074746512330224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.899737, "value1":0.49932818019000214, "value2":5247723900283585273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205758, "value1":0.752048623202744, "value2":405978535011434243, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.492517, "value1":0.7587591048194695, "value2":570855823822973980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025167, "value1":0.9352906688766658, "value2":409025014085718364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524374, "value1":0.456131184246001, "value2":6492711389883342442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.173025, "value1":0.7216031002138218, "value2":1900883846963453532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991241, "value1":0.9435537729789849, "value2":6744125432328938384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126845, "value1":0.9483471232225058, "value2":2429204263257823531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079973, "value1":0.14998274238182085, "value2":1609358502486987778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257883, "value1":0.09750371692825593, "value2":5050315729103715843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.074081, "value1":0.21018141055409362, "value2":9169596995481234183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218738, "value1":0.9047134659722221, "value2":9140209877004134552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.745794, "value1":0.39056909591189853, "value2":4170147818303980952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179295, "value1":0.9580414905588824, "value2":4615545745316982257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442990, "value1":0.018901929544229163, "value2":2360761388936503714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.422188, "value1":0.5272513199907374, "value2":3200950480296318673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192831, "value1":0.33577000839194515, "value2":8465993285574620045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.227691, "value1":0.814891459750041, "value2":5020817184863163049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_7":"f","key_1":"d","key_2":"h", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171624, "value1":0.32281185598307915, "value2":8148752929610663940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_2":"c","key_7":"h","key_1":"b", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.490347, "value1":0.4777481785589013, "value2":2903978983753469512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217872, "value1":0.6839140142887666, "value2":3130769003103218800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.992440, "value1":0.1702388495618268, "value2":6704155277469548917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.834388, "value1":0.855009067710764, "value2":8656769438285137265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230809, "value1":0.6026285375258597, "value2":1622052925759175826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344218, "value1":0.6418025631389815, "value2":8245500230948114788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086815, "value1":0.450124296857116, "value2":4866053118697224871, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802304, "value1":0.9289196028774398, "value2":6637963067322995548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_6":"j","key_7":"d","key_1":"e", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.840991, "value1":0.6695124623580792, "value2":5437276365728147032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.833585, "value1":0.22730504686256575, "value2":1520963367111518952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.744045, "value1":0.8193501328748979, "value2":279541774649644194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_9":"g","key_0":"b","key_2":"h", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.736126, "value1":0.6177732267307673, "value2":738077807298530261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131891, "value1":0.46856485152824584, "value2":1245296884531108950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419961, "value1":0.9237029205265445, "value2":1223547652782219884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512633, "value1":0.3170684617445516, "value2":3777467556351824325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630355, "value1":0.021704073637768814, "value2":8674907845809369167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.309670, "value1":0.16245539540033113, "value2":5206956291412580457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.348146, "value1":0.7658591727792013, "value2":8113096525411729139, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346099, "value1":0.09833103805087655, "value2":3329067421410580481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086337, "value1":0.8850305283927763, "value2":8626010840258936486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479373, "value1":0.20404187483062786, "value2":2412557827504873850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921982, "value1":0.7871473016496364, "value2":1874563226125050237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415457, "value1":0.9305937676701617, "value2":4246181816757480088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035701, "value1":0.11812212256088726, "value2":7168381109733048473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.225156, "value1":0.9869870812720318, "value2":137813480312065053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836380, "value1":0.860500054517146, "value2":6804060104865284962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289062, "value1":0.385288303480441, "value2":3917001550422656910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_7":"g","key_1":"k","key_2":"a", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885780, "value1":0.1315685249986314, "value2":2596575576892432777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948363, "value1":0.36071231455630776, "value2":628986020889092104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369788, "value1":0.40617296244272144, "value2":1817522335798570734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086188, "value1":0.6439285363835136, "value2":4867721675008841698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.995659, "value1":0.2954211415738649, "value2":316663728175376955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.129393, "value1":0.050215829131005675, "value2":6231361900716566583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400907, "value1":0.39112211803964236, "value2":5808809619414218326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_7":"f","key_0":"d","key_4":"d", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717436, "value1":0.943850779238098, "value2":1447465700474277383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217700, "value1":0.1707128166287673, "value2":4689212155367810810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_9":"e","key_0":"b","key_7":"c", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.908215, "value1":0.13836337191309858, "value2":3286491733476954470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383068, "value1":0.3171639131993519, "value2":5768120780013343252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758654, "value1":0.8248294031178783, "value2":1259633164417337834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885742, "value1":0.7054910487633926, "value2":965825539457944695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450755, "value1":0.5160791051802902, "value2":980291621965651603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280407, "value1":0.20023478153485316, "value2":216774102527727538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953286, "value1":0.17708932991625895, "value2":7861520056013490110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428154, "value1":0.3840775072053381, "value2":5128389599856507984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650144, "value1":0.5475742595996844, "value2":37238921657180760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880799, "value1":0.27347132520371564, "value2":7086278310918665364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179258, "value1":0.09483479166767275, "value2":3722096865477015794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025175, "value1":0.24738190446733826, "value2":2326403401160366476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680444, "value1":0.6441894024482266, "value2":2704266700740984761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830593, "value1":0.6314118537062262, "value2":8997005770669681872, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_9":"h","key_3":"c","key_5":"b", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177647, "value1":0.3203303212434556, "value2":8965923512665629393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923489, "value1":0.29642566667944875, "value2":1203313530737226131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.811446, "value1":0.5561455146353509, "value2":570880137152142782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544802, "value1":0.9164364466036977, "value2":3723655635493793654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.542566, "value1":0.7689437007202478, "value2":3938961297360550598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819459, "value1":0.4225560622422357, "value2":2765920289473813577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971131, "value1":0.7588578513174423, "value2":554701584189491661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638701, "value1":0.9383542993382983, "value2":2675775339679034911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658985, "value1":0.8385957143790167, "value2":9118144992996965702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449591, "value1":0.6354046681746691, "value2":4796404340287643460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.851526, "value1":0.7827632730352198, "value2":4383742830241177102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929626, "value1":0.7374713580232243, "value2":6076523832966313810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.289625, "value1":0.6865021261714863, "value2":2978049869639889816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394738, "value1":0.6344060546968167, "value2":293482016535092319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.487799, "value1":0.01884442354627017, "value2":6314666407653758310, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.189751, "value1":0.5131352606856101, "value2":5370657937415537626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.532641, "value1":0.7500243297537671, "value2":5814407549612083184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413938, "value1":0.35879356972502013, "value2":7060341260629410136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464627, "value1":0.5028910273480031, "value2":4767839332233680092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.840321, "value1":0.13633703551235915, "value2":2307803453888302997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177282, "value1":0.5212414108183161, "value2":7948818597964566740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394656, "value1":0.22032364884959718, "value2":8010953658147063588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_4":"b","key_2":"j", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.193561, "value1":0.6737972615452449, "value2":7205681986933877897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919748, "value1":0.5508022412545106, "value2":1935155841798114792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.549841, "value1":0.6412542942797744, "value2":5725068566993480801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755602, "value1":0.7570856701300407, "value2":2582449631680230244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569314, "value1":0.9682571063393881, "value2":3213998604408735177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220995, "value1":0.4682404519517274, "value2":1296712240822053674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030188, "value1":0.13463362539315563, "value2":6167313691161388680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955102, "value1":0.12893740940580098, "value2":937550317375879979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912054, "value1":0.23584661351992364, "value2":8270059203922837294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248552, "value1":0.873739706342713, "value2":5483897643292875706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.668856, "value1":0.06480463060086489, "value2":3429839268398826587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628990, "value1":0.06856495811246685, "value2":2353082280387987376, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.364427, "value1":0.8257731797802594, "value2":8162649044337833619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488610, "value1":0.47476344691483213, "value2":7563742944450884141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.659600, "value1":0.22983785061318257, "value2":5566259052586123346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715473, "value1":0.5778415124343756, "value2":379000163266106555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265567, "value1":0.856852436111815, "value2":96718295321822248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647902, "value1":0.9996700403145788, "value2":6837057660421261848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.866334, "value1":0.8219801167193398, "value2":8280931000915145951, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144409, "value1":0.604589772741639, "value2":4205152374038619548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192185, "value1":0.27115827101755235, "value2":5770338217899591097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300761, "value1":0.009332804989256857, "value2":4018403627826051427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.312951, "value1":0.45431720098201106, "value2":2937253947933810175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_4":"d","key_0":"j","key_1":"c", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647625, "value1":0.3268497558683786, "value2":5914220923444612314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878656, "value1":0.9409966965511731, "value2":1289273008938800365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057134, "value1":0.3301792558402197, "value2":3318812578583185116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.439955, "value1":0.2652202727470005, "value2":6508684150448276421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827995, "value1":0.5808204202626653, "value2":2858312188918612316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863057, "value1":0.013094412054074, "value2":8217074167029879656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936866, "value1":0.17076998229838808, "value2":1405785613574281058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.188817, "value1":0.4102266284907626, "value2":9200292054736167191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919291, "value1":0.849823159364745, "value2":8783211669019586620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.684825, "value1":0.5612249810486886, "value2":2459193643509253683, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.132931, "value1":0.06165133682856612, "value2":3724912889225979746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939721, "value1":0.7022523416408233, "value2":3787551523413319921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423514, "value1":0.16397574471331047, "value2":2217544936229011402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556341, "value1":0.3363631636072514, "value2":8296724122152500212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.797045, "value1":0.9775102685569385, "value2":8717038963718857217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.037101, "value1":0.5188119627862039, "value2":8253271773680540946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.472369, "value1":0.723269226446299, "value2":1830427992517853225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350765, "value1":0.45427831592086265, "value2":2459543486390463113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.172879, "value1":0.9405309891223543, "value2":2118657389939971315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902535, "value1":0.42952097429562314, "value2":3142662533255542276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863286, "value1":0.06876792355358291, "value2":1443459731022840202, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.137921, "value1":0.6964987593861792, "value2":7256675183245953924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.477178, "value1":0.3581558179516239, "value2":6351662172251250145, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286204, "value1":0.148299090562339, "value2":1443868893374130520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286734, "value1":0.45842257280884463, "value2":1988366284391476937, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787180, "value1":0.5028191040745897, "value2":4148707399092925318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.586935, "value1":0.0518139414168187, "value2":8586911735374624592, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_9":"k","key_0":"g","key_3":"g", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021854, "value1":0.8978228143933898, "value2":1413164249310483673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_6":"b","key_4":"k", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318004, "value1":0.5329901050180244, "value2":700502281930980696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381834, "value1":0.37533803419078043, "value2":1751644850942508358, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781575, "value1":0.7741607009730129, "value2":7455242385880444942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.901091, "value1":0.9760509636456571, "value2":1834882752759920223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805005, "value1":0.18640631542045302, "value2":3539140922926641426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734757, "value1":0.14536274157280238, "value2":6893258320350162281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.340380, "value1":0.2042782161421506, "value2":2006565586318459391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.271132, "value1":0.008492488378786304, "value2":6129670920743570493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947658, "value1":0.35165969319536833, "value2":4951697844001184553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.735049, "value1":0.4770090728759761, "value2":2452955784370225782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337551, "value1":0.7050663230338188, "value2":4249578695875539573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.633134, "value1":0.6874967232601639, "value2":8449525705234658507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412613, "value1":0.8000775398308927, "value2":5399334563984433319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.520666, "value1":0.3288020539774752, "value2":1684836256027900746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063862, "value1":0.4985615503389622, "value2":5250749533704010894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.776154, "value1":0.039036691522280384, "value2":8622652029902283303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.115396, "value1":0.2649092470624877, "value2":678667215803198769, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.728254, "value1":0.3531001293726284, "value2":2004607802946409884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242732, "value1":0.9820165334533849, "value2":7506111952193927451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969092, "value1":0.45874224514029954, "value2":6402112089715656350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.528740, "value1":0.8320354986171156, "value2":8412309334445466976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.907053, "value1":0.18136550029947487, "value2":6212227004856794281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630067, "value1":0.8986694902601496, "value2":9213857752862359137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133736, "value1":0.8699930415468984, "value2":4081508478635244211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.198697, "value1":0.017191657622434766, "value2":1486415534842367537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.690493, "value1":0.3223491896226482, "value2":7372352323440542564, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.404617, "value1":0.6240663182478166, "value2":4691037631120171189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120685, "value1":0.9919062682339983, "value2":8460618157376864427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.688005, "value1":0.3953177302348419, "value2":5961626802588462897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093767, "value1":0.23061987147968935, "value2":2892986034831352810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.543984, "value1":0.16501271900248737, "value2":317259514400114949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842969, "value1":0.25446808569640833, "value2":6002379582827585880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269131, "value1":0.8842302265963506, "value2":2140205221312415384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972033, "value1":0.9857791825041854, "value2":8003941482296671818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508672, "value1":0.5492264125933453, "value2":4184449934578666153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.812228, "value1":0.3691868332062656, "value2":940690450246364953, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033340, "value1":0.4347207480857335, "value2":7925423134984867831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921934, "value1":0.7008140011057534, "value2":3950204529429726081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_8":"e","key_3":"i", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.159043, "value1":0.4015464920656329, "value2":6141367601228839763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_8":"h","key_4":"e","key_5":"i", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207886, "value1":0.923323917554864, "value2":2538142530573084913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560000, "value1":0.988479108310322, "value2":3590995919192964810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324806, "value1":0.14623600559543842, "value2":7983994507018983485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.278078, "value1":0.47875025775481017, "value2":9135606324353342316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_9":"e","key_5":"i", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978376, "value1":0.646562029258319, "value2":6969654958747462453, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.038811, "value1":0.27278148584909157, "value2":3766949452289754200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174153, "value1":0.456204859878279, "value2":5412969439184865040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152394, "value1":0.6304591834176713, "value2":7508490770671251540, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079217, "value1":0.7093512830025359, "value2":8041107853739821078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736508, "value1":0.9456361684627089, "value2":2695850503301130596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.689579, "value1":0.8717416155878271, "value2":914033909063513507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_7":"d","key_2":"d","key_3":"e", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716602, "value1":0.6181082501584888, "value2":7525886338688800823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485128, "value1":0.805498923411989, "value2":3515190992320449261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_2":"c","key_0":"k","key_1":"a", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761534, "value1":0.3987574394917293, "value2":565378419404884224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234797, "value1":0.32687033897000584, "value2":4678721697918141366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_9":"k","key_1":"g","key_8":"i", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099258, "value1":0.5165730450377122, "value2":7897877163898991964, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.855254, "value1":0.6113796894495002, "value2":5439835292581168290, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098218, "value1":0.6331475060559588, "value2":7978662530169854255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.392907, "value1":0.7806228764363937, "value2":5349308434124377933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174842, "value1":0.7994951238912167, "value2":213268455628772714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.654871, "value1":0.23130216238112586, "value2":6353589500271587044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255121, "value1":0.029192800621726373, "value2":2049991280590108935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.454836, "value1":0.6345686250008671, "value2":8305250329173845884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.196946, "value1":0.9679432223428265, "value2":4005113690213723694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.514102, "value1":0.9460847081604973, "value2":2275533825263312731, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.267365, "value1":0.5628401308203138, "value2":2718632724301463408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.702558, "value1":0.3615743138551236, "value2":5265258619209069561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.157371, "value1":0.9914192491349261, "value2":7107333633697743515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367362, "value1":0.30046228876641146, "value2":8731213027442541680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502940, "value1":0.33029188840622603, "value2":7282340644676767522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773440, "value1":0.7398342089000881, "value2":6064862964524030137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370133, "value1":0.5464083248908671, "value2":8595565634194089280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819624, "value1":0.22563478104006193, "value2":6291072404703346422, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.822444, "value1":0.6431602385348965, "value2":3571751115605471571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154889, "value1":0.661415650101091, "value2":2897716721956096259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.706177, "value1":0.9541112843149568, "value2":4382405639585131280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.892582, "value1":0.273637580698836, "value2":1569537455975017743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912404, "value1":0.02958676932622315, "value2":864163789414463895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164999, "value1":0.8749507184007825, "value2":9170076213645968437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.609248, "value1":0.6230590575325244, "value2":1182671592625390639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402824, "value1":0.8555668782169765, "value2":3430464294959504849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.905017, "value1":0.6152490194289306, "value2":3258848281050130827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_8":"j","key_9":"d","key_5":"i", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.669648, "value1":0.6859907392101928, "value2":2219730699877135652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435602, "value1":0.12441938240097333, "value2":3663118450934184743, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736951, "value1":0.9806346124707411, "value2":6495060662787801507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502755, "value1":0.6962847488848957, "value2":7479724851955507611, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.618421, "value1":0.31789340504585667, "value2":8738918799232514086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827476, "value1":0.7869780281846211, "value2":5474883946012414730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490095, "value1":0.6962736110006601, "value2":6409713241665137545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985050, "value1":0.2615773286934893, "value2":6009116516814474343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502430, "value1":0.19305751383330208, "value2":1654152222563323901, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_3":"e","key_8":"b","key_2":"g", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349955, "value1":0.7772270603571774, "value2":73068209272270733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781804, "value1":0.6130002187106758, "value2":5094562490162409190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.158107, "value1":0.12199479281505482, "value2":887695971080523742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.575029, "value1":0.9090539881797902, "value2":3726306097696429975, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.197382, "value1":0.010351058305108105, "value2":3390177787518625882, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959617, "value1":0.10545857069830916, "value2":8523151888175341811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789446, "value1":0.7606780493785521, "value2":2782849609854347711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384471, "value1":0.3297433712153455, "value2":8732496713018613730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.440642, "value1":0.05991406624275138, "value2":8221843418889420393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485145, "value1":0.4324948321911064, "value2":1223798989569651011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930239, "value1":0.797530519094042, "value2":3933281842105360464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_7":"j","key_0":"h","key_4":"c", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921694, "value1":0.3827550255479697, "value2":1163470615431943983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780691, "value1":0.8316468698353685, "value2":3969005124884722997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563279, "value1":0.5313651121371132, "value2":1275907147042029939, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099445, "value1":0.8066245132140291, "value2":1404833370385176869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.897823, "value1":0.395354453907472, "value2":4125787827950753058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562742, "value1":0.0008610845550682808, "value2":2683131454862577523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981050, "value1":0.6811899501971241, "value2":2963479936998599518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.090076, "value1":0.7872489837661779, "value2":5920042185451264661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904987, "value1":0.582385162743904, "value2":3052394546288639081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.807620, "value1":0.8276864738323398, "value2":1770112128995343558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.313153, "value1":0.7000557306246208, "value2":8804754539464742313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648178, "value1":0.6035738426070452, "value2":274869451073352526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382503, "value1":0.7570596198853545, "value2":8494223821423922555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981825, "value1":0.4954131435773338, "value2":5546139769442373925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.896817, "value1":0.3057668662697838, "value2":2146747890118623175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_5":"k","key_1":"b", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857258, "value1":0.7535991261099572, "value2":7241562330594412223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.954553, "value1":0.6272550393596944, "value2":7059824369010485707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611570, "value1":0.896770550906892, "value2":4157172203390286832, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.200060, "value1":0.6395440920752251, "value2":2313959820318356273, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_9":"k","key_2":"g","key_3":"j", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.305817, "value1":0.7944717065357103, "value2":9079049510197634026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771177, "value1":0.5686457722223018, "value2":5926649416008895009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.194460, "value1":0.2030568091950128, "value2":5211606615226569619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_2":"c","key_6":"b","key_1":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541777, "value1":0.38229879472765593, "value2":8330651996424197311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566620, "value1":0.24307591832233774, "value2":2839998819511281148, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376323, "value1":0.5739727137071835, "value2":5399793491142182010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480111, "value1":0.7895749118902282, "value2":7450043826116159674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453645, "value1":0.4875344798695633, "value2":6926040231429456296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734559, "value1":0.3865838466523982, "value2":9001676782517223243, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.171659, "value1":0.33364493236506704, "value2":5739131266488499620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.966196, "value1":0.1807801172943221, "value2":7703455497589255747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087000, "value1":0.14745151659831435, "value2":3780273793092000978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537557, "value1":0.6679609579098379, "value2":7524235382942600940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624433, "value1":0.43524838478159544, "value2":8707151351953730491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.764878, "value1":0.9857854697655962, "value2":2628899540411422018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087516, "value1":0.9235134986127471, "value2":1244667248757759893, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.092707, "value1":0.8123903328089516, "value2":1510741329638496293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212617, "value1":0.7803051371802152, "value2":1782941121051923216, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110139, "value1":0.9520250869731262, "value2":8515109734542519287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217411, "value1":0.569745410096317, "value2":7158729472403289306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578681, "value1":0.4421036889589163, "value2":5172660091174561754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.917747, "value1":0.9646587109518411, "value2":4932202474624997807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490742, "value1":0.18375289991112742, "value2":5134730937919546380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698691, "value1":0.8462874020691947, "value2":6789099466073417633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987429, "value1":0.47456618667904216, "value2":7333730409059985654, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.552031, "value1":0.8731675557704086, "value2":2383267067188016407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.024510, "value1":0.7398931410894473, "value2":4688375563960023614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.885940, "value1":0.5267822661948509, "value2":3106942003445818164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.128634, "value1":0.17304999217293912, "value2":4202058405826758012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919245, "value1":0.5246293149604686, "value2":7962001396490479168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.784582, "value1":0.9051007718281217, "value2":6345441367106704431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.631365, "value1":0.18200897376922306, "value2":2743187358723124435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632586, "value1":0.49203338164778787, "value2":310459975834593918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.986910, "value1":0.21502125960121787, "value2":9213029499233048926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.903530, "value1":0.20307359325640195, "value2":3425839773997981556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495175, "value1":0.42906594953086846, "value2":1910613263092063181, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560265, "value1":0.27498766656590357, "value2":8893579102721900369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.298348, "value1":0.05982261910728163, "value2":8036644582013512610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624831, "value1":0.8394666399726175, "value2":1345154507982673038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.123368, "value1":0.13834539586421774, "value2":6342191610731262762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.086336, "value1":0.17439021756766837, "value2":6128400320706208802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421248, "value1":0.39843282468447905, "value2":8680280397492385017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139454, "value1":0.456434331043375, "value2":4836743646420773621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351219, "value1":0.777135306661456, "value2":3903934605632263710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.414090, "value1":0.8143501039400677, "value2":4413351639615300728, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_7":"f","key_0":"d","key_5":"h", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231721, "value1":0.3138412091160347, "value2":9168517044165068794, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_1":"h","key_0":"d", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.230702, "value1":0.8804508000643604, "value2":512236854100511456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.708957, "value1":0.47311614120900003, "value2":7136526398161878544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.475928, "value1":0.32108039730398563, "value2":7292530251105851041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.643383, "value1":0.9689860631255371, "value2":1734391035548707204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.792203, "value1":0.8844015947765959, "value2":6318884852252048224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.323993, "value1":0.18316994233066625, "value2":3785806626345479252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.055153, "value1":0.9975090536646855, "value2":9101914663697333394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762055, "value1":0.14538112505308043, "value2":6589674161435979608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457017, "value1":0.9313588309515796, "value2":2386410717081263989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118975, "value1":0.2538090940895061, "value2":2359952847484695171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887012, "value1":0.26127955373376677, "value2":7323797183712585078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465777, "value1":0.39956363244564946, "value2":4290799816610955241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919817, "value1":0.24654070770474018, "value2":6141631506827241212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878721, "value1":0.7298060681766352, "value2":7628091662411871299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308905, "value1":0.8062336262819177, "value2":3004783841504994012, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630086, "value1":0.9740479758667072, "value2":7021198218943014146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661157, "value1":0.0030467411917742306, "value2":8705979164600447258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865574, "value1":0.36919272421680055, "value2":4523649159933689100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.185605, "value1":0.5443213859941823, "value2":3228781454351122262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865038, "value1":0.09385639771871399, "value2":7227272424843868171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798023, "value1":0.8684611657481685, "value2":2233953571760268466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730804, "value1":0.08710486562835452, "value2":6236391362613657084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.202824, "value1":0.4792114852099944, "value2":445497349923776863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173553, "value1":0.039235946095809125, "value2":7869971288440786845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908688, "value1":0.5052600498486294, "value2":4519674082380595002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341267, "value1":0.007052586435709472, "value2":3150628489852334883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881706, "value1":0.2885539517449333, "value2":1797762712648874995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216914, "value1":0.4200170713130975, "value2":8711469515389655710, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.219622, "value1":0.8598302806140433, "value2":3863311400727592540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.656021, "value1":0.6463537908056233, "value2":1050074674903654190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641122, "value1":0.40560841010225995, "value2":742946648471490108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_9":"d","key_6":"j","key_8":"c", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755478, "value1":0.42687719134929036, "value2":8589479839598188558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_8":"d","key_0":"a","key_1":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.848432, "value1":0.9962729980353733, "value2":780000159187900550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.195275, "value1":0.6773988604333073, "value2":5522000829662160867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787345, "value1":0.6486468864516003, "value2":4138825123686123358, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.553619, "value1":0.7566207604613129, "value2":5341523136140526945, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982202, "value1":0.8174248836774913, "value2":4875324336295746206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317829, "value1":0.25574317763537074, "value2":3850513628767917465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366612, "value1":0.634692324496315, "value2":8508176811169559100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.931082, "value1":0.9070818575294899, "value2":2941126198839449331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419466, "value1":0.6035953996525765, "value2":5226492465996637298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207446, "value1":0.7148273386681443, "value2":6830161919863899962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339115, "value1":0.9680080839353958, "value2":3346579129519946511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329518, "value1":0.5052033589082856, "value2":8326166807060477796, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_2":"c","key_0":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314104, "value1":0.23744990369573346, "value2":6679180757317175227, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177954, "value1":0.6916907655144761, "value2":6598624424043999988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220478, "value1":0.9419151786449748, "value2":6868454728302182507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.962386, "value1":0.29200153388880246, "value2":3531740125944089227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.522912, "value1":0.4384617611060238, "value2":7012028395173793023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.701530, "value1":0.008494038169472137, "value2":4096870035163582532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019574, "value1":0.540551382306373, "value2":7575443997806932703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558742, "value1":0.30242558718747614, "value2":1312291174476606343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.597198, "value1":0.9990604033963557, "value2":5377437181323103863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321750, "value1":0.8571357709325697, "value2":3223326577646383979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976585, "value1":0.794736424631542, "value2":7297990135456560387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053373, "value1":0.9010563829072253, "value2":5702408670366972568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.500145, "value1":0.16016689363963474, "value2":7655127244946192653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.347755, "value1":0.19254449629837572, "value2":1438411533080776332, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.047080, "value1":0.9596252631833848, "value2":4943940353426671568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.737278, "value1":0.6304724450380977, "value2":5075506433971212474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842472, "value1":0.9682019821931608, "value2":7537248780629599850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210066, "value1":0.016073835132125416, "value2":1148941512571603438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019003, "value1":0.15376503679496498, "value2":6744619422647456093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972351, "value1":0.6522824510722263, "value2":3635978187235320184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464404, "value1":0.1987173098042302, "value2":2325620189736714816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505822, "value1":0.8969967220271704, "value2":2772428523245878847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.858017, "value1":0.42532104055952885, "value2":7057664582771584406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827767, "value1":0.11843519151662998, "value2":4642257980991996610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457807, "value1":0.6680540521232995, "value2":1413093760337990232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307237, "value1":0.2483558233491924, "value2":1657535478313961463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187495, "value1":0.8787274497214357, "value2":1012440122187350073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_5":"h","key_6":"k","key_2":"b", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.935106, "value1":0.6622345557285527, "value2":873018521033719372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212483, "value1":0.03664769373990424, "value2":8662944034974337497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.279577, "value1":0.06940570148672788, "value2":3397490112016095138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.679928, "value1":0.94467398885254, "value2":8140288354763339321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348611, "value1":0.09177297933527545, "value2":2243053697955925595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033795, "value1":0.004266699548950342, "value2":8869596053230724066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716957, "value1":0.952311087977132, "value2":3488323066281405120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275244, "value1":0.9153132837467353, "value2":6017024334903831134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523199, "value1":0.8894820330529598, "value2":2495751532034294665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.548536, "value1":0.36452752239967207, "value2":4087473006792877554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_9":"g","key_2":"b", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452175, "value1":0.03794808741352215, "value2":8378297363369384496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156999, "value1":0.3422567916396813, "value2":3950482866588863620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508543, "value1":0.5441836551476523, "value2":5324891351187729379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397072, "value1":0.8991563074648984, "value2":484650303993845367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074791, "value1":0.08635154390096551, "value2":7636185572736407362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214033, "value1":0.529318647161814, "value2":3428935700531030330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023178, "value1":0.9863231971076605, "value2":952801547411635885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.655562, "value1":0.1342499141610202, "value2":6341506872735720793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834066, "value1":0.5801423696428913, "value2":2989261336292506730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.107225, "value1":0.08595942695174727, "value2":7162816956772874074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.303930, "value1":0.5732092807009482, "value2":6606617148160680700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.915970, "value1":0.10263255284529503, "value2":8480457956761770794, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356355, "value1":0.6141228451134201, "value2":7156945200312274543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645757, "value1":0.7597072830851564, "value2":2605675246955821391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_9":"e","key_1":"f","key_4":"b", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760044, "value1":0.048059905194606296, "value2":3102725962018871003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949450, "value1":0.803383211503761, "value2":5157179277981661443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124187, "value1":0.16836795463840862, "value2":5627024937490827799, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621202, "value1":0.11873453496853606, "value2":971627245321019634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762026, "value1":0.5957950719970895, "value2":1706809321238862597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391493, "value1":0.7101377073031195, "value2":6966030075201405586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.168903, "value1":0.22863352792292077, "value2":1427332489767053365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698944, "value1":0.5031422927319381, "value2":2536670790706398066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985917, "value1":0.7010791398723792, "value2":2424373004486759234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611987, "value1":0.023052053167488012, "value2":1572028785346213886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.084732, "value1":0.8225285464369793, "value2":7247308476502074358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476584, "value1":0.6320619568566006, "value2":3013845267574452926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.358865, "value1":0.9597368730404013, "value2":1895755548879639735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559280, "value1":0.7939551495955407, "value2":9185811549617457738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338153, "value1":0.6433609372521104, "value2":5456961662990895069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120489, "value1":0.02598857402797232, "value2":2690067127374441413, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.026856, "value1":0.3509430162667547, "value2":5912250260618117609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.911457, "value1":0.5658091707632997, "value2":3970272443516058585, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563432, "value1":0.24659033075994388, "value2":728046743750740729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.596212, "value1":0.5615056517168183, "value2":7033619460114819908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051460, "value1":0.5771358250664421, "value2":8203484790496543671, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.824484, "value1":0.05697931696566816, "value2":6635808805223685263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627182, "value1":0.9130566878289693, "value2":8110244852760276756, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_3":"e","key_9":"e","key_2":"d", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207192, "value1":0.7274854390257628, "value2":5772335372245409001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799146, "value1":0.5204858317687999, "value2":128757778781441708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014826, "value1":0.7097519460607616, "value2":9006806179184131719, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648719, "value1":0.01042589454449313, "value2":1630317652722526040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_6":"j","key_9":"k","key_3":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.665998, "value1":0.6184281613297867, "value2":3187261205020385384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.501321, "value1":0.23807711629991066, "value2":1285207877035639838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184165, "value1":0.7331865941823509, "value2":8439731025981805138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_9":"f","key_0":"a","key_5":"j", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801569, "value1":0.9860558447882487, "value2":6774600722253198808, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384223, "value1":0.22369926816026842, "value2":5826656674460169842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.980495, "value1":0.6229185444454209, "value2":4872634527752777210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.380826, "value1":0.7806130098804899, "value2":384791734560367497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163193, "value1":0.5398716860823138, "value2":4387950486971806786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.823934, "value1":0.5810656874809516, "value2":450490831163611562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.608002, "value1":0.7930931659235, "value2":7050949938710758790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363629, "value1":0.5705690869328316, "value2":4003836970877260330, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_6":"e","key_7":"a","key_5":"i", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.997480, "value1":0.2946419071937674, "value2":5707538095602523121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.835470, "value1":0.023853067484955352, "value2":5038338303194879356, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.720798, "value1":0.48474572095528634, "value2":682322783035637435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.573983, "value1":0.3514813766672496, "value2":2903766540993385647, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730487, "value1":0.8217554814463921, "value2":2948047805979067123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016747, "value1":0.12213673109770266, "value2":3888549493731213502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.081414, "value1":0.18665321280658956, "value2":6792269259144278363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300754, "value1":0.9731729557371976, "value2":7090692680090167231, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337262, "value1":0.4023504772804693, "value2":4873228147801762854, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990413, "value1":0.6497908067795414, "value2":3504328384430870621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.276664, "value1":0.785362017819063, "value2":1481672280882279109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.357896, "value1":0.03050056454630596, "value2":6649840382760115378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.095552, "value1":0.31901448906359003, "value2":7956338378631849376, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.518227, "value1":0.922032137621878, "value2":785067472984268171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094243, "value1":0.6170957228718421, "value2":6792110304537802959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782822, "value1":0.20221744914264872, "value2":3178187263071942184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.539464, "value1":0.9703158465709303, "value2":1060145839109092280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773211, "value1":0.9517836987483416, "value2":7463838442968425219, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.695794, "value1":0.34514076166527163, "value2":1111746250317888633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074230, "value1":0.19136080585905593, "value2":4573587278465589025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_6":"c","key_7":"b","key_0":"j", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982043, "value1":0.2232136951539441, "value2":7688263110504645334, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939724, "value1":0.3301646825680086, "value2":5317348954333303134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275048, "value1":0.1567976387714283, "value2":8628385701502049400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.251237, "value1":0.601867452525591, "value2":4258975777323734306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051509, "value1":0.87447596502846, "value2":390569829027012116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453014, "value1":0.7517900109611292, "value2":5319112069730300764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.859148, "value1":0.03968223769223721, "value2":6092044766196787249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564895, "value1":0.8282334480284984, "value2":8177460795118738605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.213064, "value1":0.37245338326241034, "value2":5313011500830153125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234759, "value1":0.6391009807318464, "value2":3598159575817066989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922540, "value1":0.8881740732484977, "value2":4573970560902118758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247842, "value1":0.6867454879676517, "value2":2359279288069431907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350225, "value1":0.6867879732162172, "value2":37306184729818872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881388, "value1":0.4406197726220826, "value2":724124821973247241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.950564, "value1":0.1322036312488787, "value2":4198819767892784951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.666395, "value1":0.2251980228042216, "value2":606006971675717615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.344497, "value1":0.029652052432054064, "value2":5846707393638380751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.703773, "value1":0.6376539079560296, "value2":6681127841717987695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295458, "value1":0.5526352586149329, "value2":7043209016069060231, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435502, "value1":0.09713433968048614, "value2":1019638678991819627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598132, "value1":0.963189125504533, "value2":6458016500274290636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627640, "value1":0.4641061727729553, "value2":8689246494106761338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254760, "value1":0.8326486356175677, "value2":4407797931839765907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805732, "value1":0.6423132619728287, "value2":6833967417402080897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.326823, "value1":0.9067956612338547, "value2":2841002873709302821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.088104, "value1":0.06965395874499884, "value2":1789284717431249823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.948712, "value1":0.9884448336440479, "value2":1290858986155893057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511709, "value1":0.8105556502461225, "value2":3470144572708895882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163730, "value1":0.8658645130208062, "value2":6466367002189635673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260153, "value1":0.5010858345134921, "value2":555883752624156654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.274796, "value1":0.8267418905611275, "value2":7590347778036203754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338150, "value1":0.7906313079639156, "value2":7837780129396014797, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938845, "value1":0.03371523590847688, "value2":2642591228031199330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254822, "value1":0.42059549793106427, "value2":4614016824012787995, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003989, "value1":0.05975709544316542, "value2":1382695454770978636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.577151, "value1":0.8937534168973357, "value2":6521815205949311706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.253675, "value1":0.40184282199800475, "value2":8286880530272716169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205638, "value1":0.7633247932732301, "value2":6913337973948968095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939093, "value1":0.03322965819693288, "value2":2797917596305577577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248192, "value1":0.5307465178536894, "value2":6134584955455781673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_9":"j","key_1":"j","key_4":"c", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.593487, "value1":0.05419262497736568, "value2":9190929989995856492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.182515, "value1":0.035610068777040586, "value2":6469776877211677470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160124, "value1":0.662164487805106, "value2":8768604308735587138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317998, "value1":0.9757873511779303, "value2":1199801679672428467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922339, "value1":0.9554985330074205, "value2":8286943158249961765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023973, "value1":0.01669695080382125, "value2":4408497776354893523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.399088, "value1":0.37811424022789003, "value2":499643435878747492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.974576, "value1":0.6392617923906977, "value2":1832499551397243524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423426, "value1":0.1431493881970496, "value2":4016805275891833987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220477, "value1":0.006541995474356055, "value2":5608461794122610089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014455, "value1":0.20051860544075079, "value2":4298831902740539404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771702, "value1":0.9711913949733121, "value2":4548136488165596312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837484, "value1":0.3199900890395432, "value2":7619719494433917806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632963, "value1":0.8179930116909135, "value2":42611344634328983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739321, "value1":0.8571068088303897, "value2":1929086780074260377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169110, "value1":0.2241172770388991, "value2":3015343857994873164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558214, "value1":0.5245963976733786, "value2":4636974583954671385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551826, "value1":0.8529460688457188, "value2":7676686801578586657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317389, "value1":0.34597315939279244, "value2":8526544610592186971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131556, "value1":0.9023875402465826, "value2":3147425851236763117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330907, "value1":0.7187683220932978, "value2":6304935909987923644, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419527, "value1":0.3464275048663091, "value2":800017209533647436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325070, "value1":0.32409707654758807, "value2":3093719017131899877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094166, "value1":0.6576755239099957, "value2":4375841020494708801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184361, "value1":0.454784333846635, "value2":8511384704032602153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424067, "value1":0.7746556148202395, "value2":8409657959538616305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.282726, "value1":0.19274194173178774, "value2":4484284290811930974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.044679, "value1":0.17881938606509734, "value2":2991918820431220404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622917, "value1":0.5058188298556251, "value2":2680090310035550156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.686600, "value1":0.019991692028205214, "value2":8990563930001611676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.977493, "value1":0.8045660581528546, "value2":3965120458727174879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.910280, "value1":0.15532301386230277, "value2":2751777764504449883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981678, "value1":0.6634934084628059, "value2":3338856320416894701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_5":"a","key_0":"g","key_3":"e", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099077, "value1":0.4991967563162609, "value2":7696501412266603819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_3":"c","key_0":"a","key_1":"j", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169242, "value1":0.7573800949559079, "value2":4682651688122452104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.854077, "value1":0.38315712266426233, "value2":7277684317066282125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017721, "value1":0.43030519598600153, "value2":8176728734487897933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449823, "value1":0.0905672633924478, "value2":8038615141368276704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505199, "value1":0.29347749204963386, "value2":5939807103722962422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.429412, "value1":0.3821527042310373, "value2":7616377560194630586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645364, "value1":0.31800894756504583, "value2":5324276910842211792, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789951, "value1":0.12346800778619027, "value2":7282935021388766519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.005934, "value1":0.4652775738745803, "value2":5150985627139247681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770208, "value1":0.8318069314007921, "value2":8693296121103057557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.036795, "value1":0.31504722243353955, "value2":5332439897432100886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.557676, "value1":0.04547498827953966, "value2":1747115516814035342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295720, "value1":0.25765308010583876, "value2":7172964053161542723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.862280, "value1":0.6690305275979546, "value2":8530225967231964009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464755, "value1":0.015096554191777058, "value2":2158727717387461667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902724, "value1":0.3729818170388733, "value2":2937474665692773587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.150211, "value1":0.9010548939518067, "value2":8689173712211659171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502794, "value1":0.637718327451074, "value2":1058416603563820737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756334, "value1":0.9093486580620052, "value2":3914139843013624351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109763, "value1":0.3482544181233465, "value2":3984218999735374107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_7":"g","key_2":"f","key_3":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.007357, "value1":0.530193826214922, "value2":7784257234281286192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133042, "value1":0.14994561696634548, "value2":6913972686763727632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.499117, "value1":0.9693770819840257, "value2":292923979595115495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.256986, "value1":0.8560085418753359, "value2":1633179891830212156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348017, "value1":0.4525195870224779, "value2":1498113059165586202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.389164, "value1":0.4747861877903318, "value2":5663253941505962378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.712219, "value1":0.23770447169372633, "value2":7127281555547067699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799547, "value1":0.4771589732318728, "value2":2601851094636806560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761432, "value1":0.9902875537117704, "value2":2441735233721216793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.594915, "value1":0.9327820854039164, "value2":7435962671657247487, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.018250, "value1":0.35410376862709275, "value2":6088069648662464618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.640810, "value1":0.6282310495408681, "value2":4206520774588356560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929377, "value1":0.30194565379846455, "value2":371174649294622523, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.817275, "value1":0.4745483085572834, "value2":2810603774900787473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.046061, "value1":0.9540450542751275, "value2":6567731530813672613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976385, "value1":0.2747180574854192, "value2":7052984344810070260, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833030, "value1":0.8111270330849011, "value2":2406611793520084033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146154, "value1":0.019440584474240373, "value2":1431804086688415352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541319, "value1":0.42910271299796143, "value2":5008131544913964869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.478974, "value1":0.3992297014608565, "value2":6638251650936294767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.591553, "value1":0.9631516171967569, "value2":6328755963396316225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.836675, "value1":0.8925642028256048, "value2":2169296529295363291, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_6":"g","key_3":"e","key_5":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240233, "value1":0.5601423681829663, "value2":2372802153601657174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926558, "value1":0.8669054217402387, "value2":8750941328640682000, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.785340, "value1":0.9294643708923529, "value2":8967392703697598518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321372, "value1":0.6020812110006013, "value2":1942102877471692500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578919, "value1":0.17388396608421544, "value2":5054354296681821565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830121, "value1":0.06504953435269434, "value2":7812868363877570134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110821, "value1":0.12907416408116543, "value2":1398896999021006979, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318471, "value1":0.6665917244504806, "value2":5816507337427212221, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881196, "value1":0.35509697330525436, "value2":3799008703902351704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503152, "value1":0.686811705668727, "value2":1281918055642507847, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.393488, "value1":0.8980510009425112, "value2":3157487117727981952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993113, "value1":0.8339917765115039, "value2":1528182230124691520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441452, "value1":0.5723022066521427, "value2":8611586563060872543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902474, "value1":0.21553093452664723, "value2":1591141288904852990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.937761, "value1":0.5975557556565823, "value2":1778079885483684894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.509879, "value1":0.8107703383515009, "value2":7338854303608988296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321562, "value1":0.7839743525651934, "value2":407341572079351661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_5":"k","key_0":"g","key_3":"g", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.083126, "value1":0.8340311472033722, "value2":414058498508156341, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.266082, "value1":0.7848792912531005, "value2":1487975780946804498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.001354, "value1":0.042658133657431034, "value2":2686477010080712855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984348, "value1":0.12772415816911367, "value2":1744454398245616628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.545130, "value1":0.012749865379431171, "value2":3614758343709591135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.211075, "value1":0.26931470974880833, "value2":5335597373851226907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517993, "value1":0.8277823010534222, "value2":441076351209513172, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.783905, "value1":0.6446774753163863, "value2":2444284131966329588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140755, "value1":0.7307559658816861, "value2":1750453360454837174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.243464, "value1":0.5611645445748624, "value2":5031122288522829688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247597, "value1":0.26059857605419207, "value2":6822156384989813472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053905, "value1":0.6298269719196294, "value2":6661136217666523640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.058995, "value1":0.7306532383370615, "value2":5311315770055711299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030527, "value1":0.08351380687475202, "value2":7155821886727737087, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.498371, "value1":0.2731396400770269, "value2":89197923125259866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397662, "value1":0.0870595280214176, "value2":4020978501682465140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173834, "value1":0.7945264755888277, "value2":3882559205765407193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497055, "value1":0.09364593584875654, "value2":891682366412177732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065765, "value1":0.835147994300049, "value2":2505979555724543511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.765901, "value1":0.992206355507561, "value2":5788507149763623428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_6":"a","key_2":"k","key_3":"h", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622912, "value1":0.44608293916585673, "value2":946502417712586156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366924, "value1":0.9938167708003688, "value2":7200121887993962206, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.072114, "value1":0.001073896039030629, "value2":8685581835197772410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265774, "value1":0.30130760063683054, "value2":8411046196525683207, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.749589, "value1":0.9239863104020335, "value2":4219975890726145199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190061, "value1":0.9734172533312344, "value2":8137079207203857103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.519945, "value1":0.07187194342466732, "value2":7825744617647803155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.649495, "value1":0.6408236940193898, "value2":3048798030528052662, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320456, "value1":0.5081060875649162, "value2":719352320345764599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476698, "value1":0.052307893890373255, "value2":2742427547083463898, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_8":"d","key_2":"i", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571531, "value1":0.4132003718642955, "value2":6484766831533191693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682702, "value1":0.44751287991255284, "value2":4055042383084807325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_4":"e","key_6":"b","key_3":"i", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.890718, "value1":0.33741266801359987, "value2":6934783766842267527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452747, "value1":0.3663537726308516, "value2":3312655545498739922, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199996, "value1":0.15068464159094366, "value2":5665646131255744951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.983795, "value1":0.8775221552243728, "value2":9069927522091992178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.583294, "value1":0.838152847228304, "value2":1035319401212201491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212587, "value1":0.8012210929096193, "value2":5585326063344558523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641779, "value1":0.9247211843697717, "value2":2117637859665911236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.406293, "value1":0.6198872109290668, "value2":5911640262763562610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.277897, "value1":0.027423430536495416, "value2":7628618445567261117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.832579, "value1":0.5331161695907243, "value2":8305462638082561375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205268, "value1":0.018199222166683245, "value2":475925606235710366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.615075, "value1":0.9915416739226851, "value2":2204646069144435635, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_6":"d","key_7":"e","key_1":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424612, "value1":0.9757938651409276, "value2":6266511548869428739, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286078, "value1":0.0846398426026624, "value2":3081995387268873440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_9":"g","key_4":"f", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787171, "value1":0.8365446036859473, "value2":5979460348923075463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381115, "value1":0.9123835513620147, "value2":3398482109219075054, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.114157, "value1":0.804291931301114, "value2":6012424958858906101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134911, "value1":0.00736667824808292, "value2":2400000632738793301, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164780, "value1":0.21287762980212552, "value2":8645521771535465924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710538, "value1":0.7411138160092244, "value2":8685307998868002992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585644, "value1":0.08786909591629084, "value2":7343395510792995060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125543, "value1":0.7332366768127239, "value2":8370868046930361421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.932834, "value1":0.9015644945676353, "value2":5236314268804910193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093732, "value1":0.497896904599941, "value2":6397325408101186539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486123, "value1":0.5097658314843312, "value2":1337466807316653828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904889, "value1":0.6556792189077274, "value2":4516335409033295305, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.387498, "value1":0.9910872699779568, "value2":1557997601588004148, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710931, "value1":0.5132845499506883, "value2":3319795795293053140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.515215, "value1":0.049432502275940204, "value2":2300709145422274852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821813, "value1":0.6825139883510115, "value2":8194491426131939536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865373, "value1":0.7044541486613806, "value2":3703555177588303304, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837580, "value1":0.8572176121238669, "value2":2344100903459752932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314910, "value1":0.4347186053772807, "value2":7676829676819086499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.946948, "value1":0.7950628854157453, "value2":2267010392465955325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015107, "value1":0.8624602552899475, "value2":9148330689311945183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488041, "value1":0.4618418780870866, "value2":8123711349904092581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710726, "value1":0.4226915062812881, "value2":5491521860827839646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841455, "value1":0.9825903973183027, "value2":3077499782633025057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.592689, "value1":0.9193972918242918, "value2":929238328644383650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993922, "value1":0.14695440679147787, "value2":8882303717901413025, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_4":"b","key_8":"b","key_0":"a", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731320, "value1":0.9934902241842611, "value2":1321453009633438522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604377, "value1":0.24174531205655925, "value2":5870091064000157680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628164, "value1":0.8064136026821553, "value2":8786888366338651141, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.484350, "value1":0.5079394885547678, "value2":6920245211342331930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978330, "value1":0.9354707910737625, "value2":7581764658078201917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632366, "value1":0.23595321295755245, "value2":1511878176205605753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142194, "value1":0.6306719216610874, "value2":2431345756658073693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397195, "value1":0.7323027634538346, "value2":8521159495965572956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.996298, "value1":0.8484427529278864, "value2":7246857551445151219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.328211, "value1":0.8531255802404514, "value2":4506921974707672140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308884, "value1":0.5324739320542478, "value2":6505796324920546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_4":"g","key_9":"i","key_1":"e", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.721813, "value1":0.2030510736245931, "value2":493933611483820748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325138, "value1":0.6318807488600625, "value2":3427677815511198978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.717950, "value1":0.13850468444662895, "value2":3714125763364851426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984959, "value1":0.2313158857459622, "value2":6179178857252455230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767352, "value1":0.04737365094297706, "value2":7312736110130720225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517326, "value1":0.38308887661557994, "value2":4397262624851606256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537458, "value1":0.8630190277228629, "value2":1405251953788249605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.589024, "value1":0.6853812314292282, "value2":6568591835038822877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013121, "value1":0.9022971605517441, "value2":5391410131187089227, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585652, "value1":0.027029007730938633, "value2":6270487841856618503, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.069148, "value1":0.5452016240009335, "value2":7230673973301658030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870318, "value1":0.8893099847439015, "value2":2072311642337421510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.525589, "value1":0.9489296395713415, "value2":1934395693866121179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.411127, "value1":0.5543622037481519, "value2":869973100557162310, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870905, "value1":0.1429227422154167, "value2":1580405259981052384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214057, "value1":0.007127408341584659, "value2":5021161294907689526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324636, "value1":0.2886203159980936, "value2":3536603792281429135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566190, "value1":0.2435195285039059, "value2":7314231811000308948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.302446, "value1":0.3969373764014023, "value2":861549335043188440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265270, "value1":0.8392596283575425, "value2":229012885919848214, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306041, "value1":0.6755326135952276, "value2":687305007623331029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.920842, "value1":0.14122582181388352, "value2":8912506495776009399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970414, "value1":0.32375225588910955, "value2":3874667148173594242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098047, "value1":0.5080753238940934, "value2":259459341225408666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814708, "value1":0.5851579994151463, "value2":4279032101744191170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857725, "value1":0.8750883468056723, "value2":8160658677842295703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288649, "value1":0.3318827172603761, "value2":8934632128032959148, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.651119, "value1":0.31525481325301247, "value2":8244510685212193822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065672, "value1":0.08637516713839134, "value2":707728667641140855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620357, "value1":0.8983386747310453, "value2":3756826533006983142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707292, "value1":0.4495030238963459, "value2":5996025393915497140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.443961, "value1":0.930457120039426, "value2":6519425791992110608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109167, "value1":0.09168467762421384, "value2":2057706949013098262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.481603, "value1":0.3529978146205129, "value2":6544227882618708766, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770746, "value1":0.6378677748113686, "value2":1182963631296444077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967875, "value1":0.8756904550723122, "value2":516292034493240162, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324624, "value1":0.03689035176673176, "value2":6723940798795470811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556389, "value1":0.7611742354824225, "value2":6343475929372871725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538576, "value1":0.1188977043392838, "value2":1679649620772949796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_6":"k","key_7":"a","key_1":"j", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.531730, "value1":0.2793539457328025, "value2":8309265375086957584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488778, "value1":0.8806255322454197, "value2":2637346612964090286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174493, "value1":0.4058829790367517, "value2":4908420813809791761, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.875676, "value1":0.8084081237917962, "value2":1370185119560523418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214353, "value1":0.22148397653369978, "value2":3829075843579764833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402832, "value1":0.9983967808941802, "value2":373850521427271723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.574399, "value1":0.1403913922108672, "value2":1025325851242540472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_4":"j","key_2":"i", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538957, "value1":0.5637672549308663, "value2":180146514999582015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043198, "value1":0.6621042931314224, "value2":4672271958823003044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.455391, "value1":0.37463255563131875, "value2":7050585404363795282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118353, "value1":0.7019235890065948, "value2":6561897677595087821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562842, "value1":0.4216296582302376, "value2":4671864964861835857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.613979, "value1":0.9101669225413574, "value2":5476069179621211969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079079, "value1":0.6285830126630634, "value2":1800090753316916399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990168, "value1":0.4228502254109105, "value2":4769618752656230984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630399, "value1":0.0593879368852638, "value2":8425926359679957872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381514, "value1":0.3170447182462336, "value2":134775839032388004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306533, "value1":0.09290124329204048, "value2":3878589204829724343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363988, "value1":0.7494592976903763, "value2":588119535129085693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.726401, "value1":0.2843848594033847, "value2":797824336547840313, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.425192, "value1":0.6747059449207641, "value2":6618945540166900762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486703, "value1":0.9234732601693532, "value2":7284867992059891857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.162651, "value1":0.4998767239554866, "value2":3758664262196916666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638502, "value1":0.14222618431131662, "value2":7183266992692325102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814797, "value1":0.04744514500578937, "value2":8807075724873574935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617563, "value1":0.7113712918981732, "value2":7369990382580999103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391047, "value1":0.4667345592294561, "value2":2593217151341486604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423210, "value1":0.2993388901152061, "value2":7736063295280636221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.233156, "value1":0.2669621811468438, "value2":9163725732269873210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780343, "value1":0.4836321272043857, "value2":6239122865745239252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270794, "value1":0.8124904828437979, "value2":5664361933553425625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_2":"k","key_5":"k","key_0":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.567390, "value1":0.1509148834766825, "value2":769811157679024909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502511, "value1":0.3496329905819851, "value2":8064435486484275503, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.995318, "value1":0.07944294849213016, "value2":7290799161307742730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160027, "value1":0.9323372972664894, "value2":8694402707307208350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265068, "value1":0.7804375599856387, "value2":2823069570137165166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852790, "value1":0.04269382129150226, "value2":4247785862022220763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.813337, "value1":0.3041270110008107, "value2":1162341534698170174, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_6":"a","key_9":"c","key_1":"k", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821104, "value1":0.05863251524747595, "value2":5517633377182805397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801029, "value1":0.7039209651517294, "value2":5777192245942090986, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.292328, "value1":0.0789099713548181, "value2":4341141001609491414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.650140, "value1":0.6547556555322551, "value2":488100188330068974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881952, "value1":0.6384170556890851, "value2":7393134503131526080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019410, "value1":0.4521619097561246, "value2":2730473646049580719, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_7":"b","key_8":"c","key_2":"b", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413658, "value1":0.35836104075629194, "value2":5913006016396086189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.388542, "value1":0.347328329631466, "value2":3415671885617145536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908514, "value1":0.682932258250745, "value2":1535326911373174401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453424, "value1":0.011430118976046925, "value2":6615057602694952188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661892, "value1":0.04490375141966574, "value2":3586691570128338490, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598897, "value1":0.9840071764206562, "value2":6756637664735447989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.614741, "value1":0.45336506256132186, "value2":7160462685010833845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959437, "value1":0.33491802794663117, "value2":4263644093884006866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394710, "value1":0.09508520954191016, "value2":8086669194950309707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.400389, "value1":0.748914928293926, "value2":7561318401530983897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_2":"a","key_7":"c","key_0":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551520, "value1":0.29033239272123923, "value2":219973352444233301, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.433012, "value1":0.8026759332563937, "value2":1035833541375117050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.097876, "value1":0.3123782080487283, "value2":97482352559342166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971923, "value1":0.09350074090215915, "value2":6603209689656348295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385293, "value1":0.025071498526727982, "value2":7821611042801827653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.582177, "value1":0.17895122462677535, "value2":4851272893829158392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715577, "value1":0.12145793831580125, "value2":6773868244129033160, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_9":"d","key_5":"e","key_6":"h", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.245667, "value1":0.5083106874300092, "value2":5342790051223313569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.191947, "value1":0.7213188287503117, "value2":8184500849109232134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.527410, "value1":0.2848785581313827, "value2":8059297953437098619, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023841, "value1":0.08963330011439892, "value2":8744048608508050406, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.861975, "value1":0.9569428255590289, "value2":5863440324678791040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_4":"h","key_6":"i","key_0":"j", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.020139, "value1":0.5328792585389814, "value2":6851298079925452135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383316, "value1":0.6736870667139099, "value2":1034982274545531879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490723, "value1":0.7069141316227696, "value2":1238860616186948989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774619, "value1":0.15017350186839917, "value2":4145739713631541991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.724625, "value1":0.41136382513830944, "value2":2470375246542099625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014004, "value1":0.45322459474776666, "value2":3424033124770728498, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370213, "value1":0.7936480778398273, "value2":7651653355869440262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930663, "value1":0.28375382304907193, "value2":4573094024293120951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.108477, "value1":0.2514498975622512, "value2":6565807048098733104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318641, "value1":0.09067602148053568, "value2":2118726105929309608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341849, "value1":0.5146408700742566, "value2":5544935195040202056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016279, "value1":0.2887329883061575, "value2":4016937139552222061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696532, "value1":0.35345697419681527, "value2":7198819764217760607, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356409, "value1":0.6101596793669094, "value2":2338966817998610022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882128, "value1":0.8757983585066079, "value2":1264113150578508254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214730, "value1":0.04033669277556926, "value2":2311999714610864651, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125040, "value1":0.8510776359004306, "value2":9203972476999185975, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617466, "value1":0.4880211576700948, "value2":7876488006268555933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.580824, "value1":0.738663404243217, "value2":6159559973328325759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.894051, "value1":0.0394639970488642, "value2":477392450960926806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112539, "value1":0.11952574252776728, "value2":2511563277039044978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.918404, "value1":0.30833608301592924, "value2":8563798643952041377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385588, "value1":0.17478857683593713, "value2":6552271330519870932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139243, "value1":0.42492669927426513, "value2":7691897138631144930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715207, "value1":0.8647651137464607, "value2":8497862595581854195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760625, "value1":0.24627316716260067, "value2":1409311771773591601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.880305, "value1":0.7511958117841795, "value2":8663780143091433962, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.121986, "value1":0.503001087851081, "value2":86404765376501910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.378220, "value1":0.7154879418930685, "value2":4699613906263091859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367776, "value1":0.3357733171654874, "value2":2109430818036060968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051853, "value1":0.46132527877007656, "value2":1864815246368296251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503984, "value1":0.5450705451757059, "value2":2199582755716092199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270031, "value1":0.15857947720543178, "value2":5111440735129620852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376563, "value1":0.08077432861716789, "value2":7964637453463198864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017382, "value1":0.4697079883099492, "value2":6862028548685305626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938781, "value1":0.3698227408392578, "value2":1642534448554432249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.869838, "value1":0.4366942978360045, "value2":4007380928114996020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.309900, "value1":0.05509289000926218, "value2":8881026026944444737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321699, "value1":0.5508642725465765, "value2":1065357963175430930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242134, "value1":0.05194281440569991, "value2":2955861771024227742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394641, "value1":0.6802778418350491, "value2":7829903676952590571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.011752, "value1":0.2679251298930048, "value2":46390372055260426, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156821, "value1":0.8759623901682069, "value2":6663991521677358276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621515, "value1":0.8461818248462066, "value2":4793482631115954367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.906690, "value1":0.4454231810933155, "value2":1968123317376605778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833508, "value1":0.5580322939662696, "value2":542437654648170086, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.864441, "value1":0.13709821675852493, "value2":3775094265545831122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349796, "value1":0.5463888647509524, "value2":3843833923457334958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210241, "value1":0.13053751777334896, "value2":2547919115148938351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.534237, "value1":0.13104275846768723, "value2":2757438080188257397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.479346, "value1":0.6936882347927468, "value2":3593937018182399578, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013374, "value1":0.647079383488511, "value2":6027310083597491507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353768, "value1":0.5713686637635252, "value2":3396180021694291926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704388, "value1":0.028813882790100154, "value2":6631675215659864441, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350375, "value1":0.09828506398906811, "value2":262432636815220332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130984, "value1":0.05874581283090899, "value2":5455068607213223389, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269711, "value1":0.16897643510231944, "value2":2188619802248362391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798909, "value1":0.49628972013165096, "value2":8871333460189078193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314472, "value1":0.003776179814725882, "value2":5465069878443166467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841977, "value1":0.016019153138836187, "value2":5090055543864076033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.431741, "value1":0.4503254906693805, "value2":3437452976536184232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021144, "value1":0.7157770425720729, "value2":1627465028436054242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.492677, "value1":0.170722769795088, "value2":5333195898301269961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.572483, "value1":0.07518223050679318, "value2":749837753058622914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.070547, "value1":0.7819934575507779, "value2":1257841836293236196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834097, "value1":0.7836504674507244, "value2":7822820024554629611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.725415, "value1":0.6205675290840149, "value2":7546425291277331498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756846, "value1":0.4577877981753711, "value2":521428696388054794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551591, "value1":0.4656287234062602, "value2":3666151433771478256, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367967, "value1":0.3303667849726779, "value2":1336037347437447384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124888, "value1":0.9362913257587775, "value2":8565731764409689287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356964, "value1":0.763103278688178, "value2":7384734667573077593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.336557, "value1":0.18917586865284508, "value2":3164250330171533065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_4":"e","key_1":"c", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947920, "value1":0.7279862820897369, "value2":5585150661666129475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320214, "value1":0.02069659588289354, "value2":4096509130796828102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339884, "value1":0.3585015534256289, "value2":1745908303614221947, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736733, "value1":0.6300320540961489, "value2":4976479107859231308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843896, "value1":0.8618710521424175, "value2":409303658417169135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351779, "value1":0.31536023315738654, "value2":7469097912657923197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146297, "value1":0.47961745828082863, "value2":120290133475149079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.299654, "value1":0.953430875530432, "value2":1804061300635059106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.653880, "value1":0.6130300298494126, "value2":3600820249700194803, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.241886, "value1":0.4261506936601424, "value2":6410123065530378279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.719862, "value1":0.5557094068088962, "value2":6619604443134241152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192985, "value1":0.4372446351220488, "value2":5419318851099904514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_5":"i","key_8":"b","key_0":"j", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.062254, "value1":0.35078106654250335, "value2":6337545142680530695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_8":"i","key_9":"d","key_1":"f", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.826832, "value1":0.03911586835937395, "value2":8510739605077299393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.466577, "value1":0.8386295972027331, "value2":3712956826274262169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774816, "value1":0.2874652343932132, "value2":1661160295946815788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497396, "value1":0.37671291600810797, "value2":7809177916397546184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.061112, "value1":0.6756138122485794, "value2":3821150629117787840, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.711530, "value1":0.9726549946550717, "value2":4361793701491206585, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419723, "value1":0.8422780123510716, "value2":8387473970870580835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740430, "value1":0.3687691277865257, "value2":49347396348935758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544648, "value1":0.34026181496926355, "value2":3168616117094841931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.031693, "value1":0.35157464321590376, "value2":6956034409157435335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682310, "value1":0.4968537230062955, "value2":1715909753935447521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571570, "value1":0.7444971868018181, "value2":773947913317871760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412817, "value1":0.17790194335965473, "value2":453971816672635527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154029, "value1":0.695415820029077, "value2":4412491024371312379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187573, "value1":0.025946190955721947, "value2":8268616070900349780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297634, "value1":0.39036160577230816, "value2":9117936615987168462, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882305, "value1":0.6498141826508745, "value2":1021392582780518091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288985, "value1":0.6094023317572287, "value2":5522861781820654768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.516532, "value1":0.4913308899910666, "value2":2877526282810678571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.663856, "value1":0.6052287079408516, "value2":5244892044841001586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177812, "value1":0.8518899507803316, "value2":4611087097036165557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_5":"f","key_7":"d","key_3":"c", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789804, "value1":0.9253854154062459, "value2":4746855558380871130, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163177, "value1":0.054868844952025875, "value2":4671327502125519320, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753554, "value1":0.0480631824565462, "value2":7078677034582099346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537888, "value1":0.3970428718849328, "value2":8317481519599523582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887867, "value1":0.9625491057630078, "value2":851265403943296561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739174, "value1":0.3876147534715181, "value2":5378120884849587893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263149, "value1":0.6578316627430515, "value2":199930804061246062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926006, "value1":0.9018811245039401, "value2":8736925807617681839, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559532, "value1":0.017637548071686292, "value2":2677656350646305112, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.940793, "value1":0.36532273109204866, "value2":7505680103024683972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416010, "value1":0.7581313246147069, "value2":7561931289424116286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130651, "value1":0.31761722951334315, "value2":7760092958155535983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330390, "value1":0.3514672157667692, "value2":1963737467345840913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731094, "value1":0.5668209441148129, "value2":6834927574629705768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003464, "value1":0.44994944703983886, "value2":1268238147301227399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488880, "value1":0.15568001025824393, "value2":3802105015027414434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099752, "value1":0.49578700566450296, "value2":6660233803101117286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131762, "value1":0.8279103415624635, "value2":8189053738835694283, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.818080, "value1":0.7683869146409037, "value2":4901733131698699140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502240, "value1":0.2102196895388282, "value2":8419378719359168551, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130382, "value1":0.00017527290277822845, "value2":7209832064279367432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697720, "value1":0.977742512542472, "value2":5886772442159349223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152164, "value1":0.5452125862262094, "value2":2301031936204454118, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_9":"i","key_0":"f", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307306, "value1":0.27827706723478657, "value2":8114605959354081308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.687815, "value1":0.407696242982513, "value2":8251434258695514324, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.957403, "value1":0.9248822561074296, "value2":5435740408050442827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297405, "value1":0.735268728902142, "value2":6544542317901773513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505182, "value1":0.6434041992526011, "value2":6723613820733288827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953853, "value1":0.08239495901880306, "value2":7797622216144272838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.816223, "value1":0.026034555587018983, "value2":8247070392223609361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781228, "value1":0.009350455430296063, "value2":6344110503212444401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993318, "value1":0.7246275363827985, "value2":1117977296074531052, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.346481, "value1":0.04745532854484892, "value2":8841440982349072666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240566, "value1":0.6244142217945766, "value2":4416869640928141403, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627039, "value1":0.8121528941309705, "value2":2364077225142023978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.605014, "value1":0.041614775390040444, "value2":5587679092002855017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_2":"f","key_4":"g","key_1":"d", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.675250, "value1":0.22265983684713628, "value2":852855496341453985, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.626117, "value1":0.13978101794521924, "value2":4898929345737475246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.272053, "value1":0.8981952631418685, "value2":4155867998384044766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216853, "value1":0.7930982218994412, "value2":1990843980684870249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231757, "value1":0.11381414833270587, "value2":5629821232198202573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967616, "value1":0.9325867021628451, "value2":6458012454076604601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.105465, "value1":0.20026280910311015, "value2":1519145667363005058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959715, "value1":0.8692424311015335, "value2":3441946116265084781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.916752, "value1":0.09650654250745302, "value2":7435925512873628832, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314127, "value1":0.5339606052113064, "value2":8918601923652673265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955659, "value1":0.21237559939985842, "value2":6700000271323408810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.462571, "value1":0.8031758318689627, "value2":4200089724641662061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704628, "value1":0.42388353536260726, "value2":1676230088334959718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781871, "value1":0.7853951443498608, "value2":4677884824803639265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093020, "value1":0.44592927039430036, "value2":5106772041668223293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.066421, "value1":0.7167334468150509, "value2":916960599462889530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753333, "value1":0.8855172765987326, "value2":3498076904045993942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263558, "value1":0.039242126444255614, "value2":1994687158267196250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.529319, "value1":0.11390721956642542, "value2":6770255083967052135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.642302, "value1":0.25917237536080345, "value2":3561054032858646761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.800222, "value1":0.5648139239977721, "value2":7859467413386922358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.714908, "value1":0.3172565417584279, "value2":4018189432904936827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495891, "value1":0.1699695672081134, "value2":2951696383925672800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.927247, "value1":0.5193998219352067, "value2":7461947996252867047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465786, "value1":0.04561481590524083, "value2":8852624735734959732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970044, "value1":0.9502884615051707, "value2":5544298343674474830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.224365, "value1":0.7835642667167426, "value2":7989661444310811565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970171, "value1":0.044357973684653176, "value2":6240172232063272107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.022705, "value1":0.2909464675793184, "value2":6791113566169749215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.791002, "value1":0.30758766636926194, "value2":4087253744537420622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.526835, "value1":0.26218100233396685, "value2":761773481307699191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_9":"c","key_3":"d","key_6":"a", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057757, "value1":0.5941262990661297, "value2":5203993881422592468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621587, "value1":0.9726557103153585, "value2":4065263619832011048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936496, "value1":0.4220278511438201, "value2":537814119030374569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476922, "value1":0.7453981222005723, "value2":3086149467770264289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544208, "value1":0.1648752329298873, "value2":1528219915951406823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480134, "value1":0.5864292944345278, "value2":1915038657623169872, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.634200, "value1":0.06085546420380519, "value2":1586772996290355012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.290157, "value1":0.4479966729250899, "value2":603667588545059551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523397, "value1":0.8019975659364824, "value2":2263229113306312715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260677, "value1":0.49446211956319075, "value2":5384294748496824064, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.040570, "value1":0.2634688640512974, "value2":7081899407358309984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.670159, "value1":0.4242108328120063, "value2":1003697559007887859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.181699, "value1":0.7357308047023233, "value2":7674299813279760629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.437960, "value1":0.34081697732260335, "value2":1366760825001014571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_5":"f","key_0":"b","key_3":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585340, "value1":0.9880202469237767, "value2":5993013748473116382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758920, "value1":0.528974137063132, "value2":5387464413435497334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_4":"c","key_7":"j","key_0":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190405, "value1":0.24309504115260316, "value2":8699961182370479292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.045104, "value1":0.42560682798506067, "value2":3749517767528292571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.239270, "value1":0.11837048427737668, "value2":4852849985587057450, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647950, "value1":0.24581023177518593, "value2":7946632180521405772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969355, "value1":0.36754556888022905, "value2":7897267252653597746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416692, "value1":0.9421287451192997, "value2":7881806374967586751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.052040, "value1":0.27660174901670026, "value2":5294738319793885165, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112174, "value1":0.8996109424352536, "value2":5125031699148488781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696416, "value1":0.42330986040349045, "value2":7061433961041368821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127933, "value1":0.6743423459526591, "value2":5703788929519053213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_9":"a","key_3":"k","key_4":"h", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421658, "value1":0.6878542321395752, "value2":3553548366690488868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697493, "value1":0.169783750302659, "value2":7055001682108205827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.550811, "value1":0.35231569219578235, "value2":1455572534223159241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.493018, "value1":0.08084254027780423, "value2":2900266808751124367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949248, "value1":0.8847972628692381, "value2":4051840894997158475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142802, "value1":0.3270345064861823, "value2":2685380632911712697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.004387, "value1":0.822755239334881, "value2":7165522026622587078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620895, "value1":0.40285181555826477, "value2":7279952302942542040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.988553, "value1":0.42488091295486163, "value2":7195769969726296289, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314916, "value1":0.24528560160904253, "value2":3412797507763244235, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_7":"b","key_3":"e","key_4":"e", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740841, "value1":0.7678978195926474, "value2":7353627394146413053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767489, "value1":0.9849856324971001, "value2":4180092134684532856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.681082, "value1":0.08407929345646925, "value2":2247545351887675892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987608, "value1":0.3158564571415852, "value2":3372134575580371247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953355, "value1":0.6808029749520165, "value2":6416401290847025754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.934118, "value1":0.3815229371385279, "value2":5398231187188543220, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349522, "value1":0.008036202059570214, "value2":6632834525194225583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852636, "value1":0.6082938917169154, "value2":2209452475312580755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.491711, "value1":0.7070391677298287, "value2":5867779644024249420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329284, "value1":0.23697436114335285, "value2":2747492895656100802, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217160, "value1":0.013558152161070156, "value2":543215008088311315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376202, "value1":0.3967940240541657, "value2":4348396634707940652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134419, "value1":0.2570940106942101, "value2":5495331795379375428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.693581, "value1":0.6383501980446401, "value2":7332225459965791621, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_8":"i","key_7":"h", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843591, "value1":0.374550163160159, "value2":1957979143426115517, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127314, "value1":0.16562410193780275, "value2":5514782476287105847, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564210, "value1":0.47834402715548674, "value2":7864140566183633471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.673777, "value1":0.656108656648855, "value2":6919590747679310147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.071957, "value1":0.3923477172589352, "value2":7954657800672945455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497858, "value1":0.21899200976683347, "value2":5922902740218283725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063776, "value1":0.819166087262725, "value2":497846112853863498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.039234, "value1":0.06262728093114268, "value2":976242972769343091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658955, "value1":0.48196452599993433, "value2":2417102300700963835, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255288, "value1":0.13639741045424703, "value2":7715812700116132347, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.496112, "value1":0.1915457705996175, "value2":7008066197109595718, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144334, "value1":0.011357361010319224, "value2":7600237475494790795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.129843, "value1":0.45829887966275845, "value2":1540110795133843915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.828068, "value1":0.8516996075016723, "value2":8405984298577097957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870240, "value1":0.16467566190225996, "value2":1556056753506941522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199050, "value1":0.7512917767548035, "value2":3201410658564101147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830765, "value1":0.6115828253610872, "value2":1648965358200873623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385351, "value1":0.44894635523974136, "value2":2127367263922899732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.555613, "value1":0.566989554108829, "value2":4275387093192470829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758445, "value1":0.45045350568682, "value2":6540058212119502734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051410, "value1":0.07872483972360628, "value2":7661444664176600116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.027756, "value1":0.1866583235803357, "value2":4402492796625360355, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_1":"i","key_5":"g","key_0":"c", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043704, "value1":0.3399629891033194, "value2":2843620218251838442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.218394, "value1":0.6792705695737844, "value2":3717810518285734277, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.489979, "value1":0.2797515798899019, "value2":5603667040320572095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394844, "value1":0.47076120967369356, "value2":2551470543073926967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.176040, "value1":0.28649026862589944, "value2":8089352485313652968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015062, "value1":0.8503661846365486, "value2":9115866206649167217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.867138, "value1":0.18222497145716, "value2":748036164126661849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560015, "value1":0.16282129139411827, "value2":7758713833273415824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782776, "value1":0.3488004874020465, "value2":715341283187680350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.547186, "value1":0.07214699982761937, "value2":8071660169425611932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383243, "value1":0.8304505195235525, "value2":4079430116382080627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551280, "value1":0.6676351538773236, "value2":4287835647581016602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.853493, "value1":0.3478301116617329, "value2":7815908195604525371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511951, "value1":0.7049888722888338, "value2":1948135132689353800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173192, "value1":0.5458720430431239, "value2":5871330428575335057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413126, "value1":0.46553694703913845, "value2":7586060029018039815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781439, "value1":0.705616760521343, "value2":1127274508185282406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.732371, "value1":0.37205742400318453, "value2":5213749986182819618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569115, "value1":0.03375202087271923, "value2":118632103331380976, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.102614, "value1":0.2822023902913289, "value2":1391807523431966730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140423, "value1":0.44611401204912965, "value2":828680952801675744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.085991, "value1":0.7893667946565419, "value2":8411361994319110857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.975471, "value1":0.7162325854468116, "value2":79462509030833666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.244787, "value1":0.40407729683075916, "value2":2829176965864438571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.304099, "value1":0.2959757161094293, "value2":5764137406549726855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837106, "value1":0.4258452743236353, "value2":5341156407592196309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707606, "value1":0.031525280019894485, "value2":868621351492758022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_9":"g","key_0":"k","key_3":"f", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441177, "value1":0.3225860911008584, "value2":3070966375799941978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.077150, "value1":0.044429243112689185, "value2":8848387799149411370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604969, "value1":0.8353491972869558, "value2":5277008629054064552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537157, "value1":0.4103644112452139, "value2":9148697464094805661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353178, "value1":0.8020014291715372, "value2":611786907029670311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505264, "value1":0.10898025782239415, "value2":5937127177824419205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_2":"h","key_7":"f","key_0":"c", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.228410, "value1":0.43346447533111176, "value2":7102784791187505051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321463, "value1":0.9635846235850585, "value2":7091075060854093514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.999019, "value1":0.15889354502430938, "value2":7074771352556045066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_5":"d","key_6":"a","key_4":"k", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.766619, "value1":0.4943151198279696, "value2":6293058242069710093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382183, "value1":0.47579561094609324, "value2":2482575383850980092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.054382, "value1":0.7191240626399533, "value2":7914822293313169877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255873, "value1":0.055171339528865125, "value2":1928364223437432733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.535958, "value1":0.40417404005201557, "value2":8512533852699686666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394085, "value1":0.8317795195864898, "value2":6061937875766891692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.752931, "value1":0.1811338646540346, "value2":3103290849364510613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.049507, "value1":0.11124247410560267, "value2":8253232345223333946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.067068, "value1":0.02792863936331565, "value2":1942738411033963266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746589, "value1":0.43603502537000616, "value2":5153565802559198697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903064, "value1":0.3556713649253779, "value2":3489353708315351817, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.856458, "value1":0.28933348763855665, "value2":5247367677027973150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397179, "value1":0.7130148119357395, "value2":5728198037062323847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.927222, "value1":0.7639936733901646, "value2":101990787192301191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005808, "value1":0.03609120161396754, "value2":9216428818715898202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812165, "value1":0.4866254594918793, "value2":3026112033227979237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420851, "value1":0.7402262167603453, "value2":6177484091405440598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836705, "value1":0.0336701243488239, "value2":4778937227074309153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097260, "value1":0.6626429057561898, "value2":7257306969130399155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496088, "value1":0.36910009322506554, "value2":3657880390271983286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870639, "value1":0.7067162570632016, "value2":1102103625009499222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.020747, "value1":0.46587247219742656, "value2":5967339025562916268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.724139, "value1":0.8163768770313428, "value2":2067500216739340556, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.369554, "value1":0.7671983408631192, "value2":4937837819229078201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382589, "value1":0.94426076786917, "value2":575895696952869704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115327, "value1":0.47575759482061636, "value2":2541949700573197783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260734, "value1":0.8252666210219215, "value2":5316874961818228049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.872358, "value1":0.8794099168533623, "value2":5800876034859982246, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.054141, "value1":0.05686166977091221, "value2":5301631175885078161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759774, "value1":0.8076817854451119, "value2":2675452361460215935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921855, "value1":0.0467410376485931, "value2":7851926061831999904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578515, "value1":0.9029496688391901, "value2":3737429676030030627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812804, "value1":0.47887479306222563, "value2":1687173944192158201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472625, "value1":0.8441160499425835, "value2":1515286134892734672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760814, "value1":0.33274037254007843, "value2":1188458814270546919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788501, "value1":0.15942957594654944, "value2":2916736695606434573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098854, "value1":0.8030098284257553, "value2":2336601872331852302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894386, "value1":0.4139981184160349, "value2":7897864995270774175, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_9":"k","key_5":"b","key_8":"g", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117046, "value1":0.5674622805765219, "value2":4791162537839256650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.542843, "value1":0.19807908776500033, "value2":6807107659814067311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_1":"h","key_9":"f","key_0":"i", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703834, "value1":0.5788042510612239, "value2":5007564675868943302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945291, "value1":0.3701740719620502, "value2":2459541142030887319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070410, "value1":0.2627776276203505, "value2":9110732671786561088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.305776, "value1":0.24403108521016115, "value2":3139654072672113070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.417910, "value1":0.549110839930364, "value2":2406040915101429610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410216, "value1":0.8784609507591513, "value2":3467112924963980779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186300, "value1":0.13500098796508653, "value2":4869858265003017538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934357, "value1":0.7572145499314314, "value2":3619701318200032332, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871110, "value1":0.4402673304510715, "value2":5018886154821992035, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.493200, "value1":0.1348835909121323, "value2":3845890133642201057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875968, "value1":0.1364875892800979, "value2":570804136752641105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.469938, "value1":0.5776925075890766, "value2":8313002338216576665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.667608, "value1":0.9950302066492306, "value2":1913698187097493343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497144, "value1":0.7591363060077458, "value2":4304322015184203986, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731147, "value1":0.743025674257032, "value2":3480752580915944185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878760, "value1":0.5947269209561762, "value2":7998247484475194678, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.606309, "value1":0.663213778377581, "value2":3067341192985682212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.207494, "value1":0.8601914108563131, "value2":256790036338793553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.144417, "value1":0.30286956006124033, "value2":5377965606661311252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.134377, "value1":0.609971735912583, "value2":3774215546443285103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.434483, "value1":0.986170039019965, "value2":5064754870308124270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.509040, "value1":0.5522321797777098, "value2":5012812814115566068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903746, "value1":0.20330422291674133, "value2":6986910292987554191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282547, "value1":0.04694512830804489, "value2":6329857494201458927, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354275, "value1":0.16833606088021735, "value2":3136536256813456866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774256, "value1":0.5410085627082251, "value2":3587475793865363544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.762751, "value1":0.806841924275979, "value2":7358998298622408616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584082, "value1":0.6563492986452016, "value2":387593409104388354, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.080055, "value1":0.5509982288474451, "value2":6251069533288673484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695506, "value1":0.8632479367125148, "value2":2061552818267451584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489099, "value1":0.6265315870043634, "value2":4304737020249244240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098523, "value1":0.5339406052485941, "value2":4381041775949231103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945882, "value1":0.23785010902188283, "value2":6729562900414289120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.167154, "value1":0.3701120650676118, "value2":622381534969203518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775199, "value1":0.6184507954864329, "value2":5210419683928766855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_9":"j","key_3":"f","key_4":"g", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.730074, "value1":0.38121631002699524, "value2":419350096079979395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.013737, "value1":0.8747949109587233, "value2":1018960453244065142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.091852, "value1":0.629803885859078, "value2":859062102274314528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166974, "value1":0.47395530253799945, "value2":6411055007256508887, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.524472, "value1":0.9186585978945081, "value2":3492305084628232737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241236, "value1":0.3787867675534582, "value2":4760531059679736084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918712, "value1":0.05290704362542312, "value2":1966436853098488271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636045, "value1":0.9510956232200477, "value2":8704302710827007347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187946, "value1":0.6635616634234764, "value2":1433605612315108919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180096, "value1":0.173261531213523, "value2":3306945339384428282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.453459, "value1":0.11094008182846338, "value2":5724966973695631624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562127, "value1":0.3576118834730829, "value2":3463667757785807622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137341, "value1":0.6255329096036213, "value2":1381955744525597553, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913181, "value1":0.29162415237583095, "value2":2202030046648119531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.443734, "value1":0.3209051100037936, "value2":3274712051745788108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.302887, "value1":0.38589486316701666, "value2":1489273904023316120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205658, "value1":0.07820373463872678, "value2":7665901880569799873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.211623, "value1":0.32477633025580727, "value2":6808357456829544888, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.697819, "value1":0.26244647649977987, "value2":6232754250974380613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853360, "value1":0.3310597786987246, "value2":7565924568483154342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.135295, "value1":0.049613381396899345, "value2":6155724101399152110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701749, "value1":0.9716574010135762, "value2":3704214151049295904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527554, "value1":0.6532141955951636, "value2":2786469958733724605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866043, "value1":0.03422399382505321, "value2":5496039574170303988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.415307, "value1":0.682109162147481, "value2":4924263240012975638, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631078, "value1":0.7274897566111301, "value2":8520095366826456731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.549725, "value1":0.7077865773466269, "value2":2977543758339035237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.154150, "value1":0.14439730391589514, "value2":7810498955756317569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007570, "value1":0.8087681893470107, "value2":4017899853890170979, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.055629, "value1":0.44977291321386215, "value2":3453485470431150033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189056, "value1":0.012064619774392913, "value2":3564594888238224552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.545459, "value1":0.412297537176583, "value2":7021178975362987850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.116215, "value1":0.3276649448479836, "value2":3143364922694929733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529456, "value1":0.4708787441244093, "value2":2999813825732785635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_8":"d","key_1":"i", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.966668, "value1":0.04918146120370962, "value2":870818154766064140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601524, "value1":0.4723442759712491, "value2":5773596887056998694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712431, "value1":0.04260252713403969, "value2":4497318416983710614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452968, "value1":0.03699415334331326, "value2":3190104546355404213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.394341, "value1":0.05586282401938057, "value2":2019354346274899062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.084367, "value1":0.35383668982583555, "value2":14041413767842483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227017, "value1":0.7278732120270098, "value2":8901905120302064125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494493, "value1":0.0910027308098097, "value2":3614458258424175747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059547, "value1":0.2958045744232146, "value2":5813283620430694858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.423389, "value1":0.9221509815116762, "value2":6742495498273189642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501855, "value1":0.44198471706869047, "value2":7662744388645110665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293650, "value1":0.512834114916101, "value2":5984393328938374467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614743, "value1":0.8714098760733937, "value2":937734757736172502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406459, "value1":0.17711840072487878, "value2":749048739837121472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711112, "value1":0.5465711310503617, "value2":2943773951314932011, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335697, "value1":0.8515828942170133, "value2":7615557480020888685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548405, "value1":0.13026413723454033, "value2":4513263208882824306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.351448, "value1":0.28264554746804893, "value2":1701229548455066314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501558, "value1":0.7474271542447126, "value2":4300562906938627060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107125, "value1":0.5220893826927367, "value2":3433346642469038441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968552, "value1":0.32921550916944375, "value2":4204134061287881453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.438315, "value1":0.5786006279022659, "value2":7016921001360584415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.834753, "value1":0.15159677478724748, "value2":3404421917920731295, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454269, "value1":0.5292362536342466, "value2":6241633082661695873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137463, "value1":0.4976726181585776, "value2":7634359338923847776, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744484, "value1":0.7709542443985684, "value2":2984932547540133289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.653649, "value1":0.40168424113567774, "value2":9208485552143870835, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420303, "value1":0.8834331384252292, "value2":6602388659890449882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.995276, "value1":0.35489802779155194, "value2":3627299399432728520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.609717, "value1":0.32106453173709887, "value2":6997589541390821763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.764384, "value1":0.448976213692369, "value2":2970007228856945436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548393, "value1":0.15204462152678552, "value2":7646037909450468928, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379101, "value1":0.7807002939639928, "value2":783999596388900221, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.278309, "value1":0.014487780492841778, "value2":1057643982300790379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.016491, "value1":0.40903973612790867, "value2":453932377401006766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347001, "value1":0.5622168932310977, "value2":5705011066230310463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.086911, "value1":0.6494793319729653, "value2":861844073039964568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.997904, "value1":0.8338228010696771, "value2":1349114457177401043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266213, "value1":0.43023507437229846, "value2":2397780341037792730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068462, "value1":0.6058806411980994, "value2":78381759723174420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358282, "value1":0.2854157607268251, "value2":9047800747480831869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845468, "value1":0.009988708005244632, "value2":2440593977353687165, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780415, "value1":0.6187822246822943, "value2":4482602300599359335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_1":"g","key_4":"k","key_0":"c", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700223, "value1":0.018684194191095932, "value2":6529685586040726217, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.792345, "value1":0.9111940168010332, "value2":1575039147844860051, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275696, "value1":0.01410734034670126, "value2":8535583193313743852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458977, "value1":0.23636701604224677, "value2":314843807550918715, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_9":"d","key_3":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711597, "value1":0.1315269488618045, "value2":914748291669479577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003734, "value1":0.48566417416368673, "value2":5809881749485776988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.587116, "value1":0.411572893699433, "value2":582776826400065747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000207, "value1":0.25509895950578154, "value2":5802214523620869062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082969, "value1":0.48435989736978513, "value2":7352735247711683766, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.042350, "value1":0.16157141460949973, "value2":1607874054697622469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.765374, "value1":0.15664293959132114, "value2":3675493379037113802, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.359224, "value1":0.887036256695556, "value2":7786059543942584229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293533, "value1":0.267785564160632, "value2":2723125994900979360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964294, "value1":0.716239726237387, "value2":8651041612345523274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076213, "value1":0.9749455558859894, "value2":5297461003834502762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.896241, "value1":0.7102104646771853, "value2":2795778846362770396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903120, "value1":0.14414851964547873, "value2":7334142904098197, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.843431, "value1":0.8361555205773405, "value2":7976205336110881786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731615, "value1":0.4179919061811569, "value2":1554283199870929425, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.953374, "value1":0.7537780463182495, "value2":2591672750977463865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400999, "value1":0.7471726171159019, "value2":7300350647929857982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232296, "value1":0.7720730243870363, "value2":1060384392061020703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219798, "value1":0.16805543570986609, "value2":1462620146278820176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364330, "value1":0.7255148218772242, "value2":713753994079127610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152830, "value1":0.13410318940561705, "value2":9140011796352402602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585566, "value1":0.4130103851752984, "value2":2970454230901035575, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.718094, "value1":0.5500539331076559, "value2":8305617631847453693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.624058, "value1":0.9500416769324029, "value2":2935568409193402069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539303, "value1":0.05177136725388866, "value2":7442175860869173746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.172986, "value1":0.4456449462659514, "value2":137655062794989901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631420, "value1":0.9356892224700029, "value2":8946945917645913041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948627, "value1":0.9988907819529511, "value2":7978995836658846886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946323, "value1":0.5613474012283862, "value2":648588536937617868, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361054, "value1":0.4585620050578102, "value2":5573215420419545492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.817729, "value1":0.3241298418342054, "value2":2657202509964663855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_6":"i","key_7":"h","key_2":"b", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713670, "value1":0.7338301625220077, "value2":7217765288314019300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.521076, "value1":0.5030076209647331, "value2":3777966789579493364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.855317, "value1":0.7172152098421418, "value2":8060266815999977878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849239, "value1":0.9561486509896251, "value2":266727525021228946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008199, "value1":0.11740640397171373, "value2":3689690733597943394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936773, "value1":0.795334335887194, "value2":4733404788345141419, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_5":"d","key_7":"k","key_1":"g", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888974, "value1":0.9298030625919304, "value2":6088282595701162644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247054, "value1":0.1558941325474189, "value2":460928608228918469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853182, "value1":0.8511278083029166, "value2":5598797292292719359, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.252753, "value1":0.16156992511009938, "value2":4368635811765896864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309449, "value1":0.5849973411963314, "value2":683197153873995001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.222187, "value1":0.6637363404506509, "value2":201431956551451886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805299, "value1":0.03082292423828661, "value2":3135350466088347105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489193, "value1":0.47023015827549575, "value2":301183049195776618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_6":"b","key_1":"b","key_2":"c", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.058774, "value1":0.7620086190415307, "value2":2178333587571299612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098933, "value1":0.696442402358351, "value2":427216344556980093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497026, "value1":0.18415993654905655, "value2":8786493859199944811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257640, "value1":0.6674105512961352, "value2":395753979079709412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_9":"g","key_1":"d","key_4":"d", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490944, "value1":0.5114071451614219, "value2":2042645288775439509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455445, "value1":0.7431063214848364, "value2":8165863124378358117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.796800, "value1":0.6414858212054014, "value2":2442066548061613486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616766, "value1":0.17517079750483502, "value2":2134452812266479738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060207, "value1":0.5103041206672632, "value2":6515531846051600666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677324, "value1":0.2542456767596465, "value2":3968114223247313347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.487523, "value1":0.6903207412936282, "value2":1424416428597884416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601220, "value1":0.7953800597000228, "value2":3247496941215159385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.642959, "value1":0.2541441152051556, "value2":7121652419723619875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268832, "value1":0.025435318199453064, "value2":8277309445690585191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591576, "value1":0.98502525758626, "value2":3261712292707439368, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696131, "value1":0.602844278741062, "value2":9150721424292798196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.799050, "value1":0.6397730903214256, "value2":322841440821332477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703698, "value1":0.7179811740811757, "value2":6911091771320522730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.276968, "value1":0.16623158682521033, "value2":5763688015485978321, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.932237, "value1":0.9953773456479392, "value2":1648531648235955285, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562810, "value1":0.4786357292651346, "value2":3349388012721867031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496221, "value1":0.10819101220823944, "value2":5211051877655840528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720431, "value1":0.552698162040604, "value2":7021843927329324470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.087213, "value1":0.8472571863880398, "value2":7745061885728922306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.039167, "value1":0.21533794157440472, "value2":3121229544727240655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309356, "value1":0.1590321315931928, "value2":710479249184112134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.191661, "value1":0.040375310871212516, "value2":1055365248220450266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189905, "value1":0.8222460270476257, "value2":9109111690041797580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517425, "value1":0.7168387126170568, "value2":6945048447223853682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916881, "value1":0.37193142727878464, "value2":2747126651524473231, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.395094, "value1":0.6414572510838905, "value2":2905014786439561176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.163189, "value1":0.7311236407483467, "value2":5070208982215207577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827333, "value1":0.19235777061627696, "value2":1042609530408390018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107479, "value1":0.11539837467269841, "value2":7629188841800355160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186100, "value1":0.23799692868617986, "value2":3114291156883869525, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790442, "value1":0.4974549500784167, "value2":3142531162087829163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.797183, "value1":0.252808246395566, "value2":7867371410034903023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269019, "value1":0.4459897405815389, "value2":5769972800266784623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299688, "value1":0.7900084672305991, "value2":4400086899085085411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.295251, "value1":0.4992500267841033, "value2":8698415822608825668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.992903, "value1":0.9703688948654993, "value2":699484569689785049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694010, "value1":0.24590491974037657, "value2":6378096972799959498, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916779, "value1":0.03667685970441408, "value2":8236016681410946158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.018930, "value1":0.7762719211488223, "value2":5653992369569699642, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673243, "value1":0.9005498092358467, "value2":2541259280078576557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.403564, "value1":0.7066442393226117, "value2":7468478788293853391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912162, "value1":0.4479784585493673, "value2":2451129253364350697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622846, "value1":0.6985884362308626, "value2":1822728881940552123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707469, "value1":0.052057120254889885, "value2":640515677830640949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433576, "value1":0.4097754058261397, "value2":2475392911974322330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374459, "value1":0.1353067477342801, "value2":7368967313707141232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_7":"k","key_3":"g","key_5":"g", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993651, "value1":0.06711618906287087, "value2":1750993237587336816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219084, "value1":0.5316119999916927, "value2":5790597415731545344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527236, "value1":0.6470232539834205, "value2":5706809590317379695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566174, "value1":0.9745838789323467, "value2":774234656544468494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803752, "value1":0.8708197206158679, "value2":3775873873449515760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.388879, "value1":0.4862649086023325, "value2":8962983929778901050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471944, "value1":0.25578979603582636, "value2":7732635688969517720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017485, "value1":0.4968624551232297, "value2":202940911271524303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_8":"c","key_1":"f","key_3":"b", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.333780, "value1":0.8299780273551013, "value2":895214629529935810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.164082, "value1":0.9321061863901389, "value2":7407307236544459993, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225068, "value1":0.9163062543062077, "value2":8739271132887488724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304547, "value1":0.45077586944816295, "value2":1933302507685789986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.538110, "value1":0.4128481092985977, "value2":3178182543294139274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541173, "value1":0.7483924955860858, "value2":212944257013368531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255386, "value1":0.14438260405328707, "value2":7645966424158688784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.073000, "value1":0.7628193459385832, "value2":4973597494190231390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408853, "value1":0.5564603800067697, "value2":4401411283807625564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447512, "value1":0.8939878640344981, "value2":2693396599053914337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048017, "value1":0.34230394362828026, "value2":3182689568942606164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057151, "value1":0.8042880864594656, "value2":2051969816473375198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746476, "value1":0.7332341667442952, "value2":461488002348760082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_6":"d","key_1":"i","key_4":"h", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180890, "value1":0.31049348146155, "value2":1020023460971269956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550108, "value1":0.4362790210713837, "value2":7901529415931806303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345598, "value1":0.11823802615751057, "value2":8356166230167153323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804625, "value1":0.5643553567149021, "value2":2154734444228835687, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_8":"c","key_9":"d","key_6":"j", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247313, "value1":0.6975496564473892, "value2":3820052334665018388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805735, "value1":0.31084346436090265, "value2":4462974662487547374, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364439, "value1":0.1744566217814337, "value2":607910964577919864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589313, "value1":0.37010824428777245, "value2":5796079422541877022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931293, "value1":0.5772458152930102, "value2":1027336122468786461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527679, "value1":0.2977670251954975, "value2":3769527680254605539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.404475, "value1":0.38496439646172187, "value2":2284351654690866371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772590, "value1":0.07775425303960136, "value2":8477700234367515830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722168, "value1":0.17927054370632736, "value2":7192360571641392544, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712775, "value1":0.576069645932529, "value2":4594581975868935835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.126652, "value1":0.5061094013712955, "value2":3728550795113093627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321145, "value1":0.13357161718693558, "value2":588806792143157793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.101550, "value1":0.6394371793311785, "value2":3045802217092318405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.547834, "value1":0.5303880277608923, "value2":244812322497139670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_5":"f","key_6":"f","key_2":"k", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.022447, "value1":0.813105541632356, "value2":4932627202785941294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_8":"b","key_9":"k","key_5":"b", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382289, "value1":0.3508103161551219, "value2":2376325264202595095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.611464, "value1":0.9618639190404236, "value2":2197184281244482985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.911985, "value1":0.3625352195059055, "value2":211893969967676184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268188, "value1":0.6899041268799436, "value2":3263903700628732234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229932, "value1":0.7830409765107325, "value2":3581295224187821255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921675, "value1":0.15132831757259205, "value2":8941349375775958121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.738768, "value1":0.28818505448700765, "value2":5311365998633458584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.670459, "value1":0.7891122563117983, "value2":5569764792017894880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.893664, "value1":0.863874518524225, "value2":4812640494718764785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484101, "value1":0.6065100518640419, "value2":3350972220626617309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048572, "value1":0.15248106917852358, "value2":6593923637797377418, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.043823, "value1":0.6456350127530551, "value2":3888964250592362865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109495, "value1":0.6845812758644145, "value2":167613517010834758, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.328450, "value1":0.42643027428907515, "value2":8129659398848977292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.429304, "value1":0.14015794848590404, "value2":4681962038728020905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.776267, "value1":0.3289665984746857, "value2":3318083720052608693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496512, "value1":0.3624675397742157, "value2":6289410424653796398, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441324, "value1":0.9094233421689666, "value2":2912289985138568601, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.650459, "value1":0.3752516427539381, "value2":4398293649161961566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.350383, "value1":0.1741603700667637, "value2":3124160867383954120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.051974, "value1":0.5242492694985263, "value2":6760802540060503552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_9":"i","key_5":"d","key_8":"f", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.291840, "value1":0.5576830705012547, "value2":4155187805029620559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853505, "value1":0.13344813988799406, "value2":7220245991757698014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_4":"b","key_1":"f","key_2":"f", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.480822, "value1":0.34390971406763976, "value2":4047856794594370906, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192205, "value1":0.46496172552445103, "value2":2904944897204510913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668616, "value1":0.9800566015891223, "value2":3768818727218430516, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.940469, "value1":0.8502278446477077, "value2":5643864232780485221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_4":"c","key_9":"i","key_0":"a", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775796, "value1":0.26560945264856717, "value2":4386556505603475589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_4":"c","key_9":"g","key_2":"f", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484224, "value1":0.10363743205437767, "value2":7594526116903338826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.889136, "value1":0.4627114619785784, "value2":4680730415934145352, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739229, "value1":0.1719548250883568, "value2":6803717844294234894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_8":"i","key_4":"f","key_6":"k", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057765, "value1":0.3081555912049451, "value2":8499651141401941047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.450693, "value1":0.8168675860307004, "value2":7949382588313472478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770167, "value1":0.272960033553786, "value2":1543554906153054869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411881, "value1":0.5252975573521564, "value2":7223448923269287286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.892105, "value1":0.15152104472471403, "value2":3531358618763240055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.882618, "value1":0.45516649656010977, "value2":2288125701245215412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913392, "value1":0.6620512228658078, "value2":4363674990969315932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.835393, "value1":0.7513525395048825, "value2":5883431192279251653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.563057, "value1":0.9008981984383352, "value2":6465724162584986904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007001, "value1":0.9502622268086611, "value2":6989543064471041687, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688255, "value1":0.07451084769674933, "value2":2957263551734111550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751629, "value1":0.2617947338456188, "value2":684618542420066244, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930787, "value1":0.6308597904360509, "value2":6601900249320434792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.363243, "value1":0.8139409689083822, "value2":1497160325123262680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676085, "value1":0.9711732869605988, "value2":1463546357374490230, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.161858, "value1":0.6374277110955211, "value2":6435065381978315913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757941, "value1":0.910443361797238, "value2":2854035357843167779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.331899, "value1":0.6895825352637633, "value2":621471764809456095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473746, "value1":0.2844187612168192, "value2":5704986401969616545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048060, "value1":0.9984260267119236, "value2":4771344600707329825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.030112, "value1":0.9637634957448155, "value2":5568130719531472451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257160, "value1":0.5940271387669745, "value2":142865511754737819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760326, "value1":0.4334581523321684, "value2":6413870941968139402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827687, "value1":0.8136794802681319, "value2":746965665674196204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447680, "value1":0.3168283761550826, "value2":7144161377877398366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690596, "value1":0.3996428785716089, "value2":3598801694226963836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.009204, "value1":0.21753660582669873, "value2":1790271338088923331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.432520, "value1":0.8760470338201594, "value2":2329654518443264597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558450, "value1":0.5241764449812006, "value2":8062760419700144076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.999165, "value1":0.4211253053940594, "value2":6241537222130424287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.492382, "value1":0.19419908429465416, "value2":8132562796409382183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968488, "value1":0.8740668460370395, "value2":5462263580744941212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566020, "value1":0.3116629548456918, "value2":1660846180499855539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899292, "value1":0.4455952655758791, "value2":5641720897227183391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213999, "value1":0.7107285930376039, "value2":320006539559036783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177859, "value1":0.6189254938745993, "value2":3916861895859871884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.942059, "value1":0.3464916414346185, "value2":1739217875853687994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637964, "value1":0.4027137396066144, "value2":498622351647583844, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994610, "value1":0.7530828834500288, "value2":5890892402021233384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.727487, "value1":0.28182955033112955, "value2":4630833361364112333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.915590, "value1":0.5470194104324091, "value2":1666728635470642191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757932, "value1":0.5371051329548212, "value2":1493513093655584392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.254302, "value1":0.06977656180754654, "value2":5979425073849108121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.475877, "value1":0.8550799756217883, "value2":624522672976326814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247089, "value1":0.39743079305559637, "value2":4830779617580090488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015595, "value1":0.2577347123153436, "value2":2406835300488043720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945719, "value1":0.3129335496013211, "value2":820286347352483926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081969, "value1":0.6051672450326654, "value2":8261558360059795606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948207, "value1":0.7815897187663656, "value2":3429909037501105433, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.935696, "value1":0.909809980891043, "value2":6385383912360129741, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.248852, "value1":0.9247408795385116, "value2":9175721939467544545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934738, "value1":0.6334884200328613, "value2":327826698570505025, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102837, "value1":0.44571438142580455, "value2":4747306485524936018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.885921, "value1":0.35994074928816305, "value2":1395907621788196782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.949368, "value1":0.7895021421389461, "value2":7949753331449968910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.633040, "value1":0.5458972679689471, "value2":4245189254820710416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.150358, "value1":0.7548359500337994, "value2":8481176564399105584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.557167, "value1":0.37427466994480085, "value2":1133639012241894316, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.093134, "value1":0.29043113295756745, "value2":6947797053182077820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_7":"d","key_1":"j","key_6":"a", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.136231, "value1":0.7872857259243544, "value2":2466956534940008420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.329501, "value1":0.01079935258997052, "value2":4968522926518293278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.825131, "value1":0.6767719760068325, "value2":8529334985450219291, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807971, "value1":0.7351117134307579, "value2":3019346910002173461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_9":"k","key_3":"f","key_7":"a", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017959, "value1":0.6253725541571399, "value2":8835344453005145545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_1":"f","key_4":"j","key_0":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.280599, "value1":0.16481336771423033, "value2":1417072867431315310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673036, "value1":0.8178932673930933, "value2":6530240293167023011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192297, "value1":0.7505738676572953, "value2":6861957449422419505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517236, "value1":0.639000099787322, "value2":5815570450456127187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739883, "value1":0.17493563192408712, "value2":7179940437785343272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.531912, "value1":0.08360001327419847, "value2":2755543678440921421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452307, "value1":0.8726532834145122, "value2":187164282595967676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517272, "value1":0.9444444024596198, "value2":5607680802025895328, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289453, "value1":0.15619031798053584, "value2":810705789880173134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397508, "value1":0.9544477853731341, "value2":7641027031682219689, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812107, "value1":0.9448725135844366, "value2":5145031592575451663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.063548, "value1":0.922662738511914, "value2":2569888172146766439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.959005, "value1":0.7947728545231653, "value2":4794325670292386969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.570203, "value1":0.9426031560649749, "value2":8759901536059040166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.514218, "value1":0.47519036709767065, "value2":1247765910819397634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541370, "value1":0.3774216285225394, "value2":7667033726860743169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.938282, "value1":0.020047520852319107, "value2":7849085976338159942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874020, "value1":0.47048043771282444, "value2":2196635070664843443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_8":"c","key_3":"a","key_6":"e", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513588, "value1":0.012232986779526358, "value2":8478348320905895865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109700, "value1":0.7338145643706604, "value2":6797416603289211588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.488779, "value1":0.6031093288566471, "value2":5300319596464382751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931594, "value1":0.42339447548233033, "value2":969762807855801088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.553860, "value1":0.676076748018672, "value2":1192139536623982501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.132006, "value1":0.6728558661354047, "value2":1380355336108501777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.669226, "value1":0.1577792131508693, "value2":4512376080159929307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921683, "value1":0.24615483388690507, "value2":5797393083888487885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.800085, "value1":0.768303750055719, "value2":6971653838071612447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468961, "value1":0.45803974634578515, "value2":1199483949895079721, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356489, "value1":0.49471640885346424, "value2":4138046086256944727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225992, "value1":0.6741558407603926, "value2":5216648174057731100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.779458, "value1":0.11692394331277675, "value2":9011198363894466670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.683290, "value1":0.48772002686762633, "value2":7212512979389072906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053472, "value1":0.5939009672642634, "value2":1146669122452133702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790089, "value1":0.8261492714018603, "value2":6290707817110359790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145153, "value1":0.6394833078139336, "value2":7849898590496421466, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.367040, "value1":0.6490663681468899, "value2":3967326627567255091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_7":"k","key_9":"e","key_4":"j", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930388, "value1":0.6376629127038135, "value2":3197193413798224716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471623, "value1":0.06021884177520976, "value2":4773928340410962251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_7":"e","key_1":"c","key_6":"i", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201531, "value1":0.7136591237382468, "value2":6260011441045155926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064553, "value1":0.2818642070197672, "value2":2495939851413003901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282911, "value1":0.1452427125541155, "value2":8365230104234658997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279591, "value1":0.8354266947002554, "value2":1254307733558365753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849657, "value1":0.6912085697845711, "value2":1824933885382646361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_2":"d","key_8":"b","key_1":"i", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.092000, "value1":0.053974617818740134, "value2":8465803905922172931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408792, "value1":0.3821176162488874, "value2":4135301501032886839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.728999, "value1":0.9460512775720927, "value2":5232062325949200093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.706765, "value1":0.05808795299595995, "value2":3187473873599955497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689611, "value1":0.07108495627051868, "value2":8720638688812490559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.686555, "value1":0.11535487696002504, "value2":3943663806047625686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815986, "value1":0.6951101939977824, "value2":2103945278295322543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991212, "value1":0.9405521412873721, "value2":3787480236551923641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665833, "value1":0.21165663807431181, "value2":3904059491287927557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183315, "value1":0.5795855388985459, "value2":1854253684581147206, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344965, "value1":0.8721578775869344, "value2":2246100236955108961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671953, "value1":0.31034394563958956, "value2":193813594216774336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.127315, "value1":0.4080665667532778, "value2":360930115537783747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.947191, "value1":0.901998732832304, "value2":1708333379014319670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894585, "value1":0.9195635726221545, "value2":8084121448593915190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004792, "value1":0.9591461919610487, "value2":8675745838539187803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900181, "value1":0.4274569433819623, "value2":6055452196459435249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948861, "value1":0.954396067309185, "value2":2700832246849384276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_8":"k","key_9":"g","key_3":"h", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656208, "value1":0.4983661330975837, "value2":3291693517576289438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_4":"h","key_5":"c","key_1":"a", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.937253, "value1":0.2620189984734357, "value2":8181737649910995938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867100, "value1":0.06224158128949376, "value2":4580692356241748765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827749, "value1":0.03871136162211054, "value2":7089705182458313228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241402, "value1":0.4482242754297532, "value2":9114765255689214165, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.602059, "value1":0.29735202117340814, "value2":3162172555777612351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.249371, "value1":0.23918886296603434, "value2":9067747281684729145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725148, "value1":0.8102090769702104, "value2":4626069739522513533, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.928403, "value1":0.3838571333237863, "value2":8647736244601572769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358211, "value1":0.32922133279873583, "value2":8174411590575614648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347825, "value1":0.7167150468238062, "value2":8311537141574084420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875897, "value1":0.26919747077534717, "value2":9053609159881163713, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392772, "value1":0.2800052196883224, "value2":7203447031464069407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849662, "value1":0.5009846634201038, "value2":939832331440858045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.024201, "value1":0.9391704477843268, "value2":2885798586864811264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952269, "value1":0.2664477356829927, "value2":847075590517072325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.307777, "value1":0.6456685214669929, "value2":3218983200036055999, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_5":"a","key_9":"h","key_3":"j", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.832715, "value1":0.2591746701265793, "value2":3426357136923132270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.510375, "value1":0.5688430382533638, "value2":7025749889591390600, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.850762, "value1":0.6438552947023466, "value2":1197489157433338286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_6":"f","key_1":"j", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.905930, "value1":0.54095928495375, "value2":67603635153932551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636437, "value1":0.5854506016456503, "value2":5128251159570110333, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836009, "value1":0.4067978187295729, "value2":6225162557978433985, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.781439, "value1":0.8809907639802731, "value2":1945481468549859562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722225, "value1":0.6928638550983661, "value2":7984089816823103368, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631076, "value1":0.0305606186093776, "value2":3704333676945193488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.515877, "value1":0.03574457632343526, "value2":8116748669267665996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.062029, "value1":0.8880344844442181, "value2":7205869492721817899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.753934, "value1":0.5309770343574366, "value2":3276732533413239990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598973, "value1":0.9458250274999828, "value2":4588112807086674988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284154, "value1":0.6864088859089572, "value2":8516198907569397198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.337595, "value1":0.45387405891882765, "value2":2878690183576728505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743636, "value1":0.5411320007572493, "value2":7926291410401458702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751569, "value1":0.29015226479219164, "value2":550259652101408680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_6":"b","key_9":"f","key_4":"g", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696737, "value1":0.19318283293628083, "value2":8059270019889405703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558974, "value1":0.5527141281446059, "value2":4981805365285955545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.499157, "value1":0.3593055667554787, "value2":8763899128009608550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725437, "value1":0.38132949820890993, "value2":2593622500148824140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575337, "value1":0.27925006017882537, "value2":4225105729105614711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.168187, "value1":0.8455269167046058, "value2":4706079022430056900, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.223652, "value1":0.14768410638621007, "value2":2848574409182395374, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.655719, "value1":0.26992469602574687, "value2":4754721503564673004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972901, "value1":0.22308996900209793, "value2":4121883961065445917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513268, "value1":0.763713042109811, "value2":1389619580667003266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335708, "value1":0.9988121809461258, "value2":8201212630370607829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874837, "value1":0.37662051730821194, "value2":1239286216512679885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.761409, "value1":0.08352300660036191, "value2":788197175065166874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993753, "value1":0.08212627471454952, "value2":2431495613386540436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513306, "value1":0.09225555422185021, "value2":6434312790658522957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899387, "value1":0.2747071960330611, "value2":3838090571407685488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498947, "value1":0.19881833202736185, "value2":645491056031774982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851971, "value1":0.9432399401550566, "value2":3792284814302154775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912337, "value1":0.6898029391029513, "value2":8695381256661600831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060802, "value1":0.4672690888050405, "value2":509110642459148760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.988275, "value1":0.6949519126514042, "value2":2125445866842945317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.537780, "value1":0.9644299177479598, "value2":2844641921809856363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.129171, "value1":0.2870831250353682, "value2":5356758362083554180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.699225, "value1":0.7476204022496519, "value2":4681919372415529338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.798856, "value1":0.5178345021294356, "value2":8950909328927419020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356529, "value1":0.9689016091606751, "value2":778967583500360424, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.740297, "value1":0.48229504021118547, "value2":319778397891873168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082023, "value1":0.8672498335471771, "value2":2761738695533665878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.737582, "value1":0.21539065762206674, "value2":1173702480391317018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.203204, "value1":0.04428967569336166, "value2":6796467582468653168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197343, "value1":0.5038157716364858, "value2":5419554006840711744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_5":"g","key_6":"g","key_3":"e", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166454, "value1":0.4520211285816449, "value2":4140960458683685597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.810425, "value1":0.9556399667218088, "value2":2345821074968960700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708611, "value1":0.36562484484170277, "value2":7482183983302020104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614909, "value1":0.010690625631515951, "value2":4802163912353727023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529200, "value1":0.23500509922901588, "value2":1518336865591090789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957588, "value1":0.8999494296908893, "value2":5272580661109552970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900198, "value1":0.3844554073160095, "value2":4170365040112200144, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.579116, "value1":0.16747277518786968, "value2":6804977291625507700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250708, "value1":0.8577877748125339, "value2":590691746141274872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721810, "value1":0.7147800394287138, "value2":4035652902850373019, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.922402, "value1":0.0627394141655773, "value2":4185528067707010977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.198128, "value1":0.08114002676829289, "value2":2239252166892691326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803033, "value1":0.0019520065249596544, "value2":1312973954738137976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.174637, "value1":0.09909477348410087, "value2":93134409084713078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250984, "value1":0.49581172361364056, "value2":9059996107352612477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381289, "value1":0.5781636662002811, "value2":3051612121774242926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.046509, "value1":0.758530400177533, "value2":67989907532933127, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.933345, "value1":0.9339175035199608, "value2":377586629316226209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963510, "value1":0.7004527478348754, "value2":1390034556068597934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918093, "value1":0.17000680115237268, "value2":4215018843369459603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.569630, "value1":0.20175634343868196, "value2":8381338885007503558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.883668, "value1":0.06695323081792075, "value2":1589024156310802370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.178981, "value1":0.37314887727591756, "value2":518499450705948775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187208, "value1":0.4599180344115974, "value2":4887550148987301900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952934, "value1":0.8319538072999415, "value2":3671167315474130299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.588275, "value1":0.5426790448893685, "value2":6523903627951338788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.128001, "value1":0.37530546288075156, "value2":5616976404881722053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004862, "value1":0.8346907316991129, "value2":4019033006145719569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826250, "value1":0.5254457071340125, "value2":972903594117286740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344623, "value1":0.6999472357226462, "value2":1701492823670778543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639041, "value1":0.2248828857383105, "value2":4810585903388633809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.571165, "value1":0.22946219998225406, "value2":8055176566662404630, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117315, "value1":0.7792786347828345, "value2":8503346841194050818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.646178, "value1":0.29610828190559946, "value2":6707117461343778078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804384, "value1":0.8994527553958773, "value2":3055603759085197982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.503237, "value1":0.038643663095115535, "value2":97929435615191250, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.917565, "value1":0.06570063912325382, "value2":7659557506835508250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.194133, "value1":0.7280519586189449, "value2":607993872873285299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197005, "value1":0.34702978428971004, "value2":2119181822487481024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578715, "value1":0.4531709992663944, "value2":1719249508626109183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.221581, "value1":0.5476991652235295, "value2":8006956843834072386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177885, "value1":0.5137634978637854, "value2":5009892345362966157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596214, "value1":0.2368591303155911, "value2":8792401284340375040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182004, "value1":0.5444756866358181, "value2":1210706677095755681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972249, "value1":0.9909726775988351, "value2":1108183309913410572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225911, "value1":0.47194616841751835, "value2":3490783413049237610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321843, "value1":0.5905642407899484, "value2":4893835600551421123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.552992, "value1":0.8654702264058397, "value2":5106927081449112108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782381, "value1":0.8285431175840194, "value2":9168936255854371872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410849, "value1":0.31656794437122665, "value2":2071986113321022167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780581, "value1":0.051832044232056354, "value2":6484981440389875558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.072700, "value1":0.698857333675772, "value2":3443549406704488268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689951, "value1":0.5808073564555776, "value2":5048983612386176894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.733768, "value1":0.8029317643963917, "value2":7359595780765172048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364358, "value1":0.34011958562739014, "value2":999331274737013831, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.512816, "value1":0.34358105240557685, "value2":6714721903573983722, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711860, "value1":0.9821515464895118, "value2":5477696623940889135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.857930, "value1":0.697732523770638, "value2":6661494860267552531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_8":"k","key_4":"d","key_5":"d", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114607, "value1":0.8236051117026364, "value2":4748182480100358673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.628228, "value1":0.168082038026373, "value2":7102000252406916356, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.687012, "value1":0.3638576608952103, "value2":8540460395700338240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_9":"c","key_2":"k","key_5":"b", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169601, "value1":0.3111748596945155, "value2":8571165720141489367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498080, "value1":0.6187426468242411, "value2":2451438877433776073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866907, "value1":0.5131483959448067, "value2":8869138925582259535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.396171, "value1":0.28292656018705137, "value2":5391262893377663378, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114785, "value1":0.0649996585433473, "value2":3978180949228199287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671381, "value1":0.13833508369091968, "value2":5430930553797657293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.923816, "value1":0.8519197270200854, "value2":9127083562371495095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786218, "value1":0.02969493094352536, "value2":7671441472993176714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774561, "value1":0.2102387127527105, "value2":4039185760762390304, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192344, "value1":0.144819186927003, "value2":378491549245728191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076506, "value1":0.3187033255519536, "value2":8885020385104236250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213717, "value1":0.8128476179407741, "value2":2181064442818506560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528442, "value1":0.5936064804578587, "value2":5648031374498714859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.767476, "value1":0.1348520099403021, "value2":4074505568425329493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.449677, "value1":0.09144593933086395, "value2":8719212717808997934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411794, "value1":0.8990540517914553, "value2":8624156888058647365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.457637, "value1":0.3197628975782331, "value2":16763285997559821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.330356, "value1":0.8095806602517515, "value2":4384365921475656332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584148, "value1":0.16829289997155406, "value2":380241693738836834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455567, "value1":0.7529538853574959, "value2":1681989614930760275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.844936, "value1":0.6074576588778431, "value2":7722066461223215119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926267, "value1":0.5797545736876416, "value2":7883616781252732331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238789, "value1":0.014736618507623488, "value2":4669225171297242340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_9":"g","key_0":"k", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361896, "value1":0.17876601404118195, "value2":5115665659478830691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756049, "value1":0.04403510631259223, "value2":7660735889331080164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.031957, "value1":0.33256068212920453, "value2":4453878565561794031, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.828157, "value1":0.39237914319665085, "value2":7249969180646160990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952636, "value1":0.372690613847562, "value2":6192025452112268327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991675, "value1":0.5494769946982794, "value2":90791903671198766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240830, "value1":0.9786829569747533, "value2":5978162907079513697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279525, "value1":0.1494973578442592, "value2":6869066553359800738, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_8":"h","key_5":"d", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.939830, "value1":0.37432428722372013, "value2":4226472512491590319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.564559, "value1":0.7159468168647539, "value2":1512961341899043326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_7":"e","key_6":"k", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048658, "value1":0.5401236641884074, "value2":8347872809477134111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.340095, "value1":0.4533223235157631, "value2":1670286383442367211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293295, "value1":0.4258630060523724, "value2":5674260405699571928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.239916, "value1":0.9150796129317534, "value2":8474870548791768473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744309, "value1":0.6470478918019383, "value2":7704476841600519211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458393, "value1":0.05603920506738755, "value2":1942285693271412968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720119, "value1":0.6222144605927992, "value2":5009900948732974588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435194, "value1":0.6528463342414516, "value2":2372394551953155351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.847658, "value1":0.12821161771950726, "value2":1481519210091358924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.083324, "value1":0.3648356544882518, "value2":15076769986125901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_5":"h","key_0":"h","key_1":"i", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696866, "value1":0.5460078372418026, "value2":8482317071463194706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878357, "value1":0.9158893232321995, "value2":4112830082593015136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815751, "value1":0.7832798664333082, "value2":6621546193633794291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.036509, "value1":0.7258315650178256, "value2":2697123926599878172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786542, "value1":0.050969235189444476, "value2":960522431282362262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293909, "value1":0.1637483242537312, "value2":4063809715912746117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_6":"d","key_1":"g","key_2":"e", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960513, "value1":0.15412859037505927, "value2":3609332046279378552, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845902, "value1":0.9308703210994502, "value2":6609460372849170625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750261, "value1":0.12234055832157559, "value2":8912704681070762337, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458834, "value1":0.9318510467298435, "value2":8026158416345347023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859979, "value1":0.9301579137721765, "value2":7925354941740095610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.783146, "value1":0.4471773356574157, "value2":8481603802647763313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.505375, "value1":0.5948200403445978, "value2":52874652951483269, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005495, "value1":0.829951436170396, "value2":3721373420722024974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867411, "value1":0.6049905245498157, "value2":8221468316724652544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435857, "value1":0.8205378674820365, "value2":4321340481613778781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.618588, "value1":0.6552813364407445, "value2":3804139748589484208, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.027076, "value1":0.23841518035800105, "value2":7013010516589186218, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285111, "value1":0.5613630899205873, "value2":1517169132984045486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117978, "value1":0.3226349150452925, "value2":5930250973192218816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238673, "value1":0.398077529845209, "value2":8495752825324479706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_7":"d","key_8":"j","key_3":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258260, "value1":0.3301633034128501, "value2":3240643823549476632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.373493, "value1":0.3529131970207001, "value2":3035971070657682432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665486, "value1":0.1772430533425256, "value2":5999482327853244380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358796, "value1":0.18816869038874143, "value2":5134838279453664617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140177, "value1":0.8387362327402264, "value2":2310199526324609410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_9":"j","key_0":"a","key_1":"c", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059616, "value1":0.8521328351977854, "value2":8719119695607623072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121795, "value1":0.3605246283964201, "value2":1540950724302287517, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760661, "value1":0.21327358886251288, "value2":5483717677008529333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.019253, "value1":0.7234842919041711, "value2":3995891303198614565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468371, "value1":0.026572272284047523, "value2":2431068728438057213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397658, "value1":0.22598294609730654, "value2":2698499054966120972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755700, "value1":0.09282245685151808, "value2":1822149750384470026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304716, "value1":0.6358956357038871, "value2":1038515895167880998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673410, "value1":0.01619219159654624, "value2":999540262235584439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406798, "value1":0.2801610335950404, "value2":5737646822014765045, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472994, "value1":0.14109660013219333, "value2":5384594772062199779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601871, "value1":0.291815873607527, "value2":1847519318868439297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994725, "value1":0.6730562186913949, "value2":5489090928652625049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205639, "value1":0.42334867153762545, "value2":1823044221508369143, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.904945, "value1":0.598173556849254, "value2":1029832085092338611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507464, "value1":0.8462370553826191, "value2":8938945324991684299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105783, "value1":0.4362851240411875, "value2":8230814679005715011, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.130819, "value1":0.5311370726009594, "value2":2799369947502242194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.436621, "value1":0.5406911040823768, "value2":5927551572729154830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_6":"e","key_0":"b", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081043, "value1":0.9897107711937903, "value2":1409810039307886257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.360124, "value1":0.8898589942584547, "value2":1182666162202869439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473406, "value1":0.9785032169444227, "value2":7933797459692204307, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711755, "value1":0.8440901040738314, "value2":1266347640739805258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836845, "value1":0.13202825619743236, "value2":6494324863003452768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.814171, "value1":0.22371325434459086, "value2":3679612694652027947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707264, "value1":0.43001516107021553, "value2":4022260893191171434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585576, "value1":0.9028218878505339, "value2":3312996140868409539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_8":"h","key_2":"j","key_3":"d", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.253947, "value1":0.5596795145791367, "value2":9053991858276197521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.793355, "value1":0.655432892066219, "value2":2171965578246650022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003349, "value1":0.32642930281521576, "value2":4724329284506985861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532693, "value1":0.47384645211929527, "value2":166053326506373649, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258148, "value1":0.8755464478705562, "value2":1367687629182661630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284672, "value1":0.969983045672162, "value2":7709886208589610791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656648, "value1":0.7178009692517222, "value2":378528392759859903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.138190, "value1":0.17961568288211016, "value2":6086554307721297156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755588, "value1":0.7476161070292083, "value2":3339991721000583202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.160231, "value1":0.4667023272863336, "value2":8520356596039544954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888583, "value1":0.20049555103413333, "value2":2444783554472337568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008015, "value1":0.08008213078983399, "value2":3073434633965527342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094964, "value1":0.7132542439294741, "value2":5312205610465131789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.234374, "value1":0.3178507521158574, "value2":1341935169276159022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772701, "value1":0.9799507521983674, "value2":4679043722393447880, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.742787, "value1":0.8436883610291797, "value2":8206108321797356735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285496, "value1":0.02119049506813719, "value2":5643416354242611531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041734, "value1":0.8593746773495375, "value2":7177072845753511018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102405, "value1":0.258959468834628, "value2":5506364318868451234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.617097, "value1":0.5213134879577571, "value2":7820417886370593685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_9":"c","key_1":"k","key_6":"a", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614285, "value1":0.8586887753126803, "value2":146426955523604107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094475, "value1":0.3100760743003405, "value2":3216906708607385583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400149, "value1":0.40305327199936813, "value2":4063666163388559666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_4":"a","key_8":"g","key_1":"d", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.741707, "value1":0.07472258132971145, "value2":4583642984114208044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_8":"c","key_2":"b","key_7":"b", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289818, "value1":0.21965034622340104, "value2":548484284878385021, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788770, "value1":0.8645491106416102, "value2":6428112142936498186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589911, "value1":0.8352505870700639, "value2":8807067943589232363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980194, "value1":0.4222199616754522, "value2":1431823521353243628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347713, "value1":0.3792858568507692, "value2":5699682291213510899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790574, "value1":0.48263080283098414, "value2":3899441712388650234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.586360, "value1":0.39598813053011617, "value2":8951581838322949271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147821, "value1":0.696249186695191, "value2":5997811343683052027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_9":"g","key_6":"a","key_8":"d", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375267, "value1":0.30594268061015806, "value2":7589869429394133120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.662739, "value1":0.8061361815679017, "value2":5904903629197529499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_4":"h","key_1":"d","key_2":"d", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447051, "value1":0.37750215516426633, "value2":8371518067848222420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121391, "value1":0.6696130432092745, "value2":3528483712948148367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_7":"a","key_2":"h","key_5":"i", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114395, "value1":0.9303974176092045, "value2":539479413974722614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.440024, "value1":0.33141470193064243, "value2":492711817812030768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830103, "value1":0.7998316524208386, "value2":4274003622712511224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.343594, "value1":0.948502518427523, "value2":1092062054441297015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.820370, "value1":0.6468806610248711, "value2":5700638394649707417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169752, "value1":0.427387565910367, "value2":7968698722232869415, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.352398, "value1":0.008797452976395297, "value2":5389560588922529986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.574184, "value1":0.6925729377012323, "value2":7921127567234590177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708589, "value1":0.6874360024316648, "value2":7727336634389454587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.647250, "value1":0.2053256936728798, "value2":8730436625808972720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522110, "value1":0.7058378270168358, "value2":5838333049897839477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894605, "value1":0.3141493596388785, "value2":2076790219296988613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008782, "value1":0.590566630449788, "value2":571655682516576680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.424761, "value1":0.6098530285996386, "value2":7846500926381318806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591196, "value1":0.33824124551827933, "value2":646303391431941929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964204, "value1":0.7366285135213106, "value2":4137944794465811468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643081, "value1":0.3218928504433246, "value2":9121706025453589663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_9":"e","key_0":"b","key_4":"f", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804724, "value1":0.014940637092911, "value2":7451060690570027498, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694428, "value1":0.21460306796333273, "value2":2982813118003123474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_6":"g","key_9":"e","key_5":"b", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.122258, "value1":0.49140759630280006, "value2":8832276444519664682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.996796, "value1":0.2474526024511197, "value2":7613005431630356063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238269, "value1":0.8267663978098108, "value2":2724285849945025892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.644529, "value1":0.4649514742650587, "value2":7589769671176102226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.376035, "value1":0.7135202605627624, "value2":4131507979199286159, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958098, "value1":0.5742451996367589, "value2":7655662522626761261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478186, "value1":0.30259713893200063, "value2":7366343481003162441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_6":"g","key_9":"d","key_3":"a", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299728, "value1":0.5700777203840441, "value2":8077274544410191414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.685643, "value1":0.9403232361639245, "value2":809458262071959965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478225, "value1":0.7234775762369925, "value2":1256131637116314402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.445423, "value1":0.17591193935592764, "value2":6211570042356723652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.362681, "value1":0.09007902579380467, "value2":5590217114587646147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140908, "value1":0.3872422494802931, "value2":3824505998771737796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.312510, "value1":0.26492926145602147, "value2":2580605589364820036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639094, "value1":0.8695708814560201, "value2":828997494663746747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532778, "value1":0.8659642362585516, "value2":489767661854306328, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.332818, "value1":0.5085353492466338, "value2":7131627003266463948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616148, "value1":0.6665482572291218, "value2":9183191141357087143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807708, "value1":0.03703617074385954, "value2":1634911105603876624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.990782, "value1":0.8721197542810023, "value2":829062251335888311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.678575, "value1":0.8518891534609417, "value2":5490128933307846252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676580, "value1":0.032403043603468545, "value2":1860775688299500894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.011776, "value1":0.3310374930985375, "value2":6454291476355278297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.902226, "value1":0.03335506845841715, "value2":4281002708262490682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.884426, "value1":0.26587448707952976, "value2":2888077545939138944, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721322, "value1":0.9352434968753679, "value2":7587503328880398082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674491, "value1":0.13470131213667097, "value2":733064028083204302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.422073, "value1":0.34800250717845166, "value2":5657610491232907815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232687, "value1":0.8974445402000435, "value2":5670871752075567567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.603175, "value1":0.8044440847017621, "value2":8279540241317103114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_5":"b","key_6":"j","key_3":"e", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354995, "value1":0.9370809546532062, "value2":2794379758286031831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598521, "value1":0.03162604483850602, "value2":8898609192002715752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489635, "value1":0.09343989179985779, "value2":6354680197733735029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_5":"k","key_2":"d","key_4":"i", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.766631, "value1":0.6557065025532866, "value2":8867906113412686551, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.110067, "value1":0.37336348255028134, "value2":8359961637534300323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.734771, "value1":0.6467506386274888, "value2":1558727129876488311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751073, "value1":0.5011707406625697, "value2":165276254030404304, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903806, "value1":0.883542811993649, "value2":5713497784481830170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_9":"b","key_3":"a","key_5":"h", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068830, "value1":0.41287620615552584, "value2":110550725395415731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.481998, "value1":0.9176610757654294, "value2":3543653242038441064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.206639, "value1":0.6881985031716263, "value2":3991865736711226608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682333, "value1":0.7354938188464508, "value2":2336942143484925177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.909363, "value1":0.6550859218093187, "value2":7848025848676081478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.439129, "value1":0.4735897904277586, "value2":1730101214885033885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788447, "value1":0.7029400400744759, "value2":7348647096857266019, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656519, "value1":0.8226813316731755, "value2":8565628989265102630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.286018, "value1":0.1715470955708772, "value2":281139728200192436, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.316461, "value1":0.970829440713552, "value2":8266115606666039974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.131938, "value1":0.06029126490427053, "value2":6088969321338942970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936137, "value1":0.16630484422294922, "value2":368627481682565524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.632956, "value1":0.4597748954914025, "value2":614883467360042989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441534, "value1":0.3169134259464135, "value2":4894244187289767341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.536363, "value1":0.21988172452701402, "value2":3848642916965059480, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722069, "value1":0.0035036640993316324, "value2":1784910019753576066, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189179, "value1":0.5999084267877618, "value2":589928855378135734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070147, "value1":0.2816071736469606, "value2":1978576359562742327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.474328, "value1":0.12233813345274026, "value2":7757125810591739471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756802, "value1":0.8734910453077231, "value2":2858305999840238155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830462, "value1":0.7809071944843656, "value2":6070440638222443699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.158081, "value1":0.8007440801809812, "value2":3893225795949780633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_7":"g","key_1":"b","key_3":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266044, "value1":0.5291100849459248, "value2":8570983485516004790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375100, "value1":0.43060898088087474, "value2":5072556087103000232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433713, "value1":0.4065332038272028, "value2":7498601821703546789, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293147, "value1":0.2008523459507916, "value2":3469755278652299519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.464083, "value1":0.7193413705182999, "value2":8360590737420028572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182168, "value1":0.12869784027227613, "value2":2436732966596869973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152849, "value1":0.915235247509805, "value2":6575468662860533653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.106299, "value1":0.6574795299867987, "value2":5839146977664216835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.842382, "value1":0.6684180756051546, "value2":6255233029731307668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.976893, "value1":0.9921834371308854, "value2":286112572755130788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.747197, "value1":0.6687884895336262, "value2":5026368687057544137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227443, "value1":0.8444750407098351, "value2":3238045843783198862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575432, "value1":0.04959245001196866, "value2":4763760968874648742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484574, "value1":0.8794886478305205, "value2":2929187105321944273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.613332, "value1":0.4649488124035846, "value2":8038984692264433406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.156389, "value1":0.013056913554735015, "value2":1073629030912865258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614657, "value1":0.21062410476220902, "value2":6415897327379032774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238729, "value1":0.10820884210252545, "value2":9083864450943421396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241678, "value1":0.22605677165183255, "value2":191747579342742576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958794, "value1":0.08847610578305673, "value2":7897538472323716766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335228, "value1":0.7181578715519717, "value2":7450823110233453413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408629, "value1":0.007049664704797943, "value2":1505104843936344580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558681, "value1":0.17466226992103584, "value2":4472498115626926601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682989, "value1":0.9645272985758268, "value2":1982644188431990793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.179200, "value1":0.6439691808346755, "value2":8186363896835482044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.823991, "value1":0.6478071089364331, "value2":8599197823923500800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109730, "value1":0.43770117688691335, "value2":8534783631376673343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.659077, "value1":0.2388079100822681, "value2":5323293096195049116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.120405, "value1":0.7373368673794832, "value2":8559881035375574344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.355361, "value1":0.6166673498020887, "value2":5456145421934358189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.824086, "value1":0.4534358999465994, "value2":6384310233648456918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871013, "value1":0.5576949556795737, "value2":1428733241052481930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507168, "value1":0.14733425604177453, "value2":4932780474472793744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.399272, "value1":0.7455596536595573, "value2":7959578558660693956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015542, "value1":0.3661445195957949, "value2":5873286557095287645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.103786, "value1":0.04914833432556202, "value2":3473465672536909160, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201038, "value1":0.09243610446589845, "value2":317387159009776683, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637510, "value1":0.05764711949860248, "value2":683074398669967457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000515, "value1":0.27555419865710506, "value2":1427932259617448392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_6":"d","key_4":"e", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455871, "value1":0.4421289415542984, "value2":2179707427691179241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.816179, "value1":0.0315328103269199, "value2":8955771787657622788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.012467, "value1":0.2583652156574764, "value2":3152397034013397022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.323088, "value1":0.6137213421267609, "value2":6194154265710187463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361691, "value1":0.2728847707358428, "value2":1410878768807423021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507208, "value1":0.40681095070477, "value2":2031127740762892841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.943788, "value1":0.4688930142953297, "value2":4135242725922960343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528649, "value1":0.6786255340569864, "value2":5287486612492903502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_3":"f","key_5":"a","key_0":"f", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225718, "value1":0.4601298705932887, "value2":9052049426787719860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.308880, "value1":0.4295083103049244, "value2":5651329885903965921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759618, "value1":0.16482488772851353, "value2":7857083585674436753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416052, "value1":0.9111867810311565, "value2":6723934981662609251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.962464, "value1":0.4935260151955853, "value2":3632302155175562257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.771117, "value1":0.5157133736913164, "value2":2578240794064018102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690480, "value1":0.6806227775589547, "value2":478506980366332127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_3":"h","key_0":"e", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.819739, "value1":0.8780417506767497, "value2":5796860147973283852, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.393173, "value1":0.42102685254727695, "value2":655129410007603946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.648983, "value1":0.7792161980083644, "value2":3901696553081135725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746096, "value1":0.4415965960336573, "value2":5128962497088709971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550611, "value1":0.9164668411003939, "value2":8252426555782396131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.026169, "value1":0.7150672668373745, "value2":4764749266652855333, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.684226, "value1":0.7136962442214909, "value2":5607090679190953591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382560, "value1":0.4327657862928645, "value2":2199828856833521565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_7":"c","key_2":"b","key_3":"i", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.467808, "value1":0.22363679307659168, "value2":2768232122997302049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575206, "value1":0.7602213610975361, "value2":2779299954374668017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591751, "value1":0.6270567668817301, "value2":8638942281069975034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374899, "value1":0.3628294711980554, "value2":2256958770162190672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490274, "value1":0.7562635891941201, "value2":828105904736409259, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.170118, "value1":0.5429488039839387, "value2":2772778506127755922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240769, "value1":0.2820088361870865, "value2":5639438631779439142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.052386, "value1":0.43149632078191064, "value2":1748438881846630926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507233, "value1":0.961520455982136, "value2":538372936759435184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_4":"e","key_3":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705937, "value1":0.5267821110187133, "value2":2615203611933488638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750925, "value1":0.12125960803758137, "value2":9007375284837027095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631618, "value1":0.07157188871273036, "value2":8155465841105503917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695843, "value1":0.7797612580730544, "value2":174508687346238960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269295, "value1":0.7420818036952436, "value2":4380281606468315484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566418, "value1":0.31419999154507045, "value2":1448626920753535043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.125439, "value1":0.23644172282231143, "value2":6454497651916669000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260012, "value1":0.7675706994384546, "value2":5131372429329822614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674484, "value1":0.024443192261290648, "value2":1060802392645575205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870912, "value1":0.40232027987742836, "value2":7398514903536659789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.193840, "value1":0.5573502520764032, "value2":2891807756570316806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496711, "value1":0.0561679292808207, "value2":7275940738251409744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007546, "value1":0.4548616529539407, "value2":3391843142730199327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532516, "value1":0.5229547783776293, "value2":306719517858726688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.879419, "value1":0.738955824150393, "value2":4551796041817468625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.037090, "value1":0.16862399179431758, "value2":4308060124699455866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.748879, "value1":0.537219693168335, "value2":464744057316162498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.610036, "value1":0.35423811263377813, "value2":8058264218672212470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.212290, "value1":0.8726353918052155, "value2":7894596078524999317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.876606, "value1":0.6546517022010405, "value2":6758959777685985912, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140504, "value1":0.347953522010423, "value2":6223000072814932347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.829533, "value1":0.7297577355885363, "value2":7098834979468843426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770147, "value1":0.5158958137849098, "value2":7721786354129545110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960566, "value1":0.08984803039947419, "value2":7183642332674708296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.002511, "value1":0.7991212300299283, "value2":1907144646133978540, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.969474, "value1":0.8490958586264525, "value2":2475634499457331111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757044, "value1":0.08525717165380897, "value2":6126821447855017519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826913, "value1":0.3832744407117778, "value2":7549472912902563562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713539, "value1":0.040929240199395034, "value2":8725477692408636856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.235038, "value1":0.21655074702892513, "value2":6340750264489982261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713507, "value1":0.09698561541865447, "value2":6976532341223849395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926365, "value1":0.6869464705551463, "value2":1770406309916093131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_8":"h","key_0":"a", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.159778, "value1":0.37025745483297273, "value2":1667176166944781722, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964175, "value1":0.3460815492793407, "value2":8584884445165219279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.789212, "value1":0.5752530695794612, "value2":7518543931942002634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946193, "value1":0.9389444654094391, "value2":4798866777286425255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183973, "value1":0.42689788996190486, "value2":2707564478214913458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381159, "value1":0.19968138225970908, "value2":8498929039632010906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.535645, "value1":0.9768502318817727, "value2":6273017261118239502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957340, "value1":0.1584492922790321, "value2":1071530234423723880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622320, "value1":0.6014449518439046, "value2":5905866866815287771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.839825, "value1":0.6412078194818629, "value2":8581664149275929445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.925054, "value1":0.6499072539736884, "value2":3213784800648371267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.654382, "value1":0.9107398509050536, "value2":6290871463312648713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.795697, "value1":0.8789558932554133, "value2":9118966352973366626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299533, "value1":0.18449853142128472, "value2":6924501956061845944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993498, "value1":0.10358257618685637, "value2":949727120096577179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192327, "value1":0.14944601875044164, "value2":3650903859653754777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115925, "value1":0.7654222794387283, "value2":1807303188151583637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290372, "value1":0.022817260035797225, "value2":3306384333491756709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.402670, "value1":0.5073298240798392, "value2":4756233885952638662, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682572, "value1":0.44581853587828163, "value2":8977717914108281095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.176704, "value1":0.0686762673056191, "value2":418784156784186861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676421, "value1":0.9540024144812602, "value2":5040838132393700704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851525, "value1":0.4501599756607321, "value2":7291682933774791859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.593343, "value1":0.506206698395899, "value2":622013338659516904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454600, "value1":0.4498330838292854, "value2":2669650474822317581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527805, "value1":0.10877922841749062, "value2":2131131607855096268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.196472, "value1":0.04264644660597854, "value2":2656288085872651609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743799, "value1":0.2248320170707705, "value2":6330060840606217610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137446, "value1":0.5799142292246671, "value2":4848799016740596142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.405919, "value1":0.3378516880177889, "value2":3532844197361780130, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.085405, "value1":0.11768848717895854, "value2":3011043909687794254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494421, "value1":0.45496539093555294, "value2":1968423041425508639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_5":"f","key_0":"b","key_4":"h", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.846925, "value1":0.23911281903346318, "value2":1229857216789047456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.320172, "value1":0.7911586278138405, "value2":145982991700455098, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688187, "value1":0.9272721413003945, "value2":5752958588041193324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.069720, "value1":0.006222818414235851, "value2":7251719396221186260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.974434, "value1":0.7327419748667513, "value2":8892941322690231954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.891519, "value1":0.2559114110516683, "value2":8453661615833220464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596204, "value1":0.8329616164402396, "value2":6548226786173702319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539908, "value1":0.7401840588881007, "value2":2823865774842747097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865375, "value1":0.23950780446408748, "value2":6387790804623201730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121268, "value1":0.4818898421020435, "value2":5213834205313537165, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865802, "value1":0.3275415536098801, "value2":2802490407634150279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105641, "value1":0.6210077776605651, "value2":7224328772436945002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592291, "value1":0.821033330430236, "value2":3130486219263774432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497809, "value1":0.5925199380020667, "value2":341686057277963037, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097734, "value1":0.7071847698151175, "value2":3681912566029517435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.763574, "value1":0.5357729458141934, "value2":4576393986863014129, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626276, "value1":0.4467400550839313, "value2":8832239215373133572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626952, "value1":0.8566744061055088, "value2":2681684884383601627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980666, "value1":0.620719669726131, "value2":3866315867973394400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.970686, "value1":0.492660470887635, "value2":8584579737939983838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205981, "value1":0.41980273151462594, "value2":1272721076115788334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643071, "value1":0.9850402079127909, "value2":6866475251892359276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.520241, "value1":0.021531538514302357, "value2":3680419636166133733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689781, "value1":0.7246711061095096, "value2":9156928465977729149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936886, "value1":0.30593890582238353, "value2":8650088989944818585, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064882, "value1":0.9419400037260646, "value2":5139525288533697167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255204, "value1":0.6562362511494719, "value2":5377088753944330401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147096, "value1":0.12576537131594923, "value2":696189011986700613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.199366, "value1":0.8703136109731814, "value2":8758678284656449473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782148, "value1":0.21588917591543605, "value2":7157001910263594492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053985, "value1":0.9864976225072658, "value2":3999466034196061068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.409718, "value1":0.4052988292496616, "value2":7239010836693528474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918878, "value1":0.9460279119002769, "value2":8629146616157881411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744254, "value1":0.12263478198823631, "value2":8698623269262602240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_9":"i","key_4":"i","key_8":"h", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701868, "value1":0.3178667417931335, "value2":8069435536591676411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_6":"k","key_7":"h","key_5":"b", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.794445, "value1":0.7546856348043545, "value2":1056257035145843208, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.318803, "value1":0.8976234734615749, "value2":7848445975050376420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.146350, "value1":0.31748244669278847, "value2":2926268572806049868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392349, "value1":0.655725592207486, "value2":4190046781739044777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.749690, "value1":0.6239130127907853, "value2":5206683397328774242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744300, "value1":0.6764999956977692, "value2":1477381647586507515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.758303, "value1":0.5258481908865111, "value2":5235026862755190183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379762, "value1":0.890935177217856, "value2":6322796509408703364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592912, "value1":0.6886535422027805, "value2":3968885283796366911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859203, "value1":0.3806206868492738, "value2":6588306706789304138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.565175, "value1":0.823280011831958, "value2":1840222942885696142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.095858, "value1":0.18658533157232132, "value2":3040729957448861027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.567863, "value1":0.1544724870114228, "value2":5739061219420103119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739840, "value1":0.5915071280708923, "value2":6986464734573649185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.200755, "value1":0.4500618730121175, "value2":4400580498849630870, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358028, "value1":0.8986628799372285, "value2":3320447208954210899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017181, "value1":0.4972731292168526, "value2":5687205385943047330, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229539, "value1":0.5257635254456374, "value2":1562089048942850119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.171044, "value1":0.8736127789799626, "value2":7151271249368522054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.384626, "value1":0.2370246172687121, "value2":2826036906774872004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522883, "value1":0.6013577976454938, "value2":890649018657102134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.877768, "value1":0.6077458920362099, "value2":8966895480129060070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.224033, "value1":0.0076863111785604905, "value2":6569047840639595510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.311682, "value1":0.07593318166504835, "value2":3253540135578122766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490459, "value1":0.4609451528285016, "value2":2994036544656680091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.378898, "value1":0.9545401129487334, "value2":3668542248171100045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416785, "value1":0.35541532907488144, "value2":94039845626229270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275554, "value1":0.421200916650073, "value2":5818521501894570082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619269, "value1":0.30772799684273167, "value2":4876402407267868155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.702725, "value1":0.15927682148224467, "value2":1903456341847904299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232604, "value1":0.7711772747260455, "value2":3866638254479737513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.217772, "value1":0.8540182257688077, "value2":7309608218029816756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705505, "value1":0.48163362162995105, "value2":1550283929814058088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900193, "value1":0.23760307179291948, "value2":8646186763820142769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.208626, "value1":0.38931880434870053, "value2":5483530341831301025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.568347, "value1":0.2801336860166125, "value2":9164397534745927432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117458, "value1":0.981405746517363, "value2":6802896340427379587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963035, "value1":0.7214166769862517, "value2":8440456704015434339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700137, "value1":0.5934611351717767, "value2":6333818613628602547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145303, "value1":0.6922047329264945, "value2":3009832451157760657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345925, "value1":0.7232346453151443, "value2":4791014890475627712, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619327, "value1":0.9932237622196707, "value2":3624811523135333449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269730, "value1":0.7866373764087783, "value2":3746435711914598124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.265782, "value1":0.05991121042342668, "value2":1951849832097700576, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.175543, "value1":0.5444637318639066, "value2":5220283889625062347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_8":"h","key_5":"i", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677825, "value1":0.904620506580413, "value2":4669605336529676512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668600, "value1":0.17366696171797397, "value2":8249554071220606769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041222, "value1":0.3254693615668756, "value2":5262353711290630554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.243762, "value1":0.8759302538831941, "value2":7311945236881805486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958625, "value1":0.5492731854343449, "value2":5435276136514133338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290213, "value1":0.7949670770789082, "value2":7111766404096808279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025988, "value1":0.1026128579407924, "value2":1755616401770874869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595099, "value1":0.18775955592545962, "value2":1656134813479826833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.671098, "value1":0.8079739199189518, "value2":3665225572712988915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.542084, "value1":0.5181312401815146, "value2":1481804189008155276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.784956, "value1":0.40357178109169595, "value2":9035114082415618141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.276517, "value1":0.24301724893563487, "value2":4602891919019880467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064974, "value1":0.37706581069603073, "value2":7053383221578853097, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698510, "value1":0.45184858839742487, "value2":2258520036203869092, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.218133, "value1":0.3113727295782559, "value2":6841422843138500205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668881, "value1":0.25909755219748914, "value2":1185172632185740020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.049479, "value1":0.9359900704241361, "value2":8443878057371326705, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.674394, "value1":0.5330986303950987, "value2":4597086230952392696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195073, "value1":0.23095513043718424, "value2":886484372856556987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_6":"a","key_1":"e","key_5":"c", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.380352, "value1":0.9222632147295795, "value2":2011984636052581606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.302156, "value1":0.7967664965647748, "value2":1936857391665696956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673174, "value1":0.8085420465867998, "value2":6974688906284239573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.013224, "value1":0.01982064999844403, "value2":3955016096628456095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090855, "value1":0.01304461484505885, "value2":8778318340262765164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.795784, "value1":0.12217080131703652, "value2":2014947645475742270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284442, "value1":0.42738762978686595, "value2":2584606560267930457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.741760, "value1":0.5561052082820417, "value2":7263332550390986826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035524, "value1":0.908173032025188, "value2":5685807943763240264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025764, "value1":0.20925710637592332, "value2":6562668387404626293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.530188, "value1":0.6447955243050295, "value2":3888614173619999623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424157, "value1":0.17487330381635957, "value2":5301061833157028268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026934, "value1":0.0422857387218024, "value2":8303721108125129087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841847, "value1":0.5206472086096076, "value2":74348635869177909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.414469, "value1":0.4278094238360654, "value2":7010327929309184782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491788, "value1":0.747156270778759, "value2":4790193170750716406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399141, "value1":0.47388249427808166, "value2":8836306836420904183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.160250, "value1":0.29496035919361546, "value2":1937679940636334558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787511, "value1":0.40198955034703965, "value2":5313833463463440565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684988, "value1":0.6177512623201419, "value2":1449869359073755283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803253, "value1":0.8970896940692594, "value2":6178067064755804966, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964550, "value1":0.8199555931693112, "value2":140777809986023618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456407, "value1":0.2401625723920763, "value2":4030355648738753679, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849585, "value1":0.2629587147468046, "value2":7076050191155576404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.824331, "value1":0.6892589926802947, "value2":6399755277819070915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_4":"e","key_6":"h","key_2":"k", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388915, "value1":0.8127879446503501, "value2":5588278675440700300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782971, "value1":0.22771808116681516, "value2":6261899644042013732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712419, "value1":0.2905286238822736, "value2":5614218841386062674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_3":"h","key_9":"j","key_1":"d", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249696, "value1":0.9445558673553629, "value2":1404486883141712140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601814, "value1":0.6990549633969345, "value2":7198014296771075924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543813, "value1":0.07857387425059083, "value2":1217712980435541695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123893, "value1":0.07983470162468347, "value2":7171526612504871775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_4":"a","key_7":"k","key_1":"f", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703648, "value1":0.36447696895349185, "value2":5146632909527815894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166353, "value1":0.7098874569397392, "value2":5324193865568739754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399594, "value1":0.7745072356158436, "value2":7831862313374173936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.439724, "value1":0.2037239485962775, "value2":3755069786706852308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839998, "value1":0.6732355725193216, "value2":2639271267504971364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228435, "value1":0.897840654678523, "value2":5116662644663764475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973287, "value1":0.9461886626837847, "value2":5279391211817478931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.441360, "value1":0.9419535299333034, "value2":2098864241711696584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.780978, "value1":0.7212145622817746, "value2":8825784157879516761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.342306, "value1":0.40763808526506856, "value2":615440271373894957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964992, "value1":0.8428995412660205, "value2":5458378648341607708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386901, "value1":0.3242872455426699, "value2":1659876068547712992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.960102, "value1":0.2464238639205716, "value2":699750419933475491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585099, "value1":0.11986390379255613, "value2":3678288424535052375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.154037, "value1":0.7785920099800233, "value2":8275166075671164173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211892, "value1":0.7853941226491261, "value2":6369532531235861115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.746677, "value1":0.6664817644749024, "value2":4658328841108884689, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.941862, "value1":0.846994201539624, "value2":1274645788690532949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.733827, "value1":0.13169457879746155, "value2":3221287182313964764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870127, "value1":0.37466507316122794, "value2":8856106508914542017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798298, "value1":0.22462880312469063, "value2":6798911070080579970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993366, "value1":0.9961738964349245, "value2":1193549068611970903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.157412, "value1":0.4452570786392375, "value2":3487651503828456956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.529021, "value1":0.26905395113611574, "value2":2025155869621977272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.701922, "value1":0.8730175680632801, "value2":997347931310956157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_1":"a","key_9":"h","key_0":"b", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234577, "value1":0.7588051800901922, "value2":5319874977525060911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943591, "value1":0.47603601658101, "value2":5946410545801626105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.292747, "value1":0.03173464098642328, "value2":7226691638812613138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_3":"b","key_5":"j","key_2":"a", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608463, "value1":0.2957253043607161, "value2":122362680185330457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.643245, "value1":0.7167827619023723, "value2":5186237060875400709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456051, "value1":0.2188755629521717, "value2":1845588654842798990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231176, "value1":0.6418253580281563, "value2":3690123298454825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_4":"d","key_5":"d","key_3":"f", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158253, "value1":0.7945876103242618, "value2":9004322525035166688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234079, "value1":0.5202371436378874, "value2":1591535835129382659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.875339, "value1":0.37493709867362873, "value2":8328159676060260451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907034, "value1":0.20583691616211497, "value2":3631192064418800289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.518876, "value1":0.16335464977880865, "value2":4643059165781979124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185787, "value1":0.018176867671764584, "value2":3270163687436586406, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308787, "value1":0.4773555846410255, "value2":3253306657765008963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297154, "value1":0.246942999840671, "value2":6872861714776356785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_3":"k","key_9":"j","key_1":"c", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.817582, "value1":0.8165235911801363, "value2":8310469403306376604, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493041, "value1":0.38500411997916434, "value2":6747567659837060435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224106, "value1":0.9994238326374754, "value2":7099587552820760691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031476, "value1":0.09066884358892856, "value2":5945790442760569800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311356, "value1":0.5806991883130863, "value2":6895176067401972683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653725, "value1":0.15503072276739358, "value2":7449412216901454949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.598231, "value1":0.20373471056711764, "value2":1437658094331870355, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798108, "value1":0.6097451467316525, "value2":7133130757829716058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709054, "value1":0.7371923755130838, "value2":7543405219351917468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104790, "value1":0.21083306139991156, "value2":7929604552171608387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.187877, "value1":0.5608708053678212, "value2":3368623446076987813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749130, "value1":0.657437311445075, "value2":8949805346712651308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582481, "value1":0.42362389624895797, "value2":8717288037606444575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.558438, "value1":0.370755803762819, "value2":1107999166234163628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802675, "value1":0.8677299277452752, "value2":5915000168574046056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_3":"j","key_0":"j","key_1":"c", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.208039, "value1":0.38781051864576976, "value2":3375198001539598542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680278, "value1":0.886048145499295, "value2":4081527324669320081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162287, "value1":0.7118801419064047, "value2":2390093012206667849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.092015, "value1":0.9942758150465311, "value2":4638994738698984564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039345, "value1":0.641165676244722, "value2":2976540841974698815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.408130, "value1":0.5042712547968512, "value2":8947846924433036861, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259088, "value1":0.514500988249709, "value2":2770449998520564629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_4":"b","key_0":"j","key_3":"i", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293773, "value1":0.36521689212647834, "value2":67527608081891502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646136, "value1":0.04048325982754749, "value2":4743191660759249600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.047497, "value1":0.5502497475964416, "value2":7169106217640232030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876800, "value1":0.3799501707452096, "value2":6757458314368411993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968057, "value1":0.7499783045753542, "value2":1200059614173097893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.871681, "value1":0.6987940576652765, "value2":3787158661733344974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.998419, "value1":0.7974184353878208, "value2":3011281681090708365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663978, "value1":0.1875336848417326, "value2":7682956450262415743, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799777, "value1":0.13431952299825461, "value2":8472008336913527353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224565, "value1":0.7432835827699946, "value2":5363247813194187485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896272, "value1":0.1238866666650729, "value2":1249324477450532329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630720, "value1":0.15283582580094462, "value2":1819923308050662735, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405798, "value1":0.9413697480070442, "value2":6170928651103063980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249174, "value1":0.3594131764729996, "value2":6196624819261733193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628627, "value1":0.5009434579166352, "value2":4913727766417338788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_6":"d","key_9":"g","key_2":"e", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949656, "value1":0.677579274449204, "value2":4321897610912596942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.452302, "value1":0.8465391778403211, "value2":8767695810058898547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224741, "value1":0.9419255467202373, "value2":3942608565088456699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832311, "value1":0.1844332411242929, "value2":7897621420940762914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364450, "value1":0.6062623207548122, "value2":3048338263269528616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458762, "value1":0.8964243000562225, "value2":6980106557138925502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090342, "value1":0.9353992150482529, "value2":3024974544306240484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680282, "value1":0.32391766175077186, "value2":855225884463637074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949004, "value1":0.8992840018801868, "value2":5343754317476027418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.564072, "value1":0.04591172153502947, "value2":15119713343690272, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.015169, "value1":0.2775250663576938, "value2":8329492228577328961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035102, "value1":0.8955268703368635, "value2":1413978015063203049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.922310, "value1":0.5335365949387284, "value2":7752415045685811238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.862402, "value1":0.328052481246616, "value2":1049471938369225232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.632372, "value1":0.702701110691373, "value2":736041481181376478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886888, "value1":0.8185435772395079, "value2":6423113939133946645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087072, "value1":0.4700081672384728, "value2":4642202730229975792, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.936650, "value1":0.18762496890040795, "value2":3355401686890613388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032044, "value1":0.12307064568471555, "value2":5991756874915773532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093295, "value1":0.22401921911686248, "value2":6120641321792430058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.777098, "value1":0.29744828015347885, "value2":3084065102603938152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.437450, "value1":0.2884592585353877, "value2":4044254904136084327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.017135, "value1":0.7608822818277434, "value2":1205572766308158879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_7":"a","key_4":"g","key_6":"d", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907629, "value1":0.7081298253215954, "value2":7688268403734929948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924934, "value1":0.7192898599410195, "value2":1421354730652485196, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.736050, "value1":0.7502596301905261, "value2":2582210329738662673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515235, "value1":0.09310016531489594, "value2":5123514648664672005, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.117213, "value1":0.07556633746315061, "value2":3501827326756011617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.485929, "value1":0.9272991096008794, "value2":5159915940333201308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296263, "value1":0.11108812397005534, "value2":8753717861080819638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990089, "value1":0.527982415726353, "value2":5638704340740792965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835455, "value1":0.062168150269998286, "value2":1944114908616022523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297201, "value1":0.30562887567201646, "value2":2937626694886920620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974272, "value1":0.7221303752638616, "value2":4841762343070531864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.477606, "value1":0.011147194457821816, "value2":7938710785429203576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_9":"h","key_0":"d","key_2":"k", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129918, "value1":0.5408385148641213, "value2":9195358942783956586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364995, "value1":0.0773648729840119, "value2":2987992944358913043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.776833, "value1":0.8273262149027085, "value2":3110126394788152042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.610517, "value1":0.6781728258681893, "value2":4793737503812454856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896976, "value1":0.37188254064396126, "value2":3144974162911084078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202562, "value1":0.5271477787382647, "value2":6411589275983706377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906770, "value1":0.4476129119528191, "value2":424159247916997254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_9":"j","key_0":"e","key_3":"c", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289456, "value1":0.541155755467242, "value2":2999265325007368522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.860878, "value1":0.4758705080993404, "value2":6690006483968082589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.365448, "value1":0.027420735129761006, "value2":825754437394215422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212551, "value1":0.16839070098231337, "value2":1942763578810483030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870900, "value1":0.4994077584006896, "value2":5993483889801645858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.455124, "value1":0.26138016691309135, "value2":4427691071851307843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.569012, "value1":0.4778238812129043, "value2":8556858598621845387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.362641, "value1":0.2203054974232932, "value2":3477406531290022646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493850, "value1":0.4787384781915333, "value2":2198408185614770688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.513028, "value1":0.6429388402116987, "value2":7593443154020639176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386729, "value1":0.17475131034566962, "value2":3647348637356138676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053464, "value1":0.6091636844532319, "value2":5763485176228727883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628348, "value1":0.023668653461306262, "value2":6178270391876561387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913114, "value1":0.637992847841559, "value2":8248068010165238913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.413158, "value1":0.5132505144116425, "value2":3488568643829613122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517474, "value1":0.21449952384417484, "value2":4829637720497890669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_8":"d","key_4":"i","key_7":"c", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.659125, "value1":0.35348852774479717, "value2":929980299829000703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071642, "value1":0.5598789974197187, "value2":7463399784012985552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993416, "value1":0.6475276350489354, "value2":3252439000616730752, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003658, "value1":0.897666564581801, "value2":508174253835823210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.629566, "value1":0.49532352920736916, "value2":3384274756820723223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.084906, "value1":0.40471552937588173, "value2":2358566109120320700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.108543, "value1":0.8814759719911462, "value2":940297244819349653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138889, "value1":0.48057446422385713, "value2":5651682080133821762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.468024, "value1":0.6607643026338559, "value2":3567273041505939519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.278010, "value1":0.05185404011612328, "value2":9174087231001182435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.095328, "value1":0.4998002617147629, "value2":4028395944074542796, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.962889, "value1":0.7966578010574328, "value2":907724426763786137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.434695, "value1":0.596944451662117, "value2":6599947217349296930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.695515, "value1":0.23670726317490795, "value2":2923186045419354751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830693, "value1":0.43683284863611044, "value2":9110976313628947704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311809, "value1":0.18924644468054935, "value2":2283072307874475430, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405030, "value1":0.5026514010742291, "value2":7499615277819756210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.985393, "value1":0.2214787811403981, "value2":1167173892132804942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123017, "value1":0.24050302556695696, "value2":3512829503479749438, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.275825, "value1":0.8989518053152799, "value2":7332940306078599750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_3":"g","key_0":"k","key_1":"e", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.935513, "value1":0.422470893258827, "value2":1965139573445622612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_4":"e","key_2":"a", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697239, "value1":0.9715985007012725, "value2":2441260821113474270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297829, "value1":0.4121340516781959, "value2":7327020552945137669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.488215, "value1":0.22392461646214185, "value2":5911938621135820572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.206243, "value1":0.020325317058324376, "value2":9207086370751028426, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183572, "value1":0.23206347801714602, "value2":6792315880866236659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524533, "value1":0.9124779133103079, "value2":3221463708308920869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.893528, "value1":0.4848003839426891, "value2":3814492558819679029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.752371, "value1":0.8170507918067189, "value2":6527155317426998401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916551, "value1":0.342137272729241, "value2":1602041662641372111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.037592, "value1":0.6500639685761466, "value2":3567065955735150085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902253, "value1":0.7531115895009749, "value2":2543364566980292046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683579, "value1":0.30275430657154007, "value2":7640650810082868472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956614, "value1":0.5403322951696091, "value2":332198111972672347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202908, "value1":0.3832878669702297, "value2":880756788127038920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.820586, "value1":0.34286605495032285, "value2":6526500882733787563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930022, "value1":0.3539159969493719, "value2":370073818816792799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023457, "value1":0.047546161290410185, "value2":5398553854250083599, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303770, "value1":0.7186586727923895, "value2":9169471131017267259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.756889, "value1":0.31103630013784395, "value2":3594098461981248414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_8":"a","key_2":"k","key_7":"j", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212371, "value1":0.3882294279634762, "value2":2428299028227660876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.472113, "value1":0.19389271171622363, "value2":7737936344522555466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051213, "value1":0.12920469147183916, "value2":7837057207029745863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907093, "value1":0.017956122904157993, "value2":4878755396476119431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_4":"e","key_7":"a","key_2":"g", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.525258, "value1":0.49782305829726053, "value2":2466466392623808778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913492, "value1":0.4704351769152427, "value2":7908561685667216260, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290702, "value1":0.4198914182070395, "value2":6819583912532498504, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125217, "value1":0.04410603241691789, "value2":6832161683827996201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004937, "value1":0.13145801093930243, "value2":5314457688060891725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.656896, "value1":0.41192120722520353, "value2":5457893604153821743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_5":"k","key_7":"e","key_4":"a", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.700497, "value1":0.8047394500892703, "value2":8180560267702238361, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553716, "value1":0.7484558334038893, "value2":1003966921986785454, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966183, "value1":0.8167158291471999, "value2":2250434045285686743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663750, "value1":0.7127353578019723, "value2":3741760060639503306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686363, "value1":0.9712701187599743, "value2":3622116724705792870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.103723, "value1":0.4724493721726449, "value2":8181583911299132150, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.484649, "value1":0.18448759636607337, "value2":4053994543260017135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.925440, "value1":0.06382834538328273, "value2":6916231632621539461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939805, "value1":0.22424399111258897, "value2":4351472309231431136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792665, "value1":0.33436392964606154, "value2":2696421529224538356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054274, "value1":0.9272819232934056, "value2":7264556561094806515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.983823, "value1":0.7989127551970725, "value2":5239695063979297186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714766, "value1":0.6892792624795272, "value2":270252571976728726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.263106, "value1":0.64122047400478, "value2":3421795383581628735, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720260, "value1":0.8087795736115097, "value2":8426217720617766298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839070, "value1":0.5917446619321536, "value2":390571547986932413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966990, "value1":0.2432955344040808, "value2":8461111581798764026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.808072, "value1":0.3216500857620197, "value2":4440941489040196963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091958, "value1":0.3995806295500445, "value2":1180768478663237716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139366, "value1":0.11507703668400583, "value2":4516311843825616388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.251512, "value1":0.7632852678548858, "value2":6418768969046377858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963533, "value1":0.8213013619231073, "value2":6101164365690068241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.466433, "value1":0.28350424710105476, "value2":5563469458031181794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831271, "value1":0.8821293582204699, "value2":3430349356503249989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003553, "value1":0.7323647142880576, "value2":5739925461894436779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.241144, "value1":0.3850622637508932, "value2":6137203197960360313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.872100, "value1":0.3184658084196574, "value2":2061373587207508318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.900532, "value1":0.2888524955130242, "value2":6600516250776964704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_9":"c","key_3":"g","key_4":"b", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142058, "value1":0.7572218923237818, "value2":5437617110907280837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.167933, "value1":0.20087813542991115, "value2":8497531403193171323, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977657, "value1":0.49027491644969856, "value2":6846007575513730281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.337497, "value1":0.7535889949814326, "value2":4191915280375173080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091305, "value1":0.41827531366800946, "value2":3553824659208645760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.260475, "value1":0.39071909544867817, "value2":3828966522631037754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421117, "value1":0.6070501451115596, "value2":637677502764251310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_6":"d","key_3":"a","key_5":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949074, "value1":0.8719785824911194, "value2":739855289497042435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005847, "value1":0.564214486086032, "value2":3879428464931256295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608985, "value1":0.8120552257475274, "value2":6222621751128276018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.505971, "value1":0.44491083173920337, "value2":2966233843840872053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_6":"b","key_1":"d","key_2":"f", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686524, "value1":0.910573536000882, "value2":7259657495340557823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386181, "value1":0.5690887314353372, "value2":7696797346941049468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004437, "value1":0.2837421940276855, "value2":3630048754825451133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197835, "value1":0.3478392366591849, "value2":8924518323717745322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404754, "value1":0.14818758453959469, "value2":3841840031846521039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.847397, "value1":0.10611042816524652, "value2":3749767974486199531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773493, "value1":0.16710420925739453, "value2":2156267597938791257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253265, "value1":0.39770730374803015, "value2":8078016986748379936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720298, "value1":0.3023956927195079, "value2":2183933305568131133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.306860, "value1":0.13870609943567333, "value2":5783110454957795198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189190, "value1":0.3825441585131095, "value2":2493897460883724129, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842956, "value1":0.712749117246484, "value2":1337306979661530320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045437, "value1":0.9278250040101162, "value2":6116936767056337825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.606083, "value1":0.4732072236705937, "value2":6967738612762397759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094844, "value1":0.9269374913746184, "value2":2591132307871233919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369784, "value1":0.09045009486516034, "value2":7834710052863465672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032637, "value1":0.16606039490952637, "value2":2737666598235173614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.394163, "value1":0.47507719343340277, "value2":313537922371752138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.987576, "value1":0.5780445815848603, "value2":4108970207058212125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906463, "value1":0.03465970899762972, "value2":6841259554497483309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.266096, "value1":0.6742131298888875, "value2":7025803245311237704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963062, "value1":0.5904971275955457, "value2":2254362569087327315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.130304, "value1":0.868734941900672, "value2":1538504759997860509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.955104, "value1":0.5994289988135004, "value2":3881947199005484386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.480748, "value1":0.49853130606809243, "value2":727938692981473193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.682757, "value1":0.38003013064353247, "value2":8479678523613035623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792753, "value1":0.16576560662385773, "value2":7797109252869584262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.510440, "value1":0.6049726150607012, "value2":8669482832353600257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.638136, "value1":0.15014964585075946, "value2":6225706505785608080, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.522674, "value1":0.3512111717300567, "value2":3641814037368457348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785473, "value1":0.4544503987969124, "value2":4635364210597620767, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_2":"f","key_4":"b","key_1":"f", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110745, "value1":0.5221211963164464, "value2":4903849187407455848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195405, "value1":0.08594736781118724, "value2":2859597321511204836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051298, "value1":0.8256101721258909, "value2":290470632378621656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284220, "value1":0.27211501738111415, "value2":2134420797562982003, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792536, "value1":0.9372077558354763, "value2":7927244557057049645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.681776, "value1":0.4477409740262762, "value2":3995977364357841936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231810, "value1":0.6078487538143919, "value2":3261959160546744959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.131613, "value1":0.7340176818759, "value2":3565658759712151137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.908393, "value1":0.28199967714832963, "value2":3350748733246898915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698846, "value1":0.043919603046489966, "value2":4515802998753688491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803882, "value1":0.9014217952934828, "value2":8345388974528376469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_2":"a","key_3":"i","key_0":"d", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501652, "value1":0.45069419646062914, "value2":4400331011414194610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.240902, "value1":0.34849466754791153, "value2":3738192721082184330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.950263, "value1":0.5119937423488377, "value2":8919897235633479114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.443897, "value1":0.08318284858972097, "value2":8878946431124018702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653076, "value1":0.2711114129019499, "value2":4904832384185588956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199562, "value1":0.5179430768249581, "value2":2338622690587545147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902254, "value1":0.8456128690164353, "value2":3512723913452353513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424568, "value1":0.358235963026908, "value2":1951463002559117594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.891020, "value1":0.899730727943035, "value2":384925737643795292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242327, "value1":0.8194521049409685, "value2":1651591731883214718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064957, "value1":0.5094969888256025, "value2":1137790773894543912, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032406, "value1":0.3455331481317415, "value2":2958859262258749395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553817, "value1":0.484360827667417, "value2":3796071195831724862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197274, "value1":0.4874943149488065, "value2":1044578434864087492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.384646, "value1":0.3107814255944672, "value2":1519235564474675279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286271, "value1":0.42853641527210906, "value2":4722523490137371754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175308, "value1":0.3946055220580569, "value2":6954032044064595773, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.793904, "value1":0.06297728493572644, "value2":5259172994496601557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.062223, "value1":0.013288825435356888, "value2":6140160587654305146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.768622, "value1":0.39922765836242, "value2":6629678017270316706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_5":"b","key_9":"g","key_4":"k", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.203380, "value1":0.9727752066385791, "value2":960040697676306380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_8":"i","key_0":"i","key_3":"d", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.845192, "value1":0.9218669222513459, "value2":8161692681657646744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625176, "value1":0.7818690860470178, "value2":6286486510404426651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943535, "value1":0.004848635402133272, "value2":8498014405632225004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.204456, "value1":0.5833134681081519, "value2":7410659829376107064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.576114, "value1":0.9192372118815992, "value2":6657353407785586037, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118284, "value1":0.8313517670516266, "value2":7764880758391828110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.535569, "value1":0.24827312684994104, "value2":2040106521001592824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.549323, "value1":0.8166897779780931, "value2":1564430022376434258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881818, "value1":0.09926733907609549, "value2":3471220368243877954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374110, "value1":0.5149369120619428, "value2":2786214942305343351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770127, "value1":0.7580522569953325, "value2":4637323015723129783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178499, "value1":0.146709310096475, "value2":7936042836162333987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274162, "value1":0.4372447015799375, "value2":9083170531856859501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491205, "value1":0.022796016225098064, "value2":8123705023600354614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016451, "value1":0.9256503393490508, "value2":6970208660239633828, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087308, "value1":0.4462899441715343, "value2":1749521530182375024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094183, "value1":0.3475409881711034, "value2":5390535090747181247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.735505, "value1":0.53372762504523, "value2":8372678000097135865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.114823, "value1":0.029693120979849812, "value2":6638472287910475525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242928, "value1":0.3050035672228757, "value2":1792917800424529821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703291, "value1":0.31694612370345837, "value2":2183523770285388049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.331904, "value1":0.47208239473287594, "value2":6204971001548635721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.345432, "value1":0.4951178136444847, "value2":3855507055566939240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136145, "value1":0.6933172444157865, "value2":1571591012651895136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303587, "value1":0.05226903625691088, "value2":6994464805121874342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.604568, "value1":0.14657045529476723, "value2":2698838307937260948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376282, "value1":0.8133509413304958, "value2":8456674940436282992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843511, "value1":0.6514108937273068, "value2":7171882174493021649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888039, "value1":0.34434154548426427, "value2":5546478000801091154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663092, "value1":0.2145932369532521, "value2":1924108819168508153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178424, "value1":0.3811642005135661, "value2":5277232346465674780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.982420, "value1":0.509903018966164, "value2":1366973490493465213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.789914, "value1":0.5590817594064089, "value2":2545283978114147454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.858245, "value1":0.2990011327668893, "value2":7402472987921684090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045358, "value1":0.4729812652892598, "value2":6922132388747645101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212994, "value1":0.6311902604147308, "value2":4414936740257073237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357341, "value1":0.06207502931784455, "value2":4475293686563973956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.975194, "value1":0.41868388111482774, "value2":1854059337783469990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841014, "value1":0.21676180216590327, "value2":4591196137073503812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.978140, "value1":0.3174552368999545, "value2":1901352595947591853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.267125, "value1":0.18049579402909974, "value2":4625546630316797154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916965, "value1":0.5612711107070505, "value2":6029229351915879062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.927166, "value1":0.40285609194050886, "value2":5993132840270480501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791545, "value1":0.4639522865587751, "value2":7463548107279858536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770063, "value1":0.26886367149111556, "value2":3097459774572125541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077189, "value1":0.7572977836842385, "value2":8298753827870878088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.287375, "value1":0.621000003066286, "value2":5005851683916784910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517180, "value1":0.6660153705066499, "value2":8387087977559252312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.675034, "value1":0.9813881401454424, "value2":3741676304938251291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.554896, "value1":0.7715073370679849, "value2":6589060977722716090, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319288, "value1":0.12279901479898898, "value2":3422770912327659239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703895, "value1":0.8358860148624778, "value2":8257794540400854550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493064, "value1":0.048118109532117544, "value2":4134373887037968422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799328, "value1":0.11593192650373021, "value2":1597084067805135642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.848966, "value1":0.3986720724086452, "value2":2405658508790196207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792447, "value1":0.5248994141540482, "value2":6715402324504357457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376734, "value1":0.7992683845538152, "value2":636776368616526730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_8":"b","key_2":"j","key_6":"i", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646082, "value1":0.7040359351145827, "value2":8792341972386350386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.358717, "value1":0.15149972457126396, "value2":8525880284771508381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038854, "value1":0.6775307407388963, "value2":4497884453622086771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313173, "value1":0.6408664944912432, "value2":9170385529548968990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.209148, "value1":0.8824832965439914, "value2":6994782182036294339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.349829, "value1":0.6332729838723153, "value2":8895300460695986955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877356, "value1":0.9629884067987685, "value2":5874587318078108417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175639, "value1":0.2215504394755138, "value2":1645626442028267495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.867334, "value1":0.17245948139280137, "value2":7635160136876279780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.304089, "value1":0.4417775050588339, "value2":4663940359268319500, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.377260, "value1":0.3699273314519082, "value2":3098095013643812538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.687645, "value1":0.49899496293822504, "value2":3483082050221607102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.764258, "value1":0.5854336662763441, "value2":9096847150126021331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.395360, "value1":0.4621587456489886, "value2":4751829496417501448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702903, "value1":0.41598100403233135, "value2":3929201336867975918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107090, "value1":0.7914140452826858, "value2":3589076543012868088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.487563, "value1":0.4474861481593704, "value2":2779456470910748695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_6":"j","key_7":"f","key_2":"i", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512119, "value1":0.5021386207405886, "value2":5678715906223819621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_4":"k","key_7":"j","key_2":"h", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346070, "value1":0.55706872148054, "value2":2039287696528568559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.574280, "value1":0.3713448367655447, "value2":3929829805998447613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.892079, "value1":0.6942759872823322, "value2":7729227663415283404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.344002, "value1":0.42092378887559295, "value2":4794883782773019081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078411, "value1":0.6154878270099419, "value2":6707716797669517904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.571429, "value1":0.1378751349573672, "value2":8074751840319910755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.040055, "value1":0.6408943986817053, "value2":1079211030051415138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162869, "value1":0.5000621894976172, "value2":5126515351324361210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.390768, "value1":0.5289053805740587, "value2":7495442762687396988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259180, "value1":0.5649142133543891, "value2":4328071162032108601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842704, "value1":0.21495927580985347, "value2":2932451180465678632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650755, "value1":0.6984634581926075, "value2":5766024562422825714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612080, "value1":0.22851599558447847, "value2":5344041318095855549, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990865, "value1":0.7354893379622883, "value2":9025665650100041943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787926, "value1":0.33568512834020503, "value2":6804173344109850974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172355, "value1":0.9139245088414795, "value2":6155807217840657379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.979541, "value1":0.24199196392953898, "value2":5042790445163279739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.314666, "value1":0.3110154918771059, "value2":8906652806658780575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357408, "value1":0.8063999303874789, "value2":1651751776595868392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.288676, "value1":0.9293974960471005, "value2":7670397636585827003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293262, "value1":0.7818217756805212, "value2":8606918715162923765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.083534, "value1":0.9670843857632683, "value2":5887729702260327460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149981, "value1":0.6552447379991068, "value2":6725961755903338686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.492842, "value1":0.7092364133108918, "value2":7796995215212529618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944648, "value1":0.6256326288941183, "value2":4831063196252879183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.895337, "value1":0.4995752655149396, "value2":1408597026195247818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944023, "value1":0.634711000263352, "value2":5584697240807054271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809484, "value1":0.7426498850085607, "value2":8020687455958940501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289743, "value1":0.008820277060811819, "value2":6337782472142563494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.537111, "value1":0.14436676295527084, "value2":1783901321543975972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749042, "value1":0.9467021656046419, "value2":279529980240550876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.261740, "value1":0.20334209964882055, "value2":3677408684840846866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.272200, "value1":0.5140769192808836, "value2":3965788713393972569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391452, "value1":0.5620808926889774, "value2":3432925103242153786, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.797201, "value1":0.24306630830309864, "value2":2212488327265931040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296686, "value1":0.4958413853936345, "value2":2745010941408731624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.508150, "value1":0.1764099610149601, "value2":6087888705783067229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.150030, "value1":0.6475061379725164, "value2":4152410772475568903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406997, "value1":0.8552543065924282, "value2":6943121315248371505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798595, "value1":0.4482032174944134, "value2":4591949889203793788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427353, "value1":0.23427032816286456, "value2":8455901174434850105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.426947, "value1":0.2157937977395769, "value2":4812396978108020858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553964, "value1":0.6569888020417926, "value2":4304633456065882312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830280, "value1":0.6753558122351391, "value2":5709360383833750840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335737, "value1":0.6229413264866215, "value2":774141312393774432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934503, "value1":0.3184401780744072, "value2":992485213419150513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286280, "value1":0.19701195292124982, "value2":5131579316119627914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.027527, "value1":0.5595644624118316, "value2":7324008691917344768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.445835, "value1":0.3878002605330551, "value2":5283397242149349986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.317903, "value1":0.34865399024368093, "value2":919918660600814082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.988293, "value1":0.520877683344731, "value2":1362287203583282026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451918, "value1":0.17797518451009597, "value2":8069259017842130934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.416614, "value1":0.49438798293161773, "value2":8221530176741871409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930143, "value1":0.11046223267799947, "value2":1569220098038190448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841002, "value1":0.2351074608491269, "value2":6987153620614536862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172031, "value1":0.30378326240838155, "value2":3438948688273621480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427034, "value1":0.3726321157197697, "value2":3892410333661915590, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.996688, "value1":0.06148106153681236, "value2":7012776674540552021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178139, "value1":0.24652480255103007, "value2":7264782641186044646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.641890, "value1":0.18896088271258277, "value2":5281363384350384982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020470, "value1":0.5427453076719968, "value2":4211018551963613162, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548627, "value1":0.9467235670877983, "value2":1356346215173760766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923887, "value1":0.16868132666008845, "value2":2419587552380208808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388273, "value1":0.34607395217688514, "value2":6360018437765039879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199475, "value1":0.48700761251474795, "value2":4207130869528000977, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212632, "value1":0.7298887109862953, "value2":3855225371708173636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869081, "value1":0.3040832001677732, "value2":1636252957386132843, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458358, "value1":0.25499479265263875, "value2":7951534915845928364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673427, "value1":0.3560914688953808, "value2":4486505974707646981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934304, "value1":0.9086314287420252, "value2":6454407382033994812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077472, "value1":0.272412312785447, "value2":4055844210948982160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391470, "value1":0.4094306269697527, "value2":8737172085844354362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.507196, "value1":0.4293932426928758, "value2":7752579176339832618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.354922, "value1":0.8782836749723913, "value2":3092074071663841238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290283, "value1":0.6466038651665884, "value2":6445966108572410248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930993, "value1":0.2534798517547614, "value2":2212538268968967382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078571, "value1":0.766186870171555, "value2":2276988139922811436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512143, "value1":0.26492881415396713, "value2":7960469334919381452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.664663, "value1":0.9293891578195701, "value2":8363525489715122179, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.859809, "value1":0.5317709711783597, "value2":5646146960428890136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755810, "value1":0.5033479266486215, "value2":1442438201922477703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.014944, "value1":0.2736949295034319, "value2":4543402002213224798, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.019842, "value1":0.06301734338783647, "value2":7848421926410748778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.006460, "value1":0.7283128915704968, "value2":8696821133231766119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.495665, "value1":0.22357458621454834, "value2":5013935622275517771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_5":"j","key_9":"c","key_0":"k", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.829438, "value1":0.08516153321035196, "value2":6346747105122069788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120479, "value1":0.6330301971897442, "value2":3074605270601257967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406377, "value1":0.9646902889346889, "value2":9116844958473667659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258382, "value1":0.9290313183449712, "value2":7925410787408077827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340705, "value1":0.7460138699765932, "value2":1834174806073680705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.660263, "value1":0.9250659950303504, "value2":6999248688568055694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_6":"g","key_0":"h","key_5":"f", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252290, "value1":0.718659365662016, "value2":414864145410727491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404912, "value1":0.5143974391105166, "value2":6178244170296726344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.355231, "value1":0.7285417682392774, "value2":4099314188219635809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232613, "value1":0.9628251247534773, "value2":2753347640662156751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042238, "value1":0.40260777202369225, "value2":6405687740970757268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.059614, "value1":0.6706802975294311, "value2":155105754975301806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335427, "value1":0.03875943420721546, "value2":2621496673126808846, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791994, "value1":0.4069226188854093, "value2":8911123850930154192, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.152311, "value1":0.001001700535447483, "value2":8160805255392424399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.806126, "value1":0.09279090865200551, "value2":5165528109888127485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181230, "value1":0.34297422042642206, "value2":5952935302808125089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308758, "value1":0.39399952685268963, "value2":2291587630268260397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438229, "value1":0.008760206538330227, "value2":4899262456651209905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253853, "value1":0.7708129134752543, "value2":5591299897276344295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388133, "value1":0.4749440177332357, "value2":1978415349188199488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612871, "value1":0.07573141562457633, "value2":5953047090784791116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.481277, "value1":0.12674805322449534, "value2":8950535310944284362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026417, "value1":0.13620073115890582, "value2":5838422267573225081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491782, "value1":0.6697421435610075, "value2":5470323240276494634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599889, "value1":0.2218252683274155, "value2":3863404473966363047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028452, "value1":0.8531657224160458, "value2":1800467248475944001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234267, "value1":0.2783481987951185, "value2":6997165241525177240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369705, "value1":0.5636516367093837, "value2":4958650393346745090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201031, "value1":0.628281679414999, "value2":3983210515283198963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.519582, "value1":0.950842451530494, "value2":8333725115642196357, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.800667, "value1":0.47362573201727787, "value2":5531339313881866007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.022715, "value1":0.7456986400451252, "value2":6375649752818307396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026963, "value1":0.013441045212029653, "value2":8879122156886303116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849167, "value1":0.7209297073484169, "value2":5571977114960029707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916214, "value1":0.07478238462060836, "value2":4957974530052652715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.801842, "value1":0.5497676040427942, "value2":7068572572815929980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.180884, "value1":0.16934771421243305, "value2":4129529446885753081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742335, "value1":0.17996128783258833, "value2":1612889506435491518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.124131, "value1":0.6211285761421723, "value2":6936437753872174981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_4":"j","key_2":"c","key_3":"h", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.626664, "value1":0.1412142921060844, "value2":7340286601023012024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_9":"k","key_7":"e", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534599, "value1":0.1824543867650024, "value2":2710596260731528329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.198428, "value1":0.563515863993397, "value2":8658671925349980639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939981, "value1":0.5602221119588965, "value2":1332664463057731952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749870, "value1":0.27530155912076365, "value2":3294589175109845443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501991, "value1":0.28276078133217086, "value2":6074437039967216201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886807, "value1":0.3878328960046618, "value2":6751632027412168403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_8":"e","key_3":"d","key_7":"f", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.911289, "value1":0.6411363324230984, "value2":8252067497790790379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.696807, "value1":0.43753398310350267, "value2":3790908907624313334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319319, "value1":0.8437649419939369, "value2":4536112071098596928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.734427, "value1":0.7768336856428384, "value2":577814262800235815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232482, "value1":0.4639181971383516, "value2":915233793101361418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033045, "value1":0.6840737140076493, "value2":6077817095892263388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427451, "value1":0.7412294644798527, "value2":5067651533453787714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.086307, "value1":0.884916959850894, "value2":6213157740517107791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032588, "value1":0.7692661642362982, "value2":3255905758159811447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129133, "value1":0.9868818324607139, "value2":2840159926392322636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005933, "value1":0.4856659261691267, "value2":5401836471710156191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078678, "value1":0.43872499297363293, "value2":7610348477992750069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.101709, "value1":0.6553052079447295, "value2":5590613299241898040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.691765, "value1":0.5500719022328135, "value2":2311787078857040546, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.609841, "value1":0.20437362444327167, "value2":5976709159513367946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.262383, "value1":0.988890785552926, "value2":5073782211732867167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_5":"g","key_9":"g","key_1":"e", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382099, "value1":0.7056400640121613, "value2":1362786206799762754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.910924, "value1":0.740118438943497, "value2":5204241364776707180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526567, "value1":0.33926045568377805, "value2":8282435047009117311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259549, "value1":0.6968073477407744, "value2":5508984811117632066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_6":"a","key_2":"i","key_4":"b", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.575379, "value1":0.9204583134801098, "value2":7328623658034172463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.021181, "value1":0.8037794671210114, "value2":8763012613490416638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666802, "value1":0.735290222214112, "value2":5898346647756703163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836023, "value1":0.9774841401477888, "value2":262883488175479807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085239, "value1":0.2006523822964799, "value2":6927806981819194391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722778, "value1":0.29750991877070054, "value2":7269146382542606407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585823, "value1":0.7790122698056057, "value2":5454870737254023995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966399, "value1":0.9342783027092735, "value2":1736810747318512793, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.363378, "value1":0.5421736888212692, "value2":8680345426115136675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548518, "value1":0.6180502927475818, "value2":576215651567739722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169252, "value1":0.8808696517971845, "value2":3266464067570846970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032461, "value1":0.3854322648479189, "value2":5351736110848205547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418352, "value1":0.9282836374954182, "value2":4950655664039540565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.567245, "value1":0.06221685577996024, "value2":6376294794655595743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.607825, "value1":0.8284877360321535, "value2":8836357162970050736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.002720, "value1":0.9429757239354525, "value2":3712886085226721402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772087, "value1":0.8045038867848936, "value2":4759138460922621996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515031, "value1":0.2984015395169622, "value2":2069584478283962675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.141150, "value1":0.6666900797494949, "value2":268184099180001198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.102173, "value1":0.08736073078451814, "value2":3669960405293392951, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.825677, "value1":0.883585003117732, "value2":4900464079888733154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.613326, "value1":0.2003226722632331, "value2":3796686759340146598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.965080, "value1":0.24041472099907274, "value2":7454089011589464567, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139395, "value1":0.8846967167208682, "value2":3451086131264105278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709618, "value1":0.40080113749800483, "value2":7347906568423400039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.490979, "value1":0.10232414632757307, "value2":6898384399737921493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.419810, "value1":0.9521194603097142, "value2":4165710553126975748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142251, "value1":0.3656369505236563, "value2":2656042717790241170, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578775, "value1":0.42161623774225904, "value2":7959202830983535548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340081, "value1":0.47208224231080925, "value2":1119972655203312039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_3":"e","key_6":"j","key_2":"k", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201301, "value1":0.686282703890728, "value2":1347056528311053195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.905450, "value1":0.1504198445233754, "value2":4153183079049561671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.774660, "value1":0.04197441149840889, "value2":2892741854923361469, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.223828, "value1":0.9913278205978917, "value2":2634921019433133187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145135, "value1":0.9764589636119458, "value2":1863333686324511619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051457, "value1":0.9664880803510422, "value2":5468675095205300977, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.471202, "value1":0.9438594012185837, "value2":8711551131933841419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438432, "value1":0.7332734301616686, "value2":6736601196370014167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.018444, "value1":0.6166410041335356, "value2":7430067770903432071, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042785, "value1":0.3083845760668554, "value2":7529797289952981250, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.500528, "value1":0.41360737530731173, "value2":527946418421906650, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.639029, "value1":0.28120003518234893, "value2":7333246850823447562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118602, "value1":0.22780592472876143, "value2":5562449967641903101, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773447, "value1":0.34556648930141964, "value2":3454919255414457289, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.751050, "value1":0.5691145183911931, "value2":3420985612149611112, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.967122, "value1":0.010774686710818307, "value2":1857363403020110299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211203, "value1":0.5071144540997159, "value2":3201415543625346596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966034, "value1":0.10396047428400347, "value2":9014157472738440520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253319, "value1":0.43609058248995825, "value2":1183645881443278471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104187, "value1":0.4324653404306875, "value2":2865322183450789108, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185518, "value1":0.8897028996083935, "value2":7566651396044601727, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930391, "value1":0.8041371194633276, "value2":7871648174569158725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163568, "value1":0.9301616644769536, "value2":7748526202149251971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869278, "value1":0.88349271636991, "value2":554007324522749523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.213408, "value1":0.664832495384472, "value2":2758153864408048603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451136, "value1":0.3702265627491039, "value2":494580882184796857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915009, "value1":0.6519575592422996, "value2":6479471856073874185, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.581364, "value1":0.0703141245253348, "value2":899491222964044388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142057, "value1":0.9957531092014932, "value2":5996532724034214303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618561, "value1":0.31275866348570913, "value2":4261560321529985020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526426, "value1":0.8632813331612903, "value2":2280841193196363442, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.640741, "value1":0.4671010753968709, "value2":2523475617435456826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686774, "value1":0.37684937337218677, "value2":6568317226871778750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.547284, "value1":0.9412532971618462, "value2":6396494980432088364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630465, "value1":0.10697292615695254, "value2":6790210135628014818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156319, "value1":0.10115374361141051, "value2":2774605952238778168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742080, "value1":0.4345761934130475, "value2":6896687242300333410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662119, "value1":0.3886014719849511, "value2":2171378209788759856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158805, "value1":0.4473032074837117, "value2":586039993535786710, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025398, "value1":0.0651048756141861, "value2":3136658443010387115, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189314, "value1":0.11574796728414168, "value2":5756732023607570312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_9":"d","key_1":"i","key_8":"g", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469000, "value1":0.3187212262703272, "value2":4880377874531438837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698910, "value1":0.9021085577061128, "value2":8314934978791640839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228153, "value1":0.7939593629687762, "value2":5721748474729655997, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.174833, "value1":0.47180057752688764, "value2":74850886504396865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.373682, "value1":0.9894401032981868, "value2":2443814427210362559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.230070, "value1":0.840516361148932, "value2":5295744989016708862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595645, "value1":0.2788762608865926, "value2":898774770804749270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583440, "value1":0.38764558337295657, "value2":4879715356875855821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.818979, "value1":0.08550208893092236, "value2":7063465379400979205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990629, "value1":0.9735905702213057, "value2":9121417987396463994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.904697, "value1":0.06007475161136332, "value2":1399260524832392153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832693, "value1":0.9560818456694564, "value2":6313771830503199817, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.273330, "value1":0.5911156951363433, "value2":6957059227202260296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.069691, "value1":0.43486640701864104, "value2":5573463606213032969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107074, "value1":0.28194511495703845, "value2":5008401247473276713, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.133554, "value1":0.6572805795366599, "value2":1430282776047161882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.994425, "value1":0.7317293344785567, "value2":2161864319170825977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703844, "value1":0.05169084281485511, "value2":7389845001390437940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667112, "value1":0.912614805848696, "value2":7494408396730454879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175192, "value1":0.5654088358678295, "value2":7414573375849683263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.205436, "value1":0.5764549187518897, "value2":4315302504250390302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.816073, "value1":0.3021595298462732, "value2":7835612651483818643, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465800, "value1":0.7224511737802353, "value2":1475815608612889931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836860, "value1":0.13043025990273924, "value2":8725370796816365579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110112, "value1":0.2182570483232002, "value2":4212625314776126266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.401613, "value1":0.3660560702023913, "value2":7136594683517469774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772305, "value1":0.6420871129325768, "value2":346199260296482441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142556, "value1":0.18492252698973582, "value2":2410884666534888495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.710999, "value1":0.03498389655141747, "value2":3857815905441742992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588844, "value1":0.6843616917435196, "value2":925508006135613377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902010, "value1":0.7892717951683124, "value2":7453201615304707478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071475, "value1":0.8597541524726022, "value2":5247483360729978449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886399, "value1":0.896477399685476, "value2":3245685435537096080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683519, "value1":0.7125828488615462, "value2":9221328263547905112, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650600, "value1":0.8976371405804583, "value2":6562020796772331341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.316077, "value1":0.8989654150875539, "value2":2315139769444859937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.844010, "value1":0.027932169490326376, "value2":8266191397214577403, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_3":"b","key_1":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.732165, "value1":0.9768852287807686, "value2":7871357409427659276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.644252, "value1":0.3652027068824268, "value2":4777422302644847840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016877, "value1":0.4774488788727276, "value2":1359637185519527638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.121368, "value1":0.07637107971325026, "value2":225307864650781063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.661741, "value1":0.9526993459994305, "value2":1751890497684728484, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534867, "value1":0.5672837246804853, "value2":3948370290656301760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.400506, "value1":0.312527577003677, "value2":3200609992091774001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023204, "value1":0.9498281171738407, "value2":1775257641053296989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_6":"c","key_7":"f","key_1":"a", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650859, "value1":0.7200243703698949, "value2":6459489446120996262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125434, "value1":0.7513756216852647, "value2":6502722962310909561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_5":"k","key_0":"k","key_2":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.861634, "value1":0.22132024628333183, "value2":4213082281103637151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.616718, "value1":0.7492815151768418, "value2":8785198245169443750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618495, "value1":0.5367662090666573, "value2":311068731789452160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090805, "value1":0.6461806005354355, "value2":1567126231539851190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.704866, "value1":0.6742157058257261, "value2":8258394004417852586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178045, "value1":0.9347807954648532, "value2":5391643132437615590, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.903981, "value1":0.04205578457439769, "value2":4267367541005494603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038592, "value1":0.9632414356901877, "value2":7847330231249131972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753078, "value1":0.025682952887214512, "value2":4407969986032631485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.551387, "value1":0.9127538521307936, "value2":5532887277300530226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340315, "value1":0.357568695323674, "value2":5839322022227051810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253088, "value1":0.37455084749823464, "value2":919408856650004667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.061612, "value1":0.08807754228035809, "value2":7674978240888726818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350886, "value1":0.9187800273401259, "value2":3985979173089262193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.997832, "value1":0.3332675485121975, "value2":6884650671561667205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376107, "value1":0.882485292013383, "value2":980580007402095853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_5":"h","key_6":"h","key_0":"b", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666200, "value1":0.543727834482791, "value2":7683502472635473948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.690345, "value1":0.9227402770528923, "value2":5963421474121503099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831896, "value1":0.8899731417411955, "value2":3119002755449635021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583901, "value1":0.46723052632915907, "value2":4738569017001620646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254234, "value1":0.07205706866452416, "value2":1464209096722519553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.940932, "value1":0.4880590550999508, "value2":3249029228406137695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.819723, "value1":0.9180382594935601, "value2":5531777988899534137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.899553, "value1":0.5017555686141989, "value2":3526732457056334830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364242, "value1":0.726219019872585, "value2":7463421321863859910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591997, "value1":0.18380773538675038, "value2":6489374408484526555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268627, "value1":0.43767329135389277, "value2":7591255891965320507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668426, "value1":0.8905993514345967, "value2":8025287728753470033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_4":"c","key_2":"a","key_3":"b", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980089, "value1":0.9629455057472474, "value2":8613343366791814003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_6":"g","key_3":"i","key_4":"d", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.561494, "value1":0.11038616417110533, "value2":5718549668479051731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031316, "value1":0.5572446441362652, "value2":9073355869031665981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913579, "value1":0.40513173097972344, "value2":1392267543875933322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469802, "value1":0.7197223556739805, "value2":8312522501295372114, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216239, "value1":0.38424010939993836, "value2":3429512093250859609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053128, "value1":0.12167032010014549, "value2":2500133776481013473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_9":"g","key_1":"d","key_4":"b", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755093, "value1":0.1432079670350677, "value2":4460076765766388831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939117, "value1":0.022396858430100586, "value2":1677736517466813137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730771, "value1":0.5975163721595746, "value2":8564236690483063617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.763913, "value1":0.11111651544548351, "value2":2128389252611471929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225483, "value1":0.1857169909861624, "value2":1459664257900170929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.557521, "value1":0.10580492023702627, "value2":9211924519949290338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.398250, "value1":0.8985634717833737, "value2":2690743666091944740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.523236, "value1":0.8737716900990798, "value2":3717036232567418933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465788, "value1":0.8763409298655005, "value2":1486778905892801100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185627, "value1":0.4834833602007871, "value2":8283437701946785636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129438, "value1":0.4506377660858689, "value2":8577339860951120030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.937075, "value1":0.6005010987020736, "value2":4798896188528165681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.387681, "value1":0.8777928078176396, "value2":4677891530420984908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.074680, "value1":0.015131030176877407, "value2":2652924707092709319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032439, "value1":0.7762590461709376, "value2":5555968285860120909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107619, "value1":0.0013683674938652786, "value2":8724767287173452904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199777, "value1":0.2705573241568207, "value2":5731172518242608315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.883455, "value1":0.440980354957089, "value2":717450023283511313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.065153, "value1":0.7805040031236675, "value2":8249819238828017763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028771, "value1":0.4266508385350679, "value2":3525220539697375602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417030, "value1":0.9344530676800743, "value2":2840008251192450241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169783, "value1":0.4944137362673325, "value2":8143727104076341090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309365, "value1":0.8233798931001785, "value2":4143752746550875358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405563, "value1":0.36473772798628423, "value2":8368477347330201864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.080493, "value1":0.45312759136632275, "value2":5757367910796479605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591589, "value1":0.5574497820130245, "value2":4692942231335997930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770417, "value1":0.20880923062348417, "value2":4939040557737590500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923414, "value1":0.8006055497749137, "value2":6979326850811422524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249012, "value1":0.022621459630026125, "value2":6570512292324514521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787566, "value1":0.5692664113038692, "value2":3307063324843670478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.236668, "value1":0.706981031305401, "value2":6240895241125781235, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590979, "value1":0.9325091299580149, "value2":5714058795401825599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782982, "value1":0.8159590900452445, "value2":3981533983292904484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449106, "value1":0.2421958898621009, "value2":8819094433559346153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_3":"d","key_8":"c","key_1":"i", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.721732, "value1":0.7320938316381037, "value2":1840371677400026121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417920, "value1":0.3777544081714696, "value2":2517494055558796825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.467240, "value1":0.07736271112081412, "value2":3355581288729910614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802307, "value1":0.06164218213969723, "value2":6962215396306940249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599871, "value1":0.2948698648725146, "value2":1065059716477496547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.109220, "value1":0.11737528756979723, "value2":4468678220083400352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156222, "value1":0.6294465098457002, "value2":2835781199255992964, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697592, "value1":0.9317953948461911, "value2":2221981181398322475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785752, "value1":0.4235402956347778, "value2":5956435747949559681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943686, "value1":0.28355998213096456, "value2":3638916038813769869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.010104, "value1":0.22773696044954786, "value2":2360395284842638669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.239178, "value1":0.14419564768805598, "value2":5315118297850048019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136093, "value1":0.3244550022291072, "value2":164222067931855660, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_6":"i","key_7":"b","key_2":"j", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968733, "value1":0.7486844786821671, "value2":6287914493893452990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252999, "value1":0.24537563867085085, "value2":1626574969302079091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.885743, "value1":0.9307243682752692, "value2":2858030878483913913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.631011, "value1":0.7084136481624964, "value2":2044467985498488215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.853026, "value1":0.04425979450750048, "value2":1653452661642691666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.991236, "value1":0.40886253057738015, "value2":7871621921656029433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.207156, "value1":0.37687587966259445, "value2":1943347287070225574, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.708313, "value1":0.5785225454675275, "value2":1380941757138302107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770913, "value1":0.9188180749639426, "value2":3884885787383779114, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418857, "value1":0.9998412287287592, "value2":1886115929568662728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.850257, "value1":0.9796804114853653, "value2":4397109710918249601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339971, "value1":0.5089698391144044, "value2":8237713822108701891, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913742, "value1":0.9646995309003717, "value2":1173067889283022862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067798, "value1":0.6899692743360695, "value2":1520636933221389194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533696, "value1":0.8606713340189989, "value2":5737213954118900238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212415, "value1":0.39075235526474594, "value2":6592432468398327445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054486, "value1":0.34784858262092483, "value2":7843113252021213719, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479147, "value1":0.5614107726843738, "value2":8690368331747046125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_6":"d","key_9":"k","key_5":"j", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.622976, "value1":0.06775340365098863, "value2":5564636394288500497, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.315983, "value1":0.33546786831791003, "value2":3930900696554115736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_6":"j","key_7":"g","key_3":"h", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.370627, "value1":0.779233269421853, "value2":3716543858406653649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.969610, "value1":0.006014494345107452, "value2":8710808689518483016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.724048, "value1":0.07768942947746518, "value2":496409043717037404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590602, "value1":0.8288610653786054, "value2":1102610595846648990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297543, "value1":0.37578423768641916, "value2":5536977376920161050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.155394, "value1":0.7083524353328224, "value2":4002932567237644088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.459881, "value1":0.4873646265203241, "value2":5916271319076231574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791505, "value1":0.5808298895011346, "value2":7841892156688279555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067541, "value1":0.09121699649729094, "value2":8699612203169033522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138436, "value1":0.8850890106644654, "value2":9069485496265005186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.812668, "value1":0.7098616610527902, "value2":8132915524494658053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.190668, "value1":0.2761598493860578, "value2":651770389362313830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.220527, "value1":0.17936744276680264, "value2":310375881133353905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428315, "value1":0.49075022964169346, "value2":6493345006460964913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595999, "value1":0.2755442227044263, "value2":4862778030716507189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.719052, "value1":0.21758648924177262, "value2":9168976451471290619, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_4":"c","key_8":"i","key_0":"j", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166119, "value1":0.5187037640605351, "value2":7936720027148899998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.361344, "value1":0.7201073761748152, "value2":8725593566170852352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254415, "value1":0.07641284618924576, "value2":7978515171853518726, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.981898, "value1":0.027541030165662613, "value2":6121611839007131360, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625954, "value1":0.22759978693770738, "value2":2989185053134172992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350547, "value1":0.566849398731457, "value2":4327099372915065254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799208, "value1":0.11746266545752859, "value2":7451515689230242023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.322064, "value1":0.26907348347198895, "value2":262434464387240754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.919974, "value1":0.6405108138252255, "value2":7284316699831173435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_9":"f","key_3":"k","key_7":"c", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534467, "value1":0.9814271230191226, "value2":9214896283898892364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.291823, "value1":0.7119821636486502, "value2":4137668561036791084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773725, "value1":0.05006867968524374, "value2":4350082823034910080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.642706, "value1":0.8326077028765216, "value2":6301127653055186823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.624652, "value1":0.6013140948710498, "value2":7150151669380473161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663653, "value1":0.6471420893487286, "value2":2842026748176794650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533665, "value1":0.40332997931033865, "value2":8939087210798770840, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915959, "value1":0.20834521315367086, "value2":7895351758501362895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.238383, "value1":0.5265667505785955, "value2":2676293106518263900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582878, "value1":0.3420886897063641, "value2":1233598437288047014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.489464, "value1":0.6718160824398975, "value2":4468628860088387255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.649028, "value1":0.7293826474439736, "value2":3127060444172041547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254010, "value1":0.8457444172560414, "value2":6200016452741633767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.536370, "value1":0.16874770298054162, "value2":7575698105239291032, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809584, "value1":0.343753090691686, "value2":4854239248106071203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.334375, "value1":0.5540567040963329, "value2":2646076573445947055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826524, "value1":0.7273180695893363, "value2":6564419565708307637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071563, "value1":0.26385268835278003, "value2":4070457585628644142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225965, "value1":0.9321283899737617, "value2":7702159669474089190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458779, "value1":0.10195409940683517, "value2":5987602857042259052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501592, "value1":0.07688705594071264, "value2":5259725214710983339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.385240, "value1":0.8254376912417642, "value2":6760238975249610189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_6":"f","key_3":"c","key_4":"g", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.760329, "value1":0.7530171422146449, "value2":1097173296466796516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.597269, "value1":0.2403481802375547, "value2":5050041855629937310, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149349, "value1":0.43132198586517284, "value2":3287400838900732872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.499626, "value1":0.737892470781718, "value2":4461721794139931373, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449953, "value1":0.8639558735483506, "value2":2903703899322529602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.912253, "value1":0.01605090964788225, "value2":5781647102967996198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.4054065908363409, "value2":6440329633133258472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038844, "value1":0.3763702930516151, "value2":1302491802347098686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.453739, "value1":0.6265318797324549, "value2":819075302260626614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590829, "value1":0.5479451127212884, "value2":8557856578535483978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714164, "value1":0.8513319596110408, "value2":3981392010073089983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.767546, "value1":0.9500048124431112, "value2":2193055476612398128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.685203, "value1":0.9241278767545394, "value2":5146494836784446946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171752, "value1":0.05193751579817584, "value2":2407542541423061968, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620585, "value1":0.9449477088124408, "value2":293003481714135625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197927, "value1":0.9081143587822024, "value2":336470947794228646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_6":"b","key_1":"i","key_2":"d", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730165, "value1":0.4106378302625472, "value2":2305406403284031522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993455, "value1":0.5015628664485756, "value2":3942941750281172936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376150, "value1":0.08846010494510266, "value2":451409939393211294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578966, "value1":0.5184884584460681, "value2":5013311361860229047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980632, "value1":0.6414794039381226, "value2":4661775189685128413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543153, "value1":0.7779597819383781, "value2":1678453405944867509, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.728157, "value1":0.4026694195455883, "value2":1247937331490084335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039033, "value1":0.6078130901686731, "value2":6572519819620597387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020089, "value1":0.16501689332992148, "value2":2135299756770180375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159279, "value1":0.06812322597368, "value2":1853312335979487824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225355, "value1":0.31761717143878, "value2":1547635249117783016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.579591, "value1":0.45080215259792744, "value2":830280592135153306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.989155, "value1":0.3246650009893744, "value2":3832412454221483471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.898794, "value1":0.9353227689941251, "value2":5019805485542488656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_4":"g","key_1":"d","key_2":"f", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.325402, "value1":0.40452074217950085, "value2":3779976710102146862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.846682, "value1":0.8078556003817017, "value2":4563926776477066326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.538107, "value1":0.6916244463696708, "value2":2616499684497809047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601707, "value1":0.0005961819411869942, "value2":8421789457526552579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.474772, "value1":0.40247038234287125, "value2":691809847413019723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835874, "value1":0.24276779175523852, "value2":6899854861109897252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.366826, "value1":0.8778001737651254, "value2":4321676554804962434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826448, "value1":0.8031889549998205, "value2":5554188972077808449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.432681, "value1":0.15017093541000798, "value2":7513115457016757405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.353975, "value1":0.23702527674889587, "value2":1810989557365945672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303934, "value1":0.2677306312444, "value2":430799962318472061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163809, "value1":0.8138323281483592, "value2":8246241283973870498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662355, "value1":0.9144882781553422, "value2":4359470640610896382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120568, "value1":0.8056479342778862, "value2":6170593949410616781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.541822, "value1":0.248047143451303, "value2":8344606419798048221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.147354, "value1":0.35385149944297156, "value2":6006537614889971434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_8":"j","key_3":"i","key_4":"i", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.116235, "value1":0.6774547184266492, "value2":5503266725598369859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.563395, "value1":0.978792023344712, "value2":8139355376515169099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202304, "value1":0.42453838538082794, "value2":8767087562530261434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553168, "value1":0.813931872578598, "value2":838817774250793063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.476339, "value1":0.2705647850316454, "value2":6004383053661805919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067348, "value1":0.23127791420697405, "value2":614040873255409897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.063340, "value1":0.04494057741571453, "value2":7945808906854676018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877369, "value1":0.188685408716376, "value2":5653852277013510880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145318, "value1":0.6394711610065485, "value2":6391486639411558835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171038, "value1":0.9902721723985418, "value2":3163235560331954355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773084, "value1":0.40123815528773027, "value2":6772197516975478771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.976169, "value1":0.608513336182585, "value2":3898075135558136236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.460540, "value1":0.027884469593113505, "value2":1806632468596114826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.411909, "value1":0.6437101253853217, "value2":4628507915366383990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.055892, "value1":0.4209111843259915, "value2":8498738002897413709, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684315, "value1":0.3608627471653678, "value2":8911984339628672502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181887, "value1":0.04106030144724243, "value2":912973087706429869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.127221, "value1":0.8723975492127802, "value2":4688994221703849298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881735, "value1":0.8013556912079172, "value2":643258613153646569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093171, "value1":0.9983142632275949, "value2":2242609892773836818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702516, "value1":0.9951260414368723, "value2":9110402286568364937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712584, "value1":0.10494403348655837, "value2":4745554958401980793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.699467, "value1":0.9010413033988799, "value2":9007036540377523860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.132646, "value1":0.06467587338211019, "value2":1060827532336676254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.552157, "value1":0.4330362437559917, "value2":7496787358324320393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888799, "value1":0.2576583993305694, "value2":4018360458617386281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.498269, "value1":0.2224059001256479, "value2":4065063745853139952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588721, "value1":0.14516718870277823, "value2":3512331805580372297, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583233, "value1":0.7489522543167118, "value2":5610699138607696953, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290155, "value1":0.44273304014865483, "value2":427744180869204041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_9":"e","key_5":"a","key_8":"c", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183829, "value1":0.6345267428968718, "value2":1846381248733979645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.323098, "value1":0.012439339538142696, "value2":8869073391299009493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634145, "value1":0.9267919956221178, "value2":2118172993893609219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794582, "value1":0.8353549954749553, "value2":1351674467301929869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.008550, "value1":0.6040059322178531, "value2":753170312236461842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268043, "value1":0.31922185771526074, "value2":2365194637514291305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.857933, "value1":0.8196275302693926, "value2":4478227043901542114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.645585, "value1":0.17157442528490047, "value2":3055405974571687053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.277791, "value1":0.8508212157393535, "value2":475170132890875463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123450, "value1":0.35015180460716977, "value2":7931493690032322348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.918435, "value1":0.30789613557477924, "value2":4634204529169845529, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.747396, "value1":0.7119174217020244, "value2":5113967535733338451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346547, "value1":0.4105703222986521, "value2":5634326652610778742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.873833, "value1":0.5726619572582797, "value2":4834039219162125836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839553, "value1":0.3474160702051541, "value2":5912402851303338785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924943, "value1":0.25193375971882537, "value2":3157228766646586798, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309520, "value1":0.4856065585160012, "value2":12512784025771269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612385, "value1":0.06742962293903079, "value2":5561454087904294314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634744, "value1":0.8158802826640159, "value2":4407063105915178233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524442, "value1":0.05056002141365119, "value2":1069310785755082516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.122461, "value1":0.8648592352685478, "value2":2607760955410972754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308553, "value1":0.8340608354185821, "value2":5091217112123016682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.073014, "value1":0.412293314616913, "value2":1754188674597018639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382506, "value1":0.7914084871078703, "value2":3113230598922027169, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.494019, "value1":0.22683994673921629, "value2":345509562584529791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770833, "value1":0.36946692588064045, "value2":3634543347045698698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374724, "value1":0.7674203492653544, "value2":409087874753033650, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.33760436450813625, "value2":6378216485000854161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753898, "value1":0.08146706117124274, "value2":4619087622758440080, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.058417, "value1":0.0895913841094616, "value2":7775148933653129917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.889748, "value1":0.9025957275835974, "value2":9060698938664882878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.429099, "value1":0.45935248830386766, "value2":5813431686543708617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.415280, "value1":0.8845089283715811, "value2":6384109853170568567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956220, "value1":0.6163283411653032, "value2":7939487848850431764, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.813125, "value1":0.3306736514104797, "value2":717112968732186535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.464126, "value1":0.23519387446018714, "value2":2061991374504688132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.336194, "value1":0.6334359034532542, "value2":5464410230276800258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722970, "value1":0.6104432497384834, "value2":3298076355265173187, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.246842, "value1":0.8069861206050694, "value2":7793368021912415700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465645, "value1":0.6879012241509205, "value2":4196934016416565115, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876004, "value1":0.06724546167889978, "value2":7203308436755461322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667298, "value1":0.6826712318664662, "value2":6402839614176163878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.329694, "value1":0.7040366253423608, "value2":1262392626609016984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591566, "value1":0.9312106031712911, "value2":6293442498282971529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.559072, "value1":0.4448009704050545, "value2":2669159126008899565, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431852, "value1":0.8238849009316198, "value2":1021818236630467563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722762, "value1":0.5948459056041813, "value2":6253806025921861085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339501, "value1":0.14816011629555237, "value2":2437202342857552183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313849, "value1":0.14517234363613402, "value2":3032490535727906731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974064, "value1":0.5282428591902778, "value2":7306937940345849875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930949, "value1":0.6595519497499615, "value2":2818227237339798252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.995836, "value1":0.41346641045143234, "value2":457793590293209131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.527926, "value1":0.17089664849686367, "value2":5825724904068162921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_7":"d","key_1":"e","key_6":"j", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258906, "value1":0.5587901020638738, "value2":2247971244413895345, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232248, "value1":0.9182710496214209, "value2":2114818465791862400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.075755, "value1":0.36489084685089246, "value2":294204895726685309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.726685, "value1":0.6513663450138257, "value2":741257611118540876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794966, "value1":0.07728058455296861, "value2":6115920270410886698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.954148, "value1":0.6576477115837028, "value2":8442517348621889353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.305446, "value1":0.8417215600157095, "value2":3960819244598684350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749210, "value1":0.905397326364665, "value2":7392448658664215427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479499, "value1":0.8192850944227448, "value2":1109448037735485747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_8":"b","key_4":"h","key_5":"a", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228850, "value1":0.7549298352802428, "value2":2475887390541907125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.256344, "value1":0.6191375565525401, "value2":6629221972448932333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843715, "value1":0.4048974244394111, "value2":2052496354441233400, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.257771, "value1":0.11131625744600283, "value2":5527432282921695963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033616, "value1":0.18479886541727453, "value2":9028612437715086928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832552, "value1":0.6959404377768793, "value2":2173084661524542898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431933, "value1":0.7816557853866045, "value2":7424533234329790131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.264080, "value1":0.7382347817860379, "value2":5213561271316161382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085647, "value1":0.4523244522101, "value2":7889921400021958159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004893, "value1":0.6556889205138186, "value2":3025561804921198898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339616, "value1":0.2999169933424259, "value2":8184676717410157627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216679, "value1":0.42313229610434316, "value2":4008780382662754625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418495, "value1":0.49555776117242517, "value2":1018253790078366875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_7":"j","key_0":"a", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033786, "value1":0.9209475895817386, "value2":5569575246133894540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620702, "value1":0.6494468637932249, "value2":4953667604093058532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_5":"b","key_7":"e","key_2":"a", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.706504, "value1":0.12760690080118256, "value2":6290715519152270329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.483588, "value1":0.5095141564714446, "value2":6862972202733799902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159429, "value1":0.90417660237305, "value2":8461823351624136781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.868532, "value1":0.20009311827615117, "value2":2269303027016825167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.030853, "value1":0.07427016371278365, "value2":6524107618104112365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_7":"c","key_9":"e","key_0":"b", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625191, "value1":0.3705374970999065, "value2":8940487375802886850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274579, "value1":0.43745692673872216, "value2":7664930411577220439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428630, "value1":0.527310522539046, "value2":6084859156132058763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.594595, "value1":0.21322470607998698, "value2":3416879710307779950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.920870, "value1":0.47127845699263055, "value2":4564621285620268941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930512, "value1":0.41802427118606883, "value2":8549501771062307593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977996, "value1":0.1610907563584878, "value2":6578022034712623821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973727, "value1":0.9224145581231933, "value2":8668945457313971984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421891, "value1":0.40172827078543016, "value2":2817865116264178423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683482, "value1":0.8428158825760677, "value2":1087743773064912864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.001802, "value1":0.5075672608227982, "value2":3748937813625482336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924538, "value1":0.6184538471340483, "value2":8049426804122873881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427400, "value1":0.23484522887445405, "value2":360795956748828928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.371453, "value1":0.14713938237360089, "value2":2782450449945692027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105401, "value1":0.7787234859135739, "value2":7103390400724679707, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608837, "value1":0.9083492888699478, "value2":591947190434298439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839636, "value1":0.20678896707817307, "value2":1816652983048222981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.798496, "value1":0.27959153848419394, "value2":6453301245071446957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_8":"f","key_1":"a","key_2":"g", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079294, "value1":0.9921346825877959, "value2":2610009567825131673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.932174, "value1":0.616531393243407, "value2":5793240958146632836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352378, "value1":0.5817027431697587, "value2":1001435181122015935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.892678, "value1":0.09115452403074104, "value2":5100882205034466347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.976748, "value1":0.223689480612581, "value2":4832346905388106952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348707, "value1":0.4785116775888147, "value2":1474405574460091730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285197, "value1":0.2545105724094511, "value2":2069757700743153534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_8":"a","key_3":"e","key_4":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886496, "value1":0.6267242677514145, "value2":5390965713876988864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_3":"k","key_8":"b","key_0":"h", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.106653, "value1":0.29720904047717794, "value2":3776030286792185916, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.360157, "value1":0.5546347320924953, "value2":330359320098225031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229710, "value1":0.36539460584181066, "value2":2852256551896569615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.107272, "value1":0.7519447135299673, "value2":479042341091737730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.653206, "value1":0.055664103121505173, "value2":4715105139689145584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339724, "value1":0.65279778916842, "value2":5051290581444035933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605595, "value1":0.7185823471699857, "value2":6840084614633305448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923752, "value1":0.4375825544173804, "value2":5512879974255226786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_5":"d","key_7":"e","key_0":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259124, "value1":0.1976251344530473, "value2":3865204942280496136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833243, "value1":0.1327195116975274, "value2":25036382112222312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181954, "value1":0.03893696644874014, "value2":8620048848181171214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241452, "value1":0.08765547856551259, "value2":478115772614972254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.346035, "value1":0.8594527006005106, "value2":7112082294884150846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_3":"f","key_9":"c","key_2":"i", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171651, "value1":0.03206744023244803, "value2":1423544456047900511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863325, "value1":0.8536761474388656, "value2":6060590565202496674, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.441838, "value1":0.172156773217081, "value2":5954759661339252215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_4":"j","key_5":"b","key_0":"a", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960980, "value1":0.32980876505437323, "value2":8725964433492010874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_7":"d","key_1":"f","key_2":"c", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000707, "value1":0.24709283071336913, "value2":6676514692655151346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.618373, "value1":0.9009517094483285, "value2":3595072304978000605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.170508, "value1":0.9280407408957537, "value2":5488749324303924425, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863252, "value1":0.12661256202486373, "value2":8664732043269436820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.849403, "value1":0.16431470929214623, "value2":7019138744327165116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464027, "value1":0.09323138351556215, "value2":7725624855401413056, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_7":"e","key_9":"c","key_6":"b", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.236399, "value1":0.8723974444748736, "value2":4852776678069519279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748433, "value1":0.955013940926474, "value2":1647976132077310759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519277, "value1":0.9878333472759755, "value2":1234045371574302701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098817, "value1":0.01452665827731407, "value2":1567075824080468956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.086812, "value1":0.507976874584688, "value2":7088565570837880582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.442740, "value1":0.05234017389082447, "value2":6744325328771713826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200886, "value1":0.26486555470244233, "value2":7063921451775904700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542647, "value1":0.44652572463811874, "value2":881040498652574470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403848, "value1":0.15638403887386948, "value2":881480808583221270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.456051, "value1":0.8059172501323808, "value2":7272729849107639168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529307, "value1":0.34486654778342174, "value2":392684728493023069, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171436, "value1":0.3729515434430422, "value2":3944928681545148043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308638, "value1":0.7632957285940241, "value2":92442464729948625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515220, "value1":0.6925288324736131, "value2":8180048442965609063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584191, "value1":0.021892941959738798, "value2":8483219855752350520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.062243, "value1":0.4180089249967854, "value2":6354312606503764778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_9":"j","key_1":"d","key_3":"h", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341779, "value1":0.7379607607767219, "value2":5150239222005969615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843413, "value1":0.6687750316343205, "value2":2316529591090571268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.402005, "value1":0.9141363896275393, "value2":6270246938486716288, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128379, "value1":0.5150067708119237, "value2":678394980878451314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108482, "value1":0.3129644434816646, "value2":8384633845422523721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_4":"a","key_9":"g","key_3":"c", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.594874, "value1":0.2519824660631957, "value2":6886759477377939773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445205, "value1":0.903159708784011, "value2":7034171128847343546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824198, "value1":0.6204070552794421, "value2":1059263092990498226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.159620, "value1":0.8686561618779289, "value2":5404253444446120581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.365387, "value1":0.4572890031222697, "value2":7934471101097198056, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521775, "value1":0.36902024174358433, "value2":2258863231460147617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.553157, "value1":0.44513850096108715, "value2":6217475414573430294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141433, "value1":0.21220874698804243, "value2":93944351046750038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.811456, "value1":0.8670934692477834, "value2":7403345830074134000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003892, "value1":0.4239005643048954, "value2":7762289626422779835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.067882, "value1":0.18864986614817061, "value2":2907310668904743062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.750570, "value1":0.6373780024843834, "value2":3686123513663951346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.386866, "value1":0.2003273945539795, "value2":5992425398301380670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515245, "value1":0.516857446811694, "value2":4876668434287590384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389071, "value1":0.9340489387136409, "value2":7504250901331893981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098316, "value1":0.29683731723021556, "value2":7480572590262205533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906295, "value1":0.7596148064915852, "value2":2005574389285214762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.469199, "value1":0.3474919541045514, "value2":4513611972684272623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735221, "value1":0.44321964186413737, "value2":284056865289241089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382330, "value1":0.4698238258381697, "value2":6544780705530113911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644228, "value1":0.34547957427274706, "value2":5643558758908257061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025533, "value1":0.9831418878764304, "value2":34382825795091331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564530, "value1":0.1974264877985391, "value2":7202346148984902707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_8":"h","key_4":"e", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.582038, "value1":0.6658499962262118, "value2":4779411307587076137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.048223, "value1":0.28118182352648263, "value2":2406664371776445226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.277281, "value1":0.3293088133074837, "value2":3125287577587960916, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711593, "value1":0.6513318644160435, "value2":6304109109655854952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.136044, "value1":0.15479605681668362, "value2":4561542839689997911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.143594, "value1":0.03881037667597162, "value2":5145183720039449728, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766270, "value1":0.5913740900789849, "value2":2880750585927778789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.330272, "value1":0.03345193869225257, "value2":5177206809108614692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893588, "value1":0.9545617077887952, "value2":1994751413642673810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200065, "value1":0.7615854624227617, "value2":2555245387882162412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.637143, "value1":0.8697167932948927, "value2":3163242108044176287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.640389, "value1":0.5659009743641423, "value2":7954937488899706120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.558497, "value1":0.928513605246244, "value2":1207841403052993714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.244339, "value1":0.9485629290996225, "value2":8066024682557174519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.368292, "value1":0.1771040772082859, "value2":5158146682102067527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883369, "value1":0.26796315495485606, "value2":500188854907829642, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.603831, "value1":0.695040513930677, "value2":7670486818087410343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635402, "value1":0.17089841071375036, "value2":2188177332357604180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246584, "value1":0.638326848995027, "value2":2467157573588247428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297753, "value1":0.4028871413838985, "value2":549392368948968713, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061298, "value1":0.28566196711825703, "value2":4345219419785285366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468603, "value1":0.15137756224435941, "value2":5857705640176057374, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705772, "value1":0.9900652870935787, "value2":4160023456685330050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628560, "value1":0.47755802914007767, "value2":2886069569872931687, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.201527, "value1":0.3765200270511815, "value2":8192740497964394547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405600, "value1":0.16568713498682006, "value2":807681981221494025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.070083, "value1":0.5411063354146068, "value2":6815859051389366015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339848, "value1":0.09610559077312085, "value2":5872072911320364504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571126, "value1":0.2497057720663749, "value2":924893218806837423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659128, "value1":0.026188484488162033, "value2":8931856805131420281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.127455, "value1":0.3964141576554178, "value2":2662798392903843365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780076, "value1":0.15040609462861912, "value2":7522359721454017998, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108734, "value1":0.6578186778027336, "value2":2079921522814480295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.476594, "value1":0.5388243902425934, "value2":1295082543190183614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377611, "value1":0.4899107617022405, "value2":1810844261495017790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.980760, "value1":0.052036560099483146, "value2":4808591658339710229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.662657, "value1":0.739342249293886, "value2":4457016924188070830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573154, "value1":0.043565004929414355, "value2":2934360743315418976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.792186, "value1":0.7194337918198027, "value2":8482147182498656703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687869, "value1":0.5159874448621534, "value2":5870856463348491189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.522440, "value1":0.8824876555801262, "value2":2534178016362914654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132515, "value1":0.8599678891940701, "value2":8395948226155967317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.096926, "value1":0.9245913183435254, "value2":2358549994381165596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_3":"e","key_7":"d","key_2":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486752, "value1":0.2842029485021584, "value2":2801356184403378281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.475238, "value1":0.22293224326621178, "value2":443811864922674372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132657, "value1":0.9371037777036672, "value2":4385576391999515299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_5":"a","key_7":"h","key_3":"e", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.549121, "value1":0.03204732941506213, "value2":8714811939867543319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796275, "value1":0.21070900388130828, "value2":8667819799137478248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927343, "value1":0.9159887689007843, "value2":7689292400204431331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685507, "value1":0.7348923960578333, "value2":330857683592078866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.865459, "value1":0.24066397037006473, "value2":203169051837200909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.827976, "value1":0.4488303067264579, "value2":3646968045887419113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900145, "value1":0.31889183176683844, "value2":5999122195912397909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.251489, "value1":0.5593865051855272, "value2":5772018587784459447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191087, "value1":0.2891985736646055, "value2":4399230413942175266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087488, "value1":0.5425699812510858, "value2":3965285909604337765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680134, "value1":0.18266599397918218, "value2":5571070206339456089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551156, "value1":0.8876027936303559, "value2":6010745863587972423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.803161, "value1":0.9394184911877158, "value2":571658979107886488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.054336, "value1":0.03957759543141472, "value2":3918244459790702522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_3":"b","key_9":"e","key_0":"g", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685390, "value1":0.5264564714839759, "value2":2232108588919897592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.161082, "value1":0.32998817001502934, "value2":8460874534662447333, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778535, "value1":0.06811024060980113, "value2":7835773491677016452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832955, "value1":0.028669939437680664, "value2":4299578964430488535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004645, "value1":0.8188081000238006, "value2":9221293293613853860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205474, "value1":0.40545870686171354, "value2":1575377844766226698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639268, "value1":0.8908404617416188, "value2":1540158992274106467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.139661, "value1":0.3668772992670227, "value2":6483649363648949947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924909, "value1":0.16190741337415684, "value2":5257368157508278570, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297500, "value1":0.4330469601614637, "value2":4632520395480129342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101765, "value1":0.4685291307083107, "value2":1614399458247460286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291465, "value1":0.6213831794241761, "value2":7754123020970248013, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369394, "value1":0.5153486584517465, "value2":2263160774029528717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719680, "value1":0.5383036464533613, "value2":3589028105702605678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262982, "value1":0.02336562179973987, "value2":2500396082328773649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.790458, "value1":0.7125005920534319, "value2":8453423485925643300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.937011, "value1":0.9029577396659265, "value2":3493444148478497784, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815751, "value1":0.34331005780857, "value2":4896770691763395578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.732239, "value1":0.5781653462685661, "value2":2815554048397289034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073195, "value1":0.6318398759739543, "value2":1443128294631046938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.335280, "value1":0.2900799960272814, "value2":319439076295666505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375292, "value1":0.8416100389726322, "value2":1104228901216289966, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543223, "value1":0.30539264418826334, "value2":2529051273510780065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850347, "value1":0.9475422757893678, "value2":3815335775316587120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010486, "value1":0.9233328326166781, "value2":1132448560381334143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509503, "value1":0.43078826324145314, "value2":2585538196535057958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440516, "value1":0.36575868540627, "value2":764298189191108299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.576334, "value1":0.9992971397552057, "value2":8607203945347812848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.970781, "value1":0.43725394836722203, "value2":8039899278253467744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746791, "value1":0.5653040341344706, "value2":6676447946324724349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636245, "value1":0.8420862976065959, "value2":1562613302672270132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285262, "value1":0.32142916834577895, "value2":5289001385309911194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.417392, "value1":0.049995803596490404, "value2":6607894685909801752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584296, "value1":0.2409216093884783, "value2":1637965971530230928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325720, "value1":0.028777043283946674, "value2":963675377332588860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566004, "value1":0.9002123352889874, "value2":4604125644132750665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.882236, "value1":0.33330368402055666, "value2":156824449049228711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.767642, "value1":0.13763101488639068, "value2":8064121309553799557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_2":"k","key_9":"h","key_0":"d", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719598, "value1":0.8986180062766446, "value2":1742955026017794842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886873, "value1":0.010775899740746808, "value2":1351720915728762737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484279, "value1":0.5158913513515657, "value2":6695543448765155807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.120818, "value1":0.38000311042969326, "value2":8387696033998291, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.471169, "value1":0.5449952216814127, "value2":7977583153767662789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.692704, "value1":0.644651424950153, "value2":1344814886763463146, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434917, "value1":0.5336620566427158, "value2":7974250102837601477, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.231472, "value1":0.18429722211173638, "value2":239717814769298968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351874, "value1":0.3302574440246314, "value2":116725965202686758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.652256, "value1":0.8832602328157859, "value2":2443207413955073666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392058, "value1":0.3666546013150111, "value2":5949156227430002451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940471, "value1":0.038777989285728266, "value2":2871780422161853348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412070, "value1":0.5197207843322247, "value2":406042026849178989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571008, "value1":0.7027610078055025, "value2":1383746034138340292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900415, "value1":0.17910816553895145, "value2":401749706711859248, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.718726, "value1":0.2276914412180115, "value2":9221447498523708978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.381086, "value1":0.9677704240195594, "value2":1303001585301279474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893466, "value1":0.2579622668275943, "value2":3971563895105696581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_9":"e","key_2":"h","key_4":"a", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.400237, "value1":0.23652480428212666, "value2":6009422694154041952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481908, "value1":0.625595272995919, "value2":2204722405376257638, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.936595, "value1":0.2680275935509345, "value2":324738255121972023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.983646, "value1":0.3650479381459544, "value2":6655499432411034428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.489964, "value1":0.6666717547984726, "value2":2225549269590513897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641209, "value1":0.578327662161616, "value2":6547750503066251015, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134486, "value1":0.67209109592655, "value2":2885152331577461710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073911, "value1":0.08294753363850706, "value2":7590236594974272032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181170, "value1":0.5500056163737558, "value2":6771546940482209584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.029045, "value1":0.6184324246909119, "value2":8362437134419045707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162843, "value1":0.4243438443026443, "value2":8729664805668824103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.808018, "value1":0.9740191657641797, "value2":811057613633969138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509186, "value1":0.5860682567407373, "value2":4715370569695436166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664283, "value1":0.9354682602818816, "value2":4771667149292350278, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220707, "value1":0.9895245632099, "value2":7384968594835669289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389342, "value1":0.5213334242916959, "value2":9064052527115170337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977547, "value1":0.9968413793490465, "value2":1018668683107683896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122093, "value1":0.22725801593699438, "value2":471678573457243344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474960, "value1":0.6106407332478643, "value2":6142821675445007328, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770612, "value1":0.38366813133853406, "value2":4771686788392964587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166046, "value1":0.6731562334089229, "value2":6765933549203517189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000435, "value1":0.7725469520964668, "value2":1526254098614108268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.596567, "value1":0.33143109459424, "value2":6988963074743099921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_2":"f","key_0":"f","key_1":"h", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832348, "value1":0.5082718126919952, "value2":7366669828299681751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_5":"j","key_7":"b","key_1":"j", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.933461, "value1":0.2177858524105436, "value2":1927680313692641282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_4":"a","key_9":"h","key_1":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800386, "value1":0.37752422688739345, "value2":3623415676274096956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602632, "value1":0.9591290646171173, "value2":149144865605034806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805853, "value1":0.46005104238615857, "value2":5399494644643471050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023064, "value1":0.2589954384060476, "value2":2161234059148173478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.663052, "value1":0.4340276216577473, "value2":1651362675477804080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425648, "value1":0.5284587146267693, "value2":2204338428823542625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297837, "value1":0.9930278173662268, "value2":1153109222914733395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.661230, "value1":0.3189372627835483, "value2":8950530915947476877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997529, "value1":0.9931406569786665, "value2":3171376634461957806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785991, "value1":0.6115707472871859, "value2":4480386318218501447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909727, "value1":0.29199552201380746, "value2":3686357880544622730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155119, "value1":0.5470008889969509, "value2":7858820522318450111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_7":"e","key_4":"a","key_5":"k", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.530629, "value1":0.9873625937302516, "value2":5471095001682678938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512132, "value1":0.14465321479404997, "value2":8852396394387214520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.643903, "value1":0.81607168234715, "value2":7750000008183770029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088371, "value1":0.9834997161657205, "value2":8265123019714389041, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.840291, "value1":0.31720704974294733, "value2":8838522133270072113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607394, "value1":0.6158114593013463, "value2":3920524769932432062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.891225, "value1":0.7260707014711243, "value2":8545845285837175547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155524, "value1":0.5324837252402642, "value2":6050562650984829349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463368, "value1":0.3636429297340659, "value2":7366387480034568615, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.051267, "value1":0.21240586224005645, "value2":3520112151921939439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.713935, "value1":0.6148054965944443, "value2":4435199374933360163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229994, "value1":0.877849746015665, "value2":4328772383519778289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.394088, "value1":0.7419214909083843, "value2":1533570891823401045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311200, "value1":0.6547222273874411, "value2":603076909255570468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593460, "value1":0.33723617074445394, "value2":4429464301654712087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016002, "value1":0.7850512802013139, "value2":9218293660844009579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293893, "value1":0.9024192531760731, "value2":874814059564514510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197015, "value1":0.6346018344137921, "value2":2275287342249871781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444015, "value1":0.07476763645648904, "value2":6294420894284163263, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.137036, "value1":0.7149572150590823, "value2":5890716035999924313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382203, "value1":0.2373007200794848, "value2":4237441552449229366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927338, "value1":0.5067232894842505, "value2":7166709656823836078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.856975, "value1":0.7551172442205076, "value2":3114426699624764241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290618, "value1":0.461652630545808, "value2":6870191814019559455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194719, "value1":0.28972301156720964, "value2":2266210405139781750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.930146, "value1":0.8041428167626514, "value2":3758081748375193976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.876259, "value1":0.8601408243336739, "value2":8153013110856241749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544542, "value1":0.4588432483941638, "value2":1078220017088765885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861984, "value1":0.33210767773987954, "value2":6423637803003548004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725768, "value1":0.13821724766383348, "value2":2685436575194046233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422624, "value1":0.6174306852475553, "value2":622178413482526248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.749470, "value1":0.7804428540523304, "value2":6875918761098582661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.903503, "value1":0.7328633216838554, "value2":3622697451467218798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.116528, "value1":0.9193757905595477, "value2":6857103271674313745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318972, "value1":0.35822522818261543, "value2":8829601973990551809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433538, "value1":0.5317306440222773, "value2":2556858263334639046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497594, "value1":0.1404686557588262, "value2":5182788796116063454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.731708, "value1":0.08352722566425953, "value2":57349948699707298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052659, "value1":0.5013772188087474, "value2":4075548779807119976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680970, "value1":0.7475466041318248, "value2":804362071276609030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644235, "value1":0.13991370513704515, "value2":8012476351028838411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.430709, "value1":0.07096170614998987, "value2":810033186472659399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.283029, "value1":0.49501003784947184, "value2":5335533898113400696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.213964, "value1":0.06985861165056967, "value2":3236599269070483646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899939, "value1":0.8432631992057813, "value2":8331656940556283499, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447604, "value1":0.4633606486187383, "value2":3282862092550905414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986819, "value1":0.31082999792855004, "value2":5075360107904774840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375656, "value1":0.6617351656082016, "value2":8930874302935418560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_5":"j","key_1":"c","key_4":"h", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787774, "value1":0.5242892599116226, "value2":4186364548758456678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918288, "value1":0.3268848136936015, "value2":2614642728372801254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535191, "value1":0.176268908097137, "value2":653040557920632015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529661, "value1":0.35212458808041835, "value2":1525600014751099598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.361165, "value1":0.4714568379001177, "value2":4598533494631122989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898032, "value1":0.22769956750018774, "value2":5631589457641828317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_7":"c","key_1":"f","key_5":"f", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483853, "value1":0.4317121876757055, "value2":564210499810148536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512418, "value1":0.14841146100443744, "value2":6375007633580482412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172575, "value1":0.4457725144006962, "value2":940411418055250112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109679, "value1":0.17465206890579033, "value2":619954528132439983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016784, "value1":0.6172432789077167, "value2":8694611599799563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405055, "value1":0.9156938137116516, "value2":1197968795715040120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.153068, "value1":0.6193599211433017, "value2":3101256038412073082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041261, "value1":0.0029965595633104653, "value2":1337872063966920132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.852796, "value1":0.021368920853982197, "value2":7987040863826958067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921132, "value1":0.47595655072486737, "value2":1700855145658274194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.383399, "value1":0.26371142871266584, "value2":6832011082222328508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.332123, "value1":0.8824714441278855, "value2":4889159017232292596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_5":"a","key_7":"j","key_0":"f", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847690, "value1":0.5825485094693434, "value2":8301349950043802905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004022, "value1":0.963420472883698, "value2":4714062367056435117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178319, "value1":0.21282157659328568, "value2":5741804038235426956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226035, "value1":0.614117409288991, "value2":7916317937107113091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188200, "value1":0.6715893678504037, "value2":7136230473959043902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709476, "value1":0.7492754619759127, "value2":6914951663040962797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879839, "value1":0.9373684799176515, "value2":3333434004719572493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174084, "value1":0.14484188554360505, "value2":2035242236177706450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.505125, "value1":0.57329431354178, "value2":5119683575480371621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742853, "value1":0.7629472821449486, "value2":5737967107040353117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226226, "value1":0.17730887465740228, "value2":4673386690110010077, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560859, "value1":0.9766459300795906, "value2":1996912721458005085, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262060, "value1":0.05094335959612934, "value2":1245357682330480297, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432269, "value1":0.8390455137158405, "value2":1878222408225434076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113677, "value1":0.7363256343069756, "value2":2885521712019062646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444967, "value1":0.7955117384805845, "value2":264780152192144506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162729, "value1":0.6712123871834628, "value2":1119778679298263778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.911266, "value1":0.9891407670464004, "value2":9103652755878870157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379007, "value1":0.2889256375011702, "value2":5327006092087239478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.092408, "value1":0.21632368500444782, "value2":3442219206298318473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032852, "value1":0.524214031146113, "value2":6530887848447144232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382209, "value1":0.7113041138812657, "value2":933816269978698137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.334638, "value1":0.7695287307455443, "value2":7489040093668157103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_4":"d","key_7":"f","key_1":"e", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372671, "value1":0.7439130206835773, "value2":4959199765009131376, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900273, "value1":0.9052840812209041, "value2":965461062104746729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.654196, "value1":0.8121385984212723, "value2":5665296699467259244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178571, "value1":0.19948342559708582, "value2":5100240681564871548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770922, "value1":0.3886846963277322, "value2":619589477307797843, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.623448, "value1":0.010709272715713756, "value2":2405417022197491275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.501430, "value1":0.49322743442420075, "value2":4840878662359361238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268501, "value1":0.9623480970104531, "value2":7490234459966550271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_7":"b","key_9":"c","key_4":"g", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403202, "value1":0.7953121912476196, "value2":6585060526257826515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078706, "value1":0.6781344722473363, "value2":3204877843782827661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.343846, "value1":0.7227864686817432, "value2":3271852738757870545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.026648, "value1":0.47137422416076813, "value2":3734918729615183416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.691254, "value1":0.7660887564670621, "value2":790090719852690753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262583, "value1":0.666841123381339, "value2":2785906959504169261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.234934, "value1":0.28467055157973764, "value2":5751420159217569904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110004, "value1":0.6775088488593821, "value2":3553287660015547262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766422, "value1":0.23227174275211976, "value2":623571748203397381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800936, "value1":0.31353934764817454, "value2":4270432397290565810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_7":"e","key_0":"k","key_3":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709087, "value1":0.45637800363407555, "value2":6299983262198737196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756471, "value1":0.4807492151985086, "value2":7279712801686741955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188086, "value1":0.36885137660554584, "value2":7300782007111153833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.948483, "value1":0.37900022039799186, "value2":659834049221243616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.589486, "value1":0.790378840630986, "value2":5696488550645422430, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783617, "value1":0.7580291430060272, "value2":3320953914407054988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321932, "value1":0.6227238552522466, "value2":8556762730182657496, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585825, "value1":0.8340787864264869, "value2":2874883662014238947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425246, "value1":0.4329750194122766, "value2":6011155334107441536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023097, "value1":0.474467449495024, "value2":9096564500742879948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625423, "value1":0.1682038891195739, "value2":4431847776626169211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289832, "value1":0.9685809921059807, "value2":1593648503211538110, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613185, "value1":0.10083138549838662, "value2":6012379324790188749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720225, "value1":0.22688014499451828, "value2":533494308206781076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.925929, "value1":0.3145810371686551, "value2":2521880979655219825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_5":"j","key_9":"f","key_0":"a", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073559, "value1":0.8510652378781342, "value2":2771699030834759801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540880, "value1":0.19703714704465763, "value2":2229904355406996428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_1":"d","key_0":"c", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358372, "value1":0.47865214913286414, "value2":991745116280869642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.985891, "value1":0.5712880806403865, "value2":7108232781064624938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787182, "value1":0.10623777547287752, "value2":5757177518524775613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725343, "value1":0.4881539523572101, "value2":9008775172737657154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.276129, "value1":0.7963976864976252, "value2":4233311333311619132, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.296009, "value1":0.9362536518865696, "value2":6472012225611436212, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572284, "value1":0.5256827892320936, "value2":6967364447645723612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490559, "value1":0.5321215426269078, "value2":6272951345279554009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.897929, "value1":0.9148681755770618, "value2":8519086271296505773, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438881, "value1":0.9657537469163081, "value2":1825676918798292475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.878233, "value1":0.44284605438890395, "value2":948269923000784878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.240505, "value1":0.2373413838092857, "value2":510374534141290052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041417, "value1":0.17592672383423122, "value2":3473808065501665173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_8":"k","key_9":"a","key_1":"j", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191879, "value1":0.3751219146996357, "value2":7106698191372720163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694969, "value1":0.8445486295845246, "value2":672504742133677359, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974994, "value1":0.6654201856510823, "value2":4512443800515268537, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.265020, "value1":0.3977309051692021, "value2":987280456712575618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_9":"d","key_0":"f","key_7":"k", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.837282, "value1":0.6667916772962499, "value2":2649317741597325862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584752, "value1":0.8184998685164273, "value2":6853243233010504233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956207, "value1":0.20740750871245983, "value2":3701599145151889680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785977, "value1":0.42059150301577825, "value2":4300206568693523992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206794, "value1":0.4065971742289675, "value2":5055750687883280777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.466356, "value1":0.24692397834834437, "value2":6503690772355582431, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374485, "value1":0.27439926954774063, "value2":2257827689674118963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859317, "value1":0.15734285025101918, "value2":2188339349886556422, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.888728, "value1":0.2275635074824085, "value2":3959472993254545104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552286, "value1":0.8140316248314068, "value2":28562363718590634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101732, "value1":0.9986972418263012, "value2":6625998065774179710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.064398, "value1":0.4719546471748007, "value2":5016107261432878232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540095, "value1":0.05564657854486056, "value2":2892021965316178106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052034, "value1":0.6085312696710864, "value2":267981962928859635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479153, "value1":0.1864971302120239, "value2":4274890765687650257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285637, "value1":0.8618179927379825, "value2":98273395449385741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160949, "value1":0.1665709809492651, "value2":3549197933610586412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479807, "value1":0.9792753435060556, "value2":544874846395596701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.715679, "value1":0.955575677088658, "value2":7351930305259561505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655220, "value1":0.7716326625888155, "value2":490280663710319116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799754, "value1":0.26438748028137354, "value2":2889376751051793411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625103, "value1":0.1721904108442133, "value2":7991543743244408523, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.690861, "value1":0.007633155868195746, "value2":3073219921658712367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308852, "value1":0.12391196742809665, "value2":1115580635235450437, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.013227, "value1":0.7267200692886595, "value2":1429240573369570759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268822, "value1":0.6684636760714637, "value2":4339881778709198457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_5":"f","key_2":"i","key_3":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160477, "value1":0.5396173638949372, "value2":4906545989468275563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599858, "value1":0.07893489799204395, "value2":410345370022207222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584892, "value1":0.8068626341311561, "value2":4979004915398024459, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.987465, "value1":0.8935603600443448, "value2":8149710572361609309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227032, "value1":0.3687576237691273, "value2":1391051851919204878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833201, "value1":0.32203470987301264, "value2":2081318068679055279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291063, "value1":0.034071004249405484, "value2":5342977357585868177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290178, "value1":0.3044083195032883, "value2":4841141697891251443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109841, "value1":0.9228991323104464, "value2":1691775942527270832, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587504, "value1":0.5332728718746681, "value2":1334529969932810461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.920462, "value1":0.18350284649923848, "value2":2590094361323084553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.966666, "value1":0.25923537198192, "value2":6116285159120956507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480100, "value1":0.8741323699909098, "value2":1633286453079731557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.175764, "value1":0.46174408884851265, "value2":5588956365079776400, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_7":"j","key_1":"i","key_5":"h", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357949, "value1":0.9471421832392611, "value2":2020538223652392065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087613, "value1":0.8391096943297279, "value2":1883071449916289062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883487, "value1":0.5961374967262479, "value2":6748555704132728387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.514904, "value1":0.43082646153613147, "value2":3827154946167603504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899576, "value1":0.926252451999718, "value2":6912098010507428750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.708901, "value1":0.5214095428253637, "value2":8239976493295716264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.784267, "value1":0.7119074977387614, "value2":5875315435697946397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318238, "value1":0.2857683698624022, "value2":2168440855638261641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434779, "value1":0.728469975577827, "value2":4027120624532411802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432104, "value1":0.9159655292186285, "value2":1266477315347168519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.453551, "value1":0.9820731220635184, "value2":472902679452044251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.239100, "value1":0.05657570742681277, "value2":1082500030056759801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918697, "value1":0.5552870083342735, "value2":3402695849932536811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689901, "value1":0.372073062318352, "value2":4891331242898882829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924258, "value1":0.5283619937827609, "value2":559122472611611400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172686, "value1":0.4344219735503553, "value2":1248792109975059381, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894193, "value1":0.43426860175202114, "value2":3497813026288966485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.327909, "value1":0.3878605121485491, "value2":5047756483827083362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220825, "value1":0.07655058465784402, "value2":5252985818354538541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742213, "value1":0.38430981024473343, "value2":5620787295533246773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.733232, "value1":0.5555223164703845, "value2":631270712021422829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940662, "value1":0.3098977203238577, "value2":5954879855194614083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.649184, "value1":0.3354932964670884, "value2":6051779552546236650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832762, "value1":0.9063274954717605, "value2":3724265762255192701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.385652, "value1":0.5036946685314467, "value2":5109085475601380385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.722579, "value1":0.772640832685559, "value2":3780594965713240464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655329, "value1":0.29356054638145535, "value2":4596604760791157242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.459104, "value1":0.8777447945394818, "value2":1186981345641271839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.399975, "value1":0.025674080171056102, "value2":8964167090414725476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.123918, "value1":0.9272811881045043, "value2":7218529778562540903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747920, "value1":0.9264629051459597, "value2":8464913590880556582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914960, "value1":0.32245869851475545, "value2":9109063864698425563, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.328190, "value1":0.7652891376156531, "value2":3024374861601192187, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593397, "value1":0.17904346621073747, "value2":3899031855173978925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229837, "value1":0.5384685110923013, "value2":5235158233416481121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.698328, "value1":0.5982638259890453, "value2":6740778632490166175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812614, "value1":0.1666081159883058, "value2":2800734738787135789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352669, "value1":0.33792886940688555, "value2":7496164581714310216, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128711, "value1":0.2991783007626336, "value2":7338677993190210677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552539, "value1":0.06968528327878533, "value2":8300480680068139329, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.581708, "value1":0.35231544880285015, "value2":1384864101595986439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778952, "value1":0.8415964882413857, "value2":1745115595251159346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.695991, "value1":0.5753223257206643, "value2":1886601101964605359, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664273, "value1":0.6725491194351899, "value2":5574699884022596470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711367, "value1":0.2595939895449639, "value2":7814193120722379636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.600545, "value1":0.944128951867823, "value2":6711650903238481221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452146, "value1":0.9554010846729131, "value2":3169390990279464268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.140642, "value1":0.16345235341026784, "value2":2148518750069655597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388132, "value1":0.5979152819074895, "value2":7002872128153528233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078323, "value1":0.5697531979956287, "value2":980455479278929153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499938, "value1":0.388179798040964, "value2":2192712143484318261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761109, "value1":0.2600517751619912, "value2":647733851046806648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_9":"d","key_6":"d","key_8":"g", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088032, "value1":0.2162359479396891, "value2":7960761017907576840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886480, "value1":0.21950291960892157, "value2":7881441150954657573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.538255, "value1":0.36627249562354336, "value2":2697185681710870928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956144, "value1":0.09482810661100612, "value2":3142048329987868522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.610158, "value1":0.7330550786147262, "value2":4897289423318138878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_5":"g","key_7":"h","key_0":"b", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641699, "value1":0.27867439634792107, "value2":621712637914424018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.218950, "value1":0.1329184934440547, "value2":3565116595465876544, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.247888, "value1":0.7166338804544903, "value2":5367509956565676215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.214416, "value1":0.3332699601933243, "value2":4699030477990361670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023652, "value1":0.5060732821933588, "value2":1384885559250896583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907328, "value1":0.1694015036022409, "value2":3592921690010847117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.642558, "value1":0.737568796988017, "value2":2406767446522063712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273826, "value1":0.8447613680154935, "value2":6080438111270024688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.984506, "value1":0.28262866688772725, "value2":6480278416681893463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.868136, "value1":0.8910201029103536, "value2":6867531047341490726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687877, "value1":0.8829884505686191, "value2":2249445178190997362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850677, "value1":0.6276341510695712, "value2":2252184121487700107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737726, "value1":0.12597210262772834, "value2":1348511802277935140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.810752, "value1":0.2102822839010783, "value2":4831043683200695850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317105, "value1":0.817061302570146, "value2":8324179065679406640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.426069, "value1":0.09982797254913842, "value2":6519984536301213031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.121073, "value1":0.5197148618479782, "value2":3010915938164137229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.264976, "value1":0.4225812260629186, "value2":5303229136813956536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572439, "value1":0.07380111555778088, "value2":4428887607089365223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526101, "value1":0.2368198468823957, "value2":5676247054773494672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.794040, "value1":0.8488571887152928, "value2":7926265788784400661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682501, "value1":0.6419011526811798, "value2":6127958830288275598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.508833, "value1":0.08505936393863323, "value2":6956791791775335476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.485450, "value1":0.24440134625086388, "value2":8675077803244715182, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341184, "value1":0.7204080689080778, "value2":7288860462608592695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300497, "value1":0.587674524187528, "value2":2136914643668219995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646074, "value1":0.5687487761791935, "value2":5117858885820847437, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405215, "value1":0.63703146651586, "value2":8601605955587466200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040361, "value1":0.9056350649746174, "value2":4395139702119619272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042190, "value1":0.25258468888663055, "value2":6275160558122438530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436823, "value1":0.3052140189316623, "value2":1176588437754725430, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_9":"c","key_2":"d","key_4":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.885585, "value1":0.996098364991821, "value2":7425087259003526051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.667291, "value1":0.01567259898271256, "value2":688757342279745926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560861, "value1":0.8815928724937075, "value2":5127701164424889645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921488, "value1":0.5280525030917018, "value2":8702967906065448733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906624, "value1":0.204252621066632, "value2":7869988567138398448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.855702, "value1":0.956230561465019, "value2":157946491688041108, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367342, "value1":0.6684623495570281, "value2":329650868812150871, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445102, "value1":0.6894360816538613, "value2":4920916512388188612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.609045, "value1":0.0027756305160863397, "value2":5637928110041308291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.632075, "value1":0.04166500110036427, "value2":6157318199778992604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972254, "value1":0.31480609981503055, "value2":4364758259634809973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.776493, "value1":0.13104837145124149, "value2":4732079918184171170, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238222, "value1":0.6559225899889957, "value2":3576156993215909053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434447, "value1":0.24817502461178453, "value2":7546679218765777183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040709, "value1":0.1860292914746309, "value2":1580899927050287482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273335, "value1":0.9526744406123745, "value2":5086267667987270481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052913, "value1":0.33164061518472737, "value2":6494564179935068135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093729, "value1":0.8324601543372498, "value2":4840824205416424699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.167756, "value1":0.8292588795543363, "value2":926691352131787790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602891, "value1":0.7468923005026717, "value2":1499084894678948825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939687, "value1":0.9093420550399373, "value2":7624268916058145495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859436, "value1":0.018264976223293757, "value2":8966202800040990455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540853, "value1":0.15398968210956118, "value2":1169217513145417422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927943, "value1":0.6497527028232101, "value2":8964902118693660858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032864, "value1":0.5440150103669782, "value2":3398396102381979841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440858, "value1":0.5805592700724092, "value2":598543749015746918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220522, "value1":0.7634462305829404, "value2":4744586200886838973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350814, "value1":0.8753641286301881, "value2":3594959380973533940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689230, "value1":0.11698455183819528, "value2":3362979353229964321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369880, "value1":0.9959022990686378, "value2":913069268809019275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499935, "value1":0.42435230326696954, "value2":5642082683494178997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.126493, "value1":0.24855180288733794, "value2":2553926153696879833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241812, "value1":0.19715345400514303, "value2":7966035951898297683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.282709, "value1":0.26568871987724985, "value2":5614603759761174315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634974, "value1":0.5611667195638738, "value2":719923428562838874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_9":"f","key_2":"i","key_3":"b", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.437429, "value1":0.044098595487335565, "value2":7598428545433157434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986084, "value1":0.15643080062821366, "value2":876340157580973000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350215, "value1":0.5350647081004255, "value2":2039458830523373741, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.263790, "value1":0.13663992546983536, "value2":4887595679013193650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761252, "value1":0.31778054800300737, "value2":4963237153707981360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_5":"k","key_0":"k","key_1":"c", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358509, "value1":0.4977744678036884, "value2":2417156657544845857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.825502, "value1":0.7927543298826962, "value2":7803607945365925463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463820, "value1":0.25407613728207273, "value2":7298874514736284603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_7":"d","key_5":"f", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815985, "value1":0.03419039527127868, "value2":2983524479323907938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648209, "value1":0.5111390924658735, "value2":4469943396931617311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560198, "value1":0.3027175063306067, "value2":8582749896805213520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293707, "value1":0.7381247559612524, "value2":267296433337218156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436607, "value1":0.1678795867934854, "value2":2882160820519571353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516040, "value1":0.02439990616294445, "value2":3459503829097802670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861717, "value1":0.8549050337904404, "value2":5330118776081944380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181801, "value1":0.014687212605387332, "value2":1741497345355098408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398203, "value1":0.3791649455492745, "value2":7827883443566154722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004152, "value1":0.4627140852343998, "value2":5127693472090561791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289305, "value1":0.9473077760244797, "value2":1972308969161346102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.407619, "value1":0.22478804543381714, "value2":4393947727507143193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_8":"g","key_5":"c","key_7":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500355, "value1":0.9148638467317476, "value2":2930801640270658847, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390931, "value1":0.03756752782865913, "value2":6450238118865267877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436518, "value1":0.7302171451231968, "value2":8892033149379515881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352126, "value1":0.6345536180749114, "value2":8076417892766647115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211476, "value1":0.7869548859510007, "value2":2137142575900025284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.620465, "value1":0.04777740137317064, "value2":2493346722675519040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166994, "value1":0.5908047667032886, "value2":2133480026426783424, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799110, "value1":0.7447928303219921, "value2":6367274514210109366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.258756, "value1":0.5720021685376423, "value2":7819250772844902436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178485, "value1":0.22728962622611987, "value2":5810329748220466185, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756847, "value1":0.09118290676810949, "value2":2438579961381457911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447596, "value1":0.2813020748043202, "value2":4446085339672311465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_6":"b","key_7":"i","key_1":"d", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.507148, "value1":0.8635348228135215, "value2":7235169187543778395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412107, "value1":0.6168652299086078, "value2":8460560637760416654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.916980, "value1":0.20506020965429841, "value2":1044309486752695204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358540, "value1":0.5251096014647666, "value2":6638181287796109637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_5":"b","key_9":"c","key_2":"k", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.199047, "value1":0.6942630163772678, "value2":3410541935645339151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_5":"h","key_6":"j","key_1":"g", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.754827, "value1":0.9837922147041048, "value2":3690131697027385202, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_4":"b","key_5":"h","key_2":"a", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.329509, "value1":0.8271404181330777, "value2":4227974754451880800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.513229, "value1":0.672862073275747, "value2":3810363550665232995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910574, "value1":0.9390557529752215, "value2":5133678849432201178, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181846, "value1":0.49307852075676845, "value2":6137025771035845818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572559, "value1":0.2408765387072259, "value2":8298872358883983458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078241, "value1":0.08393417297668128, "value2":8837983950648920911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_9":"e","key_4":"k", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.056551, "value1":0.9755466271988745, "value2":2207490383904730952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.814175, "value1":0.5043585864165596, "value2":5350947527006645760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.066736, "value1":0.5177229679064868, "value2":2703347438865819514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560402, "value1":0.4478041012053205, "value2":117552149576626326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079515, "value1":0.133747701259907, "value2":5643720599026556636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_9":"h","key_2":"f","key_3":"e", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246265, "value1":0.8787847052412423, "value2":8596628070583162060, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.011510, "value1":0.3733460731999948, "value2":6601446014598669145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735379, "value1":0.9116679871732262, "value2":5605502556155572766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.574257, "value1":0.3506079241295861, "value2":1800090074854288106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900580, "value1":0.8930823791186644, "value2":1843480248185265031, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.744935, "value1":0.754485340555253, "value2":1527411985430067052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369195, "value1":0.7284643223167715, "value2":2551804296404453319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500109, "value1":0.1715238288854149, "value2":3468590424262559119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196638, "value1":0.4131391791311713, "value2":2036494112995161182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660034, "value1":0.5162081878070082, "value2":683174465375000181, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.324311, "value1":0.6392052032119463, "value2":6854067358310903979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477405, "value1":0.9223995048777075, "value2":8678575474954356403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_5":"k","key_6":"b","key_3":"e", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894983, "value1":0.2879379950878977, "value2":1760029872301364905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_8":"b","key_0":"a","key_4":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404612, "value1":0.5964910019926303, "value2":5352075776414002891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_6":"e","key_1":"h","key_2":"j", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.406123, "value1":0.9911934759109439, "value2":2505572631856818583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_5":"c","key_0":"a","key_4":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098160, "value1":0.29265324156352274, "value2":673331618670269188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317156, "value1":0.5284207406945622, "value2":5801235018377104530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.278954, "value1":0.8339844494736254, "value2":142783045099103297, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144316, "value1":0.5981739630713123, "value2":4376705426457644175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.533900, "value1":0.7048057967645066, "value2":7668900249344587055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655047, "value1":0.9572743468227054, "value2":8615116979377517835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.164063, "value1":0.12955196466904195, "value2":8720744967395487030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211413, "value1":0.4645239019949383, "value2":1341011821671689315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.254073, "value1":0.9282656526373411, "value2":5675449932048560089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939659, "value1":0.3774328021259814, "value2":7756643154892601396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.789551, "value1":0.7757594275737888, "value2":8309777310097157940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_6":"a","key_4":"c","key_5":"j", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.256014, "value1":0.7599489616826112, "value2":210625827724423434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883947, "value1":0.13842755442158153, "value2":5303830768512577958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516070, "value1":0.17914029301657106, "value2":1056091608874344104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.446988, "value1":0.27173267141582086, "value2":6180352463990379834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.622518, "value1":0.8672352642207358, "value2":6931279306288268314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.002662, "value1":0.25623553544743577, "value2":4114415105877902794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.250822, "value1":0.1691133676662619, "value2":8315894568290440082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148935, "value1":0.5989068031663785, "value2":916798823552491100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404397, "value1":0.3854563629426573, "value2":1729425108588138938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155114, "value1":0.42561573956387977, "value2":3412690494545228649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_6":"f","key_4":"e","key_5":"k", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.185911, "value1":0.7064647650423553, "value2":6268941778110729170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_4":"i","key_7":"b","key_1":"g", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.349779, "value1":0.9255472682273418, "value2":1166345991907977547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.982678, "value1":0.6891602632892445, "value2":1740831383850752805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914669, "value1":0.4165113125409203, "value2":1826795354292330747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871167, "value1":0.7086334187759732, "value2":5543325343112219047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.762690, "value1":0.8505896053437381, "value2":5927978325036218166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.292162, "value1":0.6672535143315814, "value2":1327915299266347941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311132, "value1":0.47941008598606394, "value2":7437258572827323548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523170, "value1":0.7404188223668129, "value2":5505665204112874175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357326, "value1":0.9538155003965737, "value2":2702198679896121556, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.364597, "value1":0.17680019955559403, "value2":1753707621503560793, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452318, "value1":0.2992929781674707, "value2":8362411304781786581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518625, "value1":0.8647905294441987, "value2":5431066155038766988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.355524, "value1":0.6116936780714586, "value2":1234552975837284329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542514, "value1":0.8351923165774915, "value2":1733954151005989610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979827, "value1":0.2892109226113143, "value2":6049795733820800226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883185, "value1":0.011744889551642781, "value2":141159892430595908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.223860, "value1":0.4073953516179897, "value2":6458792499529133548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049058, "value1":0.12482723086065665, "value2":9009567363074084595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.298389, "value1":0.4827417824302383, "value2":4841281853843733128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.629693, "value1":0.6591071306029623, "value2":3355181768485869219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.147730, "value1":0.34536474665893674, "value2":3191505597112674507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.179092, "value1":0.7637896583466016, "value2":1179889369576374895, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.658439, "value1":0.9209377727035774, "value2":3422449294756383390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.071622, "value1":0.28143127213081115, "value2":7766511653375780058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537938, "value1":0.6024534595610256, "value2":8547099742960318907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646788, "value1":0.20378541750257417, "value2":7978976103929383445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747803, "value1":0.10859041477345463, "value2":1109588376642858177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564874, "value1":0.23647658766118684, "value2":6863917686945126643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042244, "value1":0.3078158837684314, "value2":4419488498667340703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.284250, "value1":0.9387293705744738, "value2":6769306683790898588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479123, "value1":0.4851789805412465, "value2":6507016270927900733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025704, "value1":0.8538216321323174, "value2":4020894753483999383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833612, "value1":0.2626123684082926, "value2":5033990782030407302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.881634, "value1":0.935752736774716, "value2":6384512508877447191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474427, "value1":0.8815963618627093, "value2":8929077315195001710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321721, "value1":0.8079053030903706, "value2":7509708901019477626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.854474, "value1":0.6950357389111533, "value2":6524710922297840763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229889, "value1":0.6795085741944863, "value2":5609613021607849758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.988904, "value1":0.5223586661895601, "value2":89977508053442787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374340, "value1":0.755589597933608, "value2":7955537057313043569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639189, "value1":0.7408346971613415, "value2":1902500838340597252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_9":"d","key_4":"c","key_5":"e", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481547, "value1":0.5004511492728786, "value2":164083172923357290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518019, "value1":0.729605886464792, "value2":9068451454110926149, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_7":"b","key_1":"c","key_5":"d", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.363805, "value1":0.7382344993077278, "value2":7045722716338732866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.630886, "value1":0.688064557117447, "value2":296768996976239880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_8":"k","key_4":"g","key_7":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.845466, "value1":0.13213135497669065, "value2":4879659946939633689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660200, "value1":0.2229370900047786, "value2":292379935200020145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073732, "value1":0.8115899146610889, "value2":8577867893469971833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.019908, "value1":0.49801059455223595, "value2":7683222481005280397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.347447, "value1":0.9276573673104045, "value2":2091335224644651701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321409, "value1":0.6126771681057066, "value2":1893407571657392417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.738125, "value1":0.3569881382718092, "value2":4029761814756595416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.665842, "value1":0.05170172426704011, "value2":8174581603042814007, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.449347, "value1":0.7466823594518665, "value2":6257307241351449210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.588421, "value1":0.4753472682300776, "value2":4728406440983577982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.257494, "value1":0.1386821629125743, "value2":332192645116862048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.728108, "value1":0.43976058351020564, "value2":1985435892430278941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.209628, "value1":0.8966889425339122, "value2":5787236380684241489, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.638851, "value1":0.9617304712840572, "value2":4707670589409962460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484422, "value1":0.020940662405038862, "value2":1545134502951937463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440131, "value1":0.17343525627253645, "value2":6009630316902812876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229422, "value1":0.02146978522387302, "value2":2111892615484907893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780591, "value1":0.6377996056471382, "value2":8896668703689799570, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.124061, "value1":0.7970190759525996, "value2":7452754965853045737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_2":"i","key_4":"g","key_0":"j", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.450958, "value1":0.8563337518192949, "value2":1606820680288550386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194998, "value1":0.8092586961600053, "value2":1104653518579575826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746061, "value1":0.888221791516023, "value2":6881584298104407367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.550870, "value1":0.45584434378222843, "value2":7394447363524184561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.684022, "value1":0.3067771551792505, "value2":1245627384806850593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289452, "value1":0.4582593754084005, "value2":5465272523370779338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235677, "value1":0.8840876989101745, "value2":551093905717196985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.890313, "value1":0.09198263253620918, "value2":6028451346704401787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.168914, "value1":0.40840563117271195, "value2":3931349792536134233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.717876, "value1":0.7410792258916561, "value2":5169350845698434472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.202605, "value1":0.5290546996961457, "value2":3475693437398263961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636633, "value1":0.824130818871441, "value2":5903284123492948390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.615008, "value1":0.04083331973299552, "value2":7577674506757874978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036559, "value1":0.5696805792367668, "value2":4633771544842561674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367723, "value1":0.355783015228386, "value2":5675601540429514938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907306, "value1":0.5698956093759964, "value2":9122457118633339454, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685953, "value1":0.4276033626449204, "value2":7616472412744453525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587193, "value1":0.9520532910468696, "value2":2039460897796769985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.322100, "value1":0.9372598841781506, "value2":8025865217929956411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155211, "value1":0.2725517811852112, "value2":7680103519195376114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964045, "value1":0.49419337991273576, "value2":8605447057400658561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.423455, "value1":0.28782941494899417, "value2":7631463815755268967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433388, "value1":0.6722143467332055, "value2":4743404486443339981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073044, "value1":0.9919783220125814, "value2":3546009072923452494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060903, "value1":0.09931972482798261, "value2":3396822536822006988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.952732, "value1":0.3775686488599951, "value2":7121618792612707640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521690, "value1":0.5851273414170087, "value2":4662473906180780029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227468, "value1":0.11958534947502218, "value2":3788112644660727919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196785, "value1":0.8966564462732959, "value2":4357533587354590088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901366, "value1":0.5571789697475874, "value2":7759726578542538969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.633498, "value1":0.9521589976099354, "value2":3412636563124759184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477196, "value1":0.6345321514502629, "value2":2968047284799701894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.245970, "value1":0.8602524837567829, "value2":239564384927138993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923521, "value1":0.1090517952569125, "value2":8200798429042044530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648242, "value1":0.5170091370557403, "value2":2725553204805480592, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812069, "value1":0.8884437199979248, "value2":9154279472010215199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962857, "value1":0.2140414709499265, "value2":5080056778998450413, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.561764, "value1":0.6433379926959468, "value2":1246414196804721612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288689, "value1":0.8910605919928337, "value2":8094720078703937078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.419833, "value1":0.6705423885099231, "value2":6631047350002617679, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.085812, "value1":0.03831227212231745, "value2":2735730201464302258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.046896, "value1":0.09940708362873886, "value2":600170391537990700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.884000, "value1":0.5332485748572247, "value2":5278799395808920103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.942254, "value1":0.6825570254740958, "value2":3505708958239891309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010039, "value1":0.007884797092278711, "value2":3745225427592325971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.712360, "value1":0.2908960224193339, "value2":9147580185923316377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109158, "value1":0.3741670076640341, "value2":1005531285446043532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680722, "value1":0.010722209112304005, "value2":826737562607372966, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.439540, "value1":0.4713372686055574, "value2":6754252699406092830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392281, "value1":0.9898544758145399, "value2":2353492636354645276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205843, "value1":0.6788358030320953, "value2":712272724855746801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003145, "value1":0.7114317643988356, "value2":3716874444298014530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.133514, "value1":0.37586422274898074, "value2":2416361460381490648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.904105, "value1":0.18756095304630882, "value2":6928964256284052382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171387, "value1":0.6820596249231228, "value2":172259903859496202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_6":"e","key_0":"i","key_4":"k", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.510204, "value1":0.06734756409519246, "value2":7819819964334132673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422546, "value1":0.1465987153240839, "value2":3085770726305992826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.039220, "value1":0.39594354666033404, "value2":6309816101603497959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.739730, "value1":0.8530985778840952, "value2":7379770578925142492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.672747, "value1":0.7865961192571739, "value2":7774331371169359686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.769896, "value1":0.802526565041674, "value2":3838614021162591095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.304485, "value1":0.6949310789221889, "value2":7501685449967872481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.678592, "value1":0.7504248678928968, "value2":7218481023042482207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764761, "value1":0.20348098581370722, "value2":5988835601145003290, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.968921, "value1":0.21895244723583734, "value2":6407608012483731307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480313, "value1":0.3910222274917921, "value2":935230396616102910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680717, "value1":0.7679413590875351, "value2":6737105603635406237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843792, "value1":0.928935788622385, "value2":8032690240756723112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544961, "value1":0.3803375288284628, "value2":808064213573953017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974943, "value1":0.2873027943668844, "value2":420303650121753052, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351352, "value1":0.9011332762961933, "value2":5787708950335762277, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049110, "value1":0.920403506389589, "value2":3542056558848585785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.473874, "value1":0.20499658001386006, "value2":6473623955229286943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_9":"e","key_5":"f","key_6":"c", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705527, "value1":0.3369786416226785, "value2":5565389573084278676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977283, "value1":0.5753145958771464, "value2":5509805379215659395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.969055, "value1":0.7944991097395847, "value2":5227490476089768493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.260822, "value1":0.37247993171967786, "value2":9100748121410350084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392300, "value1":0.9170516775216153, "value2":3337181235028553814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.917458, "value1":0.7005088931842404, "value2":4249344436835260528, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398079, "value1":0.4412594689419285, "value2":355338602844953023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605493, "value1":0.14953313696992326, "value2":7522955956227719413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.947612, "value1":0.24504972733419753, "value2":19900356272350185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.657164, "value1":0.7554151847082919, "value2":2083009730678034907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.303410, "value1":0.9047440140278514, "value2":2665949305315254597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720136, "value1":0.02233304294884983, "value2":2148351116720732280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605342, "value1":0.06576250199061187, "value2":3149195196917185744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.249421, "value1":0.10981969871005968, "value2":1489417694228791945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490002, "value1":0.2365349300258502, "value2":5413143921150744151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537535, "value1":0.056469545325025704, "value2":2942905392018720189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748755, "value1":0.5086783291906242, "value2":4857376881146282553, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235928, "value1":0.6784255332464089, "value2":3029395751611067716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.401233, "value1":0.753375539931517, "value2":8169340970187425814, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960595, "value1":0.5301594317579601, "value2":3908827718931366422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113359, "value1":0.7603991477763598, "value2":2119709520548445193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.875780, "value1":0.49587314465392923, "value2":8685987546300538378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.938401, "value1":0.9152835192021896, "value2":1076417540165823423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.869630, "value1":0.7973627027092416, "value2":2126356962780308110, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388566, "value1":0.6382739532774706, "value2":159979324198275701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.275145, "value1":0.5927046376681012, "value2":7080867181269495541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523278, "value1":0.5267214411412359, "value2":1728237597853111534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584632, "value1":0.5083502145633392, "value2":7227399514745308579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964267, "value1":0.6658213068549924, "value2":5519764789585080716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824564, "value1":0.9264931547852524, "value2":3998899671907293853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986213, "value1":0.027645492661625626, "value2":3185626691636394685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634127, "value1":0.858418006506526, "value2":5813619502423233874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.099856, "value1":0.5750664046005645, "value2":7125055104530490351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685619, "value1":0.3275765619786257, "value2":3507230098472264785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174236, "value1":0.36236865121536765, "value2":2097215901740320742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093835, "value1":0.8733207807127527, "value2":5480486627610559582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687722, "value1":0.42815747753178873, "value2":2463297564216975626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015930, "value1":0.035417712050281464, "value2":6849011969803492014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181519, "value1":0.9822679258679019, "value2":408242883289676228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370465, "value1":0.9916450645571185, "value2":3540916962390370495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.557960, "value1":0.814086825170977, "value2":975702025360269586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785667, "value1":0.18117401736664177, "value2":8964384630261548826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014126, "value1":0.6836156690970117, "value2":5150749937802379773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310715, "value1":0.8554125170001826, "value2":6356794838014021178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000193, "value1":0.43591866134299384, "value2":3781675008214883017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_5":"k","key_7":"e","key_3":"d", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310472, "value1":0.20656655062551638, "value2":2155883927623577685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348781, "value1":0.1874457704512197, "value2":3156445090401189095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737752, "value1":0.4107965308340836, "value2":6831391234474352496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.864771, "value1":0.7087377270632053, "value2":3852759604751603741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377175, "value1":0.302211293966416, "value2":3671141882815450126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035362, "value1":0.11868658008033231, "value2":4402958858442974997, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206005, "value1":0.9050640840246325, "value2":1571018380191846388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607719, "value1":0.5276900307886362, "value2":2938372021759409514, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847851, "value1":0.23589306520821895, "value2":8130082209207560079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181853, "value1":0.3318615047159984, "value2":2814234275379593797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093707, "value1":0.8735019416998069, "value2":7839357347839415099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.050872, "value1":0.045154096442812874, "value2":1387216811650422069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635715, "value1":0.7372871363614325, "value2":2653034997948603272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.807390, "value1":0.06360776205204009, "value2":8705617981394656683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893750, "value1":0.033458142425902254, "value2":2550969485909611240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599717, "value1":0.9713785482184008, "value2":1528762568291094955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544670, "value1":0.0942081326101987, "value2":5208646740238503092, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.460630, "value1":0.30899325368963804, "value2":6955190987114158639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348159, "value1":0.8338319484328468, "value2":382561011167415899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_5":"f","key_4":"d", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.866844, "value1":0.4559148181796855, "value2":4921190547959215054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.779864, "value1":0.8408779744303566, "value2":5151627244176235100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087888, "value1":0.9444160436820308, "value2":2636681313922893724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422949, "value1":0.45533312838966733, "value2":8719456700476032056, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909073, "value1":0.4871022495216044, "value2":7470894518121500308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526308, "value1":0.6556527723106481, "value2":2061496006850665904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483995, "value1":0.01917969989791312, "value2":6594173174887504625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_9":"g","key_1":"k","key_3":"h", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.704096, "value1":0.7335839167893562, "value2":7683462035683402688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796245, "value1":0.775721209508694, "value2":1762994312464074505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635089, "value1":0.08634955165534107, "value2":5435729701416464796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.146656, "value1":0.904887030211023, "value2":3845389016759808576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.975641, "value1":0.06011625753112877, "value2":2167561119971885977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.027722, "value1":0.7606346737865884, "value2":7806658536996851894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.255754, "value1":0.8709289771166605, "value2":2946981794368069885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682006, "value1":0.164502402573321, "value2":5375762287724896423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901286, "value1":0.32721693073724145, "value2":1600743249293213477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.844121, "value1":0.4592683330441191, "value2":5355971233932247551, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756958, "value1":0.06556927134836166, "value2":2324918632583888734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.686437, "value1":0.5613972309334896, "value2":1964277954793319022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.280286, "value1":0.210108301927216, "value2":4762745887015662113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300106, "value1":0.3451768742896802, "value2":5756080702879459521, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008833, "value1":0.8860643993351358, "value2":5562038867537669443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964560, "value1":0.5679670350248304, "value2":7270655908716143592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259598, "value1":0.1779142648395649, "value2":1989791279542261185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289316, "value1":0.27910110845131275, "value2":9108304266352627509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.380786, "value1":0.03821154554853608, "value2":5351444755572082025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_3":"d","key_8":"b","key_0":"c", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.619088, "value1":0.014955635682113628, "value2":1506343674187146948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.991863, "value1":0.4507410741879421, "value2":700372855016807730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.819751, "value1":0.47361005562943986, "value2":397573677309043949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962980, "value1":0.5040052190830692, "value2":7266525751259633187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.451541, "value1":0.1442134636404948, "value2":561640599502466121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.387998, "value1":0.5301085061271325, "value2":9134048967019410288, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300013, "value1":0.7373601287356207, "value2":342878676774549894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008200, "value1":0.7728904842469639, "value2":6539748237741452295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.697595, "value1":0.02404028274324282, "value2":5913828782152003967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.726484, "value1":0.45889774925126936, "value2":8474247967864647236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.696138, "value1":0.8390712125923527, "value2":8640587498190477408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.053954, "value1":0.3443648011087006, "value2":6899937907932201153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.702864, "value1":0.09419193983638496, "value2":1509375003037818758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379925, "value1":0.10612710322810476, "value2":7235965724589430822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.065471, "value1":0.03922963726603652, "value2":5920886218601602154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087291, "value1":0.4838942531586474, "value2":535199259720805783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977724, "value1":0.6375345212159459, "value2":4619853255519698247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566854, "value1":0.21378747510647453, "value2":3467239940916600883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110862, "value1":0.6797718237586716, "value2":7914262457466633074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440246, "value1":0.30120840846619557, "value2":3776365109572133674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.797803, "value1":0.7797287504617298, "value2":5037915665887516284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.586577, "value1":0.1709193163609849, "value2":5198104224414552394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879594, "value1":0.7450124911318364, "value2":8773848846938448870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886367, "value1":0.551654132389125, "value2":358026416808351667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464288, "value1":0.7461865171369122, "value2":8085418698319569937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197248, "value1":0.8352566024408136, "value2":5751810192674740519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.217337, "value1":0.19284687675681011, "value2":7864252915572676984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448294, "value1":0.42640546323275524, "value2":168095719577724557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.591254, "value1":0.037748263913037684, "value2":2863811536714848498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372170, "value1":0.6660431538229836, "value2":8567505344595141514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_6":"g","key_9":"f","key_3":"k", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317243, "value1":0.18445456220952994, "value2":4074456277125729011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.267738, "value1":0.2691217808870719, "value2":159261331127552033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585076, "value1":0.21078242008152992, "value2":1239129613801043878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979016, "value1":0.5956340156534701, "value2":9183614150234962352, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447286, "value1":0.10494392100402684, "value2":3770086671966420800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105069, "value1":0.2794799410983503, "value2":3469689160956336636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.173597, "value1":0.6860882854882017, "value2":6360441874685617414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.835787, "value1":0.7543174676127191, "value2":6129127352506150313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.772599, "value1":0.7516717167947417, "value2":8027123646004983290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.115846, "value1":0.3569203207491387, "value2":8576278949637442153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551211, "value1":0.9246333110829817, "value2":898669648143321253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.230459, "value1":0.04953444106043317, "value2":2309701856093198744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060038, "value1":0.7972052364471702, "value2":2490346122315313464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512371, "value1":0.09068245544499785, "value2":3847395208192505439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783506, "value1":0.7070750057247079, "value2":451775490327949567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000319, "value1":0.9107891782905447, "value2":4820343479119697886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628626, "value1":0.016138857727039164, "value2":6452512787294668321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.063279, "value1":0.39758013318800806, "value2":2380900021298276106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480402, "value1":0.014403992075819017, "value2":6901656754464212004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.462954, "value1":0.20570595420817753, "value2":4308695238369802814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308777, "value1":0.46154273334740287, "value2":533630259579000143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.006531, "value1":0.41769996010310556, "value2":1003539425076610908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.606413, "value1":0.8392935388898198, "value2":7138000920468723420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390008, "value1":0.9457032273317953, "value2":8816745864308273066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.545540, "value1":0.9980077865004171, "value2":3431998048103898458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.771291, "value1":0.9269154791448218, "value2":2868685560372963447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035244, "value1":0.15635792222626285, "value2":745780555759481875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.802078, "value1":0.49326419263144117, "value2":7061825021138296948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141863, "value1":0.6394029445292002, "value2":4052750041261099856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210519, "value1":0.4447889112985626, "value2":4491375706281220187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014644, "value1":0.5845639893426985, "value2":8504751940020912407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.457311, "value1":0.39232937571361776, "value2":8948083137978870620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497330, "value1":0.0554441906288732, "value2":8552872946126060990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778960, "value1":0.3021332369906045, "value2":5664412036675630494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151648, "value1":0.7180248405530221, "value2":4322565027390602745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.286112, "value1":0.7877440826847747, "value2":6988799327817384951, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.131604, "value1":0.8048208447034889, "value2":7951591877124931974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122845, "value1":0.26609349854031233, "value2":1608820578917270500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.059135, "value1":0.8932421413801345, "value2":6863045131855298597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.119289, "value1":0.6625474883897694, "value2":3938725492057076257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.114546, "value1":0.1285766450638473, "value2":3866539491041344116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_7":"e","key_6":"i", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134976, "value1":0.9092879262740383, "value2":3783610158682087147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.676253, "value1":0.24518334285373874, "value2":5037863892800296388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613311, "value1":0.5221731263503148, "value2":6112150162991672535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448627, "value1":0.09620166167463585, "value2":7262951598026367151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370708, "value1":0.9034926573578455, "value2":6264024668154911646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694827, "value1":0.4562036337293121, "value2":5132858458452108988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.409213, "value1":0.2400040814231921, "value2":5692228309415552332, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.272515, "value1":0.512618446397202, "value2":4245362498789762636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398497, "value1":0.9796136787594169, "value2":2854381927687220104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_3":"b","key_5":"g","key_1":"k", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573421, "value1":0.25717373724181153, "value2":6071818930618476006, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499699, "value1":0.5015171773350958, "value2":6590026308842608922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390872, "value1":0.15108026004608982, "value2":4054685864574240684, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.187721, "value1":0.11524534470163803, "value2":4129310514528385220, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486124, "value1":0.9297806553036565, "value2":2076740459139610542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.506484, "value1":0.48473773192721936, "value2":5279382479053069968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095317, "value1":0.5302104024468535, "value2":1006511502199967697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659995, "value1":0.7173304922658666, "value2":7272260713937588084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206009, "value1":0.21353139637435428, "value2":4614697991046896838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049549, "value1":0.9736103016824452, "value2":713014430698092731, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764793, "value1":0.5877828915841818, "value2":2743371147788843703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095972, "value1":0.7802447999330678, "value2":1247537151507568078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210056, "value1":0.13206812499303777, "value2":5119237508780620544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477450, "value1":0.6302618943375251, "value2":3505999540579430287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.182076, "value1":0.5504514321159135, "value2":2799023815129557532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262688, "value1":0.9279334258185475, "value2":3960329791115549039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601942, "value1":0.9333313963421735, "value2":8488152958276516414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095627, "value1":0.2423663973685341, "value2":3147375434225291522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325183, "value1":0.9698662748883076, "value2":8348906475599130816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535337, "value1":0.9108408536175355, "value2":548221601502028995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.465394, "value1":0.20049512890521093, "value2":6918344754347341650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805179, "value1":0.04912411768594746, "value2":7498908729880858823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635014, "value1":0.5227012296024662, "value2":6384611575231594987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481932, "value1":0.2492598145582209, "value2":7063605156324549396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.458503, "value1":0.3120534424622104, "value2":219352200445992919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144186, "value1":0.6043714345133836, "value2":5343022569080095368, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148408, "value1":0.7539243961545181, "value2":8912511444056366708, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357993, "value1":0.9968001216010738, "value2":3427619790755264753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519356, "value1":0.2148487452898843, "value2":387702087539510555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631948, "value1":0.4990937312036263, "value2":936214783188826913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631191, "value1":0.03578918325727108, "value2":2415177266799716415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.915725, "value1":0.6672040350791707, "value2":7911702372383761591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898111, "value1":0.2610115750745007, "value2":6604319492118960080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871853, "value1":0.700295668840288, "value2":4280817186446527854, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601683, "value1":0.48046941918756864, "value2":4803180873024223807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036853, "value1":0.059813074330959816, "value2":4421336201693669164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.870282, "value1":0.09570589043315565, "value2":2716621407072678469, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543069, "value1":0.31121787546995405, "value2":1276512731094194139, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367830, "value1":0.09434652170549433, "value2":2256603050688647078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468827, "value1":0.8402294714187805, "value2":299914447978500025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910520, "value1":0.6890435117531037, "value2":2224012689460052929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291500, "value1":0.2581942233541333, "value2":94642526795868830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972763, "value1":0.39798358703466546, "value2":711133600064208053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398873, "value1":0.007238280134283497, "value2":414930643350925093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003583, "value1":0.3709596459020853, "value2":7158538565937127848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288974, "value1":0.0033772263476846164, "value2":3505644260463770547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.989217, "value1":0.7234413065529468, "value2":6102862603791704696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151126, "value1":0.962243743917813, "value2":2796295067408464675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.313076, "value1":0.4725232716261173, "value2":5800110791744456113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.169414, "value1":0.10364598232077965, "value2":8013370755699143486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_4":"d","key_7":"f","key_0":"d", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641143, "value1":0.3874568519220252, "value2":1865317425913683745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960408, "value1":0.6444887127846117, "value2":2021509390881885067, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_7":"f","key_1":"h", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188655, "value1":0.9671102461325771, "value2":4499002439335266005, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_8":"h","key_9":"a","key_7":"e", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438594, "value1":0.23438356733095106, "value2":5880329494012383454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.043142, "value1":0.8137602822079195, "value2":5072736510183548575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566884, "value1":0.3037390436862004, "value2":1427943088173652054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.528095, "value1":0.06744248220950605, "value2":2617279385140322030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.075656, "value1":0.6248261098594748, "value2":3102580216650467270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644300, "value1":0.11077089127760144, "value2":252971746503923271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061068, "value1":0.18796880019702752, "value2":1108553918392505010, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.996996, "value1":0.5752434742054809, "value2":8128574670569848168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015917, "value1":0.06928565567804344, "value2":6283279207443485305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238217, "value1":0.29468486256767473, "value2":4879109468201269125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.302561, "value1":0.16374126420414117, "value2":1267840542849218667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.879665, "value1":0.06081772906299747, "value2":7735358902544562393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.436414, "value1":0.9145077406345081, "value2":725040121642504252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845550, "value1":0.6355646354104764, "value2":2412964869373168042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918386, "value1":0.3307533061620177, "value2":4567564826294647319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310744, "value1":0.5825111373150833, "value2":6000767898591093764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206794, "value1":0.8760021186600452, "value2":3359095928804782209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.237359, "value1":0.7515468979685085, "value2":4507098732286155978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247625, "value1":0.9212103536180063, "value2":1210374309278913705, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.531076, "value1":0.9294917705494895, "value2":6008294356031388873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258474, "value1":0.4801885775502877, "value2":4809296905208303773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687251, "value1":0.5503479981842399, "value2":1159057662539665025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294328, "value1":0.579524047000884, "value2":8591427275775614498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030329, "value1":0.3897745465299769, "value2":8195510768796559399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591813, "value1":0.4311576035013457, "value2":57188656194062821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_7":"e","key_3":"f","key_6":"k", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258460, "value1":0.19968970719511664, "value2":6006706732580329556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153359, "value1":0.6163826197572554, "value2":8075354818413450461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977991, "value1":0.47088424321623057, "value2":1437620300807340754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.554048, "value1":0.1312819884002532, "value2":2639581932761358442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.829720, "value1":0.9986118788512084, "value2":3816923447231661055, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648184, "value1":0.9987080219677139, "value2":7493304631161246772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171705, "value1":0.6479656070812448, "value2":940500361907630226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452962, "value1":0.7379240782521741, "value2":8607242240788556904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422715, "value1":0.29055682202125643, "value2":2609014884729200526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997247, "value1":0.03493075047200949, "value2":1341694184124582251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.454967, "value1":0.4083872840808467, "value2":7327414310312171249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268085, "value1":0.9598378795022791, "value2":9180787761997570552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.185765, "value1":0.869366023599698, "value2":6065348312952390593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954707, "value1":0.29066212545734305, "value2":6500179214577769106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134999, "value1":0.0746058596733823, "value2":3954294580008112843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_7":"j","key_1":"b","key_4":"h", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.056701, "value1":0.08887774822608922, "value2":2624540757429278414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503714, "value1":0.7939975169235949, "value2":1543165523760841555, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.407434, "value1":0.5263350258249397, "value2":6207484012298867707, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.216887, "value1":0.6135120862192558, "value2":6706252188296828465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950767, "value1":0.6258842267416237, "value2":6943474392706233335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409499, "value1":0.39019457036616395, "value2":5543165311657950313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.064967, "value1":0.25118206025688317, "value2":4933404061205221336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503859, "value1":0.6260829223715964, "value2":2553798156705274576, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.239884, "value1":0.0646597153433716, "value2":5357886478310061690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.560239, "value1":0.025368350153306324, "value2":1834804451034467956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021890, "value1":0.4250913620299391, "value2":7901663683744276123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.720024, "value1":0.7102429898685084, "value2":8848853087379978689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_1":"b","key_6":"f","key_0":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721708, "value1":0.267345226146574, "value2":8204105067810932755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.688701, "value1":0.8124970825519422, "value2":5072280932532136578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227719, "value1":0.19432503349760097, "value2":2763726347992488568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398501, "value1":0.6725319245300637, "value2":2029542129845625509, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220707, "value1":0.07663396351127535, "value2":6862161694126124253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464464, "value1":0.8033021261492177, "value2":500776341255510367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.369494, "value1":0.6874411246839779, "value2":5472077652556256405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429691, "value1":0.7205162840867915, "value2":2379554410345555228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.003693, "value1":0.5301205012344029, "value2":7339147645001280473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095495, "value1":0.0031080960946583136, "value2":3948518200809126624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.649621, "value1":0.9796250725140294, "value2":5076285879574984318, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.701028, "value1":0.4664187080974912, "value2":8061147057085676917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024773, "value1":0.7030139999067455, "value2":8201442408389043582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570617, "value1":0.6283295712663881, "value2":7893659755244222642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780271, "value1":0.6223497445918998, "value2":4413418515678745434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904547, "value1":0.28169993275009253, "value2":1702651611066126658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.039402, "value1":0.42131084562328136, "value2":3560241450438323083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.612707, "value1":0.19465390726157364, "value2":3161717756488318325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729159, "value1":0.25669937307380997, "value2":1479715553122219203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813686, "value1":0.049715056995271165, "value2":2079826454687289078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747376, "value1":0.7275373884923515, "value2":2435962270026982603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354388, "value1":0.8636816800782944, "value2":6027722589314572513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505532, "value1":0.1663611987871669, "value2":668594150031699725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087699, "value1":0.09888054207021492, "value2":8612173434517336899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.118211, "value1":0.9755664430251321, "value2":3670406131876856616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918355, "value1":0.2320424600489978, "value2":8375963807463485059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106082, "value1":0.2496270382815134, "value2":3218140633568902603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.356200, "value1":0.22905701139904852, "value2":4580449910037652861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272156, "value1":0.08246894655199775, "value2":8520120895086499443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.445288, "value1":0.13062843592283024, "value2":1544733522378030955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189215, "value1":0.023728801354414537, "value2":7950194843845547325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525278, "value1":0.8120711270842138, "value2":6926776820889632931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532234, "value1":0.2803691083519268, "value2":2750609942680894663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169390, "value1":0.721935860273012, "value2":323165999073131824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465184, "value1":0.14065806381420595, "value2":6373433541582043482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211347, "value1":0.5119890992803398, "value2":437881008737730578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.326310, "value1":0.8982944342798628, "value2":7550941116286319434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767054, "value1":0.058056603449873775, "value2":8172199064443221808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098754, "value1":0.802732462605452, "value2":5531373846445331861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.626701, "value1":0.9658445273400498, "value2":6144529458195478799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382019, "value1":0.6200649361393201, "value2":74370737764383341, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.162075, "value1":0.773701976515762, "value2":3415021911489973103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452128, "value1":0.4165931264471384, "value2":1592331302702594637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546168, "value1":0.9818897837453856, "value2":2770075967419206536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794037, "value1":0.6329308941123749, "value2":1855788356162220067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267186, "value1":0.5183944510585616, "value2":2971715621482767583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.386007, "value1":0.7925028746106562, "value2":5519530031313056293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194807, "value1":0.9560872102829179, "value2":8899546063621825706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786449, "value1":0.25289379431092024, "value2":3323685962729555247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722033, "value1":0.791566877264509, "value2":8569419440028565177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.527726, "value1":0.5123053424894405, "value2":5410088201011323097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991248, "value1":0.00835285017592764, "value2":7876085652270552879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971469, "value1":0.7516893928865965, "value2":1615677872423733394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457203, "value1":0.2802029059377931, "value2":3774880897765407379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429963, "value1":0.4064794421282188, "value2":7288375695455796287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807745, "value1":0.1346588472416809, "value2":7295335243976756666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247446, "value1":0.18437680636683432, "value2":2763564622590822709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505494, "value1":0.36367826246909607, "value2":279237135102501393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570322, "value1":0.807767632694814, "value2":3489957523135662213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.770864, "value1":0.12825646293816148, "value2":559050377523308246, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723654, "value1":0.48061854827645123, "value2":2625015563945376475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939025, "value1":0.31592719165163524, "value2":1661597397006530804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935809, "value1":0.16925319668616004, "value2":8449044609778795224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336027, "value1":0.7017500784708112, "value2":8428599073105193397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904159, "value1":0.7924061448124604, "value2":4266452884457119467, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847458, "value1":0.27361557683078364, "value2":3345769475113460399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378261, "value1":0.7380019020377752, "value2":432938298263940108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799977, "value1":0.25999970547607815, "value2":7119589132321429408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786435, "value1":0.812994267772834, "value2":2642960787391817015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.587571, "value1":0.5078453077544904, "value2":7413095241087719577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_2":"a","key_3":"f","key_1":"e", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189572, "value1":0.33029349839366906, "value2":182717731110127548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128493, "value1":0.3005744554380485, "value2":1813809549706584614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.751426, "value1":0.43021252312166497, "value2":1048575931846703963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.824593, "value1":0.9962985612048794, "value2":3451457313304562908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913497, "value1":0.19667409773488792, "value2":324965724484330012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280371, "value1":0.6558253377067158, "value2":3238482029594886215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953879, "value1":0.11691624934505847, "value2":3461549771600466582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189513, "value1":0.9019750283572279, "value2":7575759643142018621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465264, "value1":0.15533711497405195, "value2":8078121216216233303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857857, "value1":0.2731351567904969, "value2":6433553377051334332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.114548, "value1":0.9868338794536605, "value2":6036587149016739858, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734426, "value1":0.4227117888273388, "value2":1455241552057769722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075679, "value1":0.2701709613339434, "value2":3857058056766739646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104255, "value1":0.981970010025583, "value2":3682624724640104305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653572, "value1":0.008103832880105273, "value2":16321050615205215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.790159, "value1":0.6178089725023603, "value2":2820923101957050608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274981, "value1":0.28628503444016773, "value2":5532572594206077059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274235, "value1":0.01138331892670461, "value2":4146948379716629700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280495, "value1":0.19902196639234795, "value2":7564853449905093412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272293, "value1":0.017257126721765374, "value2":91860645774780383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447052, "value1":0.6776141143437516, "value2":5897324288007611430, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.209396, "value1":0.14657117940198489, "value2":3976211106646613048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850761, "value1":0.5201711080016932, "value2":7354069105129952055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.300233, "value1":0.1860571279843043, "value2":470481327297902326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310707, "value1":0.25674808205958144, "value2":2662172304685648211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766353, "value1":0.3692045132202804, "value2":7757321942348800299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214351, "value1":0.6936702854905948, "value2":8248415695462939576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.005940, "value1":0.9564433372149446, "value2":8800540347032950677, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009379, "value1":0.5129617748107491, "value2":2230603715203037945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867507, "value1":0.6850688842908961, "value2":4341430047057401636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640527, "value1":0.8542276865628686, "value2":6337085201397795074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186612, "value1":0.2520653493154805, "value2":3597439324405314019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168374, "value1":0.9686674775113324, "value2":5086009893745928969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.930110, "value1":0.9065446675547199, "value2":9084895148279172527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385337, "value1":0.5310871446244799, "value2":3634604672435791412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_6":"k","key_1":"k","key_4":"a", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273397, "value1":0.6978323941286791, "value2":6270777752769360572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.644440, "value1":0.7390124331493296, "value2":5633558971491993000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692014, "value1":0.5735092134024398, "value2":4018757059763515593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307407, "value1":0.9523210688679733, "value2":4664291077378357176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.864207, "value1":0.2504856063622225, "value2":5125796995165750818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.891946, "value1":0.8905322399035085, "value2":3790600090207208565, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.196262, "value1":0.02637661052612273, "value2":3345947703698603019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609259, "value1":0.9964561610316119, "value2":4297199572790537149, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.479133, "value1":0.573177630994298, "value2":4352137818057928535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.212122, "value1":0.9374174976714222, "value2":2883686750556684691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_7":"i","key_1":"c", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143194, "value1":0.5493618690016547, "value2":8675397367930704936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.911116, "value1":0.8323148117854684, "value2":3727739579448171557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.100156, "value1":0.7882098153425944, "value2":6270723454145661182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159904, "value1":0.5891414895820671, "value2":7892306327256327213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.561342, "value1":0.575909380567069, "value2":2087294011397500886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.542768, "value1":0.7419123808449162, "value2":5892135507379591161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977295, "value1":0.2537719509110862, "value2":23083079770681507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.690704, "value1":0.08136832639928646, "value2":8018440792149947537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.681388, "value1":0.6173470752976815, "value2":4725502695033185312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045833, "value1":0.9052130764103621, "value2":2335561799986333486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.621313, "value1":0.5578705772605427, "value2":5423853801041319612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362143, "value1":0.058506006415061124, "value2":1952601720041809520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.019544, "value1":0.2454839257683829, "value2":5439837990435419484, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414507, "value1":0.8898999762410835, "value2":9399980232748751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.946393, "value1":0.5231814332055934, "value2":8044307915684488670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640533, "value1":0.1401932575466956, "value2":3489584691123939236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.641620, "value1":0.7793690564908118, "value2":6745229093735363060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256954, "value1":0.9792655297711959, "value2":8340241164033802673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.578274, "value1":0.6786422376056035, "value2":2268577288780537910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168131, "value1":0.7011560783500894, "value2":5754899077173699372, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261168, "value1":0.41241864943729767, "value2":6926145892542625665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.142353, "value1":0.026897262060108916, "value2":3407492707727317952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453717, "value1":0.9480329042934568, "value2":5909842310603335433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_6":"d","key_9":"b","key_4":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.498253, "value1":0.2696408414861873, "value2":9093122739488143709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152193, "value1":0.829008206269259, "value2":7872930012981859271, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147195, "value1":0.40564509426795664, "value2":388387455399110782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.756476, "value1":0.08296241719338777, "value2":8819867312676478909, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472139, "value1":0.10622575406945087, "value2":2389837147036501120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553362, "value1":0.4640810980824681, "value2":9129773762938561987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138568, "value1":0.27153851873742235, "value2":3922539618739320056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629854, "value1":0.06560725045609712, "value2":5242856550347246211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532533, "value1":0.6685532806932011, "value2":4403464406211940116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.102100, "value1":0.30269603070417783, "value2":560972484995423105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.594212, "value1":0.3436785616541505, "value2":6118258015340076117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182345, "value1":0.5829131946786057, "value2":316161802601703652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865350, "value1":0.9510270552078569, "value2":5815202444060769343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.417157, "value1":0.07805271825644472, "value2":1671886271645798176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453428, "value1":0.07722736953435244, "value2":5062577774198478117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923875, "value1":0.3869476521576109, "value2":7644772358804477804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170062, "value1":0.771360835653392, "value2":8029933370964328416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.341390, "value1":0.538519995660261, "value2":6637857221246165272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267222, "value1":0.4666279704151431, "value2":716146669296183590, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562383, "value1":0.35102191528024995, "value2":8428363533739190009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767969, "value1":0.6173626174937589, "value2":6290463280158957319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451339, "value1":0.546379610337649, "value2":1547962669719183472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464199, "value1":0.24671185662577808, "value2":3561721323195761229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.154391, "value1":0.45596856325620033, "value2":6560907831281448338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.709403, "value1":0.6109692366076535, "value2":3486468545935536219, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570798, "value1":0.7335030585660903, "value2":1572258901091495894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.198469, "value1":0.7425214958348296, "value2":3904425769094656355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315912, "value1":0.9245437278546984, "value2":3047808486948520796, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186365, "value1":0.8721078635865479, "value2":6305488287785962360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759050, "value1":0.3487806615217454, "value2":5791750426523247447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_5":"j","key_9":"j","key_2":"f", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991504, "value1":0.5895957831739977, "value2":6282966318152098641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355229, "value1":0.507857444797784, "value2":295494213133771851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551655, "value1":0.4307208831464999, "value2":7265410065885482832, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525491, "value1":0.6404292728685821, "value2":1060635693413562315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008415, "value1":0.6916001000931801, "value2":2353592246165483773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400680, "value1":0.36661865825780576, "value2":9104767548881126156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376594, "value1":0.3981424874885838, "value2":7118511131751820911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553143, "value1":0.6460744131761798, "value2":1887476981066696157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473253, "value1":0.8348910772414322, "value2":7455007366058138980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457579, "value1":0.4708610456466168, "value2":4593338028669686470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433566, "value1":0.16598645827046257, "value2":5257287213674383265, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.357007, "value1":0.7860515830836307, "value2":2972497673348348004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.461293, "value1":0.35370822925539525, "value2":355204159426372577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.975565, "value1":0.6491904274322989, "value2":2226145731856801788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951226, "value1":0.8792346577420341, "value2":5756323624906456991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143238, "value1":0.9818710036776283, "value2":4180983964934518595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874423, "value1":0.3287251148209575, "value2":381928105696021842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648299, "value1":0.1842978254216889, "value2":5567179544229053293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055528, "value1":0.2649298342733898, "value2":3063386077105020906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551525, "value1":0.008929798118195076, "value2":1932022596647743318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078106, "value1":0.6249963076831733, "value2":3775614794809061222, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595048, "value1":0.6556888188689401, "value2":263555282139003693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270487, "value1":0.15652423199780005, "value2":8005167724994681049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986526, "value1":0.44168299905409525, "value2":3616684978189935787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648015, "value1":0.6935400265504532, "value2":3554010850084269714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082643, "value1":0.4464596047653067, "value2":7393355921025489008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.297812, "value1":0.8411130245970256, "value2":3060908524323233968, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.023535, "value1":0.8979883834169406, "value2":3955075066530261651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775710, "value1":0.803115138047979, "value2":1956023105923958421, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276355, "value1":0.35534087510852963, "value2":70554362711904210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227561, "value1":0.3475937884638202, "value2":5770217876057854108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.997058, "value1":0.9920212102015171, "value2":4179786645072950325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_9":"i","key_3":"e","key_4":"j", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330001, "value1":0.7117987721576648, "value2":1416148178901813673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.859492, "value1":0.07088427876105753, "value2":5628499584516778303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.758514, "value1":0.5315142010423931, "value2":3999275359211756172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567291, "value1":0.5976846753929923, "value2":3568009536081310459, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914132, "value1":0.8979176888984427, "value2":4778372357738152583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.470387, "value1":0.2136647847272661, "value2":4632422950916722092, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.180567, "value1":0.7349455061346736, "value2":4238367012667671950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444766, "value1":0.35029664162897833, "value2":4149159141941833270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.351876, "value1":0.719839928132912, "value2":8311035304279403168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.343283, "value1":0.5403744774398413, "value2":7831018924301288402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202600, "value1":0.07323861368397915, "value2":7512140161978615708, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953729, "value1":0.4072430540967121, "value2":6162477020927495810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206750, "value1":0.7276384026815652, "value2":3867378392166470570, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313521, "value1":0.19760060640640484, "value2":1845943836170488560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774075, "value1":0.9632484532837077, "value2":2186178755453106945, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228120, "value1":0.1138891785769252, "value2":2119625663561385331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355712, "value1":0.12321943936422765, "value2":5732137663228356419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.004820, "value1":0.1110884226197388, "value2":2265343653685159151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.630751, "value1":0.6269969213669998, "value2":3359610757917670818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.015500, "value1":0.20581219173107831, "value2":452968813923389028, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646040, "value1":0.14066092038312122, "value2":6343927209062846133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220902, "value1":0.6865529990306172, "value2":1648919416099165702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.700811, "value1":0.4621690976315305, "value2":3551937779376072568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607198, "value1":0.8914891421096152, "value2":246252330281719887, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.574035, "value1":0.7865743679720645, "value2":5713754277685792561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_7":"e","key_9":"e","key_1":"i", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473497, "value1":0.13845795308548414, "value2":259488391880499746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944779, "value1":0.9609062478569093, "value2":5705930854457323614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775443, "value1":0.1259913054905557, "value2":5080990926603035088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010365, "value1":0.4903104589413477, "value2":3314614042137410585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913172, "value1":0.7864192987989552, "value2":8344919384469197086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.796557, "value1":0.037094581426906255, "value2":9011770804417776349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.314769, "value1":0.8877112795540667, "value2":6683425457896895076, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.806104, "value1":0.5962447685862431, "value2":5435268152791547936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579388, "value1":0.6885293468734471, "value2":9064761548833010698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.391022, "value1":0.5900734892123236, "value2":4669259037774416678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762980, "value1":0.7685532202125763, "value2":2315953471058650483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.135687, "value1":0.32474681440848086, "value2":5755607255133419339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104934, "value1":0.14589803567481788, "value2":126178294464524762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.052690, "value1":0.593421743400015, "value2":5119197034484564414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870007, "value1":0.4673411129128983, "value2":2860543686633489085, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629308, "value1":0.4498616996591619, "value2":5328613189051877451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711415, "value1":0.8849559918426787, "value2":5903543494874730971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901063, "value1":0.5349473283169682, "value2":4797032563086773247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.651731, "value1":0.3127106327077303, "value2":5700343521566497813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825358, "value1":0.7283436835171048, "value2":7844311848127264742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.956544, "value1":0.8853976152911399, "value2":5661939812745981161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235737, "value1":0.26141150008124164, "value2":1680083033155058708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.537374, "value1":0.4021256295776066, "value2":412217863647126208, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238448, "value1":0.20048610554803628, "value2":6383291537020572135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506422, "value1":0.3812390668975348, "value2":184701845674968962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.456772, "value1":0.12479606082790685, "value2":7770738457381111114, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.789484, "value1":0.7412786023699602, "value2":492664655073083816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931307, "value1":0.2536425784942127, "value2":3030103981590386173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610031, "value1":0.5842291904640166, "value2":5028658995770055300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.068107, "value1":0.6578126156370956, "value2":4352952521986270134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477620, "value1":0.16120256759137608, "value2":8502715215777515370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.124418, "value1":0.2689168702571952, "value2":404701828083799050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.779829, "value1":0.4388821199327946, "value2":4899529018433185166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800101, "value1":0.2714233939432663, "value2":584157730559703393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_7":"i","key_0":"g","key_2":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860807, "value1":0.4135766617320191, "value2":3872756991712201227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.917667, "value1":0.2149289211585331, "value2":1070350813804397058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214604, "value1":0.6792613272879433, "value2":2691750153580462441, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245813, "value1":0.6913342601180135, "value2":6216928639197666035, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087487, "value1":0.9301668438033497, "value2":6232780192940116715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.069143, "value1":0.6293446581971062, "value2":7108419812354921929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635591, "value1":0.9539262040674916, "value2":2954917330417044389, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672064, "value1":0.8389645710809651, "value2":3571122666192720690, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818753, "value1":0.0576648702049322, "value2":450051931472461981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270657, "value1":0.28617962738097397, "value2":7674542934415081242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726745, "value1":0.2340744755556526, "value2":82189729669291091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.897029, "value1":0.2638345630149327, "value2":8238518585440831118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721815, "value1":0.7091199604872773, "value2":6366310396364171134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.130557, "value1":0.49724111456500786, "value2":5852311186811706792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447829, "value1":0.6230142507141527, "value2":937535691807945647, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.802396, "value1":0.6394461194574114, "value2":7242948197847068875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.424144, "value1":0.6000896127174374, "value2":8028530164864078540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.426986, "value1":0.7003205100320555, "value2":2322106140405745021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942361, "value1":0.016353745384231323, "value2":4629676907790639064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653679, "value1":0.37062297658466353, "value2":8208981100462487617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.992030, "value1":0.7072168752768998, "value2":3820991654654691772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.346953, "value1":0.4620611667835753, "value2":4760146202938852105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321463, "value1":0.9135126280713665, "value2":8417775919956392807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168908, "value1":0.05729213648419359, "value2":8181061490241314804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.230852, "value1":0.17085551083436254, "value2":3688016011930153595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.066979, "value1":0.33115248245947465, "value2":398734641986366420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825110, "value1":0.29481497745263574, "value2":7623192343447837676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.979130, "value1":0.18961515202216345, "value2":6037869636552702023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692641, "value1":0.4102441559202999, "value2":2079215473977001869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459527, "value1":0.38418621904384953, "value2":8455919021737289595, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924379, "value1":0.831174718392, "value2":8455941435197243693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433952, "value1":0.6847864876960775, "value2":6021961695231181621, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195096, "value1":0.9137826039265785, "value2":1951159443698721147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.392032, "value1":0.7140691740516873, "value2":7387270195217478014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711603, "value1":0.30553043110809663, "value2":5447610537284294754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024988, "value1":0.636153947206049, "value2":6011662688198487566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140076, "value1":0.9829818790209834, "value2":1992400557046890911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591682, "value1":0.734180708126354, "value2":8165041750078897088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.254962, "value1":0.4530139927706582, "value2":792701874387399785, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653418, "value1":0.6853469695508817, "value2":2445415548541311714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.406935, "value1":0.49945705352987035, "value2":4326827190201670587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140154, "value1":0.19200910536315854, "value2":7788114053604406936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774910, "value1":0.775205817556658, "value2":1778011502106771934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224005, "value1":0.4109368947961367, "value2":5090636250181538696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025070, "value1":0.8639998386130291, "value2":4097020029155871346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.110800, "value1":0.46475105418540896, "value2":7930781849523032659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502796, "value1":0.19123829931308983, "value2":4926709577891896623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_8":"g","key_9":"c","key_4":"e", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.483311, "value1":0.21080006327277256, "value2":1394785449775081074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194479, "value1":0.7780668170411426, "value2":3729639098545975029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_1":"c","key_4":"b","key_0":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385848, "value1":0.4299644347219865, "value2":7191790268699040877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211519, "value1":0.32521468058588077, "value2":2791560527192035681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228801, "value1":0.83757037869911, "value2":7191574037900046957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_3":"e","key_7":"e","key_0":"k", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.752930, "value1":0.8040696176893675, "value2":1942969218455296906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951018, "value1":0.07747976080254226, "value2":1630225104393013299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_9":"g","key_4":"k","key_5":"b", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.750256, "value1":0.816008410350239, "value2":6724933613410762759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583564, "value1":0.4508316797935446, "value2":6917556384524072348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697470, "value1":0.8171672522091267, "value2":2060386169134326462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.974221, "value1":0.7951082158998446, "value2":153021997721922670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134841, "value1":0.655263428147034, "value2":6943977496854991893, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_4":"b","key_9":"e","key_1":"f", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.673465, "value1":0.16441674918821836, "value2":3664243165148767961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152562, "value1":0.26528241288165916, "value2":1119343360475809102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.420604, "value1":0.2336675869017092, "value2":3279324159460647609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.622036, "value1":0.6846769209119129, "value2":7351022317760422092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687001, "value1":0.06651364619191985, "value2":4264970580734046753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106345, "value1":0.7730270909594562, "value2":6259010316309812350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018243, "value1":0.39163099713401783, "value2":2052648194611616106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576138, "value1":0.9602266945474953, "value2":5355810905366063269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398686, "value1":0.6503809877595728, "value2":8153069195959796847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.367754, "value1":0.6043902819598663, "value2":3335160234939115824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.089576, "value1":0.7877453973247192, "value2":7617166012064737014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.112529, "value1":0.42004273729990893, "value2":2084869905228279863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262886, "value1":0.6795217484416415, "value2":8937461554206260283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652433, "value1":0.15404027924570762, "value2":16114496573329591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.285612, "value1":0.36648611710364465, "value2":4517711436918873094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.213223, "value1":0.283682948749034, "value2":1960872184236727731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_9":"a","key_5":"k","key_8":"j", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096677, "value1":0.1578537002758255, "value2":7008414275581832390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_6":"a","key_7":"d","key_1":"j", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.900633, "value1":0.993822419434507, "value2":3303474850432526688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862995, "value1":0.2925883436045491, "value2":4890951234376924728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.038886, "value1":0.18850439286651008, "value2":7964663822924622191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_8":"d","key_0":"e","key_6":"f", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742880, "value1":0.8063312642154098, "value2":8220352304335077837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_7":"a","key_9":"k","key_3":"f", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545517, "value1":0.023471717745134867, "value2":5984820965643364202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.079979, "value1":0.6088556886509169, "value2":8961288376687773441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409232, "value1":0.6445899053375183, "value2":992192951010465319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224358, "value1":0.13573773649022736, "value2":1165195770092489984, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211580, "value1":0.8798345576210722, "value2":2259526712170036956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.137602, "value1":0.5470546586447269, "value2":2196698895749095736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950011, "value1":0.5109493788687889, "value2":7352128446573662528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098941, "value1":0.972587647595324, "value2":4702835811693986573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036022, "value1":0.7940355478266105, "value2":3489006783771959904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.257286, "value1":0.6629731808259061, "value2":6607686064851284173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283256, "value1":0.4735441861177576, "value2":690002256366468336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729363, "value1":0.5152988869648636, "value2":7191544072945571987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.671685, "value1":0.7719299111978523, "value2":8611060565403129118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585682, "value1":0.23795953447079246, "value2":7946332098229864997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.187797, "value1":0.09090886526352324, "value2":470342401691160594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.617428, "value1":0.5223785659162559, "value2":5891162691146013349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.053786, "value1":0.9753973493719631, "value2":1732499838549362952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.830052, "value1":0.2400470925507307, "value2":7569204709479310112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566485, "value1":0.9031315965779712, "value2":1637870392526502935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104166, "value1":0.8068833505635166, "value2":5557577539110699594, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457581, "value1":0.31154508147136156, "value2":3306826589254236098, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740978, "value1":0.6517923386066168, "value2":6929527434345999196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398554, "value1":0.7163746669385104, "value2":6895927508923594471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174253, "value1":0.37937402384141433, "value2":1418065909790834443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168613, "value1":0.8552112788507821, "value2":1201668238332263018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_6":"i","key_1":"k","key_2":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376103, "value1":0.7428463858250676, "value2":9210155210384521495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.798751, "value1":0.16215960908856522, "value2":4916478620663149838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.588227, "value1":0.7494081995223059, "value2":1969994590907491188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270909, "value1":0.8815916954125361, "value2":2264317244685656881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.466310, "value1":0.09496768292246209, "value2":5092383460942749308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543412, "value1":0.9329719667021983, "value2":5844328151679816255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.834516, "value1":0.9248915367132075, "value2":5869247588035820642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.916922, "value1":0.052803170907048405, "value2":105752679780838825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583307, "value1":0.04534276867452472, "value2":1081545276806182474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.957758, "value1":0.1893001037569564, "value2":4530859851365168030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723091, "value1":0.22925973916873355, "value2":5896808329469448104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.719902, "value1":0.5038773612977684, "value2":5428690516490754673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025587, "value1":0.7184099655587061, "value2":4108913234892685798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907580, "value1":0.24492386825368684, "value2":1156962833669313399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958121, "value1":0.9818175985414233, "value2":3857893912179537366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.657555, "value1":0.17952027005330515, "value2":3409773579631767064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829662, "value1":0.9863896179595923, "value2":7017412361402418209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382356, "value1":0.13377486891358994, "value2":2261611253942664564, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.735596, "value1":0.3414280491857885, "value2":1656516704640942493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.365266, "value1":0.08360581432684039, "value2":751089660510969379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.236047, "value1":0.6527140947106955, "value2":7257801694535836049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_8":"b","key_1":"i","key_2":"d", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.563096, "value1":0.4425122630203769, "value2":8109435442419953734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870160, "value1":0.7097748830786019, "value2":6394016408878658736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853311, "value1":0.7865716590956611, "value2":9110725490965710227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.103882, "value1":0.9153251610630555, "value2":5340001815935707095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.319691, "value1":0.32338872565142723, "value2":9182777252636363483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971522, "value1":0.5621974159988574, "value2":6803454508761360177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893225, "value1":0.5781000116088169, "value2":3230113190945788558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.910026, "value1":0.6955188069638236, "value2":3766609848094762934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.680766, "value1":0.24230859153208487, "value2":5617809008491748996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.493463, "value1":0.1697890841270668, "value2":8556153147754124462, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.849699, "value1":0.7552917967316483, "value2":2395400148768820203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091646, "value1":0.5496353344673094, "value2":8512525001992418076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104484, "value1":0.4174390270830882, "value2":8422771009020841833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210780, "value1":0.8954537390033984, "value2":4547239076695924457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.882314, "value1":0.5142219879570834, "value2":7919864591569880176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.983791, "value1":0.42885136319724737, "value2":7969816758490279476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747300, "value1":0.4699241249924091, "value2":4491569276746468237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939701, "value1":0.6800950779540557, "value2":3198358625582737117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261442, "value1":0.4575968148914831, "value2":6477186909414818810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256111, "value1":0.693918133492274, "value2":1509539897942915838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893394, "value1":0.31628967903294547, "value2":5501152751179175693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742786, "value1":0.9162968056386509, "value2":9070857057311832045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.642040, "value1":0.9183764345745785, "value2":1183434137529208194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045625, "value1":0.9635794700047972, "value2":8991017817592303034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847013, "value1":0.07723071092189422, "value2":6180620156015906931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532846, "value1":0.5943541460929466, "value2":2462231763403080171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514157, "value1":0.7898774054205837, "value2":9059207414611927294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_9":"h","key_0":"i", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.044776, "value1":0.9033139820464094, "value2":6478344621702437696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043799, "value1":0.8974102840071708, "value2":4685725453168510298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138572, "value1":0.5940072973710391, "value2":9150962581658325077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.215236, "value1":0.35464434283570645, "value2":2728457088873951162, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545063, "value1":0.4562894763140951, "value2":3160291795997224895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.277361, "value1":0.9620522510409097, "value2":2528750129940465586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477478, "value1":0.5247853848955957, "value2":7252619165572966740, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.940911, "value1":0.4288772636544275, "value2":1739941452377564494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.488981, "value1":0.07801581197604958, "value2":3785496084335402474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.037817, "value1":0.20985274999200432, "value2":3757407018866875990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477559, "value1":0.26133928393072436, "value2":4699711244003996922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.107051, "value1":0.5035109860681579, "value2":2406818127100942026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692353, "value1":0.44174492856895586, "value2":5849623391254459092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.948736, "value1":0.6131752371911697, "value2":8355568973826156875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562536, "value1":0.5314554437999132, "value2":2325922727970832181, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336595, "value1":0.8477764608771982, "value2":2899112579581457318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609668, "value1":0.7984731186839654, "value2":1035702079046481045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.446414, "value1":0.8536378880203014, "value2":1450863797869138094, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.345841, "value1":0.005289676671501025, "value2":9055262701472338992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.665956, "value1":0.5725507407225954, "value2":2388492838265413703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.972916, "value1":0.6568884959214099, "value2":4110169872571045957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.370968, "value1":0.9171054847132539, "value2":9195948027770005199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109600, "value1":0.6976295242986092, "value2":3260150427416206815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722444, "value1":0.756396263144578, "value2":5473984714896425934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.011414, "value1":0.35518020333634837, "value2":5432640933038728748, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.442479, "value1":0.9135774445018765, "value2":1658872104403529722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860707, "value1":0.1419422977742929, "value2":5745627115489673481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.925502, "value1":0.19711051262279727, "value2":4597556979704859536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.636414, "value1":0.8593669141574183, "value2":3782467267185587990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157759, "value1":0.4115235495685266, "value2":3223572108511746336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399657, "value1":0.24629418245237128, "value2":4491685955930394692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.941994, "value1":0.17062270403788604, "value2":3592023630547953731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282191, "value1":0.8843261061566057, "value2":6691527928807377675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938895, "value1":0.7167094005927336, "value2":1624441217258843945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492025, "value1":0.09243530358924243, "value2":1564407566445130438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837030, "value1":0.7918808546682014, "value2":6330206212330805579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.123530, "value1":0.594358060271596, "value2":1086840755761366155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.515405, "value1":0.8774866546466452, "value2":6471463500843130242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_7":"h","key_0":"b","key_5":"g", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814527, "value1":0.46973250770262637, "value2":617895520080191082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.712491, "value1":0.8366069393700927, "value2":8124214803409242134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.598705, "value1":0.13529834276697159, "value2":4048137400971693133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.115099, "value1":0.013374612744122648, "value2":4497239763994228002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.871918, "value1":0.5477513867264745, "value2":6929880807654898865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.600230, "value1":0.06263242199896547, "value2":7099533171627178830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.832053, "value1":0.9159517384120022, "value2":6068004913866310526, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497324, "value1":0.7077689276515862, "value2":8496454581274124817, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912067, "value1":0.9966402570764005, "value2":6508382435383037912, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394175, "value1":0.4056622098754871, "value2":7483291783499825043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931738, "value1":0.7229091089946994, "value2":6319491958713592651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.778862, "value1":0.3546846056067059, "value2":5823244617744051085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.571315, "value1":0.32603322678558494, "value2":617900084765432670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.022106, "value1":0.09634190796880056, "value2":1538435119318752571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777759, "value1":0.9991415497234644, "value2":6469274665586859562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813159, "value1":0.6009731720422214, "value2":5681496788640905646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990295, "value1":0.866629146873839, "value2":454920947769684349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577898, "value1":0.4068627360956291, "value2":8354382964548721361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652554, "value1":0.25976343730256174, "value2":4258649045980411819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031275, "value1":0.4835922460947118, "value2":1965749848088972278, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535728, "value1":0.0741614372394035, "value2":2365543114658538202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.725451, "value1":0.08897749963143643, "value2":2568717794069312737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517633, "value1":0.31412074219954, "value2":3756199754285644161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170595, "value1":0.31501680305424884, "value2":835864889280146724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.839334, "value1":0.8860221287236891, "value2":7263801537900469082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874807, "value1":0.37435025928000243, "value2":7922739501753662605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914081, "value1":0.7766285758384013, "value2":8262805777914063347, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.668141, "value1":0.6740415300239384, "value2":5791687526396481300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.302177, "value1":0.7738583815242437, "value2":4861664383009090324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711096, "value1":0.314472985957645, "value2":8509002774444373125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321149, "value1":0.44826985902573663, "value2":5704588376153110492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.438786, "value1":0.9611261969930821, "value2":8417802456083541652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894152, "value1":0.47571072484930443, "value2":22587301710957954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939556, "value1":0.9226156153333962, "value2":5568759137623866023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159870, "value1":0.6737683215814319, "value2":23284244914642959, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273159, "value1":0.7150778132424469, "value2":7795879578675436289, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.116050, "value1":0.7176538032312242, "value2":6236044932175094049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_4":"c","key_0":"k", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274847, "value1":0.2989651835291124, "value2":2096206477458603718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764340, "value1":0.5947263013675924, "value2":949703240129275173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_3":"i","key_1":"a", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.073476, "value1":0.1345576666087966, "value2":2417122739684294792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418445, "value1":0.8647442373572726, "value2":4787011153359227960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721780, "value1":0.31241733378602804, "value2":2882725760348945089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338609, "value1":0.30106180327338455, "value2":4145007555047407199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109899, "value1":0.7817730958310003, "value2":5914588161487452188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.547269, "value1":0.29274410504915255, "value2":3689334620249634917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741862, "value1":0.6895772402009777, "value2":4687778247837546339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_5":"b","key_9":"k","key_3":"a", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.744160, "value1":0.7533050071878981, "value2":183319309026503278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.880890, "value1":0.8337197335863336, "value2":5434700933804867558, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467450, "value1":0.7739450951598513, "value2":8969122970020127825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036765, "value1":0.05604892859180446, "value2":5414621267015229038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458853, "value1":0.3999783567682559, "value2":7084993602991838778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.144253, "value1":0.945939865721221, "value2":9151551595196013169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_3":"b","key_9":"f","key_2":"i", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.519078, "value1":0.28562220040037745, "value2":3466327056358787385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.425448, "value1":0.1561471042921615, "value2":8555609213379362480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.071015, "value1":0.867937414704974, "value2":7843796710543352776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936460, "value1":0.8572536817998362, "value2":3953769786680934648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886358, "value1":0.2289778721765615, "value2":7617650297619769040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.029905, "value1":0.0015729191075447354, "value2":2294660572707247909, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777761, "value1":0.9646894479758846, "value2":530724906601952281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_6":"e","key_3":"i", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294885, "value1":0.6419477950120894, "value2":3626154680940200317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_6":"g","key_3":"c", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210708, "value1":0.9629469371323777, "value2":1825251501924013732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354794, "value1":0.9560891897559984, "value2":7402084087841622412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.007599, "value1":0.8488357845508476, "value2":9129548229641808509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.949198, "value1":0.3153555717941824, "value2":7434568718980757764, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373802, "value1":0.3277196318709405, "value2":3450473392216255589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.815211, "value1":0.19640953195392433, "value2":6991753246444170561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532006, "value1":0.8004824485602414, "value2":6264834568239277372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.745472, "value1":0.20136170467216696, "value2":5514377914099787271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.296097, "value1":0.9552161212557331, "value2":1452145138149816534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350623, "value1":0.4834905958063239, "value2":3177892637454509046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894862, "value1":0.26274803670489, "value2":5904879356296841126, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663989, "value1":0.4617229653712207, "value2":5839673609391532826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920142, "value1":0.4202645422145252, "value2":6322617270554106528, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174524, "value1":0.7616258127751694, "value2":465160921360548556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829492, "value1":0.9553481357587528, "value2":1445392053101953093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.419698, "value1":0.7725216991307152, "value2":4285351248890366822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235468, "value1":0.9784345273084265, "value2":3668509819811778630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935515, "value1":0.5182554276152802, "value2":3295309486606700899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539034, "value1":0.37544115144770396, "value2":7306450144895271272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729718, "value1":0.8597111754781319, "value2":3238304594948466970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538038, "value1":0.32408768170822555, "value2":2186207806521299657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566784, "value1":0.4892156459829451, "value2":8596226688481136458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.980628, "value1":0.8327067880954203, "value2":8561137068463144173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153522, "value1":0.15467129518207473, "value2":8602088790336601513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748640, "value1":0.18204671422495655, "value2":6797988693835752844, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247682, "value1":0.09249535842888819, "value2":5067988675442272893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936986, "value1":0.9192193881284415, "value2":5123985662043884348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378293, "value1":0.815846067412878, "value2":6070693823656037795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734247, "value1":0.49161995996865615, "value2":624464230830574462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.661797, "value1":0.2056464067499432, "value2":6156118294329348494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.033052, "value1":0.35405594685495784, "value2":520776464703947921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.575933, "value1":0.9335712679806217, "value2":425476291317360309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.732828, "value1":0.4700815629744225, "value2":8550463063060596020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.899613, "value1":0.4385944862087238, "value2":7654768751774889730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165683, "value1":0.16899343532059313, "value2":5688582074969409894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986964, "value1":0.253512537787597, "value2":8166110434329136902, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766787, "value1":0.6248495968383331, "value2":8035594691046262470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338227, "value1":0.6533144293221477, "value2":3922786069578896774, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912364, "value1":0.888149605868496, "value2":7372081699617979857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_8":"f","key_1":"a","key_4":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.223399, "value1":0.21329948027080436, "value2":8415945829680890770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.922973, "value1":0.6987095535905381, "value2":8304144963505893865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.484292, "value1":0.6786625244755433, "value2":595838710724334047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977202, "value1":0.8656898577077133, "value2":3529785897914282727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.231129, "value1":0.9035229811237685, "value2":2477457900157688998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.565357, "value1":0.40499024658110055, "value2":1356497801667980388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.691153, "value1":0.5098765403884086, "value2":2450630350992474672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459909, "value1":0.07976566115765178, "value2":3732709013710166299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104179, "value1":0.3445131016649922, "value2":528513889221012225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764200, "value1":0.1348858138724795, "value2":409727276676244812, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288182, "value1":0.46853874054271116, "value2":6728001446241512599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502298, "value1":0.31577497007213595, "value2":8859554815564990855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990152, "value1":0.3102588503519357, "value2":5723742327078842798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520176, "value1":0.15090023272414005, "value2":5438285255049886987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.155921, "value1":0.6850691286515489, "value2":7357616240486203605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350044, "value1":0.8819289168133723, "value2":4656926419487785926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.383040, "value1":0.8254228746494352, "value2":2036646364854508134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451480, "value1":0.9507122214170446, "value2":4089393907406910121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516899, "value1":0.7707382433506376, "value2":5274734231531309553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467177, "value1":0.603964532592086, "value2":1716897544081104224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264870, "value1":0.9573825263948318, "value2":3412285010274210994, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.556842, "value1":0.17542611082882992, "value2":1043370942216175800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009549, "value1":0.21407922779664104, "value2":5715745607073532896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462287, "value1":0.8132271496462169, "value2":7467232826614980432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538632, "value1":0.7627133328751289, "value2":2219523491939336599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.217749, "value1":0.9344741716490049, "value2":8166313022724995390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811552, "value1":0.9190437296914767, "value2":5885018846694177935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814548, "value1":0.8390440310277583, "value2":5493133492272371657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.817102, "value1":0.6598894636593393, "value2":8537596977440503466, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.067200, "value1":0.263575315223174, "value2":2125629267937428501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_1":"i","key_5":"h","key_0":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082507, "value1":0.6268250434274911, "value2":9058686535109969227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734195, "value1":0.3167788157158676, "value2":7432987222538757392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.795085, "value1":0.4488172699242242, "value2":4473525864538142538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_6":"b","key_8":"i","key_4":"d", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.332775, "value1":0.8505535842615966, "value2":6047205731470921861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727338, "value1":0.8876718108712484, "value2":1531222095185940701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.387878, "value1":0.5788599279069843, "value2":8604920509885625807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.233954, "value1":0.6310172938155442, "value2":8925576745188878375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.805738, "value1":0.9330396354121838, "value2":1424947937953535481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726267, "value1":0.32286499179099665, "value2":3766446177817631940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229570, "value1":0.9916323313225612, "value2":3888482529661188248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167559, "value1":0.08178718223938254, "value2":7520256881259787943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845474, "value1":0.6808433459458884, "value2":7844247683138918907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400844, "value1":0.39452808620697233, "value2":7357453563119638881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206854, "value1":0.9379245513406866, "value2":6598367899784187154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640341, "value1":0.11560748654575319, "value2":414342029075207599, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497616, "value1":0.26256799403609016, "value2":7366324474884701561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.822083, "value1":0.6049405008185211, "value2":6275944038502753172, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.819954, "value1":0.373947872013572, "value2":3355608188213748317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739894, "value1":0.06143600864431769, "value2":309096149476303416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.684612, "value1":0.7692595721499597, "value2":2629612954063953162, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958165, "value1":0.6185066606297357, "value2":2824340682692220488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944290, "value1":0.3864309642344103, "value2":6500366333535522458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576232, "value1":0.6603219832087598, "value2":7174737056650755203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.047735, "value1":0.4501836136879949, "value2":7043766590114442160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282665, "value1":0.3628575018664628, "value2":3673366331802014852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.590235, "value1":0.30332064856581503, "value2":5473820163155790307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995108, "value1":0.1854914610326227, "value2":3604467487566582522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270312, "value1":0.9632251653093951, "value2":4816282908135882739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541752, "value1":0.2581250654809775, "value2":6451610961696076892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535737, "value1":0.3163976646275719, "value2":2322386485534186745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610988, "value1":0.49088683208094147, "value2":4479251775302671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.111345, "value1":0.016089752487428322, "value2":4062002565116218346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.380428, "value1":0.8878960076695425, "value2":7482838775142667379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_5":"a","key_9":"i","key_4":"f", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.286191, "value1":0.5281244570210905, "value2":4742826162899756346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.633864, "value1":0.28655049708989444, "value2":2281246548041472554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517718, "value1":0.8760337085208192, "value2":6876760617398624261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275264, "value1":0.8763572333888452, "value2":2040006649964380357, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.133688, "value1":0.03683553631174895, "value2":879315984438783745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.178775, "value1":0.7921271023115032, "value2":8974452178625132408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.559621, "value1":0.6842414958291543, "value2":4473030242509416772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.933147, "value1":0.42656042046254644, "value2":6875784776532081765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.265632, "value1":0.6577298952553573, "value2":4117148991099157160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_6":"e","key_1":"d","key_5":"k", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.291705, "value1":0.35063088013808896, "value2":6022154315116551060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.486399, "value1":0.9066996869066141, "value2":3890435706555388518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539080, "value1":0.6597581253474917, "value2":7149394823928068405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.172124, "value1":0.8972702809236033, "value2":5889011931565346539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399820, "value1":0.22420487156545466, "value2":3632173066141869984, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857491, "value1":0.658023411947534, "value2":5044439445561970794, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.960983, "value1":0.8072960706443908, "value2":4799697070254932404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672380, "value1":0.6734321372300892, "value2":7080832382541851744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.699212, "value1":0.7095725169760027, "value2":1829131464356530776, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893830, "value1":0.2302554104612147, "value2":9188062021573390442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520337, "value1":0.16075557090608333, "value2":219133269848494495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173477, "value1":0.6892053752715795, "value2":2151318569219964336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229017, "value1":0.9691975492787838, "value2":1366609608627051856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659208, "value1":0.7460794334063994, "value2":817106321355418532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.361925, "value1":0.05687743883366658, "value2":7921175806067397073, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.523858, "value1":0.7866547024502999, "value2":4071070330904780510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920001, "value1":0.5740949320374887, "value2":487306006337280717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775196, "value1":0.1341448972829432, "value2":7272783478294294123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000330, "value1":0.1260519175980482, "value2":9041625229579242321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313318, "value1":0.4146999970169919, "value2":6937035767989914959, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147057, "value1":0.5790734522844233, "value2":1184517563846638764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.331707, "value1":0.5826950424479324, "value2":7357100402735593295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.476945, "value1":0.09337572406541937, "value2":3528304154285323988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_4":"i","key_5":"d","key_2":"e", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.660357, "value1":0.9211165869524813, "value2":6490630229378478544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.978786, "value1":0.24445422171222897, "value2":6316462580906113749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031375, "value1":0.9435579557750952, "value2":3775564649397357096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786489, "value1":0.19831604296196023, "value2":4519338814156406008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845707, "value1":0.0733198408835947, "value2":6043258187561628613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607379, "value1":0.49574791837525684, "value2":2511607289885086712, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541151, "value1":0.9239913172285873, "value2":8906754784907447893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409187, "value1":0.347121325872609, "value2":4022279358676190140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179171, "value1":0.9901703571238136, "value2":1971122679978818549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807000, "value1":0.9794430062314226, "value2":4140173177331652200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514307, "value1":0.3368836183017325, "value2":6646214711319824003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_8":"d","key_9":"g","key_6":"a", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.430324, "value1":0.6783643243115747, "value2":8700387930430110357, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245716, "value1":0.5777847683942137, "value2":1772788621154859179, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825752, "value1":0.8229486912139793, "value2":3218217186376093534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.550373, "value1":0.44484600976320454, "value2":1214217372893594474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810847, "value1":0.6410630380778536, "value2":1107368725861373561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_7":"e","key_3":"g","key_4":"b", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258640, "value1":0.3699636915738955, "value2":3778046401086414746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354190, "value1":0.2787621713653213, "value2":7905127146501563870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276300, "value1":0.38144875313061755, "value2":279189270439050245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030267, "value1":0.09524338073167776, "value2":624319768896255519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395031, "value1":0.017165936825319472, "value2":1839096275149454289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_3":"c","key_8":"i","key_1":"k", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920079, "value1":0.3695378285006256, "value2":3642800979833317953, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_3":"f","key_9":"f","key_1":"e", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055296, "value1":0.5098042037064671, "value2":1508579590160432139, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128994, "value1":0.8793028393199556, "value2":5718919044160675205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418223, "value1":0.7389480744126634, "value2":5430202156875304420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458004, "value1":0.7589090833562657, "value2":7035246718540970070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_6":"k","key_0":"f","key_3":"b", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492634, "value1":0.3450062925676948, "value2":289715553515433637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965681, "value1":0.9053649041017894, "value2":2294309054504906499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234931, "value1":0.0732565141042219, "value2":2755939599945126397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939989, "value1":0.8451432422468432, "value2":2656355912924482468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862255, "value1":0.3071521473170743, "value2":4029052487902069438, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027859, "value1":0.5418227496728708, "value2":5130775345772381962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016707, "value1":0.5241900456864456, "value2":1703509093458181611, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.129222, "value1":0.45379187899537626, "value2":8646708304907761402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167400, "value1":0.4118569437426427, "value2":1933032658312744458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.448154, "value1":0.9880224154363099, "value2":1958598830665944865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095756, "value1":0.7737922723291526, "value2":1554270443791925823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.669733, "value1":0.05998044134164805, "value2":4233304263515802857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360452, "value1":0.021777318888489317, "value2":1462350357926539483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.791525, "value1":0.24698569461323847, "value2":6782513953965205791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.136814, "value1":0.13312227926717296, "value2":2973379075607635624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_5":"a","key_2":"i", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.611322, "value1":0.06800531937511199, "value2":2253428998305264546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000702, "value1":0.40505920041369187, "value2":7444308564790803777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.317483, "value1":0.5921456568045917, "value2":1159233331687655633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.207714, "value1":0.03623125609776327, "value2":4050612595565072329, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739699, "value1":0.1967627573726683, "value2":5128621003255593133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938178, "value1":0.8074789236886436, "value2":5329361240090086591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075581, "value1":0.36183126859676484, "value2":8380824340175200195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.840434, "value1":0.36564465907617033, "value2":4138989352760782569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697538, "value1":0.8089117668002023, "value2":2706669265618979267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810618, "value1":0.9822561514547918, "value2":6363847253975708775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453492, "value1":0.7521891954121415, "value2":3415954419502830788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.246679, "value1":0.291250645374521, "value2":6787758008675320095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.717578, "value1":0.21885147116003636, "value2":8249910093492361560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.584635, "value1":0.267173208281922, "value2":6350566579717368150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288596, "value1":0.8034133849819789, "value2":3057165042866020348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043225, "value1":0.8891915956366093, "value2":1172085376591869222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514557, "value1":0.7993701790896678, "value2":3530709071532758425, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_8":"h","key_3":"k","key_4":"d", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727549, "value1":0.7581402483767911, "value2":2882149021889454970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944767, "value1":0.03125902081908963, "value2":5930819025045052350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_9":"c","key_6":"c","key_7":"k", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635131, "value1":0.5563692033064526, "value2":3139117266540547008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901432, "value1":0.08725832596843655, "value2":2516400281682537507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965447, "value1":0.24555624361218226, "value2":8420031154067673423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653005, "value1":0.6015476287649423, "value2":7179718486091286242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.251270, "value1":0.6303898680513278, "value2":1976335650844570524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.902243, "value1":0.42829690884430943, "value2":7763758072113593479, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748351, "value1":0.5394339139974091, "value2":1947184765867041268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.496788, "value1":0.0776914163879659, "value2":4120402058111565140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_6":"j","key_4":"a","key_5":"g", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202592, "value1":0.3950986449717796, "value2":7867550414487529994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.475860, "value1":0.5847506627916105, "value2":1395577688027634835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307279, "value1":0.66953692642835, "value2":1884810173101069714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447867, "value1":0.30049434187635843, "value2":2021494635928386331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371456, "value1":0.9583003775042523, "value2":8408660678843228458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321951, "value1":0.673208957772375, "value2":4484934316922285119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.654317, "value1":0.34301031853223307, "value2":1377167639028225502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258833, "value1":0.5754272342729656, "value2":3672436572873719874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894800, "value1":0.27153721678184795, "value2":6771295009871301863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886915, "value1":0.4211098948169063, "value2":327314688390276361, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464071, "value1":0.8401442859431041, "value2":7237906700870505791, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179548, "value1":0.35527262545135796, "value2":6929428209196710992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348853, "value1":0.7390477554491065, "value2":6017303276519989534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228799, "value1":0.7190691653435217, "value2":3466847624797872404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.676189, "value1":0.8701176990093307, "value2":8139200890464740272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780301, "value1":0.31331610868424237, "value2":7970934533227613450, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.763684, "value1":0.6117316228567153, "value2":1342308618516991103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.510130, "value1":0.26253831176449904, "value2":223167711418991037, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.054906, "value1":0.30113096258283606, "value2":9029709188884415357, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.101830, "value1":0.6535137630775729, "value2":8047896116508601877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.816267, "value1":0.6779088610569585, "value2":402736258268648092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794465, "value1":0.6385825716360082, "value2":647146760807700438, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.342961, "value1":0.08518129482765982, "value2":3079348210933738649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503723, "value1":0.5000975675164184, "value2":3087069466664576618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.552740, "value1":0.46623198603217897, "value2":5991311418948024851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.634526, "value1":0.1623989693910568, "value2":880550310997257962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409701, "value1":0.18776946541701378, "value2":4672037814003639717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928628, "value1":0.5544683022868725, "value2":8786609556345104074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422305, "value1":0.3339761609696324, "value2":2037434470333278554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497260, "value1":0.21504973222613463, "value2":8744267736203582723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348792, "value1":0.7267165846871734, "value2":2029452385366977955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767788, "value1":0.7268443539701483, "value2":5120204611450631315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313022, "value1":0.3937734460488266, "value2":8789465407671191549, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373011, "value1":0.5815734596824621, "value2":7433149153695507174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.403959, "value1":0.19547385196106162, "value2":4536904833017601954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702004, "value1":0.787480554646893, "value2":6826871522585433391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238346, "value1":0.3158260333418349, "value2":4890876749664266809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_7":"h","key_9":"d","key_2":"g", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.184806, "value1":0.1658734659451667, "value2":2103368389172466197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646125, "value1":0.7611256397426871, "value2":3779878057324979416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.963683, "value1":0.6784369755611176, "value2":7198319968843792884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264784, "value1":0.6944677799630911, "value2":1361718740081763677, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.606235, "value1":0.9767553574110414, "value2":8172694375569168120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.555524, "value1":0.0757100455425729, "value2":6369412142150141094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.683577, "value1":0.23488908846392256, "value2":7496440285542052380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741203, "value1":0.6152800537206294, "value2":4336041330603516829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.076957, "value1":0.46349128481126883, "value2":1995149205719248029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360568, "value1":0.18125853395781905, "value2":5023669000017633099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.127735, "value1":0.2064987900720652, "value2":1090691939971047635, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182440, "value1":0.18035240548503653, "value2":2377567143003176008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173358, "value1":0.9386364543056086, "value2":6498536821606623122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_9":"k","key_5":"j","key_6":"d", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.749875, "value1":0.9851379736422945, "value2":9068316297781013801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262923, "value1":0.6215747239294878, "value2":3444654966165617334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.471762, "value1":0.9687744980352465, "value2":4008980207107557770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.449800, "value1":0.43369424684152247, "value2":1796590317391088733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799032, "value1":0.7408688395558841, "value2":1701592597588021265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727561, "value1":0.14048934174685151, "value2":1964669731520342830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393461, "value1":0.22927403588573142, "value2":6637169452408438633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577492, "value1":0.9899876418682123, "value2":417258805492276216, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.149873, "value1":0.2451895383714133, "value2":6783012468215477739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794818, "value1":0.2978983436952165, "value2":6898137216027059682, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826276, "value1":0.03031011408512658, "value2":6243267082953045335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853432, "value1":0.8319004837974954, "value2":6916103698866916518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336756, "value1":0.6573141098461979, "value2":7552682537825884423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280458, "value1":0.9944062499165491, "value2":4515137518875248587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723258, "value1":0.8389015105022624, "value2":4385100343250672688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.518510, "value1":0.262846594625773, "value2":8956848533096958437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398441, "value1":0.29945983222092276, "value2":7583507905789284927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169305, "value1":0.3509582334049355, "value2":4486670807718872528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229006, "value1":0.9454115215233667, "value2":4148482506114005261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.108860, "value1":0.7792992725899739, "value2":6662551108461731004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270948, "value1":0.22595556811111142, "value2":4858846807835096111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_3":"e","key_6":"b","key_1":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739267, "value1":0.10553288145788361, "value2":3903915559852076766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.771104, "value1":0.2713630569195371, "value2":9127087478037486061, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234974, "value1":0.4512849051156243, "value2":388475201207159224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021067, "value1":0.8525575906494762, "value2":6112677337472259317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360331, "value1":0.5466498354493777, "value2":8416483456048971785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.083431, "value1":0.6268377803161493, "value2":6874204752716819531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543079, "value1":0.356809340191254, "value2":1479162719217476511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543913, "value1":0.1823988812868871, "value2":1772497490870724163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697354, "value1":0.6008728382547858, "value2":4808659248566150951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173629, "value1":0.11694918608631227, "value2":8374732016838634205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.478338, "value1":0.6523910738528493, "value2":6127719310482278347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_8":"d","key_5":"k","key_7":"j", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195226, "value1":0.11728848022547718, "value2":2692984308355874383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472792, "value1":0.3849335854410186, "value2":5551198795362400796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.088581, "value1":0.6479908764028293, "value2":6646669667589759935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.051281, "value1":0.8063603427465644, "value2":4677280644621004022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.421107, "value1":0.8098711885881693, "value2":781451797112683390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702263, "value1":0.7459154166325007, "value2":8763722224980194522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.105837, "value1":0.6082412929946549, "value2":9076065007808782827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229179, "value1":0.5208679934438682, "value2":1614625428456495501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452972, "value1":0.6602375068241643, "value2":3665988082745333789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570733, "value1":0.8539881179697112, "value2":8291534021329215591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.854105, "value1":0.19867014748060782, "value2":4707537791933933479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134473, "value1":0.013248508886480496, "value2":8975225813346771480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.401924, "value1":0.8131686031327187, "value2":1570626633965770939, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462042, "value1":0.33875887576127867, "value2":1531594524235268547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780840, "value1":0.6167438470784343, "value2":3111876176210806342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157314, "value1":0.024850360062745134, "value2":2155248434369533975, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.892341, "value1":0.29151476400849524, "value2":3929095109066478763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811920, "value1":0.3960145984061985, "value2":8079803277636061346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010049, "value1":0.1522516905011452, "value2":3887799589835913190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.511030, "value1":0.1096312618999367, "value2":4683382419730786111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799775, "value1":0.8107638537946521, "value2":1548400788132615580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.034128, "value1":0.8145920212010633, "value2":4095930474475755353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506330, "value1":0.8145793344024634, "value2":7248985564172971873, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.632617, "value1":0.530965707458104, "value2":6582545459218253298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.427368, "value1":0.5488520970953542, "value2":7764537337432663671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128543, "value1":0.339288607631015, "value2":8135936826126620967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.783911, "value1":0.4072868133390787, "value2":398383887746388268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030437, "value1":0.40039567718797253, "value2":5009000449434898292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.301665, "value1":0.2851345919898927, "value2":196381052074952545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740740, "value1":0.481454729816193, "value2":2929970498837276050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867428, "value1":0.8246737181844912, "value2":4477437696490734479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.708771, "value1":0.44011710908297325, "value2":7995836793967148678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663309, "value1":0.07002506566825563, "value2":3582676294388926669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.628344, "value1":0.9801112824460718, "value2":196546046309844748, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330627, "value1":0.6390220685465513, "value2":5252094592319160612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.014786, "value1":0.4666494400039896, "value2":6746944521794840076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.736397, "value1":0.1650175517955452, "value2":4725786271252183916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971200, "value1":0.08835643831453004, "value2":1662264677868065737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.804675, "value1":0.3583297226255054, "value2":6764608743691309744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577888, "value1":0.7784009799351271, "value2":7389846578443042716, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942762, "value1":0.5281041922180323, "value2":225078423814016880, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807869, "value1":0.12159890893033082, "value2":2007944198207870435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393243, "value1":0.9174108973827193, "value2":5956246567273801920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842305, "value1":0.6881483813172635, "value2":51917468498479892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.625958, "value1":0.7332068726661788, "value2":1595636028517738482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.738053, "value1":0.19091798634208768, "value2":2237197021871432008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268129, "value1":0.8406719661444901, "value2":2842521222444292729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865313, "value1":0.8513846203817129, "value2":3859037819752068385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762176, "value1":0.0431523780543075, "value2":6794482877795635511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.658034, "value1":0.12118411547914006, "value2":7727119685187980691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497498, "value1":0.8736017830767735, "value2":5281383221003217421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016157, "value1":0.45635245796699264, "value2":6588970499031475943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434398, "value1":0.5982494402461963, "value2":3451343743898655419, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995659, "value1":0.41609794124038807, "value2":2577003956591436853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.402057, "value1":0.3168156284522602, "value2":7664962841417781702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874274, "value1":0.8588869341047012, "value2":6471105672211793135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.193561, "value1":0.7716291581375219, "value2":6758054142720243683, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.298108, "value1":0.8739782093698222, "value2":7600130946423390938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008667, "value1":0.15272944664848967, "value2":28862629936648465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924564, "value1":0.2262144335029645, "value2":2100662153491567952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.250052, "value1":0.2945787294595236, "value2":6018457540699249059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395074, "value1":0.2290685251999151, "value2":4865312816831141048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414183, "value1":0.2892057090700859, "value2":3284381379197405662, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800711, "value1":0.20023612385737624, "value2":5612718418445245977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096882, "value1":0.6190652975917784, "value2":4456707593278158335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811596, "value1":0.1974893316007636, "value2":8350883557246751407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.328602, "value1":0.3577823211406552, "value2":3856849717607100839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165294, "value1":0.1558044782512875, "value2":2819862014599071323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.934329, "value1":0.6967191883920082, "value2":8317801698439570519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826310, "value1":0.5889684966352713, "value2":8904148697557725595, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.544062, "value1":0.3036793591749508, "value2":1172054639314476559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104925, "value1":0.0006382055160369547, "value2":4527189243213341516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577496, "value1":0.8262646875704296, "value2":6857852785226312453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.304163, "value1":0.23448830223910028, "value2":118297432460308257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.201626, "value1":0.8649386204518231, "value2":2335277383532252247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.072447, "value1":0.305504173455374, "value2":8879950191665801641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635410, "value1":0.8646407642266409, "value2":4794888988052345760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799765, "value1":0.020261599778331852, "value2":6569370350090311777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_7":"i","key_2":"i","key_3":"d", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.643397, "value1":0.25580026537842243, "value2":8727368389050428851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886737, "value1":0.709498290535563, "value2":159638875141661569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027896, "value1":0.4252750468577108, "value2":5985063599490557923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659322, "value1":0.8283450566415441, "value2":3829413923759281391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_2":"h","key_1":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.323182, "value1":0.05823156526332208, "value2":158181159624437675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721491, "value1":0.058149447228806335, "value2":4425192040964093188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.715378, "value1":0.1468510162162874, "value2":957193015456701660, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759517, "value1":0.11145345073332971, "value2":9085390279667186751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179380, "value1":0.22290378424729548, "value2":4452524214324184228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018314, "value1":0.08677941054726214, "value2":6248669636718030352, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579864, "value1":0.1843770522783152, "value2":2395237031433429826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907863, "value1":0.19391713305151562, "value2":1831578948753070965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567206, "value1":0.1200213703180807, "value2":281170079691225837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.908312, "value1":0.16873762365247053, "value2":3349277004368724008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.203346, "value1":0.3461718122656804, "value2":2053673338515188112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.252421, "value1":0.8594831603279349, "value2":8129734681001143206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862108, "value1":0.8625875861161819, "value2":266750263503917399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394606, "value1":0.1284747365418792, "value2":511162186129513778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104334, "value1":0.348212621993861, "value2":8809834773329001166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.390817, "value1":0.056538788821091236, "value2":959306944656093342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371946, "value1":0.0354556141213121, "value2":5315204481196619882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027057, "value1":0.04051346889125732, "value2":19281603192892033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315646, "value1":0.29470865205235175, "value2":3350853301006328333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516736, "value1":0.32421705083072555, "value2":6409364233498955235, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409843, "value1":0.8431693191414327, "value2":2309002455167863238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.290066, "value1":0.769846734088458, "value2":1354615135861452763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025458, "value1":0.04383062091902626, "value2":6270990659808009038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422265, "value1":0.4215091845870574, "value2":903506811871753839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954147, "value1":0.1703656776121607, "value2":8625109899283360680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468022, "value1":0.8656906467989927, "value2":8844353092623218329, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_6":"d","key_2":"g","key_4":"j", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.572694, "value1":0.03657142931464268, "value2":7269928716214871698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.035238, "value1":0.2174690883774512, "value2":2760303323859671187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.500319, "value1":0.8848932111671852, "value2":124147493136831346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.324572, "value1":0.32024379123233365, "value2":8082835280902735033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585321, "value1":0.18246288567364422, "value2":7470239488455160357, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726932, "value1":0.8209711897357084, "value2":4430579260139045397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.851627, "value1":0.9988163498901884, "value2":7782482596272329873, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126247, "value1":0.7338638289875997, "value2":2035846164975007267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923987, "value1":0.31023714156918913, "value2":7969614988653223555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.797525, "value1":0.9308157523651853, "value2":2008465742081738773, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595532, "value1":0.6306393412084162, "value2":1794355738915834222, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078930, "value1":0.46045211861703167, "value2":2231560481693103072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.835058, "value1":0.49083826368226746, "value2":947523171682771279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.757956, "value1":0.5783027668846169, "value2":7939213319454038876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.431201, "value1":0.249271140930431, "value2":7943702047022612152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.674800, "value1":0.9050148884978706, "value2":4701267498128748767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294430, "value1":0.37838796407920183, "value2":6812226255195471591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837164, "value1":0.41192774227492235, "value2":1535119829477772546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_2":"d","key_1":"j", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464005, "value1":0.1600487895075015, "value2":1434123145157650204, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546806, "value1":0.6061159682210883, "value2":6208205100336254753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096572, "value1":0.8332138223552116, "value2":7164548384493446541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.325386, "value1":0.6865864080335002, "value2":1198200374614070582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179713, "value1":0.590117470581001, "value2":8089464430800358371, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273193, "value1":0.6452720470474251, "value2":1981356061798226474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330589, "value1":0.2830579363626061, "value2":6390191335007008364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468768, "value1":0.2265890199157533, "value2":4363523400851143524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901505, "value1":0.3297489212069802, "value2":7168425838004946986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850173, "value1":0.7443546478795617, "value2":1215583272150299781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.968362, "value1":0.6932020029271382, "value2":4461729221210949302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210225, "value1":0.36249305348248906, "value2":7617198529682822125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.001816, "value1":0.46528315796295283, "value2":553457115403327004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.564478, "value1":0.42774941008998635, "value2":6190959325547605268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126782, "value1":0.6691920311814687, "value2":8190037233750158351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.125382, "value1":0.3161902537409363, "value2":2607443448109856155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.782262, "value1":0.25500427404714787, "value2":694236189793698587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635388, "value1":0.4760006261995292, "value2":7236218797911995877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.481920, "value1":0.5382100231194429, "value2":6783652184521772255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818398, "value1":0.6906166285200835, "value2":1593698347526099227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283475, "value1":0.9162718353521024, "value2":131978695455153917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434017, "value1":0.8865622180830747, "value2":5499929802234516147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838878, "value1":0.8831606268003614, "value2":4413611476125545911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451279, "value1":0.9923758573970507, "value2":7842864722613769419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362386, "value1":0.09704282298545679, "value2":6868263842667298833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274837, "value1":0.5887807861238907, "value2":1280869300914331508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.915666, "value1":0.6149407272649627, "value2":2848990718411372775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024227, "value1":0.1623145961115003, "value2":2455593750882693548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.695944, "value1":0.6887899812661492, "value2":533109273214059474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.528843, "value1":0.5075436862092302, "value2":515274582595904032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842262, "value1":0.5447726756017138, "value2":411822948007690047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444248, "value1":0.5454758220348956, "value2":1028316813231771868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.604063, "value1":0.46187992572513836, "value2":8158929971855121147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275994, "value1":0.8154001283675578, "value2":5366649133302343998, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348933, "value1":0.903584155771458, "value2":3421400845314883405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.041062, "value1":0.1190413277495334, "value2":3602483838893891944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928372, "value1":0.19081223162638847, "value2":4430971211717726323, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.677971, "value1":0.7830672620168732, "value2":4412058601784711710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152188, "value1":0.6024595453233776, "value2":3457200300695401373, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838789, "value1":0.8421167420494099, "value2":690533435719409387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.578903, "value1":0.6328751611123744, "value2":3916711663864490625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.896839, "value1":0.9070308716530836, "value2":5545334900650073897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.712901, "value1":0.5273783009751816, "value2":7859089933688216880, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884939, "value1":0.07741816375541602, "value2":2787557417309799857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944440, "value1":0.9501843438293957, "value2":4306730282944265532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814029, "value1":0.12358405285849303, "value2":6299011328560473443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965702, "value1":0.9397419047313924, "value2":2182258508456140601, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045751, "value1":0.6945070980136847, "value2":6892165864647238687, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821781, "value1":0.27908605223075134, "value2":5742625392746876485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514196, "value1":0.8720271384105083, "value2":6651696455700077053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.705476, "value1":0.06250737971386458, "value2":1193722866498852150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.603276, "value1":0.8453600274943766, "value2":31981742286968982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.682527, "value1":0.2427465956249123, "value2":7462455737298824829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.605338, "value1":0.7488120008677794, "value2":4273030954599275940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893961, "value1":0.0012093187527627421, "value2":3798553194034218364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.593870, "value1":0.4389808952680249, "value2":4760750743940894178, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091153, "value1":0.8390848719802647, "value2":4663558817694651969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.014239, "value1":0.6803598889667898, "value2":4768916054002910461, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478582, "value1":0.8497920155872134, "value2":6014547020899484560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895949, "value1":0.2900471481778915, "value2":9139898160138156870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.613215, "value1":0.18238778666189503, "value2":1572248498405458162, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.817510, "value1":0.7988916675957994, "value2":5551320908811649706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881135, "value1":0.18477673938398376, "value2":5754586889701314224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128674, "value1":0.2730542267310359, "value2":3723967523716120986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_8":"k","key_0":"a","key_6":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392852, "value1":0.5342690346158169, "value2":2053063244515016190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931934, "value1":0.712284281615868, "value2":5743978094943818789, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537147, "value1":0.023259389444222435, "value2":5526408740424742448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.216805, "value1":0.8594083029021016, "value2":5158284428621389401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837342, "value1":0.5180882037372726, "value2":7808493880824756093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144558, "value1":0.054388678344141024, "value2":7174534666209181912, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599909, "value1":0.09725816948703289, "value2":4397869661367210515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.880028, "value1":0.4672046328070093, "value2":7512677570982024303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271509, "value1":0.8816366985657997, "value2":7431242132698500082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.752197, "value1":0.6078880045612306, "value2":3606538184952837708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_5":"b","key_0":"a","key_4":"j", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392689, "value1":0.5702352441472875, "value2":327376335260250215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.201991, "value1":0.3880870455267555, "value2":2539019737033922024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.894353, "value1":0.3785370438906491, "value2":8100099805481858212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_4":"e","key_7":"c","key_1":"f", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.193824, "value1":0.1316387900370704, "value2":2566382490326741291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.735887, "value1":0.9017592259731758, "value2":3204066872434851855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891488, "value1":0.8557864633508439, "value2":7321454426994500720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868424, "value1":0.21022389029059355, "value2":4239400393420255730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.624822, "value1":0.8941824508080144, "value2":4130019486643335780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.292285, "value1":0.2379306348830461, "value2":19245718469181538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690737, "value1":0.07804300554340773, "value2":3416010458091576918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404288, "value1":0.7382908404371364, "value2":4673144760418942783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.333079, "value1":0.7119882858599182, "value2":2804563282517597579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.472391, "value1":0.08143361630646322, "value2":8016722515581298003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_3":"i","key_7":"a","key_0":"c", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.373563, "value1":0.6580401651760573, "value2":6974099035575699967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078367, "value1":0.04245208361261986, "value2":3309677777480755698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.312294, "value1":0.08781290868459149, "value2":410348162808240186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768785, "value1":0.8804562203614901, "value2":3554533373089833157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636131, "value1":0.6764401768679541, "value2":5380729408367504508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041277, "value1":0.013798250982638604, "value2":7112024095753417757, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867267, "value1":0.8747601920631648, "value2":1340507463961878602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776242, "value1":0.08520211967435457, "value2":4488809966272125582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634645, "value1":0.8571991096706362, "value2":4813432822336330164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415856, "value1":0.16181998754042196, "value2":2843492259629836672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466761, "value1":0.6961378523412571, "value2":7745491307176492894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.594554, "value1":0.04099735827317489, "value2":8913619907287518073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.170785, "value1":0.5648256194335608, "value2":476929125931913132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.184821, "value1":0.7825477180517281, "value2":4452433023228691120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906243, "value1":0.48861587452050464, "value2":8430953017453652958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.667172, "value1":0.06605403171722925, "value2":3065000063305702983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.192260, "value1":0.12827053824062132, "value2":4944252317941317234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400652, "value1":0.728695995485893, "value2":1367272622265070542, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535047, "value1":0.4567196818143048, "value2":5331291578601537074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466316, "value1":0.5659706744457615, "value2":5221382170536673746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487405, "value1":0.8907853085096481, "value2":2583510249445277455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.375762, "value1":0.8536987085513627, "value2":1666025213570716259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.165432, "value1":0.43405723604720065, "value2":714036789144069944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175401, "value1":0.9859093721286644, "value2":4617822411248224956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.059065, "value1":0.378208510942557, "value2":8575694693763974484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_7":"j","key_0":"d","key_6":"c", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.073374, "value1":0.7874709956095631, "value2":3297271830714214500, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656159, "value1":0.11136055751977454, "value2":6742238162891439594, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102008, "value1":0.8937221842663475, "value2":6378107019082012182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.177632, "value1":0.36894176860576355, "value2":7704992361178834991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335734, "value1":0.6980181667187008, "value2":260944621673155935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.855348, "value1":0.9199066328884838, "value2":4741986190169422412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965490, "value1":0.12178040424602839, "value2":8326386976538257273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090240, "value1":0.23602165280654655, "value2":9043605027949953616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.028464, "value1":0.05960386743156977, "value2":3549887217756566974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.956988, "value1":0.5498920550278433, "value2":6597949147890816377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.171316, "value1":0.0904948299110727, "value2":3471748119473561940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_6":"b","key_9":"a","key_1":"g", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.523091, "value1":0.4454856417468202, "value2":3973537432672533989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453397, "value1":0.7881010562880141, "value2":448816938066732131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384576, "value1":0.9045802469513693, "value2":4349360112470402658, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.929660, "value1":0.1904824371420524, "value2":4775599930437109515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642052, "value1":0.9792443542954197, "value2":6092407690974464405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194609, "value1":0.8726282980309354, "value2":7591391705732812177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095352, "value1":0.384168214137503, "value2":7516656275093877057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868497, "value1":0.924966560225317, "value2":4429888702948279946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.146490, "value1":0.7150691128822396, "value2":205950019316022568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009297, "value1":0.4940665543165739, "value2":924283179287111091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904057, "value1":0.23580267125603968, "value2":531910251609622124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728011, "value1":0.8321936268339676, "value2":2170557262229675184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944000, "value1":0.7690990294985828, "value2":8227626091745901, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.871354, "value1":0.21098039407901223, "value2":4374012870679696344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.538323, "value1":0.681484837485245, "value2":6775701390216802286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230276, "value1":0.36173491575946426, "value2":8235898214428447094, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552468, "value1":0.8892513220522287, "value2":612692529250917070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782746, "value1":0.3267438552083453, "value2":5821122346950557164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908039, "value1":0.8750317891455761, "value2":6904198848902253780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124465, "value1":0.27489616104124, "value2":1813939809215341473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.739988, "value1":0.11884426917626255, "value2":6375710418065432574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246847, "value1":0.065994349338974, "value2":4501641519889306838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.675544, "value1":0.4346958284003153, "value2":8656554552944846961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907951, "value1":0.3861999534665598, "value2":7376195997991690698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.601240, "value1":0.09029192218287085, "value2":4084656981993173700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327311, "value1":0.46909482167142375, "value2":6484192245523357545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453608, "value1":0.634255139121383, "value2":8880783116370612057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562417, "value1":0.4260014133745824, "value2":1052849307102914712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708174, "value1":0.05688677429464904, "value2":5475041244330586198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.774531, "value1":0.8132424157243254, "value2":1362266175582132598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.106500, "value1":0.7014281430749205, "value2":7134854011640912176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.486559, "value1":0.28063120575400446, "value2":3192436055979504417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.864997, "value1":0.3477928996994262, "value2":735248433592549787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.657367, "value1":0.65265411185833, "value2":2443852584375783883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.172271, "value1":0.4417461489558258, "value2":7942690633839505457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419664, "value1":0.16011001601248667, "value2":6549588207213967629, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416761, "value1":0.6429956000093604, "value2":8493895113638046625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564282, "value1":0.4457588529183665, "value2":6725016002434781716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430750, "value1":0.3037956469116298, "value2":7998283396935875899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827125, "value1":0.5250484976450595, "value2":6524724358656282389, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395896, "value1":0.12287086144958013, "value2":1930681199692368788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851159, "value1":0.23103414004696357, "value2":4914748620478090577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.736709, "value1":0.6408817571256187, "value2":9105122753792897591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984808, "value1":0.2905624937515174, "value2":7851451601162680954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487306, "value1":0.605006354442018, "value2":4276465105843409598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790850, "value1":0.6268022197129945, "value2":9220780769142386741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185700, "value1":0.566519118535341, "value2":3705647574979636584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451877, "value1":0.0460137249105004, "value2":7915985022414197030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150718, "value1":0.35693559742029707, "value2":5014219743291793375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.294260, "value1":0.9247064360015096, "value2":5127524279022159080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995623, "value1":0.21178547000119013, "value2":2431494318968387519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.256959, "value1":0.8916548214121515, "value2":1002635579417294841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_5":"c","key_1":"c","key_2":"g", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110758, "value1":0.9737478776363947, "value2":7247818685878473810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825871, "value1":0.2917878380844755, "value2":6516114778766465158, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910566, "value1":0.3957632746819297, "value2":7166657573372241684, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.178800, "value1":0.5649796089766207, "value2":7775035464817162126, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634489, "value1":0.6090054970934514, "value2":6647498876825886588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610499, "value1":0.5292322793488414, "value2":8830154316280741019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574572, "value1":0.5304362659437236, "value2":3949756347058852883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_3":"a","key_9":"i","key_0":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380651, "value1":0.956019198700713, "value2":9135146156795796096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778218, "value1":0.2121518375707717, "value2":8667556184145489676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078961, "value1":0.06325131155030916, "value2":8555129638818277403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.047017, "value1":0.1027203958612216, "value2":5246272405737501952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439829, "value1":0.7417681560989317, "value2":7749536771402273968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203072, "value1":0.5621215132192839, "value2":6118954075279232029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758671, "value1":0.48389569392567205, "value2":2705225096795559234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_2":"g","key_1":"c", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665522, "value1":0.8426429373047708, "value2":8279168363918064894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_6":"b","key_8":"h","key_1":"a", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.345535, "value1":0.20088437097171843, "value2":4950747031440638845, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124655, "value1":0.41827026587025645, "value2":3430038763553236823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977760, "value1":0.10421019819450697, "value2":733644826366835807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118861, "value1":0.7989242448499801, "value2":7548777360232504083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318842, "value1":0.009939081834750191, "value2":8375328698470876119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590619, "value1":0.03636791737366475, "value2":7582770650474370737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045468, "value1":0.7763100607726895, "value2":2803435127241577664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.553650, "value1":0.9987121300161536, "value2":3948733587296676032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790022, "value1":0.793158890727088, "value2":1174928966366390949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.232763, "value1":0.07728571888784644, "value2":3240453509192284959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074815, "value1":0.5297469977127046, "value2":2019972453697918898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.126329, "value1":0.8018605847489219, "value2":4857597305010910280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.465083, "value1":0.1752053660866726, "value2":7163605506157251603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.714319, "value1":0.8220755414438086, "value2":876552025810846680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392782, "value1":0.6635667135366609, "value2":6624504880778850428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404283, "value1":0.5416570514867133, "value2":8266175224671672272, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.734325, "value1":0.7575887007274229, "value2":8864761125492843547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.164845, "value1":0.7328611642697666, "value2":1939960544899646141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.087713, "value1":0.6472001702780127, "value2":7064434641814632478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636375, "value1":0.263542227747472, "value2":5091288498651627633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439658, "value1":0.7518898990734011, "value2":6126681364245747402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580431, "value1":0.28444432068121084, "value2":8147884504685238694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408258, "value1":0.5563100440073113, "value2":4573231178010278468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923759, "value1":0.7035415767681118, "value2":4284377617122095281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069625, "value1":0.7755797987036898, "value2":7005063200108318654, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475808, "value1":0.7602915679203158, "value2":1889690102234767528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045338, "value1":0.4860219702134258, "value2":5422255506416156729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.985908, "value1":0.7866996153890412, "value2":4784461751340030748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937856, "value1":0.5239212561479751, "value2":3109818040142871590, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.376519, "value1":0.2587179526287473, "value2":1263076833301844915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943956, "value1":0.6806333311736701, "value2":6002705815114995201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.653771, "value1":0.7947376720839073, "value2":2247611778314354742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290979, "value1":0.13605627013098143, "value2":859676226210675359, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166492, "value1":0.4455170530582711, "value2":3676647784290217622, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.625732, "value1":0.26531203043868007, "value2":8360465694110411228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033924, "value1":0.027884795524999093, "value2":4290921120785429469, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.199865, "value1":0.6930021146351104, "value2":8033400918877373777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.387257, "value1":0.5315645824113636, "value2":8487597923574953049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.385317, "value1":0.687478534046926, "value2":5788685556231917492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.632370, "value1":0.4013559424653022, "value2":8640639439561304540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226297, "value1":0.2440055184343977, "value2":5133913946019355967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.763037, "value1":0.20026459422324144, "value2":8670140335638909516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695304, "value1":0.16276770698378185, "value2":1188100569829740721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595366, "value1":0.4506718132549764, "value2":5734081257378041711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.833543, "value1":0.5734402814385565, "value2":922425738240504967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709630, "value1":0.8692041834415107, "value2":1960359817596413212, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.732788, "value1":0.23444382672590003, "value2":2854071547092864254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.412896, "value1":0.19053537293020936, "value2":1082762509249072310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988462, "value1":0.4293280866481845, "value2":4780102507614458134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_8":"i","key_0":"h", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823600, "value1":0.01981381195792859, "value2":8743022558864727512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.092886, "value1":0.9862388684646872, "value2":3622022818199205736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.018231, "value1":0.4242288910229569, "value2":880886547836969584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062605, "value1":0.9179292075855371, "value2":3743117943661894467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.759025, "value1":0.5166884948154693, "value2":739459076946893696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907932, "value1":0.2882739675567565, "value2":3650155370836487214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068172, "value1":0.4347373436897239, "value2":1372308133830179280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514459, "value1":0.3578934602691126, "value2":1369826174473414234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282115, "value1":0.4836091160007032, "value2":7304317831220994830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652653, "value1":0.7715198420930932, "value2":1854799303037917400, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.873426, "value1":0.6674125872681426, "value2":1713253808702182629, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.253882, "value1":0.35618452262868266, "value2":1451362150616905038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058435, "value1":0.11619135407507769, "value2":729304989898004351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711979, "value1":0.19795958240207612, "value2":5044852283450257784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.620015, "value1":0.7494674771784868, "value2":9072157623838715380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265268, "value1":0.6332467233229876, "value2":7660280405186635297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471797, "value1":0.9586561875915051, "value2":1400450692063171507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715350, "value1":0.28992759308570154, "value2":4420882374996204645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447264, "value1":0.32845931032025727, "value2":3764283351468138749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_7":"g","key_1":"f", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799817, "value1":0.8344281598160882, "value2":3194977322046448615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225092, "value1":0.832673789719215, "value2":8242369633095170423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261842, "value1":0.1783779388932828, "value2":7591102613464952913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885476, "value1":0.33420965888644627, "value2":2231088099862071194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015564, "value1":0.23747285224855552, "value2":2211582269309113296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415709, "value1":0.43459757217905465, "value2":2297522408220839681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671302, "value1":0.5521783843346304, "value2":8797575493863836447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809774, "value1":0.3232411464090472, "value2":689408551095099282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391492, "value1":0.44145120690127054, "value2":741707985323247529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593752, "value1":0.5004004561210201, "value2":7899920698831884885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090704, "value1":0.9099972336756517, "value2":8993930752293016892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.365212, "value1":0.17134030372562564, "value2":6300597784300096913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.458419, "value1":0.03537414872175956, "value2":4931151600790868120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.266288, "value1":0.2710772740926763, "value2":5168762762336913335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.846075, "value1":0.8530137052878358, "value2":5846881912829979057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.441246, "value1":0.895274800378327, "value2":6555884777228809545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181487, "value1":0.09635807954147779, "value2":5139650932097495606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_7":"g","key_3":"a","key_4":"k", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.283526, "value1":0.6765717041541169, "value2":717191993149931255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690428, "value1":0.5868291690096701, "value2":4142969563289744689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461183, "value1":0.7388746066577521, "value2":3907321818804232131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078559, "value1":0.5509982085563974, "value2":8489285868587130769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.006442, "value1":0.17964435576390814, "value2":6385046429151027537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803653, "value1":0.007944203568736737, "value2":8630069900982836811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843384, "value1":0.5245784602574561, "value2":1611187771800002701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.322734, "value1":0.26583004470195515, "value2":5891979565833996742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019366, "value1":0.9471429425977943, "value2":3105898999363337716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583535, "value1":0.3000091753953168, "value2":1671214988273742274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_9":"d","key_5":"g","key_6":"h", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.528263, "value1":0.6167724953698004, "value2":3571326627931246323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.353615, "value1":0.09339691407017821, "value2":8955717188172767409, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.638995, "value1":0.5498958167694153, "value2":6537844553335776020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327054, "value1":0.5183869093313511, "value2":4021636019817176776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391759, "value1":0.6107482510959784, "value2":6639243589639002168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.083515, "value1":0.6947157647744896, "value2":3642097726125624072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.586405, "value1":0.683079171453159, "value2":414358593011525856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847208, "value1":0.004363913601366179, "value2":6293076524070827579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583066, "value1":0.38103391426791644, "value2":7168389465606666058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.361829, "value1":0.0757878691848108, "value2":18451857992724418, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_5":"i","key_0":"k","key_3":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481695, "value1":0.5254185806253874, "value2":3155631786417589238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.072077, "value1":0.8787002152127282, "value2":5274515234654561765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.267961, "value1":0.8648926070157235, "value2":4125897469271708547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.245076, "value1":0.06458749862097861, "value2":3727526843973668028, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602905, "value1":0.9902842013208221, "value2":3395310217472956562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776585, "value1":0.7219469790990183, "value2":77676005392087147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741224, "value1":0.9110757287517638, "value2":3235394638378837669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.616438, "value1":0.32846413845355765, "value2":5558119133284488407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.617955, "value1":0.12562192604901234, "value2":603576184957547937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.651221, "value1":0.8976398132733482, "value2":7308768939039920124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610616, "value1":0.3225584230939945, "value2":8416609441712941807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.688197, "value1":0.39642287994645425, "value2":7290339184191009077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599842, "value1":0.6105446645640619, "value2":1208112926522863577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867582, "value1":0.1840371797963492, "value2":2177170750841832105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098316, "value1":0.966593809823329, "value2":2548324970162406510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694537, "value1":0.3376604030915601, "value2":607328136959332742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748191, "value1":0.9957584594865343, "value2":67877367036436991, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.064163, "value1":0.7248743828192261, "value2":2749527480847272451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502667, "value1":0.16495830272715267, "value2":9030464128428456991, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990661, "value1":0.3949843558188797, "value2":7201429976848106189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987854, "value1":0.29870559261727064, "value2":348149959926675176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481753, "value1":0.06548607806382464, "value2":1031140396971019018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.630820, "value1":0.8992403438153617, "value2":5146298573769468519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_7":"g","key_9":"h","key_3":"k", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804082, "value1":0.2519724232639516, "value2":1298310485258981735, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_4":"h","key_5":"j","key_1":"c", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368623, "value1":0.647450377107345, "value2":1284622928379296522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.541781, "value1":0.3261143352667153, "value2":130140210100844559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543424, "value1":0.06226681703581004, "value2":726618431953897465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988533, "value1":0.49745860881748044, "value2":3406264037344093572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080362, "value1":0.7895574305241833, "value2":152509341700687015, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150372, "value1":0.9343525151823494, "value2":6076357966504332576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489577, "value1":0.16062395564859527, "value2":3405065082227979672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225570, "value1":0.28122658630857733, "value2":8458136042003760574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862952, "value1":0.7649680950314343, "value2":1756495878088169871, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009186, "value1":0.7999050849241861, "value2":3718209620537893555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327636, "value1":0.23708513628211542, "value2":1522368463913294016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.551176, "value1":0.009439114458257377, "value2":2361018304142309387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.633123, "value1":0.2821310200838197, "value2":6052402151738260293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128752, "value1":0.3359757178563009, "value2":7472471602725669166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.456386, "value1":0.46133506790277695, "value2":2296093836300686089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_3":"e","key_4":"d","key_1":"g", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420349, "value1":0.36664549905260974, "value2":3546467223240204480, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.938528, "value1":0.007977829048726685, "value2":4244809297577031047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226579, "value1":0.6725646472034443, "value2":913735509091846801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.850850, "value1":0.23068766044782393, "value2":2300681752566773332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285218, "value1":0.6989337075616671, "value2":2607116938388175821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.221331, "value1":0.11523512208048978, "value2":8241016534243475750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886294, "value1":0.8815954846610522, "value2":1129122380251841667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010095, "value1":0.4468942822052692, "value2":7305659252479204819, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007461, "value1":0.3287186449948172, "value2":8533612311279585337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841878, "value1":0.06768912562638665, "value2":3915275733365598237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239318, "value1":0.2752287175449378, "value2":6104494420782937095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364905, "value1":0.7269412172343553, "value2":7167814392288649336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030862, "value1":0.35386987370196926, "value2":568029767114417610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945737, "value1":0.8998415941144967, "value2":2346717810373152754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.093117, "value1":0.8244480275524787, "value2":3248267576826263944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.422568, "value1":0.5719502852011219, "value2":8260911644483203383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823166, "value1":0.2849312986012331, "value2":3755798306625123375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268641, "value1":0.33070121758142046, "value2":3503668812330101597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564550, "value1":0.4481321012388454, "value2":2552310395319729245, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417743, "value1":0.047809954457076996, "value2":1081553625885385604, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595860, "value1":0.2921995022802213, "value2":5059986207137621431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.505814, "value1":0.9601390700122682, "value2":1325398133306866350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357820, "value1":0.3304365801280413, "value2":5155828582049823863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963899, "value1":0.8305280855606907, "value2":2952790462431625685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395768, "value1":0.05940274489209035, "value2":2206736622008483109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_2":"f","key_5":"d","key_0":"b", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.994837, "value1":0.28131858851364544, "value2":6377323293251471217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825885, "value1":0.9524553792547288, "value2":5686625986396675519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962844, "value1":0.7824414856459302, "value2":3176085690721005555, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525393, "value1":0.2894500361431452, "value2":9170274733300331931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066894, "value1":0.9750264781896856, "value2":6481943120958967565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118777, "value1":0.49186253332263474, "value2":196535145535309391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.011455, "value1":0.25527316956832247, "value2":23216573243095801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.516739, "value1":0.7031419417768016, "value2":2051319192678332676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621518, "value1":0.7948563798954263, "value2":8394265809911073481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.008794, "value1":0.671348308630362, "value2":8908504416251388849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804142, "value1":0.34185386259040573, "value2":7569056971355045232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971215, "value1":0.795050202357287, "value2":7291081553392495608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285984, "value1":0.6233623903797153, "value2":6444347877884163801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.436168, "value1":0.122788509000057, "value2":2542694660610691923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545038, "value1":0.897272884042895, "value2":3518180018317716924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.377957, "value1":0.26500726028503324, "value2":918021549502007270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186659, "value1":0.8861606285044316, "value2":7428746411605199070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.706851, "value1":0.8539941323885553, "value2":5095874822126979293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552666, "value1":0.8209937326987259, "value2":7918997962757156260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708678, "value1":0.20172873469302396, "value2":473140516571282958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.588929, "value1":0.8992364975321605, "value2":1107792164310220135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.396543, "value1":0.9159358105409485, "value2":3240110351377657654, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949484, "value1":0.3395753649788759, "value2":1556081794159850294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627633, "value1":0.6064813434025144, "value2":504106831033400051, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760461, "value1":0.027052105627838302, "value2":2547821024594975963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690642, "value1":0.29842739674487334, "value2":4347369482397818828, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271622, "value1":0.9232891397814524, "value2":4902825584314369050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678674, "value1":0.8612335750248717, "value2":2676726884600182684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.789269, "value1":0.7153455595388135, "value2":8629094676547921616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_4":"k","key_5":"e","key_3":"h", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.274447, "value1":0.04852479796186674, "value2":520491486142586388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517924, "value1":0.5065900646461697, "value2":1305736770624753229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440510, "value1":0.8079290707038177, "value2":745163355638334416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.992658, "value1":0.33022695610085456, "value2":3745647814397920317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.142855, "value1":0.7836474840216356, "value2":1701655973854226818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.718918, "value1":0.9106879340813463, "value2":2642697174847166539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358697, "value1":0.6889718382775613, "value2":2396285724783610411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782243, "value1":0.10018629733131312, "value2":3557685057170687759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290843, "value1":0.5213600910147824, "value2":5522593432675792373, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243393, "value1":0.9922299657128142, "value2":267617765686382025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827775, "value1":0.17424595518809072, "value2":3247150599498567700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954439, "value1":0.6395234710891515, "value2":3700591746218368196, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543518, "value1":0.7123005069599985, "value2":8170425126276779400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_9":"c","key_3":"h","key_7":"j", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862097, "value1":0.7928466963741031, "value2":1388049342872581158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861104, "value1":0.44830540845614686, "value2":3638392887423846691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996353, "value1":0.9606274415719652, "value2":6260542395914742363, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269620, "value1":0.27973905679528643, "value2":1556926452504115363, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118090, "value1":0.5428339606791369, "value2":1853095038687144617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843203, "value1":0.15162943121847733, "value2":978719322151022878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.168392, "value1":0.30548225941021784, "value2":7137186554911304535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.197151, "value1":0.3975385059702385, "value2":5863175014471717006, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565354, "value1":0.9234247531019719, "value2":812121677319261736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030909, "value1":0.8792093662826526, "value2":8559138722753842843, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.484600, "value1":0.23944552081323173, "value2":7751158766892162613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.320620, "value1":0.12230291625693078, "value2":6192516194919614511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.935691, "value1":0.28006447226692, "value2":6784015217669799303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.336215, "value1":0.23094104388429443, "value2":5569809866177597446, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988466, "value1":0.5462607441657077, "value2":8311104273733417101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447804, "value1":0.4867937694484817, "value2":5062168010270449210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.190570, "value1":0.8283372201936341, "value2":5874711067636169206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904281, "value1":0.8553852749887187, "value2":3311030506444553065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.751439, "value1":0.4266991113414808, "value2":4999173498909581042, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.125309, "value1":0.5563950948301023, "value2":1187994145431678821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.973142, "value1":0.11588338763957572, "value2":8318570385943791860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.116231, "value1":0.2526143185693312, "value2":782968109827902547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.187535, "value1":0.27872170170772304, "value2":8766589428595489268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945637, "value1":0.08854405995381304, "value2":3412460749402337910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.270152, "value1":0.3638550762746485, "value2":2218032724303888987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051113, "value1":0.40794935925664466, "value2":7831097119693907093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265496, "value1":0.23954206323753308, "value2":63253225412010244, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416624, "value1":0.17710696129812786, "value2":3803821152719869396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607195, "value1":0.28582291116541503, "value2":7284275038719896556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.326362, "value1":0.17094485896565303, "value2":4663361033102261072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438839, "value1":0.3486395787421312, "value2":8809206276934706557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033597, "value1":0.8575270834812074, "value2":1940304743017854051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.065029, "value1":0.9427613360719956, "value2":271108060306002877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.488385, "value1":0.12779971800108084, "value2":3316011689829646532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961814, "value1":0.5428137306648136, "value2":1514601534044579978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263207, "value1":0.6949685012843101, "value2":895006576805426648, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282635, "value1":0.10777955836055363, "value2":564953042266607985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.001269, "value1":0.042927028237879465, "value2":1466958494603226347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682457, "value1":0.8478178196386861, "value2":107121152123493591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.159543, "value1":0.9604877997752348, "value2":7593923445754669936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015347, "value1":0.7626139882833062, "value2":5647986907617063275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209254, "value1":0.8450839759887067, "value2":1886374721985796728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066244, "value1":0.07791428229072243, "value2":919090579972011287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.699052, "value1":0.06643124534628293, "value2":2711140899074023727, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.631133, "value1":0.143383362490665, "value2":971805873892273782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426673, "value1":0.7738540415601494, "value2":4397130911534361286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.389568, "value1":0.33397631468501043, "value2":4052528315967262589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.622610, "value1":0.24246003195798918, "value2":5479722481130894419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904392, "value1":0.9774495314687007, "value2":8793165354063487461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.793581, "value1":0.4190858782174469, "value2":4926610192238705598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110855, "value1":0.2884411940593678, "value2":5227068360933879219, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_7":"d","key_0":"k","key_2":"f", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135653, "value1":0.402267966757566, "value2":6977955000760069723, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.307152, "value1":0.8927574650950808, "value2":5803177489389723115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.737685, "value1":0.21404400943023025, "value2":6732954103348577362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.121533, "value1":0.18872391644762126, "value2":8989825709899786188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.972223, "value1":0.023522846364320095, "value2":7146680715055172496, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923388, "value1":0.8860383778342971, "value2":8303048822044107475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851782, "value1":0.32631560891796313, "value2":1989916996789168449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391839, "value1":0.002802043190783618, "value2":8629895271323501696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_1":"c","key_6":"c","key_0":"f", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066714, "value1":0.031995170191600183, "value2":3319906268065732292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421535, "value1":0.635220505499552, "value2":7200157785718775016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287471, "value1":0.8948051177891431, "value2":2176610475830643902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804659, "value1":0.28415985878229416, "value2":33052675971427371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996360, "value1":0.6152217383986905, "value2":544588703505765788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.798458, "value1":0.8665558307064753, "value2":8115268483760058936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448744, "value1":0.3917339620374744, "value2":9198148727913546136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861092, "value1":0.9826532618696505, "value2":4765520374138069290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.202532, "value1":0.0052591320469092834, "value2":336652236719043103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.524950, "value1":0.8143167251197552, "value2":6529413407694528834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.485366, "value1":0.5907312441951306, "value2":3337798525793799244, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558504, "value1":0.3641804008571667, "value2":4575793560178915745, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_3":"b","key_2":"k", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642936, "value1":0.623697886602654, "value2":7776504717328671778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043344, "value1":0.9623993994613507, "value2":6602176914068250275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.241555, "value1":0.23253929846791072, "value2":685349515600255534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926727, "value1":0.21223983767190985, "value2":3951472227334441759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.131046, "value1":0.2851063666145672, "value2":6180530959986721149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.247299, "value1":0.5130082295270245, "value2":8139956405448602641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758053, "value1":0.49876246690774434, "value2":3786706615107859566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323322, "value1":0.4313958567662574, "value2":8854072512884634153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748573, "value1":0.9590027921453678, "value2":3839028144294969465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384213, "value1":0.010562139959872605, "value2":6878855550977960785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180644, "value1":0.6131439659014073, "value2":2706285599198878081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593573, "value1":0.4120750238333436, "value2":1200854625465685501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995840, "value1":0.297035809872462, "value2":6168541710158809840, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_9":"k","key_3":"e","key_8":"d", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.511756, "value1":0.006206978177219643, "value2":1958577823681147315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565859, "value1":0.4442276087687805, "value2":39986647781639917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715853, "value1":0.9381008994826828, "value2":3556376195988394411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236468, "value1":0.20999691756150096, "value2":9161072742831597145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.161371, "value1":0.738745986035695, "value2":5138216362948975345, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445330, "value1":0.14739850543555172, "value2":8667736730671884444, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.982188, "value1":0.40813076015564426, "value2":3022991256121544350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984748, "value1":0.6214792221342914, "value2":5933123653156672217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562273, "value1":0.8568295281307874, "value2":5414869267292754213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476811, "value1":0.49675643916102746, "value2":5664619325942193065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891490, "value1":0.8866201621688181, "value2":8214607251036840472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931525, "value1":0.8288823187078326, "value2":3069955632009854668, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455511, "value1":0.8774391931652025, "value2":3046641762023967525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400651, "value1":0.5254465108261448, "value2":4999803900412377548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666971, "value1":0.9993076586927376, "value2":4982431527333428358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.8618210720028192, "value2":761627529379782309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.915374, "value1":0.37053901712348225, "value2":3944125570135845867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.960156, "value1":0.9374154078802892, "value2":4747973404867949779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210720, "value1":0.8468677886051782, "value2":4852924773263065270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.332500, "value1":0.6465642869647131, "value2":1008458018130353532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.788169, "value1":0.3773654642642006, "value2":4191084511110280464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545303, "value1":0.25852749290398136, "value2":2783025552475099417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.143623, "value1":0.36024800648413563, "value2":4932311496806548620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895647, "value1":0.0053042457769262655, "value2":6923819500937313799, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_5":"g","key_8":"i","key_0":"e", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932434, "value1":0.659079228202596, "value2":4313811336647625065, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.260771, "value1":0.403057123436948, "value2":4416521488550693116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238541, "value1":0.2952177348030342, "value2":1538460010819351929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958540, "value1":0.5291106177059647, "value2":1249052435364791122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430589, "value1":0.8147197427498966, "value2":5526371695265553363, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431544, "value1":0.9355303369766881, "value2":4469095563720030593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640478, "value1":0.48586313377391904, "value2":7959068707072142404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144376, "value1":0.00848981726968387, "value2":5068460906328461885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416953, "value1":0.6734644377643653, "value2":2766510942957347250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885086, "value1":0.21498485778798626, "value2":8278007933868416694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537244, "value1":0.3154976316557619, "value2":2639365305979929980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_8":"b","key_1":"b","key_3":"f", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.020210, "value1":0.453949847740753, "value2":2024272373950233485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514022, "value1":0.3475698755294227, "value2":1372329110659095675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806258, "value1":0.9274447352712736, "value2":3920648533163431087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209427, "value1":0.014587282647209867, "value2":8695997195066251972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971326, "value1":0.2797692384280631, "value2":6419079357930277367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652286, "value1":0.5905911533568637, "value2":8712000656540499616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878040, "value1":0.48362174266483915, "value2":2926563391771284063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990329, "value1":0.5049151964579438, "value2":3650817862757599534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049823, "value1":0.838067458662296, "value2":8854264061775325606, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017442, "value1":0.8431632474037919, "value2":8852347705624487512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.544821, "value1":0.005649842126714668, "value2":2282778264986625535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.597518, "value1":0.2045781468759336, "value2":8575639431654539646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612206, "value1":0.3075892336013964, "value2":7673450384556490732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471060, "value1":0.40918086270080856, "value2":6747291739813622106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.513402, "value1":0.7422854995723118, "value2":4083993911078701361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965008, "value1":0.6372798813181509, "value2":1707224457136697946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.354283, "value1":0.9108473180297934, "value2":1020266209800871174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806140, "value1":0.14778437030634853, "value2":6105158525255417708, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194962, "value1":0.3473291379662418, "value2":3795992189004973455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.974776, "value1":0.09118542426518424, "value2":5134289787707010070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.339938, "value1":0.13774510006333723, "value2":562171884148417195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.550551, "value1":0.384009626532502, "value2":8609730106908869554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.654960, "value1":0.2639517045663951, "value2":1862456670074602807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030932, "value1":0.5143312749002444, "value2":1494847764422163119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695169, "value1":0.2423970054677799, "value2":8509443740065644494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713656, "value1":0.34108742339653203, "value2":4598279459368093139, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_2":"f","key_0":"d","key_1":"e", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.794261, "value1":0.9458347874797615, "value2":7302724440323717945, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364772, "value1":0.2365974372946532, "value2":3654435595389536462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395099, "value1":0.15838919107992194, "value2":1939247105684031089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335629, "value1":0.39545025723069543, "value2":2672917998276624419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074949, "value1":0.7876350399335282, "value2":625303995684394443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536138, "value1":0.5616610785403638, "value2":8164874977498368721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697782, "value1":0.23834340441765955, "value2":3951692500137347815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671649, "value1":0.6501720201004972, "value2":2955880776784904551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.787079, "value1":0.378893964304627, "value2":3193906322405958964, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.031359, "value1":0.29145414339441095, "value2":5615459484207384702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.306294, "value1":0.34557951146905197, "value2":2931839249030464715, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_4":"d","key_7":"e","key_1":"k", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357655, "value1":0.3923960009663459, "value2":8112783273863341075, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961896, "value1":0.23361672067408637, "value2":432132564805318676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098797, "value1":0.18420600660031175, "value2":4984332870487251645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.071876, "value1":0.6659197757567412, "value2":1553450733606247594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186866, "value1":0.7000117639690743, "value2":7329872261968800728, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406584, "value1":0.13046586013887515, "value2":1622087582581823179, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440301, "value1":0.11646408872647017, "value2":1011606322554570900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.136166, "value1":0.4328392754883894, "value2":2919499144416670917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318957, "value1":0.7075576973107754, "value2":398072267285532828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.012179, "value1":0.5668221134322494, "value2":6110074131405385081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830949, "value1":0.3334949924022553, "value2":7572722630631231675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_7":"c","key_8":"g","key_0":"k", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145965, "value1":0.365420169950548, "value2":2535895288587113465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.649060, "value1":0.16445860625759065, "value2":6921600715877525051, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517227, "value1":0.07192874427493294, "value2":6297114124078920008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832587, "value1":0.5150936291796077, "value2":181711691089923280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291455, "value1":0.5623557100802268, "value2":8762626146094505466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_5":"h","key_8":"a","key_2":"j", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.149447, "value1":0.5534139568709165, "value2":2510872908080100775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476658, "value1":0.4486907399118968, "value2":8992668719445662670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236318, "value1":0.49075032225313237, "value2":1924725427300127433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.561765, "value1":0.9668124792198439, "value2":990391456732695590, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884668, "value1":0.0011015591734681801, "value2":5327656656973278324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175200, "value1":0.9826652864577513, "value2":2876633651705447757, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379413, "value1":0.15362330796009416, "value2":7353421640256094445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.319517, "value1":0.5202924305313171, "value2":7456692117889227019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.122103, "value1":0.7187092003738956, "value2":7131563361103346240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_3":"a","key_9":"k","key_1":"a", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678103, "value1":0.7543751799594591, "value2":8526553775473008293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.643972, "value1":0.19798980984364278, "value2":5929329615236343098, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.831121, "value1":0.1528427519418499, "value2":6184723596853858850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432219, "value1":0.5515160206615604, "value2":23121424938113426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058751, "value1":0.6338015087216923, "value2":7816911191202995864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790739, "value1":0.8762741406918414, "value2":4555556160832468314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797544, "value1":0.8139645999303083, "value2":2761547225024668528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404353, "value1":0.9860244138424634, "value2":2908826677832473108, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016900, "value1":0.053160518991546266, "value2":8380107749249883562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.407973, "value1":0.7843730354485248, "value2":6824583973593836053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987174, "value1":0.7619687243629265, "value2":3319925462827235787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577835, "value1":0.20813952139985942, "value2":6437982890007555330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755320, "value1":0.3636597821442561, "value2":5470963049552133157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.897987, "value1":0.07151356373469021, "value2":1477083928227057783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388580, "value1":0.20932748154080844, "value2":3392543048891869159, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_9":"c","key_4":"i", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.025802, "value1":0.38577745314564454, "value2":868566582237921681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941032, "value1":0.17111301155402217, "value2":6961085515450408974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947055, "value1":0.9780343555878157, "value2":133683558419674847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954822, "value1":0.08800460018752372, "value2":7497584262448035095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063420, "value1":0.8100121364093217, "value2":3755200469512293547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438294, "value1":0.3424655143965952, "value2":4316249625206240732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809627, "value1":0.9673131880891396, "value2":4132748723398010856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439925, "value1":0.8075664782512888, "value2":864263057207692643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.084104, "value1":0.3818348864598292, "value2":2553923579282526899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.647912, "value1":0.46216103925581126, "value2":4639093581287452200, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369627, "value1":0.3422458784570339, "value2":4254456412295026385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.614963, "value1":0.5220645586983701, "value2":6072161543938690608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.321895, "value1":0.3049038907635511, "value2":6325538472569574229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464290, "value1":0.22336524838033933, "value2":9076126601794953657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166932, "value1":0.3553203900309477, "value2":5982106259976109422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803898, "value1":0.5280548625824253, "value2":4063002262098608883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.927974, "value1":0.17895618154031318, "value2":547646730434307817, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.303832, "value1":0.5511007572679634, "value2":2862350870063686751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520504, "value1":0.39724544886891017, "value2":6931266301104819935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941095, "value1":0.14786172741452286, "value2":3205668266543533832, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656997, "value1":0.32075936991205234, "value2":7391478133987747835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120921, "value1":0.3939864396610821, "value2":2909689659050189847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958790, "value1":0.3269314306247572, "value2":6783238272107071770, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.386136, "value1":0.861947664801586, "value2":5805659227499250652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.449365, "value1":0.5799629274092946, "value2":9193070830904141987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_3":"e","key_9":"h","key_2":"f", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.785656, "value1":0.3165816948400473, "value2":3116393083198778433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520711, "value1":0.6656585396817016, "value2":1017823568348313071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.650203, "value1":0.22215575308437716, "value2":1902945447598147226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249002, "value1":0.2961338302748422, "value2":1415432970559418672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_5":"k","key_7":"f","key_1":"e", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694015, "value1":0.7506751146797324, "value2":6142650591148786794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.304466, "value1":0.37747841635497736, "value2":5375036774405769501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.641004, "value1":0.46446471911344284, "value2":5955856288590241375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.684973, "value1":0.907579891922439, "value2":5346005034595738366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.111153, "value1":0.49826719629056415, "value2":7497650141562337241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830722, "value1":0.28960521156096847, "value2":6154503575103623089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392721, "value1":0.6834318849474948, "value2":2628637152649849276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969704, "value1":0.15429907011430974, "value2":5405475101757554186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536252, "value1":0.7000830425153579, "value2":406966410660924195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.807572, "value1":0.2012419938894414, "value2":4358282132524848204, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.820107, "value1":0.5076022682087846, "value2":4017461578732969062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996611, "value1":0.7422015873263831, "value2":436292068764222211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.922123, "value1":0.051072363737232486, "value2":268646986926660655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.729895, "value1":0.8676962023633001, "value2":2065960180781249750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636047, "value1":0.4996316174329004, "value2":3156670443682802457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431520, "value1":0.5204212362312931, "value2":8019726224533633908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.903149, "value1":0.5859452724568418, "value2":2288060944551638278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.070771, "value1":0.9932429756726906, "value2":6690408728176272233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.314453, "value1":0.3472880048665214, "value2":58548757977887750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627243, "value1":0.7139819549630214, "value2":7080915148891558502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755294, "value1":0.05408051807422248, "value2":6931097273017939069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282331, "value1":0.3203397893756164, "value2":8256062300842673383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.034396, "value1":0.9531256831279978, "value2":7765396458031448915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.115811, "value1":0.13116244832703935, "value2":3543488806097224100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666851, "value1":0.6887850450312126, "value2":5462261380989469479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525431, "value1":0.3139759113605816, "value2":8160395548303928540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036488, "value1":0.7437550776292858, "value2":905546975072552710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246868, "value1":0.7173148070188747, "value2":7273545640281703287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840343, "value1":0.20321210783475557, "value2":336636579624758589, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100735, "value1":0.8393194019944642, "value2":291716047034557387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853981, "value1":0.5202422200638643, "value2":5204943321202939039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512132, "value1":0.018314329801792806, "value2":961979132289563000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487612, "value1":0.7033581894769818, "value2":4186345519226619201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778169, "value1":0.7839413953257911, "value2":7874569661704504858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.587363, "value1":0.24066050930994098, "value2":2238921997040090404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.863823, "value1":0.22725313032865457, "value2":4922946319365439720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.462195, "value1":0.6013044688497777, "value2":137484525088955387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.967535, "value1":0.21528129199568857, "value2":5907496598149934999, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263462, "value1":0.3398984210102334, "value2":3627872884296903263, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778010, "value1":0.2649025290720317, "value2":2665113842043817940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895866, "value1":0.8505969892692369, "value2":1773042500322059709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475234, "value1":0.08631156102550058, "value2":519973765404949203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996762, "value1":0.6378946736901862, "value2":9195196376217348512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537168, "value1":0.2006763700535789, "value2":9067766312454029883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.585179, "value1":0.8887141585928169, "value2":1447763822787404407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937668, "value1":0.18798426398408305, "value2":5531072459849624180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417573, "value1":0.018969474714960195, "value2":8739232693787930523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432784, "value1":0.6772697684929758, "value2":3670574248400915486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945707, "value1":0.2017640539145947, "value2":6866599884426415644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715992, "value1":0.2770810284692845, "value2":6065196798538107388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724512, "value1":0.7694937233145257, "value2":5461354547379838063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555480, "value1":0.4116895522673444, "value2":6999095526200651536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357696, "value1":0.9694808723879291, "value2":2308259417403012034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.925321, "value1":0.9804626384591183, "value2":1344780667969714366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340972, "value1":0.8564397089342566, "value2":3335346544187347307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634021, "value1":0.2824134389967874, "value2":827846054793909627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535849, "value1":0.8383622181777869, "value2":3210193933415043724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949231, "value1":0.3423558587998688, "value2":8018588983650318493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.604636, "value1":0.6545573982070936, "value2":8784079580099061471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041080, "value1":0.6815494814472329, "value2":2413918152267969152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206623, "value1":0.7407991857310414, "value2":8688412282907699410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697192, "value1":0.9920334806937902, "value2":4058886921829417026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748394, "value1":0.3435945058384803, "value2":6002972956903494678, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421611, "value1":0.8151473391165547, "value2":261810376504255149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.027674, "value1":0.32343755952478886, "value2":8467224123375485048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205103, "value1":0.26583420276460434, "value2":912318083515976688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362886, "value1":0.6918507204150298, "value2":3042551716463426665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181414, "value1":0.9588817871484018, "value2":5500996639169097947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607786, "value1":0.1555201931458164, "value2":3667544046344919475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447991, "value1":0.6322575842993106, "value2":6581489005452708007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_6":"k","key_7":"k","key_4":"f", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602539, "value1":0.3803193364932477, "value2":501846031619014460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224149, "value1":0.14287189157740537, "value2":4390766248301994532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019349, "value1":0.9275903306872896, "value2":6065489708075344215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.753905, "value1":0.04103970388034392, "value2":3647389822252762296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.808552, "value1":0.6912948158348138, "value2":7168224379303111769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.383522, "value1":0.7213839418172744, "value2":8917533656021329315, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988681, "value1":0.4652567645908903, "value2":6088381044289659824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426511, "value1":0.8194664026578082, "value2":3767723603035206676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291449, "value1":0.11519619446576408, "value2":7042710433748403097, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.495051, "value1":0.7541019265512129, "value2":4805842413127851386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.498957, "value1":0.7657156651926107, "value2":4284045357936772257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.920671, "value1":0.557277837350798, "value2":7618404853124250629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.325130, "value1":0.8555228162118138, "value2":5568850179022711136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619588, "value1":0.8294092067729972, "value2":1498942661428438729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_5":"e","key_9":"d","key_4":"c", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932410, "value1":0.9716095495296341, "value2":6631794618463446087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.257618, "value1":0.6931656613099204, "value2":4999381616225061323, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.328148, "value1":0.3114535269637382, "value2":7689977371490904120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.280989, "value1":0.9111540107476489, "value2":2001844404871692409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722254, "value1":0.17067710980514048, "value2":9024628709036843159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367790, "value1":0.7784645051385055, "value2":4706281866949049190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368237, "value1":0.6741959290593132, "value2":2438020324428284656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885688, "value1":0.4581933301264591, "value2":977306309852850973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.856287, "value1":0.24855090559153642, "value2":5403228793051986733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.305675, "value1":0.6170232598989743, "value2":274658269233518106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908698, "value1":0.5064010761390361, "value2":5817465185742290417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757988, "value1":0.04843336165427696, "value2":5801707099278996481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.933565, "value1":0.22461787050531695, "value2":6104462847804923186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.720615, "value1":0.12587001891222954, "value2":7755893877648671182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642145, "value1":0.4752800242516813, "value2":1097468219850109153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975649, "value1":0.9950222857947861, "value2":323860279126996239, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.571701, "value1":0.13176848875241834, "value2":8974250013315108425, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841017, "value1":0.5108584547409708, "value2":1015654371149607255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.499539, "value1":0.7323245432905996, "value2":866168553464820252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.137678, "value1":0.9837633616038727, "value2":1980815750999371252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713500, "value1":0.6844141285754438, "value2":5215363796900492290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564182, "value1":0.6959446753092476, "value2":2124568431454851658, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.548476, "value1":0.8904291651425387, "value2":8960234619080614404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.334127, "value1":0.5777624482637982, "value2":3890652700139772457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.141095, "value1":0.3019838758248612, "value2":1159300889479734121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_8":"g","key_1":"d","key_4":"a", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800239, "value1":0.4131750973298026, "value2":2344128018741531119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369249, "value1":0.1589896462920189, "value2":4160545898250072075, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.198713, "value1":0.08298578373908365, "value2":50980726393447907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.773734, "value1":0.7496268790368541, "value2":7804146534404388324, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237426, "value1":0.27543463085841935, "value2":3694238253557813150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577321, "value1":0.6586006530161179, "value2":641077213271202791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368897, "value1":0.6785289242869411, "value2":2009007121053273154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.801586, "value1":0.7989756868311243, "value2":3470745118950983568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.035889, "value1":0.4089400550263047, "value2":3320771957585601823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760511, "value1":0.7386595086100225, "value2":3332011099556778058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.359596, "value1":0.8203562587923371, "value2":574506838747954548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063436, "value1":0.2579978917934866, "value2":6993045686428025271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369331, "value1":0.4769488047439803, "value2":3318100445225134934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795599, "value1":0.9012136284147806, "value2":5770204186306544532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.219070, "value1":0.20974721576307698, "value2":2356228020734101377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537221, "value1":0.1725299480275798, "value2":5073957380468101443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015458, "value1":0.4025573823516011, "value2":3282394548862748845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580078, "value1":0.9823965911617399, "value2":8476500551084420415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.979266, "value1":0.5236855845499949, "value2":680393934209420766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682327, "value1":0.844770708190905, "value2":8298068650764370173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350010, "value1":0.5806359147839931, "value2":1527219564263667157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206279, "value1":0.4978598735396454, "value2":8931241509064706089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135137, "value1":0.7138645301142236, "value2":2633600346218814000, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.981064, "value1":0.7353726130780065, "value2":268395459435449482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.526357, "value1":0.3584205029962212, "value2":7926609514114360701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145680, "value1":0.817655724537724, "value2":2809466879616878308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.877503, "value1":0.17147641272949588, "value2":1475184649319875768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049475, "value1":0.3904708182990629, "value2":8178189937335677971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419221, "value1":0.2498991121797316, "value2":2019493757980421565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646852, "value1":0.2981893785163364, "value2":4127223489573331187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194796, "value1":0.9706301430727536, "value2":3907775771119307356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135232, "value1":0.952467511486142, "value2":6824235396463011996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392387, "value1":0.19563387556471776, "value2":6531634041022343859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.133702, "value1":0.7965289882627238, "value2":6086508658030257940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.691192, "value1":0.21849926406658296, "value2":2500546592704430602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238250, "value1":0.8547288670113503, "value2":1751658935211452093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.259613, "value1":0.7399127039208734, "value2":3758431842842501346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.959851, "value1":0.06348104065773419, "value2":5803029266892181407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350885, "value1":0.07399715612950311, "value2":9184522919237034506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821915, "value1":0.9143789840874811, "value2":3150076904171453287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.554775, "value1":0.406185744411068, "value2":4390857727102845864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.026241, "value1":0.576960195810922, "value2":864775792792522430, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118029, "value1":0.6740072102143805, "value2":2738633295377346249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.669898, "value1":0.6063573387513779, "value2":8118600251510115903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406027, "value1":0.5408212514385392, "value2":836959541970766944, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_3":"j","key_0":"f","key_2":"b", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205752, "value1":0.9617267357194531, "value2":2302570688726041304, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.683346, "value1":0.7764943186065553, "value2":1516613683034572821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535753, "value1":0.8501812602308251, "value2":3781935852497505404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562975, "value1":0.5762183849456854, "value2":8803839397694965730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_9":"j","key_1":"c","key_8":"i", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224509, "value1":0.6945846689247714, "value2":8790065402136541035, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120301, "value1":0.5693297769803048, "value2":8817337009559896419, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535111, "value1":0.7168638232585879, "value2":1459446787633025168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590881, "value1":0.052466731276145914, "value2":6984133752001938547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800340, "value1":0.19161367226547096, "value2":8470840993920678933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797252, "value1":0.22382372006161758, "value2":960932677896506756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502545, "value1":0.4567752766501535, "value2":9200307528794857259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.919995, "value1":0.3074462546242729, "value2":1729230620360441809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.273301, "value1":0.5542454650469373, "value2":2007525431527262950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249396, "value1":0.30289020495328595, "value2":7647941374943306916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_5":"i","key_8":"h","key_1":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847062, "value1":0.8617179899648977, "value2":2309423222399622901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.770003, "value1":0.8185102377578417, "value2":8900535665947339098, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612076, "value1":0.2520488653737615, "value2":534199497930559417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867000, "value1":0.3056461531330615, "value2":8074358922929649800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543352, "value1":0.1267819654500433, "value2":2282337611780682090, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771337, "value1":0.3860477352422603, "value2":6614019503509668230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724939, "value1":0.35759761476037216, "value2":7730807476801277254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.769189, "value1":0.06006911990731757, "value2":6135075757488436084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.750837, "value1":0.22593301727381723, "value2":815096043795563480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.733823, "value1":0.37434887115879495, "value2":8767359811379324612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_7":"b","key_4":"g","key_5":"a", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110462, "value1":0.7613769242381963, "value2":679247224492133418, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.207493, "value1":0.26270378513285625, "value2":6127960652597929191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757551, "value1":0.17246010749021554, "value2":3746123411387113183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476665, "value1":0.530111421321598, "value2":7929787788154212077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080232, "value1":0.24610687758058403, "value2":4382073893358318099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.518795, "value1":0.13825391880428892, "value2":6023983814814871189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.411487, "value1":0.7443212678339678, "value2":3163265105788950613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_8":"g","key_4":"b","key_6":"k", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230729, "value1":0.6799575270403697, "value2":6569848391065499405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.251807, "value1":0.23368831551353111, "value2":9121098592064787356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450371, "value1":0.8049793990951134, "value2":6271606339009618863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268506, "value1":0.7393978258409317, "value2":3084814221880218023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453066, "value1":0.7167056491982889, "value2":8359177712079217325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840677, "value1":0.3999771937764131, "value2":5907070339651841877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853610, "value1":0.4459636741761099, "value2":2591090489482088505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.050602, "value1":0.9347582889921432, "value2":3586868729035531903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.460673, "value1":0.8573683555697056, "value2":77436103148660020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.791642, "value1":0.7214035249481627, "value2":4712458830503182070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711339, "value1":0.7704360355595858, "value2":4818602735855589639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_8":"e","key_2":"g","key_4":"c", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709838, "value1":0.980850892357579, "value2":5814687934592979738, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448259, "value1":0.6911929910350553, "value2":2331566896810690473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069070, "value1":0.7715792084831908, "value2":1237753144790060173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728905, "value1":0.6453677424318518, "value2":4827712214727464044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_8":"e","key_9":"b","key_7":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.581880, "value1":0.2677360178055146, "value2":8844278477287710309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969799, "value1":0.5783967942752313, "value2":5905445599124832757, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741372, "value1":0.012956038256400619, "value2":914084143172531236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862619, "value1":0.5947373167351414, "value2":3630517863485151782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.139536, "value1":0.8372408825864636, "value2":6952590048486505005, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803976, "value1":0.43722974429572603, "value2":2176830109855416805, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.534405, "value1":0.7241541542820893, "value2":4511693404596905586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_6":"d","key_5":"a", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574402, "value1":0.7055278131957177, "value2":1238394241461479434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.06494326977178205, "value2":3563720913525659633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878052, "value1":0.8371531564279497, "value2":587508487226380908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.672424, "value1":0.26126599404383394, "value2":3460643419757907518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408361, "value1":0.5841391902007207, "value2":8050508010125067469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_6":"a","key_0":"e", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.743507, "value1":0.5292956990399417, "value2":5743834222350004253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.024743, "value1":0.5111340297086123, "value2":4739422872896352154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053127, "value1":0.41791773967804086, "value2":5388222563610877961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489667, "value1":0.6344084078946665, "value2":6127655871996983541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.876031, "value1":0.49328901667737785, "value2":3143559155074364531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.278465, "value1":0.23241882776321474, "value2":805438734033729133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380252, "value1":0.6353489769585325, "value2":1619270428062881618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512159, "value1":0.8152126527698539, "value2":1368040827327524990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.317649, "value1":0.9550761539015656, "value2":8914758126172814906, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.723807, "value1":0.7854700255052053, "value2":7095402949367847209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.660856, "value1":0.4053530100702817, "value2":2145190666873464855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.659680, "value1":0.12646331821035406, "value2":1576238191679234703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.993554, "value1":0.8499325393155558, "value2":5900620451042065379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091338, "value1":0.41503145937749936, "value2":2042239402577329647, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.299870, "value1":0.687218145400586, "value2":7560464390267142060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.731227, "value1":0.15009967640134209, "value2":1886018024243215805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.348963, "value1":0.6737669308977404, "value2":5404336119559355867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162168, "value1":0.813527799284827, "value2":4016392262793002099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086162, "value1":0.7399808700194177, "value2":8264192680187392458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975595, "value1":0.11433087569708594, "value2":4671281305443756113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799363, "value1":0.3067546199025452, "value2":225546722481809384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.113667, "value1":0.6424657658728049, "value2":8718201548571703569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904642, "value1":0.5465983180036486, "value2":4877073522652743329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.467203, "value1":0.8662610389904551, "value2":399387690820391383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082862, "value1":0.5088605240671744, "value2":1531739105872671512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.039708, "value1":0.10189606169579954, "value2":5185895909677369244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068495, "value1":0.2805843145287995, "value2":486469505433183294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357082, "value1":0.75740796150096, "value2":2690075476895328471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492715, "value1":0.3004718942084548, "value2":7383744941741065497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.968147, "value1":0.5004304919533921, "value2":1731582924085558465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.991776, "value1":0.7394075904771168, "value2":4141740040514370033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_4":"k","key_6":"a","key_3":"i", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420254, "value1":0.45767543208547934, "value2":1964890506496503777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285702, "value1":0.1366965878727093, "value2":2681549871870198885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492204, "value1":0.9414045318691766, "value2":1575036830396652846, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086526, "value1":0.5271825907095232, "value2":3742001028067515529, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.296194, "value1":0.6916437451586811, "value2":384054583246095269, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478834, "value1":0.5030459095566234, "value2":3918028956922111738, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.029769, "value1":0.2164261260808611, "value2":4102211333190644591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102553, "value1":0.42618668443550417, "value2":7568518633836620405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.681434, "value1":0.7225843033115362, "value2":8160060706717748819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_9":"b","key_5":"j","key_6":"h", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210135, "value1":0.21812369739137172, "value2":5702739804858144278, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.786060, "value1":0.33904050622912296, "value2":1017560684015420231, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558541, "value1":0.14437656270299287, "value2":7701298697010262063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.818721, "value1":0.15017675533914024, "value2":7186230665665602451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910629, "value1":0.09672885609565206, "value2":6203887741849145968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069131, "value1":0.6075489323724108, "value2":9025102416378601029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_8":"h","key_1":"j","key_7":"g", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362866, "value1":0.8018878313034695, "value2":8037732655719146713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682018, "value1":0.5433657256031907, "value2":6040793575397278438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797323, "value1":0.5032038038329079, "value2":3803498450979968327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.829279, "value1":0.605237212493161, "value2":4367127372886022859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.999162, "value1":0.5082854850214861, "value2":5031512432939201812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269625, "value1":0.0007161270371379666, "value2":6545591527308584857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180859, "value1":0.6040920298819816, "value2":8926725137631299725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090602, "value1":0.5326464117623615, "value2":7752649715252147157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.191121, "value1":0.12836456615806674, "value2":7198874487475192980, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.686863, "value1":0.708342981200594, "value2":5465607373577957408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.527995, "value1":0.10447991595171255, "value2":5974156320762011232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.123930, "value1":0.6200031138885586, "value2":5406077264893957847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.356501, "value1":0.44396993176396354, "value2":6255319076363219685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.048169, "value1":0.08153644488046811, "value2":8582968949378456970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.112743, "value1":0.6933778275610673, "value2":4587548588265206284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323148, "value1":0.20142008973704545, "value2":4527540178335323769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.934744, "value1":0.08093337102458979, "value2":7857820814429250360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962496, "value1":0.6124995000870105, "value2":6489203228482067372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.160517, "value1":0.5248710269553188, "value2":5026165991297404062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932200, "value1":0.922378624964946, "value2":7930339398930216623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.900245, "value1":0.1182228551739615, "value2":1373671224306855417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790920, "value1":0.31880287032595506, "value2":6084131296995295782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237291, "value1":0.13557228914344616, "value2":4432801334887160283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.914465, "value1":0.02040606137133301, "value2":1808915457173232015, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941296, "value1":0.9439262294487866, "value2":6574505025862591303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.822459, "value1":0.9942489150539108, "value2":3506354754330173943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239733, "value1":0.4989511485969244, "value2":7173139334014716167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.444066, "value1":0.7594335589002584, "value2":3216271266780228569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866271, "value1":0.5931978644398052, "value2":3444180839017636171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421087, "value1":0.6530151828739431, "value2":19805620992839207, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771459, "value1":0.8073198535663951, "value2":4810887667402700825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832482, "value1":0.734748250215754, "value2":1123934068717543450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450110, "value1":0.15409548849312346, "value2":418445000125934927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.234597, "value1":0.3684594320773471, "value2":3167333011558865991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.403637, "value1":0.891873955682516, "value2":6523321617886226804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017496, "value1":0.5380231504237305, "value2":1565703384693902706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577818, "value1":0.942234653476486, "value2":6854337928979531257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886845, "value1":0.561600684880549, "value2":6700870407358100744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.493299, "value1":0.1842060640300563, "value2":7660717134967306529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_8":"j","key_5":"i","key_6":"b", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043840, "value1":0.2702623264568812, "value2":7040356564551372493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051963, "value1":0.5871637139837996, "value2":4534754735986232109, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.568830, "value1":0.08173891497184316, "value2":408699021018747548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694578, "value1":0.8818619724910839, "value2":5720963057839829442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095000, "value1":0.9081401378248263, "value2":8656658880665915661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977416, "value1":0.9899458292085819, "value2":665964700424941220, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.670945, "value1":0.6733933124715158, "value2":2460152217631000428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636539, "value1":0.6931186653224365, "value2":7451786524160575556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.398486, "value1":0.8808674761773307, "value2":6001178466503449076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.085292, "value1":0.9258669929119259, "value2":4480969611449139348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451321, "value1":0.8902429562501767, "value2":2399440975743970404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100110, "value1":0.4539977466656097, "value2":4906418767346623646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.725438, "value1":0.8779089763048636, "value2":7986118613837183355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.559619, "value1":0.42482197030409785, "value2":921145061767776729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885834, "value1":0.32962036175845005, "value2":5608652380611657042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619049, "value1":0.4697848554943135, "value2":6930715352479440739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768456, "value1":0.55002846829011, "value2":6856343919804361537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.052488, "value1":0.9193313692023829, "value2":8075338150518594907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.346748, "value1":0.7969202775153313, "value2":3854682510572365821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.668529, "value1":0.8901520814855493, "value2":3780299539456604836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.680780, "value1":0.9128377273315654, "value2":655380872616480476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.298499, "value1":0.8901980009691262, "value2":4124662751920860689, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.061822, "value1":0.922218588720657, "value2":6935461410888938736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.220215, "value1":0.8707922044422206, "value2":6415853705476229931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825872, "value1":0.6145357419122679, "value2":3712577567679334743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287572, "value1":0.7153750220855114, "value2":6766565708182723447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.258723, "value1":0.8623529521546892, "value2":3906675258214843555, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417686, "value1":0.6071420867010977, "value2":8210404273898330224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.939674, "value1":0.018125841559591738, "value2":2686797765813242922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062829, "value1":0.19220806032635843, "value2":4009115073519468001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.761409, "value1":0.282044445283748, "value2":2659364730071661456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340677, "value1":0.7867304449409817, "value2":4726626950116268051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.848210, "value1":0.7451256645145572, "value2":6540697575244319830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082209, "value1":0.08477415388397415, "value2":5473839595744351618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562389, "value1":0.08808192976810916, "value2":1578232710018420800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445878, "value1":0.7573342205320885, "value2":3400644244684606362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.603775, "value1":0.183625034999939, "value2":7793796792648526034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036048, "value1":0.8359424936170327, "value2":5865912253243965224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866722, "value1":0.3949434009829113, "value2":3290170127727456353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697995, "value1":0.8423729542207233, "value2":4861550929219709907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.211200, "value1":0.36512924472577757, "value2":2972205895387325421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227557, "value1":0.22341834017020207, "value2":4806018666421767946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574952, "value1":0.6007129713703638, "value2":3849253119017333239, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741721, "value1":0.5702349573898813, "value2":897876839681174700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379915, "value1":0.6564162460066056, "value2":3131015984288737246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.229369, "value1":0.456646560158155, "value2":4778476677749084799, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.242697, "value1":0.4142421381714439, "value2":7101584169529051655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.212755, "value1":0.39066964505760526, "value2":4253776777664568253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_7":"h","key_9":"k","key_5":"b", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837272, "value1":0.15824518985564762, "value2":6812701504471384455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.104360, "value1":0.9940983043766688, "value2":2203777894935310413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391080, "value1":0.5579038729371453, "value2":6131596744931664143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265096, "value1":0.7511430046090084, "value2":4283281105409640096, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881500, "value1":0.8197410181139487, "value2":2167116001911138380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964911, "value1":0.6839484284574476, "value2":2392821775915043629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_4":"f","key_9":"c","key_0":"j", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906639, "value1":0.12232760875324074, "value2":9162174110871211714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562264, "value1":0.9910780181559037, "value2":1543791401715243110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016345, "value1":0.3277590263870748, "value2":3851358761838506924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963516, "value1":0.13686033235694836, "value2":2786944428195217311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.446494, "value1":0.051498079678560585, "value2":1973784887996536653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350210, "value1":0.13860925037657423, "value2":1354426546825223222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358378, "value1":0.28813722807183184, "value2":4138306657967174744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455527, "value1":0.9787455514136848, "value2":6829598111029855692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.208210, "value1":0.4291478612877578, "value2":749843861604187954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.330867, "value1":0.6408478280255169, "value2":8395361043273182125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.176999, "value1":0.9117058417477248, "value2":8428888804090519109, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091618, "value1":0.8122911740955318, "value2":1478086552909570392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243584, "value1":0.06727976096288107, "value2":4178958373440436949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.127002, "value1":0.8012772426876036, "value2":2149729620077994596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_5":"j","key_6":"e","key_3":"h", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461395, "value1":0.21653382210662625, "value2":2558734709934645897, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932379, "value1":0.43744123497894677, "value2":5779172669858974850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180562, "value1":0.6091002318007619, "value2":2233652952119987641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_3":"h","key_9":"c","key_0":"e", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.738334, "value1":0.8147115963605253, "value2":1038206878786066414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943138, "value1":0.1805265868912326, "value2":8231024461542686979, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_7":"g","key_0":"j", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007238, "value1":0.30166077327328183, "value2":5211541986455490237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926840, "value1":0.7197646392974351, "value2":6170887305282807365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.976392, "value1":0.02561261869316712, "value2":8003552327575750503, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665296, "value1":0.38463860662667215, "value2":702575998172013860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.957627, "value1":0.8365108478779814, "value2":3172439569974822855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.779959, "value1":0.005457784803842251, "value2":1217373846044030405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814440, "value1":0.03586677782979768, "value2":8668868565483927092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590538, "value1":0.7998407978822464, "value2":5953384113011158992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.381146, "value1":0.6861901618066985, "value2":806723358919392729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.324953, "value1":0.09051509245266014, "value2":930835793453842933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.309761, "value1":0.44326721766264904, "value2":887788347593465760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722069, "value1":0.08849038862609183, "value2":7352758650578649855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558827, "value1":0.3337734956738152, "value2":8685182570627965041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715649, "value1":0.23109099835442082, "value2":1083379478056182665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_9":"h","key_7":"b","key_8":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964540, "value1":0.6330159342037405, "value2":1167652807667781397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795126, "value1":0.5778697926785736, "value2":9140994998625427103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236481, "value1":0.5802899488963301, "value2":2895083535971839167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996259, "value1":0.2785697344481934, "value2":4913023416457428502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.477938, "value1":0.484568023326054, "value2":3495500670092878690, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261065, "value1":0.11116858735231369, "value2":5869730879322934574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_2":"f","key_1":"a", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185846, "value1":0.7126003929744608, "value2":3434450920636329132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841676, "value1":0.3736127841617722, "value2":7802733756976416976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350997, "value1":0.582318689502649, "value2":372190874054145543, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555394, "value1":0.12831996488184216, "value2":4582628444562419674, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.155140, "value1":0.3906336986690914, "value2":8237405956495459279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367910, "value1":0.741845875474543, "value2":8082126523839106878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.950768, "value1":0.6652471104213182, "value2":8268690822588065094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.433269, "value1":0.7980991574588373, "value2":8513183309513339766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.874369, "value1":0.5464494520072651, "value2":5885072115794893058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537599, "value1":0.7405427933790399, "value2":6635799236257663587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.955763, "value1":0.8778283555530513, "value2":6592488434360400176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.582015, "value1":0.879034635958457, "value2":7219576376735925925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388984, "value1":0.53843340060884, "value2":2254745170299712210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646131, "value1":0.5101339977981793, "value2":3292039546456501555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033442, "value1":0.6456192738711094, "value2":4906576027749942202, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.836645, "value1":0.48635849410277077, "value2":1404791026873124424, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464295, "value1":0.5617144285499786, "value2":7990958033367681704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010923, "value1":0.9807071857311617, "value2":5683359887238098850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438005, "value1":0.5935141575872156, "value2":4865067817966075444, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227310, "value1":0.6681961275740043, "value2":8551103550465421536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186394, "value1":0.25048622317521113, "value2":2888181774636358576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053227, "value1":0.10049698552895067, "value2":4557545020451680363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621258, "value1":0.9388516977567649, "value2":4055003743201350923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227442, "value1":0.7022685194913532, "value2":3740668486633940756, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640300, "value1":0.06434066744840804, "value2":1646848528935407928, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162609, "value1":0.71410216780028, "value2":7370378990581242781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512792, "value1":0.19345496541059937, "value2":4424425935560475985, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203787, "value1":0.9476247108925144, "value2":264480497401916863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.076408, "value1":0.13927961349104323, "value2":1149972870879073163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806676, "value1":0.6172364818942355, "value2":4996985075367830737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.313484, "value1":0.6715999817727264, "value2":5237271908928184120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947617, "value1":0.1400224644800571, "value2":4212016629330280660, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.695969, "value1":0.13461968177274677, "value2":8209146459721882476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466156, "value1":0.7851823238051198, "value2":4609596535178738402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047192, "value1":0.6615735622970099, "value2":3688971736191236071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179244, "value1":0.5525116533205023, "value2":6958696123355987767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.836535, "value1":0.8752088579621973, "value2":1772355477524999885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.385661, "value1":0.6669392703121872, "value2":5026611864593335422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.122798, "value1":0.5624470316374681, "value2":973613507611825241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663323, "value1":0.2836760055083927, "value2":8274369123408107345, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988003, "value1":0.07587743066420682, "value2":4947294410032829574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.240844, "value1":0.4161174869019418, "value2":4077015391953706312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.945230, "value1":0.14944194749229664, "value2":7660491006096000021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890111, "value1":0.20296484407777726, "value2":1824051875710962915, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522740, "value1":0.021855326277136235, "value2":3218235817889269142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.679843, "value1":0.5055217891288131, "value2":4822706751533920431, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191340, "value1":0.9959364873278129, "value2":3932069232230299078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771462, "value1":0.4557053136591391, "value2":184726250007256859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581655, "value1":0.689124829253459, "value2":1817463261736430161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.894089, "value1":0.9069576891707435, "value2":7318133204837024663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.307941, "value1":0.4227283465946924, "value2":356860997794924081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085219, "value1":0.7548908197263309, "value2":2736908468355530081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510864, "value1":0.6045243894505045, "value2":3203823484629938988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194038, "value1":0.2945791412290687, "value2":8542013145729544847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.822784, "value1":0.7046664475635507, "value2":6531211714773873879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958034, "value1":0.6210073688384752, "value2":5171125204919187937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245514, "value1":0.07787807249020082, "value2":4796876093300411441, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922409, "value1":0.026305095785565266, "value2":4204774996273721582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618949, "value1":0.8021374519845728, "value2":7055530836155104119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_2":"c","key_7":"d","key_1":"f", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.842155, "value1":0.9078167657864898, "value2":659406508816846003, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.429827, "value1":0.6861191669827441, "value2":3716781946259346478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.721875, "value1":0.7721427864001064, "value2":1559707715553693565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.667085, "value1":0.41573209008012846, "value2":770484632734171477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567897, "value1":0.2688265887173671, "value2":8731672977339492909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716882, "value1":0.20533155732399708, "value2":8158121924487423466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.450278, "value1":0.45425248453866135, "value2":7472174068145252016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700766, "value1":0.38900719316872584, "value2":9070338695929499459, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296131, "value1":0.7588298197955197, "value2":1078200536047373074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644620, "value1":0.22063287558750297, "value2":1455208095653529502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826663, "value1":0.9059127191998474, "value2":163331002989197798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.375920, "value1":0.888489428778294, "value2":3478637008435710026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795211, "value1":0.7169431218180555, "value2":5240161460250645540, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686886, "value1":0.1615738416501897, "value2":4522663398730909152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.641311, "value1":0.5919216049468233, "value2":4264524771451405531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415500, "value1":0.8013193758641248, "value2":4655946330615841665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310579, "value1":0.03464731335525151, "value2":2648033740620709930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.135452, "value1":0.663217728481436, "value2":6656249157262608643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036458, "value1":0.7592021704654797, "value2":4708550885998497867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_7":"j","key_3":"k","key_6":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196106, "value1":0.9109556832409251, "value2":1182178953302502536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897060, "value1":0.30582754840738535, "value2":5110613128180106106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.386690, "value1":0.17482144522900334, "value2":208181032234291542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770888, "value1":0.5775727956988672, "value2":7308314003124624706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710417, "value1":0.9182102368115501, "value2":8001324021984156096, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.133639, "value1":0.8132459830268748, "value2":6362172937464044603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313132, "value1":0.6425828291871425, "value2":3812023207458774027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816183, "value1":0.12876194798023052, "value2":353578462778793882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.537076, "value1":0.3677093772206428, "value2":3410314947529692873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.785801, "value1":0.5650444568188182, "value2":2518267372616726657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793640, "value1":0.36840754337253134, "value2":6490561754060041624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277029, "value1":0.5742325627742796, "value2":4576556191237422408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086053, "value1":0.635715694469195, "value2":3276323627400814300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622335, "value1":0.6664883465636151, "value2":6846298203429885729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784935, "value1":0.8345083778637773, "value2":2989860444276517490, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394440, "value1":0.3881444024090245, "value2":211938452740442471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.150923, "value1":0.2049623118713732, "value2":7830583450265904275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334186, "value1":0.8877162218809062, "value2":5530254516648658024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919131, "value1":0.645089444938841, "value2":2752404297334193415, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.550415, "value1":0.8672407553696915, "value2":7296354502501329771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.251810, "value1":0.12582294112213796, "value2":3850461942759838134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.768322, "value1":0.5089660327127362, "value2":4637000702345102317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.5790505163445078, "value2":2746113057580625956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310041, "value1":0.7419516660136578, "value2":8573283812679768621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567732, "value1":0.20310332375311568, "value2":1684484347666739261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.149277, "value1":0.8537753954365711, "value2":3792414057901721222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019813, "value1":0.3193924681887993, "value2":4416687297621574996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555536, "value1":0.2678716255116121, "value2":1978875629658477083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.676000, "value1":0.268194488154906, "value2":9208242650214533822, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603232, "value1":0.7475788122275303, "value2":6003796300446172845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272526, "value1":0.7633068823447601, "value2":5387009257178750855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288629, "value1":0.0999015536587548, "value2":5230082044659723490, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198708, "value1":0.5411261720205245, "value2":1324824529248190153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754319, "value1":0.5181119376899533, "value2":7586548988147520888, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_6":"c","key_0":"d","key_3":"d", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.974535, "value1":0.6605961675334583, "value2":4149492144268522705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620393, "value1":0.0400929610839578, "value2":2611761419308325853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489379, "value1":0.20432994152077563, "value2":4859071163254701004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.497240, "value1":0.21587046902788842, "value2":1674028941320053082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227611, "value1":0.31207060679536985, "value2":3782150896328417443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.813479, "value1":0.9346689305684607, "value2":4170814668135359561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832568, "value1":0.40374034609384823, "value2":4890680963042405726, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.868761, "value1":0.7456764740945415, "value2":7985095078472385264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_5":"k","key_2":"f","key_3":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566095, "value1":0.11155632539641938, "value2":1650668369012843098, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516140, "value1":0.2921171335805605, "value2":699561345288206492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.009998, "value1":0.0698415110112407, "value2":2175710360877139700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472788, "value1":0.6894980351369026, "value2":5185256568181045963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013027, "value1":0.47916067130802836, "value2":8339740169449446777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947875, "value1":0.15641692100908322, "value2":1874863435098609183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834625, "value1":0.4725529822941945, "value2":3313442976129641776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032027, "value1":0.11526706346113103, "value2":4474958127457336496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.643426, "value1":0.2365209546681749, "value2":2452872778362311452, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399009, "value1":0.10587590188915917, "value2":5343568589706680846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_2":"j","key_6":"c","key_1":"a", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464026, "value1":0.23089518212237822, "value2":6365432144590215713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.573860, "value1":0.41263343273878106, "value2":7100420790565954895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575295, "value1":0.15635829479176244, "value2":972701597090717176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_6":"f","key_8":"j","key_5":"b", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685505, "value1":0.3500577640159365, "value2":5296267831822852754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382859, "value1":0.8962437052233099, "value2":5000561412494049933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793198, "value1":0.03594880833972008, "value2":5558834996866286026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296341, "value1":0.6801030998229524, "value2":9104706563374398078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.044529, "value1":0.43868239396683095, "value2":1119681077532090572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977458, "value1":0.7636375641015349, "value2":3738349085487316798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.769690, "value1":0.38941161495083887, "value2":4119290182206919942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604117, "value1":0.06703731437988915, "value2":2776279989058320233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.107061, "value1":0.9783306976372779, "value2":6724603539014036116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.243947, "value1":0.3058624284281911, "value2":8607064800338196111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784300, "value1":0.47413357934186534, "value2":6062775812546347462, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053929, "value1":0.8033894774680731, "value2":3798712131176287721, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997633, "value1":0.3495480156862711, "value2":6358034341550557079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399888, "value1":0.5577617871085867, "value2":7606548109917229902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_4":"k","key_2":"k","key_3":"i", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.343972, "value1":0.37033984748970067, "value2":3823813661350967339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741859, "value1":0.15725835002679114, "value2":7931796757818180793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118790, "value1":0.8710092300018802, "value2":8432660813300591545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_6":"k","key_8":"f","key_5":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.514050, "value1":0.3589657867466283, "value2":2807064518724813388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105213, "value1":0.48313119234822866, "value2":9012983220702483909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.289587, "value1":0.28297188108556603, "value2":8293007746329622374, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.682523, "value1":0.2601723288341452, "value2":5132563496112960923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.521609, "value1":0.17923659255236565, "value2":9080059564709771845, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.866470, "value1":0.8472875420381546, "value2":7662519070278706881, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139688, "value1":0.701824998288288, "value2":8626519461512263205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693655, "value1":0.3931946923075373, "value2":4088184802361327943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.733184, "value1":0.33562267701400733, "value2":5057125078949459009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530556, "value1":0.8546701921585885, "value2":7172802034424433024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890826, "value1":0.9156630138494082, "value2":1106383994017980379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736946, "value1":0.28531565085928856, "value2":7492657224478970683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184271, "value1":0.66321078827276, "value2":6482907214355252145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771765, "value1":0.8506763394943703, "value2":2997501176379232751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621435, "value1":0.4219907917118007, "value2":2371448704479482567, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522551, "value1":0.4559374479237847, "value2":8705737330976472895, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579601, "value1":0.3374291561460269, "value2":6118047213487455086, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.911838, "value1":0.24595613407282738, "value2":8543292459890755038, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.461163, "value1":0.20032563894255778, "value2":8271655059958021803, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480016, "value1":0.537301255549281, "value2":131039455325957900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264510, "value1":0.8207403430077567, "value2":448751349067313394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.908904, "value1":0.45150207048162533, "value2":1484572129461087565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279527, "value1":0.48678886025337537, "value2":4675520724179222107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.983786, "value1":0.5074627713755523, "value2":6761505393797376999, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407769, "value1":0.8974462165990488, "value2":4427875231986566903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906297, "value1":0.32841731679412833, "value2":4853751460801403413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.910866, "value1":0.22248621009204866, "value2":6153217292256934376, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481193, "value1":0.07256346779714405, "value2":8357563464914791440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819751, "value1":0.7821155606881222, "value2":6350148722815418199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925016, "value1":0.08017277463129999, "value2":8127038680866850197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380866, "value1":0.4236203787530011, "value2":6102770882422449664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.687520, "value1":0.18833602677282602, "value2":2416669714696967955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_7":"j","key_3":"j","key_4":"i", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392864, "value1":0.12162108913858492, "value2":1037740633966701101, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555618, "value1":0.5726263308688928, "value2":5340970336126619965, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041150, "value1":0.1866697214008993, "value2":6089316958837860364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.848709, "value1":0.5912728689790879, "value2":8374928084508664533, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.633435, "value1":0.34567925121735993, "value2":6074382604796755677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615342, "value1":0.43884173283368255, "value2":2552875653507489786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.420473, "value1":0.024593195105800023, "value2":6998883295968298787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.493473, "value1":0.517385578986543, "value2":7898967292563966468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398409, "value1":0.9494831815326669, "value2":1043777551847385399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.261653, "value1":0.605095923821751, "value2":5798307460445919671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734130, "value1":0.17339556093076888, "value2":3888875155355529135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.234024, "value1":0.07147953681129654, "value2":2220989894337620932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.552276, "value1":0.46428310479897317, "value2":5429095800661644988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.547014, "value1":0.9160534111766008, "value2":4258800689197649539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084253, "value1":0.06651726425775369, "value2":8202506388672279723, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137868, "value1":0.6230337836484641, "value2":8788058834810483014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.168246, "value1":0.1571304566786965, "value2":2933470821094626337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043250, "value1":0.3445304512204906, "value2":8908780418296865224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542269, "value1":0.007638770771913262, "value2":975662304048103149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016306, "value1":0.44304617075801217, "value2":4700290207925657878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967184, "value1":0.9943193022594318, "value2":7141028805491571993, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522277, "value1":0.8578463560518583, "value2":6372322792910116627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541009, "value1":0.5200416543540609, "value2":5687465452183940942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742322, "value1":0.4918651213341637, "value2":7530197332852248825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.478705, "value1":0.02234257118225825, "value2":381364684161431163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.401319, "value1":0.36716744081601327, "value2":3165286506094547515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.154587, "value1":0.8570612240935895, "value2":440607090836829314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268457, "value1":0.19708890772544582, "value2":716490672588175731, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976461, "value1":0.37921805973698547, "value2":5187015251060220885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.291361, "value1":0.0347133343718353, "value2":4081610772727292868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.831910, "value1":0.20329085836851984, "value2":1212790782212732680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_3":"j","key_9":"k","key_2":"g", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045270, "value1":0.742873126094037, "value2":1857680286082818546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719581, "value1":0.8427487279865438, "value2":1584262554399141329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.878973, "value1":0.7217975560465744, "value2":6936890459625848218, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.786254, "value1":0.7388730574867226, "value2":3591226182562159088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719189, "value1":0.6138101273145392, "value2":4767024689432495577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335758, "value1":0.6941935445321573, "value2":1310712364678635982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333397, "value1":0.7084933562313812, "value2":2679977702860384198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816868, "value1":0.5704940444863804, "value2":3982836666436033897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.806625, "value1":0.7550148139027509, "value2":2819861429300922010, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155981, "value1":0.3978060292285169, "value2":2375324886880679840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.172764, "value1":0.8807292451628046, "value2":1585412394695366401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.127700, "value1":0.8691901922846396, "value2":4006452105841009084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.937137, "value1":0.6079865671377581, "value2":9014517461395944209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601939, "value1":0.5587486969852118, "value2":896670201927694097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147910, "value1":0.7860716932621791, "value2":8828747393971238771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271640, "value1":0.8634829980251869, "value2":7534202338930685599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_7":"c","key_8":"d","key_4":"i", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510609, "value1":0.273735584071857, "value2":7995135221543970576, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.424702, "value1":0.6570665232819806, "value2":7130224512963405605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.120070, "value1":0.4366525687584962, "value2":8321790572084844233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.355286, "value1":0.9376756075719435, "value2":7612697762460955574, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057269, "value1":0.9603884122588556, "value2":5675558649074747456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_9":"b","key_3":"a","key_7":"f", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.396116, "value1":0.3241173308406254, "value2":2149479689060994101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271843, "value1":0.5095285278231975, "value2":5564561021464735924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389082, "value1":0.7189491284948987, "value2":8539354123799965742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780310, "value1":0.7959419420056967, "value2":791327040768245285, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496100, "value1":0.7137676161743793, "value2":7249423453544847575, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.320840, "value1":0.7088322490051288, "value2":2259284238716415015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.588238, "value1":0.2616266339940352, "value2":7616111814012126210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510231, "value1":0.34548313055527086, "value2":9146415948654460714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725142, "value1":0.832268560021817, "value2":8449163586612650016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722343, "value1":0.5127452250176201, "value2":8372677466999750797, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455239, "value1":0.7274771017691084, "value2":8558710521155328448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070961, "value1":0.5108403289338641, "value2":6822737602377250548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.799042, "value1":0.6579145757389928, "value2":6367849243718047663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997752, "value1":0.29826795257630173, "value2":7963803897292403489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019959, "value1":0.5329624946388745, "value2":6166072735161874859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183220, "value1":0.1192005017394279, "value2":1414525889342507585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.216504, "value1":0.10069780302425961, "value2":4147229267124117622, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184537, "value1":0.9674252449470503, "value2":7196918589322334021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.982196, "value1":0.5121645885657403, "value2":1331574421408029626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.807710, "value1":0.6453473320699041, "value2":5880434997959490048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139645, "value1":0.45151332154483154, "value2":5206983362834731524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202627, "value1":0.5085399708189182, "value2":229347926887383465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703675, "value1":0.1410911985436997, "value2":1995545208705825638, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.916538, "value1":0.9588604837943727, "value2":3014447442999476863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174096, "value1":0.8442896590468587, "value2":1669527921851537286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268618, "value1":0.05460739672623512, "value2":6145595254763691216, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184210, "value1":0.08111723273128417, "value2":2941741438359406510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419569, "value1":0.11197378055505244, "value2":7190302664501414682, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.080457, "value1":0.46189706497698463, "value2":3739595486194011632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685402, "value1":0.5911751311740201, "value2":375795342652216999, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584152, "value1":0.842391293090085, "value2":3305999218832830126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_9":"d","key_4":"e","key_8":"a", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719567, "value1":0.7117389665869482, "value2":4436375112558311543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.599224, "value1":0.7507823724841362, "value2":5289861475224212810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298087, "value1":0.03464563720261393, "value2":5487979858306400729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084762, "value1":0.4408096997046924, "value2":8782838705036394284, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153226, "value1":0.8396685378218853, "value2":2305238685769866566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875795, "value1":0.6283427563062458, "value2":1424824369894963651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.841870, "value1":0.2951891910717211, "value2":1229301210242052079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036891, "value1":0.10272064811801765, "value2":8629352909535519019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.475886, "value1":0.47884350384173996, "value2":2002152937943396093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286578, "value1":0.5511122934004854, "value2":6456656081230025536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180709, "value1":0.591154696871619, "value2":1085648256858149575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.549035, "value1":0.9073632737596287, "value2":441521694946529713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644644, "value1":0.42900060724414874, "value2":7252379057595536961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.185531, "value1":0.28679213225919725, "value2":3666625025829964686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858171, "value1":0.07899073830408582, "value2":8847544646507443860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.9086649445849005, "value2":7300771702176978507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.217129, "value1":0.8286760370555245, "value2":437314161092307860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.092457, "value1":0.3257453112147972, "value2":1241192378749271122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_4":"a","key_2":"e", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.318459, "value1":0.6218522470963161, "value2":2601033448014127071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543632, "value1":0.47194572326020234, "value2":330858668685674119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.494889, "value1":0.42021023869128465, "value2":7698787812508472546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173188, "value1":0.6333586665476976, "value2":1031119021359267335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392225, "value1":0.13669516747392368, "value2":2722787133031533732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.418321, "value1":0.8852713566769926, "value2":1145580014369872247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499498, "value1":0.37652715283253, "value2":6023088039348052644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049271, "value1":0.7428393913560761, "value2":4074778390717770530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344096, "value1":0.618674195595606, "value2":8432036346015765748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.504726, "value1":0.8381168181634259, "value2":7490362504268179930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130009, "value1":0.04787900547347569, "value2":8930977988943996511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517353, "value1":0.9123390757414803, "value2":5406757776090744464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.159780, "value1":0.2711186767871672, "value2":6650537343113966916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601454, "value1":0.036596921749510625, "value2":1801819188066798655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740877, "value1":0.9080583731663923, "value2":6623419932438799475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.743094, "value1":0.9900165531318826, "value2":383393257357043043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.939900, "value1":0.8769075090910828, "value2":487693007619851718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.102543, "value1":0.4399163118191967, "value2":5888144134235313914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917706, "value1":0.6339047693941521, "value2":6760479052672151531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108610, "value1":0.06502295112846754, "value2":8902773220268340372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.756394, "value1":0.7104082109611765, "value2":3854497302576556911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914430, "value1":0.39800381262255646, "value2":1413572596816022205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.046273, "value1":0.9197740465700138, "value2":5994606503835738604, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.042126, "value1":0.35565383574544296, "value2":206374188969693152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.753926, "value1":0.14736292648599036, "value2":5720098218146494963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138925, "value1":0.14592833793919333, "value2":5845534364661717380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_7":"e","key_0":"b","key_3":"b", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966713, "value1":0.5691036450918006, "value2":5571037157834825483, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128296, "value1":0.522798351805377, "value2":7845314774564626095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.374869, "value1":0.16619436230032464, "value2":2683834913042811436, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349899, "value1":0.0806742622823487, "value2":5654997107573600799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621137, "value1":0.6389883746416457, "value2":6718840923740903865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505989, "value1":0.5736885226925782, "value2":7660274947298694696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245218, "value1":0.06067210631905806, "value2":7481373150924938933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.961402, "value1":0.40476409107709094, "value2":1656152426641438063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064546, "value1":0.7710994608894077, "value2":5484900997853498427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686047, "value1":0.46617534097616353, "value2":1998746207850635494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195207, "value1":0.8036664106551048, "value2":21823515460647768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890318, "value1":0.49246668518608533, "value2":4290226524505777397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.662549, "value1":0.0648629447687713, "value2":2600958480076392029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867346, "value1":0.6103284095540984, "value2":959086414558873258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036038, "value1":0.22211264871806694, "value2":2970074654495356013, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772742, "value1":0.47254264158251036, "value2":5967291460715844391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.765254, "value1":0.59761349012731, "value2":5045361763860538939, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684477, "value1":0.3727521458297325, "value2":7120290254833886255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782498, "value1":0.5592140587339981, "value2":1560278611367944657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472665, "value1":0.4451202433666956, "value2":7528293771469761549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957028, "value1":0.6722101190310112, "value2":3256843532996983591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_4":"j","key_9":"b","key_3":"g", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.558411, "value1":0.33974258843823785, "value2":9219292136379640073, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686456, "value1":0.662252940687323, "value2":2872679473486076312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.664702, "value1":0.13252769744708778, "value2":3782836160183678190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340522, "value1":0.321610730663863, "value2":1993266611063130503, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_7":"i","key_9":"k","key_4":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738508, "value1":0.03455404630849952, "value2":5265512962120119813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722717, "value1":0.039393801449424636, "value2":8411289366884053387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738985, "value1":0.4374781921483304, "value2":4532229627185520929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.959594, "value1":0.06784833824743756, "value2":2235113542571170507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512001, "value1":0.45518592645372824, "value2":1219663694779720192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699152, "value1":0.9199280757111055, "value2":920010063978439254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870981, "value1":0.7153328294012858, "value2":2241596700674662533, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713651, "value1":0.1871397130196525, "value2":5525165593287246022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608421, "value1":0.2007302921644417, "value2":7082345286303554347, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.898979, "value1":0.4326799832087361, "value2":4510789740667402710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542323, "value1":0.5916017742256197, "value2":7802920355385118767, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245771, "value1":0.3115277557229622, "value2":6710572841494695733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772755, "value1":0.17023256633885028, "value2":4693930877366777744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.247931, "value1":0.4740059660910414, "value2":1391972395280563550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394287, "value1":0.02366479635408087, "value2":7477007108410228541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108172, "value1":0.971283160522281, "value2":6224196629196175206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932465, "value1":0.6431474969443486, "value2":6996302573266425119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070979, "value1":0.4324217725560448, "value2":9202735673442875059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862734, "value1":0.5425012681866345, "value2":8705521243115097026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207036, "value1":0.7438270358882513, "value2":1482607831466689197, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345448, "value1":0.3875837956213489, "value2":2480267181897079015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553288, "value1":0.3343119844132966, "value2":7890867607759034801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559986, "value1":0.2851751818045875, "value2":3231450807250993029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.008285, "value1":0.11082046459395005, "value2":5670330804396842703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620145, "value1":0.27733408115115116, "value2":4537754734930200229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.861443, "value1":0.7565577041788646, "value2":216193021440411568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584594, "value1":0.7717911844132257, "value2":7036437829052540741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.066975, "value1":0.07703464444888349, "value2":8332259918814404840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272133, "value1":0.2064168890828292, "value2":1598424380451243838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725661, "value1":0.5107106855646196, "value2":2095325894892271000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.992271, "value1":0.04606905650348332, "value2":3458383290691046043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988671, "value1":0.9012802494548691, "value2":3265213363984040214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344354, "value1":0.6315603617727772, "value2":1819854718616345426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198753, "value1":0.2443944990728996, "value2":3740319967339712109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294599, "value1":0.7030675431685813, "value2":5821390349436487736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.233177, "value1":0.5520329333372563, "value2":2208761416514593046, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.369614, "value1":0.2829108305094591, "value2":2978362585536403835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.484777, "value1":0.3351273859753424, "value2":1348038406704883991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511345, "value1":0.6897076794113861, "value2":3140482719657458044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780083, "value1":0.056509953898858746, "value2":4983435501891849142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.023345, "value1":0.08633016397064507, "value2":439606973167008387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013043, "value1":0.8750924151560185, "value2":2563435998029443275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929738, "value1":0.6942458871164995, "value2":6742189975038174280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348725, "value1":0.6477096282239411, "value2":614033611645126767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.148404, "value1":0.3122137082601719, "value2":5921845943066980452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_9":"e","key_2":"d","key_3":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.892407, "value1":0.0024857877019035433, "value2":78557787569325314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770110, "value1":0.4854945232954409, "value2":4441111957237018619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969650, "value1":0.028054363655575613, "value2":6620021166465815802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096277, "value1":0.7642864471977222, "value2":7079078098912113925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104248, "value1":0.17068081288522707, "value2":2914828831179785715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.033230, "value1":0.6691862134057828, "value2":5879037419816229961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442293, "value1":0.918687680755694, "value2":3522143514446827859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340520, "value1":0.3316554635970253, "value2":3298849581857165314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.005701, "value1":0.22902534413133244, "value2":4040792449962896607, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.101103, "value1":0.4459003073823555, "value2":8364403783112020861, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.269361, "value1":0.9829136286924077, "value2":305552246857799140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002507, "value1":0.10735284014448321, "value2":2548523454772398525, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516501, "value1":0.3420030652884641, "value2":449036192165412633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914771, "value1":0.6948526439267081, "value2":3274351745805138467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151382, "value1":0.9010351248418295, "value2":6225882894853282790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_7":"d","key_2":"c", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.797287, "value1":0.450517653615541, "value2":6406142276072623664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575226, "value1":0.6947969923557978, "value2":4732050542115525893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.256780, "value1":0.29401472322197164, "value2":8790173514967991405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791599, "value1":0.6204111480891676, "value2":1162220986630177027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615289, "value1":0.17893814852804246, "value2":3187373261649831292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738034, "value1":0.9979754782079555, "value2":6006267840987043849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735591, "value1":0.011768322537597629, "value2":7974963296156276842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.751213, "value1":0.43700619365470833, "value2":6294325004534159296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781793, "value1":0.6112369782572161, "value2":2263959578091534198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118524, "value1":0.4979095180792778, "value2":5142183829715514199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.574787, "value1":0.8076799963798312, "value2":1975720601465651847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814750, "value1":0.45891950977715895, "value2":4074714357288466936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579803, "value1":0.25446438731696874, "value2":7191932871100542216, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722158, "value1":0.19082711905600802, "value2":1422914021938617784, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342798, "value1":0.5097927561705997, "value2":2281093093995561768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164991, "value1":0.06555881689103157, "value2":7152427537795186892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.316516, "value1":0.1580165967231405, "value2":8769573796829946768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548667, "value1":0.8164945933097875, "value2":1223332369807607717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174435, "value1":0.5629514369055348, "value2":2359802095953741925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.561139, "value1":0.9847446485189013, "value2":8748242508292199474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604206, "value1":0.7084164989246987, "value2":5662112818525056068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.069641, "value1":0.9392432577384211, "value2":7977134301111651930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977024, "value1":0.8770253339233329, "value2":114419076017958040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.981104, "value1":0.0262761349528985, "value2":3404389245833544901, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.309389, "value1":0.6700115572733264, "value2":1746800498285488694, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907890, "value1":0.3847355476155844, "value2":1062772964889350348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.557849, "value1":0.8423014863985597, "value2":8693557035023467587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174807, "value1":0.14044078429336324, "value2":2899359413194652810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907265, "value1":0.7213413890977288, "value2":711482054709798828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730322, "value1":0.010569544047203483, "value2":2272852869521847556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047968, "value1":0.8749328675131659, "value2":3313417752475725526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548996, "value1":0.13476734433713453, "value2":5686844894034380593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090199, "value1":0.47038291207998856, "value2":6737581800965808102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372121, "value1":0.4982647921591242, "value2":2960213658998005236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.732328, "value1":0.6794810773912932, "value2":5614573251611720393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195822, "value1":0.7817234737401128, "value2":6738900276029001186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_9":"g","key_4":"h", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173099, "value1":0.7620323539766012, "value2":6195639668903126539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739085, "value1":0.8805144351999263, "value2":6990390850908433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851839, "value1":0.19515034480983603, "value2":5022120185556407326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.764811, "value1":0.07516380443788656, "value2":4953415333432850814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.954192, "value1":0.6068771093686028, "value2":1921210459347583907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.501412, "value1":0.20889031917804166, "value2":7266160086492287738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798358, "value1":0.44429041626860627, "value2":7311639409429341020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200103, "value1":0.5998566082715615, "value2":3968426630272414916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443243, "value1":0.7996897987041263, "value2":2955385724626539485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808129, "value1":0.12934958318030565, "value2":8640009016586784249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890136, "value1":0.395437996374284, "value2":1375026123479284573, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.726245, "value1":0.8383327509398333, "value2":1069495836938595505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464465, "value1":0.7522623057444222, "value2":3745372652823842076, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958198, "value1":0.6062226980863115, "value2":692634818001600691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336639, "value1":0.3569530703633769, "value2":5114963101620697516, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153507, "value1":0.29997973310668347, "value2":35895264299031809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496206, "value1":0.16796883917469935, "value2":2457448015461697922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699773, "value1":0.1320495753882598, "value2":2684330380511330426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_6":"i","key_9":"b","key_3":"h", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086489, "value1":0.01686874394925538, "value2":7266008503964487472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.755161, "value1":0.12039999679109185, "value2":7441080332932356910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481861, "value1":0.21023924595248417, "value2":7215904118134021296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141970, "value1":0.6959505249259283, "value2":5593000831636710131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473262, "value1":0.20484780325876686, "value2":1515840223895186260, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.610674, "value1":0.37794171859625975, "value2":7210398121561053631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784669, "value1":0.8314547953224727, "value2":6889739271959092821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348279, "value1":0.9806070629868892, "value2":6665537531662525499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_7":"h","key_2":"i","key_5":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342228, "value1":0.8292222808866697, "value2":8140150785342027171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049991, "value1":0.20259901011770426, "value2":860224417352173376, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442752, "value1":0.9271842959401567, "value2":8958731671751036609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288552, "value1":0.19606129029896613, "value2":3803062421310719257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_4":"c","key_5":"c","key_2":"h", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851767, "value1":0.8040030606139774, "value2":5022190100385685639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258187, "value1":0.06995124309795013, "value2":3290080872961653036, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967676, "value1":0.08273649902951419, "value2":5115999717780317436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.972416, "value1":0.3332024948916442, "value2":3195696687672274736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273980, "value1":0.9801180643314533, "value2":7124826086691987600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962051, "value1":0.7078535280449856, "value2":2450804349449720576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_2":"e","key_6":"f","key_1":"i", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342560, "value1":0.21100802753268208, "value2":5026066029722000095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582068, "value1":0.9724895772631098, "value2":9045330876857967867, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716662, "value1":0.5239131397726235, "value2":6699936731993788174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.630988, "value1":0.8757044551570439, "value2":6009726873690078427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.617422, "value1":0.029531083291305976, "value2":7932542565063790762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389736, "value1":0.40122875818931175, "value2":7350974819717892233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.470743, "value1":0.286118127257554, "value2":2903264335442728018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_6":"j","key_1":"i","key_4":"i", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118668, "value1":0.42900363662380264, "value2":2786819190490835132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819243, "value1":0.7832376096990535, "value2":6768946688624968401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519924, "value1":0.1784273794737174, "value2":606279348173747859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575499, "value1":0.8580592683181654, "value2":6065349113607731318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.952434, "value1":0.8198422458926743, "value2":6520729939047239531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180605, "value1":0.504162836348111, "value2":8107251100494910096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.680620, "value1":0.9192990596381249, "value2":7916466116307364506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421505, "value1":0.6014385702769156, "value2":6172295401410316312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581350, "value1":0.2695750525680962, "value2":6289934971199391350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.244031, "value1":0.7392367672457897, "value2":7438364572557990950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801331, "value1":0.014860008784846704, "value2":6873217646463864251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875352, "value1":0.3975554355639885, "value2":4390005070727485119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929440, "value1":0.40393323123767977, "value2":1148796038957803851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.817616, "value1":0.5647949954501486, "value2":1024684537861148034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.990318, "value1":0.25287778647718107, "value2":7093365367237549470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741736, "value1":0.023580263258422104, "value2":1518420105714291210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463012, "value1":0.7888469551975262, "value2":6786560784050350679, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449551, "value1":0.2636704318808076, "value2":7884141870151198099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419454, "value1":0.6010995928923429, "value2":224905435491328214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.322105, "value1":0.1331880379524159, "value2":7369022064377940075, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581063, "value1":0.603596835974457, "value2":4386876721702326915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.857080, "value1":0.5842046650044923, "value2":1066043362420191253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191051, "value1":0.4639324153416964, "value2":8874234446483833192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298751, "value1":0.20423387563647288, "value2":8774956872647827067, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924593, "value1":0.0659156088635184, "value2":3634318708597769338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404558, "value1":0.7799254034556268, "value2":4510574800615418821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962467, "value1":0.8168896478129019, "value2":7986325005390816690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622579, "value1":0.0904343638172895, "value2":8283142617455012372, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710038, "value1":0.6909066964841821, "value2":6203472414968382073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032660, "value1":0.21417322011190834, "value2":1054486238769744890, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801225, "value1":0.2765954328390815, "value2":442385272245544502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116566, "value1":0.6031158754932586, "value2":3770751187916956934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.284086, "value1":0.055471202806315845, "value2":7266763851735180811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559271, "value1":0.28967037620921054, "value2":6147439427772857051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224654, "value1":0.5513198770455077, "value2":7761756752100166184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575799, "value1":0.07079456843554317, "value2":4847399039818291257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819125, "value1":0.4806719127445899, "value2":1129446367168224920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730720, "value1":0.6013248235298089, "value2":1610990221268346838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.593103, "value1":0.8196760148078414, "value2":6644540894797627889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.218509, "value1":0.5227005302649587, "value2":2525326877519094680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.292767, "value1":0.09698193120132978, "value2":8960471678202836654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858612, "value1":0.675005592095046, "value2":6028830951911645471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736558, "value1":0.24026583264924886, "value2":1360237166500303266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826970, "value1":0.3893255587955024, "value2":2292740904231475335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128577, "value1":0.3146179907253604, "value2":164469213437964556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_3":"i","key_4":"a","key_0":"g", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003953, "value1":0.33225441021352176, "value2":5713576701192138110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352438, "value1":0.12247814806052679, "value2":976940527664803795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.434520, "value1":0.6138506637398143, "value2":1920264902732481602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.731843, "value1":0.8305061213621728, "value2":4607426852903927950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_7":"h","key_0":"c","key_5":"a", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032970, "value1":0.8273657553543087, "value2":3103821801573462769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833737, "value1":0.3431117940438562, "value2":5529881494203381864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.206561, "value1":0.03180903277463739, "value2":1534192978519074976, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598146, "value1":0.7943659660871121, "value2":2512296846803666990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.830318, "value1":0.9077939825338383, "value2":1289520021940183737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409047, "value1":0.694416059529231, "value2":520453313543014186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.949791, "value1":0.6369046983298985, "value2":371741091848582403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354931, "value1":0.7633140731862355, "value2":8981576909845465787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.061767, "value1":0.7699262325323719, "value2":79439898840104021, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.941133, "value1":0.26448399729496197, "value2":6448749789711445941, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.339821, "value1":0.5246395493337294, "value2":44122001398655312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359654, "value1":0.5954751506379091, "value2":1668054016792098588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.304719, "value1":0.17854657722051143, "value2":5240836642486639588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533350, "value1":0.9102023121053504, "value2":5223044570382986189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.495583, "value1":0.8380393694454433, "value2":2826946501048985900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_6":"c","key_0":"f","key_2":"a", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925864, "value1":0.06057194714527472, "value2":4827880370625631466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200061, "value1":0.3196222334980913, "value2":5536801387533970020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.635607, "value1":0.5926951079597541, "value2":4317404702469315811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041722, "value1":0.2038750977962134, "value2":8603762180622472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.565080, "value1":0.5851130292640377, "value2":8612148593533709646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090894, "value1":0.12207888531940794, "value2":435323305893709131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_9":"e","key_4":"c","key_8":"j", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.071849, "value1":0.7914473708034223, "value2":4833048612300779383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202326, "value1":0.6798241827808086, "value2":5321585738808934000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116141, "value1":0.8646385559973755, "value2":2431723757311861306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.074497, "value1":0.2771958266692314, "value2":2948732814035791763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.453427, "value1":0.9750934919782419, "value2":7598695164797163349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517105, "value1":0.6447380734003394, "value2":5945582855956594632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969670, "value1":0.8792534113647463, "value2":9118825598855667812, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064653, "value1":0.7641915057577142, "value2":376649068007551056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.299983, "value1":0.5328541897726483, "value2":7848443219053578824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814787, "value1":0.5822532477986045, "value2":3517444920674306021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645109, "value1":0.6788316811493683, "value2":2399412759017043209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_5":"d","key_6":"f","key_3":"h", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104970, "value1":0.5150104887448738, "value2":39162375392832614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124629, "value1":0.9415126690810315, "value2":2487867259401435813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847494, "value1":0.8236014719422879, "value2":8522949906604300533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.694932, "value1":0.5232173520376328, "value2":2865775556504465271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.824405, "value1":0.33768880558190767, "value2":1449467820646971158, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263091, "value1":0.800562879218074, "value2":4640064437824793404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.545676, "value1":0.6991321506153548, "value2":504604265126996830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685750, "value1":0.256315076419413, "value2":7805803213074768353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.955572, "value1":0.12777100392825705, "value2":4697380262379050863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.052725, "value1":0.24897441126326256, "value2":2806761823519892967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832231, "value1":0.7123036010256201, "value2":5558652604185663810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_7":"c","key_3":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.779475, "value1":0.42379979395746137, "value2":5877566589967590752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_1":"a","key_2":"j","key_0":"c", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.056823, "value1":0.7557073972848881, "value2":1462009193456216531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179551, "value1":0.33562095061974395, "value2":2905800790209406415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.946943, "value1":0.7895726694044407, "value2":4556871779015432465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.792955, "value1":0.9478490399693646, "value2":6469796413879524879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302294, "value1":0.40539394874343054, "value2":1479528610985409190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.859086, "value1":0.8533535931566633, "value2":8328258195824345992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.912801, "value1":0.4080605346123484, "value2":921699075685228177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026911, "value1":0.5160934551500268, "value2":4192638354189389027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.145578, "value1":0.4697942761887252, "value2":1007860316702817500, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224036, "value1":0.21446416617045144, "value2":2612501104515894392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.083418, "value1":0.3800566396023636, "value2":1387864613630168710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.483137, "value1":0.9606563644656662, "value2":2761802265245174240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.855910, "value1":0.606558104786761, "value2":842530041019109994, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332745, "value1":0.01260394609809363, "value2":2781673523626108476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198691, "value1":0.0952542911286906, "value2":4639530496495690336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207605, "value1":0.29598976404510996, "value2":149815254996856780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.219100, "value1":0.7173110134058144, "value2":207519709019999249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.065034, "value1":0.7683750910989782, "value2":6262446867113607884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362737, "value1":0.5652820007764047, "value2":4138085582551770816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.790498, "value1":0.45469936210178996, "value2":3874789041764013688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581214, "value1":0.8566672554079884, "value2":5701861631669426000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741360, "value1":0.7962411718913905, "value2":3357161583638308077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_9":"k","key_1":"a","key_3":"a", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.934536, "value1":0.07212743130410183, "value2":6319820377917318485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258971, "value1":0.09357358589212993, "value2":5081900353713330302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.436226, "value1":0.5553015007542973, "value2":6124312980620622172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566508, "value1":0.7132604990750537, "value2":8485961090508027563, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298396, "value1":0.4460940570451377, "value2":5709626152776289179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596697, "value1":0.3774598732286396, "value2":1819820653928002535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.758549, "value1":0.49680425119422045, "value2":1371518397741898898, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151965, "value1":0.9897517149153644, "value2":3974511699577597667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.223787, "value1":0.9859897177824236, "value2":6368178524733411270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699805, "value1":0.9240609624872184, "value2":4091684677817576336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.640771, "value1":0.3605713604653968, "value2":2196974552050802581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917234, "value1":0.3769625716544593, "value2":7094294232648752669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541154, "value1":0.07721056009394252, "value2":2450341303960944649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188174, "value1":0.511305577711157, "value2":4677738772360990938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277439, "value1":0.006756936522771483, "value2":2736398221812059244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847832, "value1":0.23362336838695744, "value2":6292362019324286334, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.840677, "value1":0.6137390254843741, "value2":6506004284907452986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.397331, "value1":0.5291020730916409, "value2":766558029100390419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.668711, "value1":0.6677953904066025, "value2":2813624786174976455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.728127, "value1":0.9455687108064199, "value2":2598623203001035108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.260429, "value1":0.5485107722176485, "value2":8238880717060307339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407939, "value1":0.7503222721986553, "value2":8088756690439902034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016860, "value1":0.6281421773683825, "value2":4463756144226416929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.592662, "value1":0.7215969122220701, "value2":2957728488172560734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596916, "value1":0.6365196560861767, "value2":3939747841173152393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.650813, "value1":0.33343709461482135, "value2":8822031185214434530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543088, "value1":0.5660730417813042, "value2":6651394201039088708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.416422, "value1":0.2907903630149851, "value2":5811650096165674198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505086, "value1":0.44660978158387593, "value2":6270587282710629926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.055558, "value1":0.394254430161103, "value2":9153429600073084016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.578079, "value1":0.6160395423413576, "value2":7388869261196091596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.347142, "value1":0.28357580684654937, "value2":7651284402887473938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089369, "value1":0.24785364470389606, "value2":2827626134616355884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_2":"f","key_5":"a","key_0":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922968, "value1":0.9908293125287131, "value2":5006327563319926460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489701, "value1":0.9764768678976257, "value2":3780590525108581330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700790, "value1":0.8524762188330061, "value2":2298426148247437198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.884155, "value1":0.17838161460811935, "value2":707461345329966226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_1":"e","key_7":"b","key_0":"k", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.877319, "value1":0.597182710013635, "value2":66780576193296735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906533, "value1":0.44570088765160515, "value2":1159099942400157753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555879, "value1":0.6225620185948739, "value2":7381136800707660104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.498993, "value1":0.9016213008618695, "value2":8432726482609273880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349195, "value1":0.16850895992044404, "value2":6332393888509943818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530337, "value1":0.8558953203814225, "value2":76710422944574508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345065, "value1":0.8230787557550705, "value2":4201898391845439571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947222, "value1":0.9688974381514411, "value2":2441269560109987314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.287425, "value1":0.20362983597374928, "value2":5349125652009205097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519644, "value1":0.3548258293385043, "value2":1594647307470342039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_5":"i","key_0":"j","key_1":"f", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.027626, "value1":0.9120926865897546, "value2":356446699135295995, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.054561, "value1":0.22396147232559485, "value2":3930608769790828524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.388931, "value1":0.5101134733621842, "value2":6568513065035568570, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567065, "value1":0.79529931520039, "value2":6002480040724485306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449050, "value1":0.9291008790205846, "value2":741995070657009132, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798788, "value1":0.9907520697579564, "value2":2343010721096789225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.927945, "value1":0.8814555609090619, "value2":6729196069806990782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917877, "value1":0.15366203128840322, "value2":2435285873402812217, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.301622, "value1":0.20876721045801763, "value2":1403445590939959710, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.745719, "value1":0.6060177757203955, "value2":2370675136825151538, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156169, "value1":0.40315211694606023, "value2":5606845635294528121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.651978, "value1":0.02572219217737101, "value2":6216284509197748039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.395139, "value1":0.4352070266196555, "value2":2437710317158637331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.132661, "value1":0.3845242564807311, "value2":1179550652058134811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_7":"j","key_1":"k","key_5":"d", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.088075, "value1":0.27124710684397163, "value2":5954367646517477044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_3":"g","key_0":"e","key_1":"j", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213664, "value1":0.9844242730115798, "value2":1430390936289238362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070775, "value1":0.6998833636894136, "value2":8373393772820367148, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213733, "value1":0.44329246485521806, "value2":6509431828213171835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608249, "value1":0.15493446331886743, "value2":4384868943457502322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.956693, "value1":0.9523530599130996, "value2":365504939691815078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.432058, "value1":0.16692183631193722, "value2":4864357898977097624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.303592, "value1":0.934106512625592, "value2":7515104000227729499, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333787, "value1":0.6736717480454248, "value2":5830960931165083327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359385, "value1":0.922695683809537, "value2":6703804740319670530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.211845, "value1":0.13745320161077518, "value2":8688931964841266452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600304, "value1":0.365028686112319, "value2":4172510140835977605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045527, "value1":0.1293058694163136, "value2":4396801118892738860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124929, "value1":0.2927998769649342, "value2":7840410733128704268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.708707, "value1":0.42917010198000677, "value2":190549078586280269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543921, "value1":0.45395915446447055, "value2":4038857743678185292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001958, "value1":0.8799882402246043, "value2":8959405737069039234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834357, "value1":0.9787241750393116, "value2":4713255732772978232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398402, "value1":0.3356690698861162, "value2":8600435717396946203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.532477, "value1":0.5790086501753019, "value2":585720405314800565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995284, "value1":0.019519382747907175, "value2":561302586535494029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.492003, "value1":0.22729751026433825, "value2":2059445045999328988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.357595, "value1":0.2653305071097244, "value2":7325567281409701416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423155, "value1":0.8492553718976581, "value2":5551377313853459727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582438, "value1":0.9616765075659243, "value2":1026983801153932760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409214, "value1":0.1250667617190597, "value2":5921255804215527024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744368, "value1":0.8513042177093529, "value2":6603153879156409175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970658, "value1":0.2306199750722779, "value2":2906238010168281574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_6":"c","key_1":"e","key_4":"d", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520199, "value1":0.9665587748785213, "value2":2458054282507819610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.098618, "value1":0.12617188969368395, "value2":2230070381051915192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279517, "value1":0.4244894688924968, "value2":7931154210918628962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394850, "value1":0.011258876506077731, "value2":3335326787820059838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.672924, "value1":0.7327755721792956, "value2":3880440053684337972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.431109, "value1":0.6741923566632704, "value2":2327920765233612965, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.659350, "value1":0.11708731624685002, "value2":276841347975235584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.115617, "value1":0.43754612855223973, "value2":3365515217954889202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533611, "value1":0.8399047779975791, "value2":2365721152492621420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265333, "value1":0.7911288127976842, "value2":5955361860868770714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966474, "value1":0.3536325699896996, "value2":7032196306810796442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684964, "value1":0.8626527282208771, "value2":7197488766581698381, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559000, "value1":0.010659679933132453, "value2":4115322864019466158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455110, "value1":0.6204915686658947, "value2":6428809732964624602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625937, "value1":0.9727096027334052, "value2":6229477409536583105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.113032, "value1":0.5903167177404812, "value2":4156394072465021166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567689, "value1":0.10316863931938577, "value2":8418749122094896416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.987774, "value1":0.7873883846984865, "value2":2030919041886568670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.811061, "value1":0.4992724231992665, "value2":3986295112752729017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194651, "value1":0.2743659705657923, "value2":583251715501918289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950390, "value1":0.5427606927000886, "value2":2986321212639280881, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598120, "value1":0.20329762636615106, "value2":5612645284463886443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950214, "value1":0.4515603514606786, "value2":259527570321604761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645723, "value1":0.13300625888317788, "value2":9108039921133288290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001569, "value1":0.01777574006312956, "value2":7180543411595530153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036902, "value1":0.19075245150411826, "value2":5889621655868191557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929644, "value1":0.8218270153578531, "value2":6338216490369302155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036799, "value1":0.8121072864023562, "value2":4644698086978403677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099804, "value1":0.41836860978181317, "value2":2663944418369780756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.607018, "value1":0.04603283781595935, "value2":7178332091114199020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075733, "value1":0.9728379954214806, "value2":4310309256078899557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795422, "value1":0.6466237769859894, "value2":344630441945165516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924050, "value1":0.5005809462467858, "value2":4504593684309524057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352892, "value1":0.9539405805618134, "value2":996952878654521350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141322, "value1":0.01789552453746491, "value2":5938050187519951412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.783153, "value1":0.1436758434821978, "value2":6441131001960903605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473891, "value1":0.3495022243472323, "value2":4306013569645375103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814977, "value1":0.38372259399528474, "value2":436380091488564183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716628, "value1":0.6699570981955947, "value2":3523603923463352771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735953, "value1":0.6776358805955892, "value2":5856380565120662945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_5":"e","key_8":"b","key_0":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265185, "value1":0.11607380521583967, "value2":5582686822797693437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516162, "value1":0.2994616004522488, "value2":2261467086953311820, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334873, "value1":0.14715552224004888, "value2":8821110853892469057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.400930, "value1":0.8143295386608986, "value2":1312967392198736440, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196057, "value1":0.5352481987346313, "value2":3859881997560184862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925763, "value1":0.823842231642209, "value2":7076092787875459652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.012976, "value1":0.08033889071311322, "value2":2194666423113168625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656711, "value1":0.979250223824098, "value2":295513105593707440, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542943, "value1":0.34216024465522854, "value2":8482137672426222632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738294, "value1":0.6164314293888586, "value2":7863948057381466211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957221, "value1":0.4133406835193543, "value2":3905154346081616319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047765, "value1":0.9114235810126055, "value2":8926222646986865918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423884, "value1":0.30572384015704446, "value2":2339988500116190675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209371, "value1":0.07849561730590998, "value2":3882974907897625962, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310209, "value1":0.2535587414218041, "value2":7470349035360557369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988088, "value1":0.2879292667105676, "value2":5363951038000132107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947720, "value1":0.958399617505033, "value2":5218176225849015970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.674130, "value1":0.9617762098247892, "value2":5270132257081817867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279827, "value1":0.8686925044217091, "value2":7240549587345744089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.257178, "value1":0.28758053646406956, "value2":7938343410176527229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870350, "value1":0.16917076304978226, "value2":9111760754875505635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.763299, "value1":0.8896924750809221, "value2":6658856857804173307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_8":"k","key_1":"g","key_5":"h", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734200, "value1":0.44507702947189576, "value2":4594583173060659683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155510, "value1":0.8897408774019222, "value2":3124062956444214018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.363722, "value1":0.07362703906856644, "value2":6124606823090260320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336581, "value1":0.7586482338064128, "value2":2691495825421343596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075863, "value1":0.7294045132994561, "value2":8264827508243049447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108381, "value1":0.45788844677640256, "value2":2616045646790504337, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901873, "value1":0.08770240902249962, "value2":6640573405482547401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.438350, "value1":0.25439050102218247, "value2":7286046839968613612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.314379, "value1":0.33872622756746645, "value2":2585280362910555462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736079, "value1":0.12615475607591273, "value2":8133018761204018534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130003, "value1":0.7963354396181769, "value2":6450859077912673686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.485449, "value1":0.39497609142163287, "value2":170013329412671465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.06350907823991539, "value2":18163724507539892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.6759106102023759, "value2":8613373636446320004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522262, "value1":0.9415564313481927, "value2":183076459384497822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.456571, "value1":0.3755600914749928, "value2":1317951286539480179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713142, "value1":0.3684027597862527, "value2":6666246661014977211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332747, "value1":0.2712525974797622, "value2":6685862283475642897, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622289, "value1":0.8099425380336615, "value2":6696254947892698280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099077, "value1":0.4218807485485893, "value2":8151022600373825319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833739, "value1":0.471092674976307, "value2":4241908968763096086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512079, "value1":0.6891384899122228, "value2":2796503546861546091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.405406, "value1":0.9060026283921703, "value2":4773855974745798571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742427, "value1":0.6129480541080172, "value2":3917404693410650609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_5":"k","key_2":"d","key_3":"e", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.613376, "value1":0.3584257069399756, "value2":6068105032813229930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.759402, "value1":0.29210317562134064, "value2":3082388413784210214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090526, "value1":0.4678159387104084, "value2":1605658432958093758, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380672, "value1":0.5086572266239265, "value2":2924449619880684423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.249203, "value1":0.6593022058411281, "value2":727243123088870599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.142921, "value1":0.5973096622193752, "value2":2056855353114653381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041929, "value1":0.03439471294074195, "value2":8729944365498889284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970035, "value1":0.629094069595545, "value2":954705184774295714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656521, "value1":0.4935396027418025, "value2":1513112375773530412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286143, "value1":0.590796213295746, "value2":3085171514913296735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.611048, "value1":0.3503905353305475, "value2":5805006250107454516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_6":"f","key_3":"e","key_4":"h", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265940, "value1":0.8719050400063297, "value2":2334709836967925044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.231945, "value1":0.9482208594424564, "value2":7605532380274260726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099422, "value1":0.6387093062831974, "value2":6416034503475989540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862301, "value1":0.0844793706976894, "value2":1724212296853776891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.882958, "value1":0.45080094639121554, "value2":6596820877226155907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851802, "value1":0.784896015737414, "value2":2267862818127250105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.871761, "value1":0.9219727566018483, "value2":9044274636085577036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704109, "value1":0.6618033501935818, "value2":8585487696070011938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.863670, "value1":0.6882846261541673, "value2":5478374167595454238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183651, "value1":0.21762989261712523, "value2":2054870959404109040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157246, "value1":0.1072057187986972, "value2":2474773268716838947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263088, "value1":0.6424740570966222, "value2":4311694979316615362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227492, "value1":0.6060012408935248, "value2":8262744174339677603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919259, "value1":0.26635737492611994, "value2":146832998544852566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_2":"e","key_0":"d", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.810229, "value1":0.6433893313883111, "value2":6020303671131449049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.021406, "value1":0.5663729785218883, "value2":6847532330100114835, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.331141, "value1":0.03123867674281506, "value2":2370510435982901478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313836, "value1":0.7842831796528092, "value2":1464971588517507800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_6":"i","key_0":"j","key_4":"h", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147234, "value1":0.232833406436683, "value2":5302769737190544079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423280, "value1":0.5646235246180139, "value2":3807301011411244203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.872535, "value1":0.3508318181055984, "value2":8035632162257252295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516267, "value1":0.7376190499237748, "value2":1779371132519351560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.020480, "value1":0.5129193623789473, "value2":6378565762108475442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421514, "value1":0.72712434334131, "value2":3908844968753938444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026297, "value1":0.8255820903801332, "value2":5180017205009891488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138610, "value1":0.5467990020300735, "value2":8134249053330675938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754861, "value1":0.5832263769294902, "value2":8216858329154999597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655676, "value1":0.45804041306603793, "value2":8403704610668639637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.091295, "value1":0.7313452902241809, "value2":6274232113944917429, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370397, "value1":0.7082796943852117, "value2":16646752364177844, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.072185, "value1":0.8009986319191017, "value2":6123195610574092125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068380, "value1":0.3124676060355689, "value2":2569282749677030904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013897, "value1":0.9321729707819904, "value2":2281258981052686264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.963470, "value1":0.9927608945020866, "value2":6050801587231871156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481229, "value1":0.3689051456876514, "value2":1924733693360792633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.295891, "value1":0.44913357839758283, "value2":8069037361024844952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_7":"i","key_1":"h", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409485, "value1":0.18423629567656025, "value2":5066355149165531813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744404, "value1":0.9737949013567385, "value2":5174295290317701561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739701, "value1":0.4976173273573324, "value2":5658855721642744182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.186283, "value1":0.419361633943128, "value2":2053614790636840527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566841, "value1":0.7094596520198607, "value2":2148895338889314160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.215921, "value1":0.34276587375112927, "value2":1645566772308625552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.701909, "value1":0.22008909623955855, "value2":2295629303145372991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002697, "value1":0.1765088617877308, "value2":4201797060324467420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354799, "value1":0.6538892310533279, "value2":4577607391337663337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345385, "value1":0.3843950816957309, "value2":8800458811056296695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988457, "value1":0.5588868714021769, "value2":8433496452370061074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415009, "value1":0.1746188760147135, "value2":6912110645446824396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655363, "value1":0.9949350156565988, "value2":8110347995943103660, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.242069, "value1":0.48932951486152193, "value2":8286237027514157124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085368, "value1":0.0689091306398061, "value2":5859648999386501040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_6":"e","key_3":"j","key_4":"d", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.661208, "value1":0.7864155291083853, "value2":7826276349217880531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.803222, "value1":0.6015539667825235, "value2":1102772758151131198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.458180, "value1":0.3809696927779604, "value2":4439664400270353254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334077, "value1":0.4097523501895673, "value2":7454062761319263955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.885572, "value1":0.548188359160327, "value2":3020054439494862697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489119, "value1":0.6763563343582067, "value2":4242764599158368849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.187849, "value1":0.7091709802238197, "value2":6867639316908364367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663732, "value1":0.42631474518610635, "value2":9001795643794270349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.585010, "value1":0.6237338911978378, "value2":4365340422086338410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.379644, "value1":0.15387674044669916, "value2":818900165755081001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791337, "value1":0.32588514831554516, "value2":1691735796263334884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.082385, "value1":0.5745615587827129, "value2":772277205802820637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086473, "value1":0.8230348349086334, "value2":8156169851471463456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808463, "value1":0.47620124507643474, "value2":2357967788477712632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380686, "value1":0.011323528011005758, "value2":2028993392634762324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.371579, "value1":0.5028955605922396, "value2":7910467950096619453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.883127, "value1":0.20067026957438594, "value2":3625887507362062386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.305224, "value1":0.7123715236402623, "value2":7522375891893442954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443461, "value1":0.7555458976714571, "value2":655917858760911221, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901204, "value1":0.7387538487064921, "value2":7525092307824794219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.376917, "value1":0.7993175855480273, "value2":2807412127532185201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_6":"h","key_5":"c", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740075, "value1":0.3001865006906782, "value2":7190874681083752315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209665, "value1":0.89276344594564, "value2":4778320938543530902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.341730, "value1":0.20200750378616714, "value2":1664606777941171586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.629099, "value1":0.017583528638071864, "value2":2707650519412422348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057519, "value1":0.04492925365315839, "value2":2578667952703739482, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.723798, "value1":0.8454916194785576, "value2":1835422305385826626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.034767, "value1":0.201322613886975, "value2":7356183312830760350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_5":"h","key_8":"a","key_1":"b", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520001, "value1":0.5251237247182675, "value2":134900877343320532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489083, "value1":0.8513994038065534, "value2":2159556869883772494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344496, "value1":0.16316310680106225, "value2":5518614284082110727, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.724230, "value1":0.6875155435793379, "value2":35737216326475935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568987, "value1":0.5138501785175705, "value2":4338888431721237426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598387, "value1":0.4390074014236514, "value2":7041842749721887181, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.626270, "value1":0.2391917526821926, "value2":3091537573368363672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.297931, "value1":0.11690748126426959, "value2":4039852836857346610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.175994, "value1":0.6789300486469183, "value2":1246840956922367253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976992, "value1":0.43367432666147043, "value2":564959136229688173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703741, "value1":0.47707265609489413, "value2":4068705543834170412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.235944, "value1":0.8012892224294691, "value2":1988300147401692461, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499512, "value1":0.9990473134151273, "value2":8394845441054007311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404130, "value1":0.6068400138809874, "value2":1935348003856793271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542560, "value1":0.12776285353397152, "value2":6795264619125251375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693012, "value1":0.2569652780369043, "value2":1475809083419006811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068970, "value1":0.3881338979751516, "value2":1218503134871756052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264926, "value1":0.19791850227382649, "value2":5513222623463327397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.408769, "value1":0.39494174800117104, "value2":1576673230492191692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.081892, "value1":0.9402614771700961, "value2":1589947956431908818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.058425, "value1":0.5327215264224379, "value2":445308327885517649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_8":"d","key_3":"b","key_4":"f", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.838442, "value1":0.9519819634366921, "value2":745087205152711718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064037, "value1":0.23777264651361862, "value2":4675095342249511184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.518857, "value1":0.8856046396694535, "value2":1109051615753148759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077463, "value1":0.8624901705258308, "value2":3793672263940517136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.921342, "value1":0.9049145350717305, "value2":3570805638374912588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157723, "value1":0.6223760989748452, "value2":2996869515277202402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155659, "value1":0.13854963250628805, "value2":6512544911629959850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977987, "value1":0.036771554508911904, "value2":5994549505435027834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265064, "value1":0.3403489462496419, "value2":2814929460502496771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404153, "value1":0.3208642154389825, "value2":5256172233350798960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.312852, "value1":0.011902750850203012, "value2":5526100237606282605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.491995, "value1":0.46992302986974577, "value2":157557322951942167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704212, "value1":0.9425835794283676, "value2":915577115687321604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.576745, "value1":0.7854877804515764, "value2":6382074091209031167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070556, "value1":0.18030779510661127, "value2":6308540602135606131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542532, "value1":0.3384327861708296, "value2":1613795669513593265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511484, "value1":0.8996232864271185, "value2":7405365074827637694, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_3":"i","key_7":"c","key_2":"b", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.562131, "value1":0.10596222820549657, "value2":3306812514398053557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.391786, "value1":0.20647295311593183, "value2":5486022828508870343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466143, "value1":0.6224930113197604, "value2":8535178573473477072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423074, "value1":0.3088852292542975, "value2":3166186037825297943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.762488, "value1":0.33490363470863166, "value2":2912130613463378266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713117, "value1":0.2993000272696047, "value2":494302122350368825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028636, "value1":0.6503466890929638, "value2":4759183611197798725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404461, "value1":0.043678508683084524, "value2":6408629579798949638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077439, "value1":0.2418209601880771, "value2":6820734383602297695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.624831, "value1":0.4718269105814766, "value2":4622812711240963076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053083, "value1":0.6875169297990352, "value2":6119677537083039891, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345042, "value1":0.7324917812164005, "value2":5511749504782560024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.623531, "value1":0.9320016848414698, "value2":744369719732623498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.182404, "value1":0.9674131847138503, "value2":3838480249361897923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.696213, "value1":0.7881956638078709, "value2":4371203118864420550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.628037, "value1":0.17572736986644139, "value2":3461303046815504013, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273728, "value1":0.6916742591847842, "value2":2382468702478522273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335163, "value1":0.9758093684084915, "value2":1680436716344950339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782020, "value1":0.1537134689497552, "value2":4414043374214499577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.134363, "value1":0.5083067111414208, "value2":4982306711291417821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.865817, "value1":0.9406393737656967, "value2":5857776137176686892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716601, "value1":0.1916346431757554, "value2":5910562859970195422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.427366, "value1":0.03546691845658599, "value2":7315776376647279090, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950000, "value1":0.5369396780416313, "value2":7524727304557507728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.325387, "value1":0.06313877213741768, "value2":7980198837673375086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950927, "value1":0.32484655923390965, "value2":8297079571694626407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832701, "value1":0.8101399444767259, "value2":6277528961017233439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969582, "value1":0.5893386387714608, "value2":8824793605581596113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302334, "value1":0.03653755659557745, "value2":2394591637437372554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_5":"i","key_1":"b", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463188, "value1":0.2645292258357236, "value2":7750296583775050063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.774073, "value1":0.7854839946508317, "value2":4045742754979149401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.729676, "value1":0.4706720879040935, "value2":4750321291295032700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618814, "value1":0.2230204325484985, "value2":9003120111243218251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603138, "value1":0.22960374502253963, "value2":6338685193000832935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.412324, "value1":0.4957126042457222, "value2":8735002444024394358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657244, "value1":0.08184942404758783, "value2":7470578294751016176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442850, "value1":0.7229538622773333, "value2":4724836800934702763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156841, "value1":0.7233638365572691, "value2":7858912978560139100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734329, "value1":0.3404208114361277, "value2":3050306358924697552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.465656, "value1":0.38391986849101767, "value2":307361261718394449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.445647, "value1":0.990731352209279, "value2":1207071782335047705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298620, "value1":0.11180836767709064, "value2":3870338901819904571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.095445, "value1":0.2870001965302557, "value2":2415945267551238695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.918448, "value1":0.7277791066957175, "value2":2821193899111524763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541594, "value1":0.09005789780178877, "value2":2925035824598662472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.631726, "value1":0.9959155652844405, "value2":3833701945431571315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049800, "value1":0.8110556334200572, "value2":2218497283809170432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070530, "value1":0.65401432282683, "value2":498838541277181702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736247, "value1":0.7394025858411815, "value2":7148599556072452144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736608, "value1":0.08218047603491815, "value2":3776524585186258439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003840, "value1":0.6757338653122574, "value2":379207741924258587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362539, "value1":0.11651117191245357, "value2":3906526956584011608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.539295, "value1":0.44782322310256395, "value2":2595894391836664475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392973, "value1":0.8520294340743481, "value2":7384112654505361476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.078027, "value1":0.3547792528217613, "value2":8155526662947982478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.776543, "value1":0.6604220360650918, "value2":4011373177447838470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.691748, "value1":0.4862664301051333, "value2":7696312329542980810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964567, "value1":0.11169294328511947, "value2":4727405940623359973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722155, "value1":0.3780569227165093, "value2":3316392130699376882, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457388, "value1":0.9243183084724984, "value2":2432925818286345186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.040558, "value1":0.7743604477646876, "value2":627834147679968207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272066, "value1":0.5917633264655242, "value2":5918933193525430349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200729, "value1":0.7327561219912169, "value2":1393602674371677220, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.905147, "value1":0.8568183299448732, "value2":9034660617242414820, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310139, "value1":0.802088771431449, "value2":3471155770712582770, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.821462, "value1":0.5273591260032361, "value2":5140674777677119228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.384953, "value1":0.4025027387592999, "value2":5097429126997347384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.829568, "value1":0.8045993326905381, "value2":2809186620050229002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294923, "value1":0.5644401567133956, "value2":3718732987675906525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.556732, "value1":0.48820899714237304, "value2":4078290062854715892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.176213, "value1":0.2510405463103239, "value2":635371208287508106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.267004, "value1":0.22999772540025595, "value2":1871342923142731692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_9":"g","key_6":"i", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.506694, "value1":0.4581021751822573, "value2":8794558811813763792, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_3":"k","key_4":"a","key_1":"c", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354208, "value1":0.6088901271960526, "value2":132806552362853695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516024, "value1":0.7664701528892452, "value2":107006237498756274, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_6":"e","key_3":"h","key_5":"j", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.750598, "value1":0.3217472443950159, "value2":8378919480372645762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419841, "value1":0.6663125192971521, "value2":2423610833256438065, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164890, "value1":0.2006090031725153, "value2":1844725035830445264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264867, "value1":0.42671643117928926, "value2":3438912836085300466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.430421, "value1":0.07257562390395779, "value2":6088518193551139346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_8":"c","key_1":"k","key_6":"g", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.673454, "value1":0.6691702132206785, "value2":8625848517577296022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995936, "value1":0.702118704788376, "value2":4086471613439852656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_5":"i","key_9":"g","key_4":"i", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964584, "value1":0.7747761523528802, "value2":1036051996047955870, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.948289, "value1":0.25140012327548283, "value2":788619002191666922, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372555, "value1":0.9999769483157629, "value2":7184076213289982875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.920761, "value1":0.868179497430529, "value2":1697965828697918238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372531, "value1":0.5396817815353727, "value2":7506498166097940651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.278206, "value1":0.17061135686969672, "value2":713580384695452010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600654, "value1":0.6616469498902554, "value2":774519242775197957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188862, "value1":0.5531871959368098, "value2":7422109554403592667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457643, "value1":0.7840548969774126, "value2":7827846462514659006, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655234, "value1":0.76526166835666, "value2":2375166258510616618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370401, "value1":0.021438154414782817, "value2":2771089523399378961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137411, "value1":0.5646878830454839, "value2":5968018682136017050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423235, "value1":0.36529823070220574, "value2":7162022842889925076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272788, "value1":0.219359460529438, "value2":181241653539382719, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.707284, "value1":0.3923075865949308, "value2":2782827649523891195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858145, "value1":0.688162185571026, "value2":5086226934464427172, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867650, "value1":0.6841047492247676, "value2":8098586901309655486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.902390, "value1":0.706932118893774, "value2":9086709640294623772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131998, "value1":0.8578147943558058, "value2":41651666626493230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932017, "value1":0.24496475938485004, "value2":7307212968529901562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.469902, "value1":0.3291978179419431, "value2":6674355534843744608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.715852, "value1":0.5949781040197312, "value2":6585582445453110248, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.439874, "value1":0.25098256408304576, "value2":9009893614730547675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_4":"d","key_6":"k","key_2":"f", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.775831, "value1":0.888090262607404, "value2":7565201070136116477, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480681, "value1":0.47062171981528383, "value2":4853013107516729574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.654578, "value1":0.4096044979675714, "value2":1606198625715352901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.881884, "value1":0.31666628836121824, "value2":5391861566395998267, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028180, "value1":0.2934267405568024, "value2":1598172822680744133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245149, "value1":0.09374074237151239, "value2":6863637452711576117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105508, "value1":0.7783021910779061, "value2":2669034341302440615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474623, "value1":0.5494532131881231, "value2":6109855504680847451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625728, "value1":0.471409564376847, "value2":5293699949847690348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_5":"g","key_0":"c","key_1":"i", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382751, "value1":0.7835774962833667, "value2":5926458098948297057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_3":"e","key_2":"c", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.527286, "value1":0.6322552498331545, "value2":437257708157331646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.166237, "value1":0.6386670258285629, "value2":2053216930342060527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.942234, "value1":0.7212756328576085, "value2":8527112750674030055, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.880290, "value1":0.19477642153523786, "value2":8136028599409004140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.477164, "value1":0.7162729706892131, "value2":3933534960467417574, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089992, "value1":0.47761006384043037, "value2":4843140910052787407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.204970, "value1":0.4917659716382411, "value2":8639172605312608175, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.428786, "value1":0.5707960425927678, "value2":757157967003444236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131753, "value1":0.32321264423119905, "value2":1903644540800881525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.540623, "value1":0.528958932538034, "value2":8999603816652982043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.991648, "value1":0.7098945581193735, "value2":8146175572944228409, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.523699, "value1":0.499825459180699, "value2":1724724639775860908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116240, "value1":0.6561999665129753, "value2":5656111967421346554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618968, "value1":0.2848979660832576, "value2":2049018652807530472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116765, "value1":0.8252075257277172, "value2":4775243296364010632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.410327, "value1":0.06668100857135961, "value2":2183721339943212726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.688645, "value1":0.41653256081146506, "value2":5969120448550176776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657571, "value1":0.9568582636724965, "value2":730216901987394123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043883, "value1":0.40826500145612177, "value2":6407138186851676502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474102, "value1":0.6914723313082367, "value2":469628036375832328, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596154, "value1":0.050104900538538846, "value2":1354999585856659679, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096028, "value1":0.6282483124845963, "value2":116602313358973711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568083, "value1":0.07619222667132636, "value2":6984709326920626754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.984466, "value1":0.3323790308005344, "value2":675504136493878123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553260, "value1":0.45304427452622936, "value2":8153841508229333988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781038, "value1":0.2512350719527022, "value2":1697664403294613401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230875, "value1":0.8575694561008984, "value2":1340681872774498770, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180011, "value1":0.7159258322890252, "value2":7263307563948982502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318368, "value1":0.7865256606198752, "value2":7176709659967207104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.594668, "value1":0.9818689490949409, "value2":4816922436179228865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.178515, "value1":0.9620215494327841, "value2":7544435408144975228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596427, "value1":0.7405089300246521, "value2":6613451887211746808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317317, "value1":0.8954816734384349, "value2":4023989770168105300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.624103, "value1":0.1565943994591959, "value2":5829676556068380878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.259139, "value1":0.08662232425939413, "value2":2137874058817765285, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.495345, "value1":0.7444165320555235, "value2":1512339128303545319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.873798, "value1":0.10364343021646215, "value2":497864294044982023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077492, "value1":0.7825627514339433, "value2":497480348120915725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.887058, "value1":0.6684575895211121, "value2":8461019606631895989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374572, "value1":0.30885911828548585, "value2":6810974248136318973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.381097, "value1":0.062395135046400096, "value2":1505084432772472894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796962, "value1":0.4796896579785329, "value2":5095295117314039911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_6":"k","key_8":"k","key_0":"a", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.625725, "value1":0.5385278812414747, "value2":7715803409892624904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430762, "value1":0.42060637147418184, "value2":7474778452563228226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886443, "value1":0.6738972816294144, "value2":582629852137142515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467512, "value1":0.5597247611878908, "value2":8349056677712047690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301942, "value1":0.5324107668792922, "value2":2601407711369777027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.959260, "value1":0.3409466272260335, "value2":495480232494613355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824942, "value1":0.690980654161434, "value2":73040149987568706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647608, "value1":0.9936863999178829, "value2":2595521887523034584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206529, "value1":0.7042253871626942, "value2":6786989253298041563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654900, "value1":0.04874287974341856, "value2":6080516591012621533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.192783, "value1":0.636728117953383, "value2":7620028626322794241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.757400, "value1":0.3932217938599448, "value2":4695382142084123513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578308, "value1":0.7154241327471093, "value2":8875337832548314386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025767, "value1":0.4896117284329862, "value2":7827833960384552450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695043, "value1":0.4735636518198123, "value2":5776645863491585306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067392, "value1":0.2568532613491557, "value2":4347058259737260198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145851, "value1":0.867279626694445, "value2":7767138234344286903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.491824, "value1":0.43935171533716344, "value2":2117420540525504913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413343, "value1":0.7815374349474034, "value2":7674175325619501370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408821, "value1":0.6468318207569199, "value2":3382854605353241895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864749, "value1":0.5018653748831062, "value2":3163263403613237522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811002, "value1":0.1264068238414509, "value2":8051527093894333669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922550, "value1":0.797889997967527, "value2":4244570533853409024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045039, "value1":0.6136304375492249, "value2":7079529098273717788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914685, "value1":0.6884426743061036, "value2":3608727831529868858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.429556, "value1":0.37532682108769366, "value2":1177251454911069616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779878, "value1":0.5585578516583662, "value2":5769602877819666427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.711503, "value1":0.3057843361151979, "value2":2044462261279390237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142192, "value1":0.0045630377024844515, "value2":4953964644129769622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.542755, "value1":0.236125279127913, "value2":5065525208816588827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578196, "value1":0.8156211595242715, "value2":1914083289002989297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110649, "value1":0.26711648319193754, "value2":2012127151944954874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041387, "value1":0.3776450082432283, "value2":6801275370589593726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_7":"h","key_2":"g","key_5":"i", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808716, "value1":0.04169212753399746, "value2":9054979711039988827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062171, "value1":0.7218616273097948, "value2":740018633881444642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146630, "value1":0.5003843971030013, "value2":1655397867016219461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.309989, "value1":0.11293107623655986, "value2":3732853355411608797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.333206, "value1":0.17591880468368, "value2":8123233873446258834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145427, "value1":0.43545559910718556, "value2":4862315899801656213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.644901, "value1":0.9950957928656415, "value2":4691155465042626084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568884, "value1":0.6163869833398097, "value2":9217755948904745777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930812, "value1":0.3180893474971966, "value2":8238800974641951308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.148564, "value1":0.7643083780125881, "value2":8977913426229482254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369460, "value1":0.37462703037399614, "value2":2532444456287017290, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134410, "value1":0.7407330348790686, "value2":2290478450455253969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.623624, "value1":0.23159512407081487, "value2":39715061290046297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497315, "value1":0.9059175925838082, "value2":9042312906596816530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.591581, "value1":0.9196881162354394, "value2":3577426841534493512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.229214, "value1":0.677998841646216, "value2":4155628140480575006, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_8":"f","key_1":"c","key_6":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419533, "value1":0.14115394562090194, "value2":4482838294646117015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352495, "value1":0.614805207196906, "value2":6313920398652894644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158009, "value1":0.13136879968484017, "value2":1444276749086955965, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_4":"d","key_9":"a","key_0":"c", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394523, "value1":0.390863430258994, "value2":1479817062186057883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084220, "value1":0.8798339682629527, "value2":5374208931949760111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.604039, "value1":0.6734519068343852, "value2":6936119518718549917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.407184, "value1":0.883448918218706, "value2":2783824499389725952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.987554, "value1":0.04159873743578623, "value2":6347778463017159524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015322, "value1":0.9633747370317121, "value2":886249751055268990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903730, "value1":0.2538648955540415, "value2":5257877330086190982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109241, "value1":0.3895615184638186, "value2":2398924749380096990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946726, "value1":0.8408201754901692, "value2":5956645672957756887, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.126734, "value1":0.8247534822394186, "value2":8919709316239049787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419044, "value1":0.20899742823905534, "value2":9208355419350776806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354221, "value1":0.38324817137377604, "value2":7243408993217101630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.620584, "value1":0.25505110416085736, "value2":2000013463174956675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306280, "value1":0.24053747539237172, "value2":104074303789974504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.694994, "value1":0.14347374003976163, "value2":2424092953659426107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399682, "value1":0.5974894049526064, "value2":6226843809112311375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.244602, "value1":0.4472550716634916, "value2":6661706815157471152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154170, "value1":0.15894047878423925, "value2":1910278084951118791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.338581, "value1":0.8000650933715514, "value2":932928130299819996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385782, "value1":0.2346060605687393, "value2":1779773261165382974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.925102, "value1":0.31969179251919677, "value2":2628705870105871940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783196, "value1":0.47412332370344556, "value2":770542511728920904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779626, "value1":0.6092963005254489, "value2":916209608160555146, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194805, "value1":0.5004447430389768, "value2":4252088119108151132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.758680, "value1":0.486921957595336, "value2":989531111026827083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477327, "value1":0.1584089004594205, "value2":8696347531856094472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162006, "value1":0.537289345346911, "value2":2077753746787895762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613845, "value1":0.3033370157767779, "value2":7793623951886359033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.373236, "value1":0.3796012074348545, "value2":6115758866481998084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_9":"b","key_4":"h","key_5":"e", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080535, "value1":0.47198866208030016, "value2":6615627174770526167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.401469, "value1":0.5986326083333331, "value2":2667038551831095428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605284, "value1":0.7390739400172244, "value2":361713221830612946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.543913, "value1":0.568574574470533, "value2":8152708069238595670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262409, "value1":0.058861415000961634, "value2":5954875590777239919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056599, "value1":0.20694947175492623, "value2":3664736464652868067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.557868, "value1":0.04995052697654973, "value2":2073326631524402431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613501, "value1":0.31045156583158495, "value2":8926511362033688194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.047167, "value1":0.7509456855802353, "value2":846763672875326934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362486, "value1":0.30051878680197913, "value2":8317722150065418470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.669592, "value1":0.6148907714481391, "value2":3070072928838789844, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061749, "value1":0.4026807862923137, "value2":2054834671958607537, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_7":"c","key_8":"e","key_1":"h", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077647, "value1":0.02456732222252374, "value2":4163774252593358275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439333, "value1":0.49417199138729123, "value2":8947661323810810369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.592816, "value1":0.25731799708888187, "value2":5220216189538554324, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.344893, "value1":0.0022390888763635323, "value2":6178831795736016910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_5":"i","key_8":"h","key_4":"e", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350091, "value1":0.39159828262162255, "value2":3395049172404740462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417133, "value1":0.19779968591074149, "value2":3982459725160448681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313282, "value1":0.5954778061697951, "value2":2609201495243589975, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416054, "value1":0.6479875646988947, "value2":6379744126737729174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416195, "value1":0.9737347842892922, "value2":3437414765355850053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.351132, "value1":0.5274165691272555, "value2":5569886633354264071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273952, "value1":0.4616579953521536, "value2":4410692243343844144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948309, "value1":0.21495634970202532, "value2":1337568920394438603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.120015, "value1":0.6827231094645871, "value2":4910087454372528211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.220281, "value1":0.5451953602332654, "value2":5295885271016141901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222991, "value1":0.38652141757520153, "value2":611291833871236608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432311, "value1":0.4952506069055437, "value2":6235358954173265246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.305907, "value1":0.7399971630668104, "value2":7853724941674753946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.701437, "value1":0.07448132935373361, "value2":1676692481709109964, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233939, "value1":0.9922833997352541, "value2":8485630674384935881, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.196307, "value1":0.3811609484351306, "value2":6762437704329339708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.598701, "value1":0.7218662020742757, "value2":5746405219362751126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535319, "value1":0.470769974287709, "value2":5160211896753607127, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455357, "value1":0.41061237500976755, "value2":3731616592511902245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682037, "value1":0.02412728786773881, "value2":8461839080730070447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618426, "value1":0.6208202835196076, "value2":3667062185468357188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094952, "value1":0.9317381860791135, "value2":8865888160428380876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295576, "value1":0.034878221959591026, "value2":4553331116235737600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605252, "value1":0.5687954362360221, "value2":4628895813773640059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.167973, "value1":0.8144348491443895, "value2":3884298273747196831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_4":"a","key_1":"k","key_2":"h", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803115, "value1":0.14234455011711838, "value2":7333273581911274613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809534, "value1":0.31251923098668316, "value2":5000940696241059800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.504196, "value1":0.37041029395255115, "value2":7518926478801994928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886399, "value1":0.2510698129904212, "value2":1686947485208902024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322861, "value1":0.6398715658295313, "value2":5260107554586637670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.575248, "value1":0.8821568194363127, "value2":2676120924049029771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.924938, "value1":0.6642600755588765, "value2":9154692137110805453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534144, "value1":0.2220263150896839, "value2":7296067315122241408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680757, "value1":0.36579467004129973, "value2":5336171210655722369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.361476, "value1":0.42049076121425283, "value2":3694543289414649508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.423620, "value1":0.9226817255397154, "value2":5312418306728579961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.398746, "value1":0.4448382205324241, "value2":5199222544329766041, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.492975, "value1":0.6490408315820692, "value2":3620354939805403786, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.732787, "value1":0.26280435119702705, "value2":7186318610805553949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385151, "value1":0.7991171293820531, "value2":1271579952216532684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067778, "value1":0.2653370821341721, "value2":8977665444967590904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941069, "value1":0.7995772580327314, "value2":7272267767125923064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.547209, "value1":0.7653567065104109, "value2":6328078604409399184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649757, "value1":0.49061605503159605, "value2":5991812000082844933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.017745, "value1":0.8857176370602151, "value2":8890483656354164816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478915, "value1":0.49344955581785127, "value2":6898479319738793663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.600610, "value1":0.1145004603393322, "value2":7934326719175266154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752045, "value1":0.7917363851798865, "value2":6655960268552880542, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.198446, "value1":0.8915785759457425, "value2":6424163196395537238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415881, "value1":0.6309493378355412, "value2":623919229201632317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.652344, "value1":0.6344286572245156, "value2":9189348363675107233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774566, "value1":0.042498471702267726, "value2":6073051427209338110, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.918626, "value1":0.4530903546971259, "value2":5667043007795158116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.340057, "value1":0.13032389853235377, "value2":2621616876191250318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363370, "value1":0.25548611447808117, "value2":7869413584380375849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316114, "value1":0.7913570973054609, "value2":714341173131110534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.726210, "value1":0.6340822577426605, "value2":7078070670524096081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809728, "value1":0.31189335942918994, "value2":2593764393145326151, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828156, "value1":0.2513167249865656, "value2":4199156936630630581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074063, "value1":0.6299866633999338, "value2":7375082593010468605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551341, "value1":0.5634049844701874, "value2":553811382543796103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395240, "value1":0.9163098471877911, "value2":2102676899677978041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812285, "value1":0.5658853008127968, "value2":860920050673825541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.699745, "value1":0.6686081106918065, "value2":4484995258456828647, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862811, "value1":0.006750603964471356, "value2":8966268587906792618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649656, "value1":0.33455087075476775, "value2":3638358249838833627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978094, "value1":0.9765011127569251, "value2":5363905042672157366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824326, "value1":0.5852292268881885, "value2":8525521016560267029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972834, "value1":0.2713220476194647, "value2":1730194033904802699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.513347, "value1":0.20240183224033417, "value2":3586323241046235606, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_7":"f","key_9":"b","key_3":"a", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165801, "value1":0.25754092027876374, "value2":3897618512711030520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245034, "value1":0.276430555426299, "value2":7235576154009687664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209920, "value1":0.33218423022426385, "value2":6956800146565179953, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802797, "value1":0.9498551679797749, "value2":4204819498986608868, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.002994, "value1":0.33304452518131644, "value2":299998551436331074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.136076, "value1":0.58569890261748, "value2":8821512503402703105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605724, "value1":0.3779582962124763, "value2":4381826070482821493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.664393, "value1":0.7508643438704989, "value2":9133405472026903918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236332, "value1":0.4456796524317141, "value2":3982698031313790805, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320127, "value1":0.5073225371595068, "value2":2378617268397591284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865511, "value1":0.36097655561074876, "value2":2650611720712784810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777541, "value1":0.27613383935735863, "value2":3133546266043557517, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647086, "value1":0.09971158325173911, "value2":7192486223907690883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_9":"j","key_2":"f","key_5":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.172420, "value1":0.6050664492556429, "value2":3816871624777837698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639560, "value1":0.40402358999541305, "value2":7105433557941999346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.945205, "value1":0.38392410498955126, "value2":1178256579293465746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277675, "value1":0.3660884472168037, "value2":6046887408166693464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219346, "value1":0.5871741254566942, "value2":2616399527910270141, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116739, "value1":0.7713481330592996, "value2":7718152730102741547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.951685, "value1":0.6692969608681443, "value2":6198709686728043657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.880602, "value1":0.3938915734742536, "value2":5600053552458054507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.831337, "value1":0.47065388482990184, "value2":5918647530397871720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865810, "value1":0.9557189505427555, "value2":1533596579704888254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578483, "value1":0.3164962822621824, "value2":4784731554506893709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158191, "value1":0.36054084773482775, "value2":6343389237742355198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761167, "value1":0.9149570761492709, "value2":6140964050711145447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.955044, "value1":0.34237203961558577, "value2":7321019711278117639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137258, "value1":0.2734770926499478, "value2":5312006546786938240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876289, "value1":0.584796985987568, "value2":2146521016660183394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.345388, "value1":0.7891673161299368, "value2":9211813724293662180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.291358, "value1":0.5443723430932246, "value2":1305196521853642686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107992, "value1":0.33317636728859645, "value2":4303995107486054548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.065010, "value1":0.792101797297302, "value2":5624493167148885444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.512622, "value1":0.026747806334027496, "value2":31336873860845918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796113, "value1":0.3713464074931213, "value2":4041884710949108537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.794333, "value1":0.5998279188033826, "value2":2263989663686243457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930716, "value1":0.7150630462147562, "value2":5925307983835648806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.631145, "value1":0.12175047903656237, "value2":2424100894887965716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889832, "value1":0.33399829473514187, "value2":3091557798470852457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.036666, "value1":0.8679594907099448, "value2":6691229273816200596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.853608, "value1":0.5774160801209688, "value2":980449146620020072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.356874, "value1":0.6962074391730027, "value2":2178872746047471215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162362, "value1":0.6573318971529195, "value2":8364158949095374627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110201, "value1":0.41287549892040376, "value2":7444688864434046520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.441462, "value1":0.960708146788197, "value2":7272057802651917640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.020362, "value1":0.14970758198915565, "value2":5476649529178658344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434430, "value1":0.7114147585195696, "value2":6784565480587871276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795723, "value1":0.9222061229277446, "value2":5415396163282444341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.092199, "value1":0.0594949540077623, "value2":733783646287392653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277448, "value1":0.36348546035264123, "value2":2527022975440439707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607130, "value1":0.21203009019347513, "value2":3167160934083348057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424117, "value1":0.42561548619630274, "value2":439532732667459958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365073, "value1":0.8492201961187499, "value2":1130712017318335183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248310, "value1":0.6476029521632828, "value2":5228653953193583715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744619, "value1":0.5713572042259071, "value2":1575714640192742698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.026559, "value1":0.543673726494076, "value2":3134877395946459112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972610, "value1":0.09611280545479016, "value2":4984229756632609213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763906, "value1":0.8517215040951974, "value2":6804317567810698432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105838, "value1":0.8805288612544102, "value2":7653928454276555905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383435, "value1":0.3683543847634455, "value2":2147384927542025505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027377, "value1":0.4869984514514779, "value2":6620303412327477269, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417188, "value1":0.24989866283446016, "value2":7011787964949226310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084529, "value1":0.9844535394576189, "value2":6876673068454117053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032097, "value1":0.6712678103298776, "value2":3320740930339209104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394244, "value1":0.7796841432655915, "value2":4446257418849643255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828297, "value1":0.2648442239286722, "value2":25756618593200917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731687, "value1":0.07632667820503707, "value2":5775744844960229233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.348738, "value1":0.4462284223326741, "value2":7761375931867531240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817431, "value1":0.9846258074003919, "value2":5377182091927671364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603703, "value1":0.1579124346010748, "value2":7829825462713051842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743604, "value1":0.012350717590309025, "value2":3299133246704731924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941914, "value1":0.37406772316207776, "value2":3581449411399106092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.323291, "value1":0.2629524147159168, "value2":2560363092546342475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206164, "value1":0.1605470780394023, "value2":1476326866755963525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376246, "value1":0.8410819090377242, "value2":6697199584394109087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_6":"c","key_0":"d", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.438458, "value1":0.05246539756995679, "value2":2689405601803288451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.807203, "value1":0.730536792123947, "value2":7996980666084265973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.842861, "value1":0.2085319390948787, "value2":750997239048285659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.097539, "value1":0.943951867959508, "value2":8362598506758468383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363032, "value1":0.5870242519277287, "value2":6264967362494680685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212907, "value1":0.4603451527715893, "value2":4171196630516497508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177573, "value1":0.8948006671577682, "value2":5720852449593066754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286908, "value1":0.7373750433446898, "value2":2889582543857330611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.780354, "value1":0.11954278924179859, "value2":206386290438050068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744582, "value1":0.5414546611545269, "value2":1573274748881736123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262127, "value1":0.8114751808169933, "value2":1156734310814584450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.627688, "value1":0.05870166177597653, "value2":8802667963431750791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432936, "value1":0.5733149806402271, "value2":8971566642401975801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252007, "value1":0.6150552921263627, "value2":6524226679068869341, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230350, "value1":0.8727510444388746, "value2":5913218581367894399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.989663, "value1":0.8376111097555229, "value2":2874987014333615633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.435356, "value1":0.7022017770410863, "value2":6778124246365180945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673069, "value1":0.2749380629969126, "value2":1514980079784151790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328789, "value1":0.865778242569301, "value2":384106145681984426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.986292, "value1":0.9273192971072537, "value2":6526516407617182264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.240749, "value1":0.7234109649728769, "value2":6790075567204476237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774780, "value1":0.5871689076780541, "value2":5434961575452625626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.406076, "value1":0.7420142270194664, "value2":4969819996381131142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146123, "value1":0.2540849065850774, "value2":4942775185551508816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.928078, "value1":0.7980654446229606, "value2":2117433885577041648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933008, "value1":0.05348899498806063, "value2":7864335047299894108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655622, "value1":0.7668514308351416, "value2":5108933691323452489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799422, "value1":0.8658511508098131, "value2":1584220631636831527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153780, "value1":0.5100425920946189, "value2":6860613724470124155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.832729, "value1":0.6352815444925497, "value2":6914920278709539881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862121, "value1":0.40978666741053005, "value2":8297372411728279444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.046814, "value1":0.6860769857954689, "value2":7207194751565200226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.106175, "value1":0.9141202615431735, "value2":8093659818506693875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568924, "value1":0.2818708416167068, "value2":4682897128638978687, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743387, "value1":0.19055511730870822, "value2":6005851706828394010, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.131634, "value1":0.786038669470468, "value2":2052265664401672629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864417, "value1":0.6509022777920431, "value2":6405639625996257390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769553, "value1":0.3306779516460208, "value2":63644576329067046, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705994, "value1":0.39211177275150355, "value2":6019747875631249901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.085415, "value1":0.989053918126363, "value2":8753375624734757247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.966642, "value1":0.146948164341443, "value2":7640844254824954510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_9":"k","key_5":"b","key_8":"b", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.548525, "value1":0.7240074879750936, "value2":8956195213201860326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073937, "value1":0.7314322966939647, "value2":254693443211564548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347726, "value1":0.8209138159763902, "value2":6620059103742183058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.378046, "value1":0.5808416882480918, "value2":6139629334504131344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_6":"a","key_1":"d","key_4":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797601, "value1":0.7784388478906132, "value2":354987022131004184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411013, "value1":0.45158233474379805, "value2":6507365193047028596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.152084, "value1":0.34791281429560394, "value2":2152822969747742943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808545, "value1":0.6451074006535574, "value2":4535693696526329424, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735815, "value1":0.5617099301887051, "value2":4545722791278282166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170237, "value1":0.7594425432841077, "value2":2121502996639945525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839462, "value1":0.42088416942799856, "value2":4668152333347445855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977985, "value1":0.9874529650666419, "value2":7279887982544559299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777993, "value1":0.32638782808208466, "value2":7945394269696237457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931446, "value1":0.5421003602914947, "value2":7796969745729042644, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746112, "value1":0.6059100687669938, "value2":8599287190529748569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.950726, "value1":0.8607618900249634, "value2":2432626342683975270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647237, "value1":0.7468406151868747, "value2":7200054131729412480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.460709, "value1":0.4549015015070196, "value2":1628296475817521290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_5":"d","key_0":"b","key_2":"f", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007652, "value1":0.3132830085813196, "value2":1943034905431272670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.103043, "value1":0.029190956951399328, "value2":5218539584338803865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506730, "value1":0.4547431011729936, "value2":3644260335220928016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564064, "value1":0.2217922532972611, "value2":3866244575842378014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.308096, "value1":0.4735939830858083, "value2":6374663223385153398, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415080, "value1":0.9760882060049622, "value2":1970837347852222146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909084, "value1":0.6294752667120725, "value2":9199477369979291397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.969935, "value1":0.10788341374154452, "value2":7002111391558217482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.622182, "value1":0.21938176298467404, "value2":4706131254461339381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.722928, "value1":0.21266609523973992, "value2":5957309419507426661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062556, "value1":0.1304962392242022, "value2":4535950458577175850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056693, "value1":0.6559280448597679, "value2":6798153078557023165, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.275009, "value1":0.8708722343930831, "value2":2992233127991751043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027324, "value1":0.8089539697327325, "value2":6582915861289307476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769519, "value1":0.6866136235615642, "value2":3188730142010270506, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471216, "value1":0.036922660768241786, "value2":69160799499838231, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.289160, "value1":0.9707556984503585, "value2":3226749743394249266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719377, "value1":0.6150053305666618, "value2":8057451250881285379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884188, "value1":0.32970677783871427, "value2":6299179245210453660, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.128583, "value1":0.46621614630474323, "value2":3810272679777334373, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.016304, "value1":0.07070759725845735, "value2":3860287779798372481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556253, "value1":0.3162226930220112, "value2":811964996210240316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.662220, "value1":0.04332778395279835, "value2":1557538607744070859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713709, "value1":0.6625325269051697, "value2":4541028658322459718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297119, "value1":0.15013885945438676, "value2":2149756203418913335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685110, "value1":0.9036182355788681, "value2":4377728436719035164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846262, "value1":0.34678741602681534, "value2":2162478118726349508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410862, "value1":0.11378608528656459, "value2":614293064254453793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.060980, "value1":0.22871224178742908, "value2":4099107042293178536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960058, "value1":0.5820948808821428, "value2":1201783870513585432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.214885, "value1":0.03550184669484978, "value2":2051901639161172531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696549, "value1":0.44298333983599963, "value2":8432193022117819833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368813, "value1":0.6629289022292644, "value2":6617784174875177478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313857, "value1":0.15526487291119703, "value2":95708758682513207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744683, "value1":0.6423949710674116, "value2":3611912809247643043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.786335, "value1":0.589184376420666, "value2":5802753188533704826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_9":"a","key_0":"a","key_5":"b", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930246, "value1":0.20872893267238513, "value2":2224049910905425698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515204, "value1":0.20960234541456155, "value2":7647445186122035759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.800072, "value1":0.20249832362898804, "value2":1073544363493262212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.420652, "value1":0.651081573013809, "value2":355187443325178523, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760554, "value1":0.28915917130829083, "value2":5965361551796616223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.852984, "value1":0.45938574348035566, "value2":2528322122553170113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352738, "value1":0.900978009547613, "value2":3975764535045467018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066294, "value1":0.08305245567845902, "value2":2394249365413597097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654208, "value1":0.08608535610142205, "value2":2036163695122831466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.988436, "value1":0.04790022290628863, "value2":6418792057544120186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884316, "value1":0.5495267159618948, "value2":4438533595454630352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154168, "value1":0.4348957861418331, "value2":8838135202914157189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874389, "value1":0.42584041610286777, "value2":2273157634255144393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431882, "value1":0.06486477270973905, "value2":3345519389285150964, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685508, "value1":0.5499323474722744, "value2":5054596449098633988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.660995, "value1":0.17858690508054326, "value2":8567568075077444285, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271767, "value1":0.8266242576408339, "value2":8856688759394626086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755581, "value1":0.7494066282765741, "value2":4893716534306029427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171573, "value1":0.6765014228252123, "value2":5524113092747902135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269557, "value1":0.24446627014298006, "value2":2739808324008179094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.967939, "value1":0.5539434802495194, "value2":3700007071044991493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.656891, "value1":0.9116275072171186, "value2":7153356187047849001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248386, "value1":0.09058562459659583, "value2":7526211020899452764, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948164, "value1":0.6887494090196015, "value2":4600657675678245799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228675, "value1":0.8494478798473957, "value2":317720425357796205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101265, "value1":0.8190567291465818, "value2":4663927178372352248, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820988, "value1":0.775820207166787, "value2":6038722642101305434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922087, "value1":0.22553959289224842, "value2":779713635671076382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.888330, "value1":0.020416312330820526, "value2":7204412129555533981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.290446, "value1":0.9852816320447644, "value2":4592385145364652482, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.679824, "value1":0.17818186899277277, "value2":252209780074899647, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686778, "value1":0.8339304122124636, "value2":3658007614464097080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257632, "value1":0.614764442927346, "value2":4642146906724062657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482850, "value1":0.18190770107834264, "value2":8954437725061891309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998993, "value1":0.7044002667070637, "value2":431084763192513731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.875898, "value1":0.7812743007979214, "value2":2705148521595588507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084211, "value1":0.8180424469178956, "value2":7499444983471620842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053317, "value1":0.3379873516583276, "value2":5874640652872975301, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_8":"i","key_1":"k","key_2":"b", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746275, "value1":0.11387537552973875, "value2":4778407492588259899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744142, "value1":0.945713466310749, "value2":3761848793498692794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.274112, "value1":0.14967892390238677, "value2":3777191830426872225, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680316, "value1":0.5765506588965501, "value2":3913164426928614622, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.582474, "value1":0.13820731268553305, "value2":1359306396719437577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.572068, "value1":0.9236046727786195, "value2":731327351641596838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.838138, "value1":0.5915185036534663, "value2":4354316161210605047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755088, "value1":0.18889070665371863, "value2":4180101489297988878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362663, "value1":0.4708250834755644, "value2":8678758888644206492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.464079, "value1":0.13171451384673036, "value2":8187780706682520691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839674, "value1":0.16354088623961133, "value2":8461213608449952033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777782, "value1":0.005150900343416017, "value2":9058250336565790520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.520149, "value1":0.6583211185838453, "value2":179671992454725443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354725, "value1":0.26618356833133044, "value2":4502556297298851018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.588275, "value1":0.40556503705171876, "value2":1831243556070013492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.991768, "value1":0.5964588343272011, "value2":7216754835822865383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.829272, "value1":0.8185283045838991, "value2":7054142095046490626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028671, "value1":0.9570557166393036, "value2":5667948933240210255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171455, "value1":0.3984068015975904, "value2":6709012624962450094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112456, "value1":0.8897199020551853, "value2":619757189037583586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.366328, "value1":0.9348857036272434, "value2":4207327317390530443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.787818, "value1":0.06046424642679486, "value2":3800391541062933624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431436, "value1":0.5742437522880309, "value2":3338797017937456724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.130697, "value1":0.8564226707318178, "value2":2563408398646331614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531522, "value1":0.21916844982697795, "value2":6377428321672110838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799330, "value1":0.2117471404567077, "value2":6841854064863394582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415853, "value1":0.7458907419184496, "value2":3858225758706811596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769460, "value1":0.9292446985209156, "value2":4294237114969210404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053422, "value1":0.19156586881852822, "value2":8401149164501095807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.634910, "value1":0.13000039932274055, "value2":4131562742009700028, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703579, "value1":0.6862332441253757, "value2":7454437023717068913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327549, "value1":0.40704854485658054, "value2":4932202555006075846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.314755, "value1":0.5681370276361754, "value2":3506285901156371358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.466686, "value1":0.42167473519309434, "value2":3558723778128133952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160258, "value1":0.4575847781643395, "value2":6023651621402119686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228508, "value1":0.5111004373311687, "value2":8895687587282710909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.111511, "value1":0.09506770522467269, "value2":2655655566098996088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.784900, "value1":0.5435091084418109, "value2":5760921829736802814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210512, "value1":0.25861533775417506, "value2":6914577108338158111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.663100, "value1":0.8688427076237359, "value2":9173486087944898117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066304, "value1":0.7818969735770589, "value2":4913317894269750379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824798, "value1":0.9535971366603737, "value2":8366422359718269509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.090470, "value1":0.7644445079507416, "value2":6086882240277966349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994994, "value1":0.046461773505700306, "value2":1146509415122743778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409080, "value1":0.028382316546416403, "value2":2068020782874554558, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.815596, "value1":0.08796047569888332, "value2":8378736063122737266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683447, "value1":0.24029660357741323, "value2":8797505888002523812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.611937, "value1":0.2489739097238809, "value2":7210056341279458271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347662, "value1":0.7553826852514858, "value2":8429002700088766063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_8":"i","key_0":"e","key_5":"g", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318289, "value1":0.5406931125323163, "value2":1745967906139345488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.900516, "value1":0.7025144762707813, "value2":456260840471493385, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153699, "value1":0.20283897976623247, "value2":7209402260011396171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.813264, "value1":0.23005364054850463, "value2":4244429874559011543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.174420, "value1":0.6527990870752745, "value2":5344843801392876512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142219, "value1":0.1627182050289727, "value2":114175501496781641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.481454, "value1":0.37374721367966557, "value2":1638215468267960931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.523380, "value1":0.48736316265440816, "value2":4581537275255860768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317190, "value1":0.0039195876566355085, "value2":5341281001441055802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619251, "value1":0.5258022392764804, "value2":4434264026251453750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.641615, "value1":0.33039615993236077, "value2":6928858155661176736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961004, "value1":0.5399790911878092, "value2":7046629202789685841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025798, "value1":0.5984426914038641, "value2":6528145482673170360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.901084, "value1":0.06259887743500082, "value2":3236298745887395724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.489095, "value1":0.21376661515997028, "value2":7928954356959433352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655274, "value1":0.5385351363841719, "value2":7272624008760795422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.082369, "value1":0.4014980361714972, "value2":7005111860489403789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519418, "value1":0.19683342336444604, "value2":1435942937723913885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226945, "value1":0.9922423305002035, "value2":867760431487143669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752765, "value1":0.9916422768378336, "value2":3430696359309587884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.078412, "value1":0.6964630260101603, "value2":2102364069319122633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328609, "value1":0.19123451863699675, "value2":8170554763374290069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.470499, "value1":0.03919799070813115, "value2":3366292364134765186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_8":"g","key_9":"d","key_6":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740670, "value1":0.08849655325518448, "value2":1691579140130982717, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.393865, "value1":0.061408201218711954, "value2":174448511358385380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.200654, "value1":0.2836024427807901, "value2":4326318289877991961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970400, "value1":0.7730199010023195, "value2":8574178501642024588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_6":"f","key_1":"f","key_3":"h", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.465169, "value1":0.542725080263732, "value2":9196833379545093375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.563410, "value1":0.7958042531287333, "value2":7393645101828557391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_5":"i","key_8":"k","key_1":"j", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213281, "value1":0.7928629505338592, "value2":1752466947098105566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954474, "value1":0.3032546869220648, "value2":1748556721646671927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.847068, "value1":0.7688745080204612, "value2":5187070635183292150, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251152, "value1":0.5682862581346041, "value2":2363792865069284658, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360666, "value1":0.08726468762579924, "value2":643126609584411238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619231, "value1":0.7434795567208692, "value2":4822313794436821661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603123, "value1":0.10532993201555661, "value2":2544972440464847934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607459, "value1":0.20036031311934685, "value2":1168760905394403690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447800, "value1":0.7195092335832372, "value2":5642489879930119928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.850819, "value1":0.6761002637886042, "value2":833307298139301862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539724, "value1":0.5294892272320822, "value2":2667359803805326763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.208291, "value1":0.8393697010611292, "value2":683397536019101988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550311, "value1":0.9963724696480404, "value2":5468577449184757808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.023114, "value1":0.32353069050613953, "value2":9187169003922087415, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286063, "value1":0.9137524095402076, "value2":6445901279543298486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_9":"k","key_0":"k","key_3":"j", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.204901, "value1":0.4076696093660646, "value2":6677345639081163017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_6":"k","key_0":"b","key_3":"f", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107007, "value1":0.5776778447129313, "value2":8463235586423448216, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271841, "value1":0.6930651904253268, "value2":6787444131360802180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.069717, "value1":0.7690475248125641, "value2":1837461702894857026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368696, "value1":0.5454486705451115, "value2":3041446083114186683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_5":"c","key_3":"g", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395274, "value1":0.24368851515268314, "value2":4392956019271269002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352289, "value1":0.22988542503215345, "value2":6550852329237611667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703725, "value1":0.026480349904897907, "value2":1885153320482283414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.256528, "value1":0.26111767999722507, "value2":6808025827523544315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252909, "value1":0.20711336158884097, "value2":3024039988582180695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_9":"h","key_1":"d","key_8":"d", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.633970, "value1":0.07845906332760666, "value2":6511716740509653564, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605465, "value1":0.5950957840062573, "value2":2806262404335314036, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.081867, "value1":0.08398189819376621, "value2":5728413114044088113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074111, "value1":0.6218602698989528, "value2":1011556855791832544, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675019, "value1":0.11095025598530854, "value2":5207952003333828816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.738057, "value1":0.5410628771237361, "value2":2540645323761255412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317014, "value1":0.5259626957217648, "value2":3505614768801064603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776152, "value1":0.302147585405568, "value2":7518755412283013617, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094787, "value1":0.5814132866660779, "value2":4822671923080781538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.517582, "value1":0.10766355589362117, "value2":8453589798026915731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882088, "value1":0.8040313368583433, "value2":8495228225039478737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803773, "value1":0.22925549140141077, "value2":8938270200397652857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403205, "value1":0.07946210415899989, "value2":7443253750975595381, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211183, "value1":0.27303249192461276, "value2":3758490705819764753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684989, "value1":0.19635657331096867, "value2":3768344870136296803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596986, "value1":0.9569957010370201, "value2":5843413270558977752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796083, "value1":0.4870882675033077, "value2":8040326882339558395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.674664, "value1":0.05017490820454691, "value2":1512965384940722080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682218, "value1":0.05038298218752238, "value2":3666135732590503511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871616, "value1":0.007362621326077148, "value2":3382155619299232500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.479139, "value1":0.48744354938142825, "value2":2536302313422526956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408860, "value1":0.5030243670352939, "value2":3709214658604138842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.079849, "value1":0.4036712273225728, "value2":2906136993668691733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735107, "value1":0.11572752191146224, "value2":9135232014249483896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.579000, "value1":0.832018905121829, "value2":4291485121105439680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297326, "value1":0.3053519839954546, "value2":7762295205550727215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077473, "value1":0.5244918634463549, "value2":3651586678818959294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211495, "value1":0.3162790532742153, "value2":8307153509301118839, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.018730, "value1":0.4578847558537525, "value2":3121380712594178301, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.254109, "value1":0.0022335788974661955, "value2":1113680598327977122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327840, "value1":0.38572838732788006, "value2":2539441632284031099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802473, "value1":0.6047951359580281, "value2":1001558191118170380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299508, "value1":0.05862512584912177, "value2":5514618695776223541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.011661, "value1":0.31240922806678983, "value2":6137561733651595636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728655, "value1":0.1278483939203575, "value2":9035381570106713199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.667349, "value1":0.5613783650549519, "value2":596862476082819858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.253799, "value1":0.021910250550882106, "value2":6533597878636060832, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_9":"k","key_3":"a","key_5":"b", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.288444, "value1":0.30057474196214407, "value2":3169010106041213703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.798940, "value1":0.9681683301198919, "value2":3021363800209965251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.981962, "value1":0.9612929543752854, "value2":9171352851679704895, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149939, "value1":0.09981385175599812, "value2":2281824057904455016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943557, "value1":0.6955975747592801, "value2":2634327756448196264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459593, "value1":0.6446822058280443, "value2":6342926358496707008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.734204, "value1":0.6639025089026711, "value2":2476897145958348918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506327, "value1":0.10053599414522371, "value2":2221394931414407943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410931, "value1":0.8059701840141592, "value2":3838253097677401347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.964398, "value1":0.603279381255061, "value2":5782827613318848456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_3":"f","key_0":"f","key_2":"d", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.278202, "value1":0.7115230775467353, "value2":8371900549612489852, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408547, "value1":0.21860759782537603, "value2":8039563012338877282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.261349, "value1":0.11227782989658946, "value2":7783215654484926349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059288, "value1":0.07895743947800146, "value2":5667050368393356509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.601079, "value1":0.6482240475140798, "value2":4251195673123293243, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.225112, "value1":0.7216063470142905, "value2":148937314023613788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977412, "value1":0.6598247780794319, "value2":5965401930753401354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194225, "value1":0.6141698639690517, "value2":8817766861356540801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.597081, "value1":0.07693962991598453, "value2":5029342104400232103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.335617, "value1":0.5583831809146028, "value2":1406686773640908295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916403, "value1":0.7018505752435668, "value2":664450605422502237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497447, "value1":0.8277122605282176, "value2":1219079943087879547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716326, "value1":0.7509816698301687, "value2":3921665603218422131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716585, "value1":0.2920975208652007, "value2":3756232698934542100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301567, "value1":0.8942632920542656, "value2":2291395595568091863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.599725, "value1":0.5659814539312558, "value2":7320359369690410226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.614192, "value1":0.7911920709201787, "value2":8781166815844312813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.195843, "value1":0.3963219582946417, "value2":4716889546078718843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555996, "value1":0.602486402959913, "value2":4687524095847830496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.917870, "value1":0.15392369008366974, "value2":7045583703847585198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.616039, "value1":0.6919323182152138, "value2":511455339796298853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930107, "value1":0.12610440100742193, "value2":6997765950733061577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194345, "value1":0.7209468362854391, "value2":556025580881809360, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797831, "value1":0.2266151348199214, "value2":2004107184327193155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_7":"i","key_1":"d","key_6":"b", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949681, "value1":0.8884059641939672, "value2":7011478951165849176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213425, "value1":0.9929181204525178, "value2":8631476778841936239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821834, "value1":0.3090901919909019, "value2":5522454281472043559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.181027, "value1":0.06720359916226379, "value2":4456725170730825432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.762490, "value1":0.474581541255306, "value2":7005342387987070026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.759790, "value1":0.5398913013405661, "value2":2663889184009973941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112196, "value1":0.8025326007392437, "value2":883355344681130779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374886, "value1":0.9718695344446878, "value2":1646239120227544644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.729457, "value1":0.9682037233981026, "value2":6151398746046248850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048424, "value1":0.7273914683080029, "value2":8000884246159354431, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.937043, "value1":0.1526646293780592, "value2":266299265780041426, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752328, "value1":0.9655606802627454, "value2":5596664612933771240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666200, "value1":0.10048257972248233, "value2":4289061240642605106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922103, "value1":0.8963130727289086, "value2":8836663020903305631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105202, "value1":0.8314839649252564, "value2":3551543011617183084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886891, "value1":0.66836051888972, "value2":72366475317978094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432056, "value1":0.745064543848614, "value2":2189880642993257536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042716, "value1":0.3956017159748215, "value2":3605211356176936944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.984359, "value1":0.2911243881226883, "value2":7856628067522273320, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295731, "value1":0.4520766897124352, "value2":610878216891620607, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.650753, "value1":0.03928359938969399, "value2":2072987167295070336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211564, "value1":0.7625006233645502, "value2":4401190374987004409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792725, "value1":0.47390601969195995, "value2":4329714295913254034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.444277, "value1":0.5857721636217157, "value2":3580156964556605157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535990, "value1":0.9433719155892893, "value2":5643233427116703402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.688127, "value1":0.7720856573472716, "value2":5631377472070167195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.697966, "value1":0.9263837993798216, "value2":4093734777340860370, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.996439, "value1":0.3910900735196092, "value2":8375564545525062485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872727, "value1":0.8324218421388521, "value2":7152928233109153205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.246110, "value1":0.2551941160691682, "value2":5602869232123697435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931727, "value1":0.5795265646922866, "value2":7403691202626500132, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673141, "value1":0.06896155961831285, "value2":4415828287435091542, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_8":"a","key_9":"h","key_7":"d", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684610, "value1":0.16872221976263158, "value2":505317720854796636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559417, "value1":0.12924994486885147, "value2":4787886921303451969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094359, "value1":0.5164211297919008, "value2":1315713014945297109, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007976, "value1":0.7735634457421625, "value2":8579409036573615608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286892, "value1":0.43624877878480095, "value2":6924175474906299981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.902926, "value1":0.6780220008663965, "value2":4822375571733927097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.486540, "value1":0.23333524037004374, "value2":6314520721238807162, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007170, "value1":0.9857337079596743, "value2":4448133028853613455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.030552, "value1":0.24355749578071725, "value2":8327413108187969392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.379606, "value1":0.4729923914146341, "value2":914941226083135930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835232, "value1":0.9602105867588695, "value2":3810448989361248064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433684, "value1":0.9595084872985944, "value2":455874222103117607, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411653, "value1":0.037665634780994134, "value2":6833101830522402640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_4":"a","key_5":"j","key_2":"e", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713632, "value1":0.5664925972399699, "value2":7794307235376671149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391266, "value1":0.6608939962095463, "value2":5860330325168511608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212303, "value1":0.2089060448996, "value2":5651881484326487830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180078, "value1":0.7876093968011537, "value2":2862206598597477486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.175199, "value1":0.14395244434800714, "value2":2376804168343245120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809965, "value1":0.12003398504952464, "value2":5187708593154594566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.851381, "value1":0.607956049195976, "value2":7288813225458184427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933118, "value1":0.7246239590339857, "value2":1234716149527350625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.939251, "value1":0.34949617152726564, "value2":1847804623713516854, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665651, "value1":0.7331985679295692, "value2":1370393562180863722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.827386, "value1":0.6896205618409067, "value2":1887650763331073033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.637520, "value1":0.13783942741851515, "value2":7482561145284406008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657534, "value1":0.26275939644072077, "value2":6263897972267140208, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740684, "value1":0.9117133919549965, "value2":6149129956357538684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032295, "value1":0.9865289627211871, "value2":5555291464918052387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545212, "value1":0.5648289415381519, "value2":6789070914120574869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.321143, "value1":0.6917810996904441, "value2":4171532082462131046, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.693006, "value1":0.9813132387635279, "value2":6633038595296515364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.895453, "value1":0.7852256444188942, "value2":143518625251882031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165192, "value1":0.9962691534924663, "value2":1006093794801043666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.353290, "value1":0.5341309097782416, "value2":1258165174425707067, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257167, "value1":0.7039937676297155, "value2":1145181626509784790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413647, "value1":0.8270957619940229, "value2":2929580620701623843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817697, "value1":0.5617132216229666, "value2":9160744958657565723, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.117851, "value1":0.20816644440142626, "value2":5357495560720255737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545126, "value1":0.5346084869679877, "value2":7401215357009964293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824766, "value1":0.44946104541199977, "value2":4136655445147421363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.292920, "value1":0.8837887841085235, "value2":7386066379061583540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949095, "value1":0.3445432872175305, "value2":7354448880530731294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705243, "value1":0.4595023646368128, "value2":1229720604223933971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_5":"c","key_6":"a","key_1":"j", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610093, "value1":0.4977300016550588, "value2":6055289293785959372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.710138, "value1":0.8494303412007083, "value2":3284734735098008783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.503055, "value1":0.7306081415151926, "value2":7335475436979640441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194198, "value1":0.4080577435044893, "value2":1817328694183447312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409059, "value1":0.8174198346097175, "value2":1028006350387911539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696870, "value1":0.3114167302109578, "value2":5761149591452007692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483258, "value1":0.6191704897662682, "value2":6062885989254649767, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534924, "value1":0.6117230725465559, "value2":193279975656637243, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639609, "value1":0.47495483196675986, "value2":5738372947955229915, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532964, "value1":0.01280767557324011, "value2":3610414855244726673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.741210, "value1":0.5736887790720044, "value2":9118221200383527859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459701, "value1":0.09043619123087489, "value2":4263857141830808661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.064891, "value1":0.8303048678510601, "value2":1417745368253260336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.193982, "value1":0.5954818624712542, "value2":1866813575466479927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728941, "value1":0.42773413448912906, "value2":5011276511494346486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978643, "value1":0.9720431088123873, "value2":5835438838603942392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376007, "value1":0.5894070491287171, "value2":1536141728264317947, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_8":"g","key_3":"k","key_6":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226693, "value1":0.26901914665508003, "value2":1625748998624229948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.963696, "value1":0.5769249508695262, "value2":9054438071997375931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.418644, "value1":0.766497146060167, "value2":5911142909752098984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564856, "value1":0.11883244261524029, "value2":1743904390192603929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055140, "value1":0.52087787431266, "value2":402388317326273396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165833, "value1":0.8230062512281381, "value2":3737986001509812244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.537380, "value1":0.3719641116241002, "value2":7620785809655703628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059138, "value1":0.3359364898323379, "value2":2258599254367397283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532103, "value1":0.07168825170242699, "value2":8419082906568750462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_7":"f","key_1":"a","key_6":"c", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286202, "value1":0.8607616531331588, "value2":5301488987302943339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477954, "value1":0.7462715469038064, "value2":8695685405375237694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_2":"e","key_9":"h","key_1":"e", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.377516, "value1":0.1251480000451169, "value2":5693123483571141635, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799701, "value1":0.377835354984664, "value2":7635656423964308029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947251, "value1":0.7193521006934637, "value2":3495998112139060094, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_9":"c","key_3":"g","key_8":"h", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.630158, "value1":0.27928250023508844, "value2":8779396817967517012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886093, "value1":0.6334013505397469, "value2":5098375435747464833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.942245, "value1":0.3390011070632797, "value2":1737381722392040741, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_7":"k","key_4":"f","key_6":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.009762, "value1":0.2515798026660567, "value2":2465310044177783178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.896699, "value1":0.584030830613032, "value2":5977614913522628306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820378, "value1":0.7811580321417821, "value2":1709536028386531684, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653683, "value1":0.7005991085562125, "value2":7295655368561133786, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522361, "value1":0.7296163959268779, "value2":5916302487026661508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.127496, "value1":0.24876472584091772, "value2":8637191483262110011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.091531, "value1":0.9069922237355105, "value2":3364698751790785415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137633, "value1":0.9475238029065713, "value2":6717128691639834645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550582, "value1":0.1420769771765079, "value2":1644264737205720610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059092, "value1":0.38378121050522995, "value2":4775737880122110580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236099, "value1":0.8215350110894001, "value2":1784075564711617189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_8":"b","key_0":"b","key_5":"e", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.636343, "value1":0.4453072911799391, "value2":3315454218709810887, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799575, "value1":0.17197278191202653, "value2":5903905346180673218, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.584133, "value1":0.4235206584790561, "value2":1691406943949519298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015622, "value1":0.34252290670938007, "value2":5505899012400247300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.021661, "value1":0.11232931832514273, "value2":6468715321661687664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.593330, "value1":0.8735006589361365, "value2":831831146444791684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273986, "value1":0.25953904800502825, "value2":5320773233674225264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357026, "value1":0.11876193765893824, "value2":8703386441096799470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903715, "value1":0.0317054368098292, "value2":2551886074115870652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.905359, "value1":0.9780528237638186, "value2":3462176768443711504, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.272907, "value1":0.5077414966128044, "value2":5351891928699763264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610476, "value1":0.5253243304839273, "value2":7118905075072261164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684508, "value1":0.3558646139061106, "value2":5157018975048237545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748383, "value1":0.7510255655970447, "value2":4180265211771950319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.801038, "value1":0.2649030050080282, "value2":509475464752529120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657950, "value1":0.2544231826829063, "value2":4768343360725290987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149149, "value1":0.24278734421658493, "value2":2310223808628977449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.558331, "value1":0.3866811766896264, "value2":3120828999446479419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.076776, "value1":0.8191629913767823, "value2":531265332159947696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881067, "value1":0.16630603149873838, "value2":8244214645789653431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385842, "value1":0.7521248304618617, "value2":4350867045350111094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914596, "value1":0.12306660567858013, "value2":5899167982504356408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.159683, "value1":0.24808395371130693, "value2":3321017992196518857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.058378, "value1":0.4115012836332614, "value2":5551372136464658518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_6":"g","key_0":"a","key_2":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819794, "value1":0.9548946315713374, "value2":2661379806705881657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.293628, "value1":0.30579616916280866, "value2":2771189581765698489, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.904159, "value1":0.33983467932998995, "value2":3076540259059103682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946959, "value1":0.14422457278803882, "value2":619548712042120909, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383295, "value1":0.23130372874617672, "value2":6179693520548172628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482434, "value1":0.8720643520387502, "value2":6593209094509926896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613596, "value1":0.9574764532675973, "value2":8706916487148773835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551130, "value1":0.9901941419895447, "value2":5559435899121181473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954673, "value1":0.7022122624865335, "value2":6926608561209244320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781834, "value1":0.1129731019718482, "value2":3356276176028256450, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_4":"f","key_9":"e","key_0":"b", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746610, "value1":0.5148787087784333, "value2":2343142454537691934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409647, "value1":0.3828843970677094, "value2":4346661084777988652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.427543, "value1":0.6220594320308941, "value2":653819026690948218, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_9":"e","key_1":"d", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467056, "value1":0.23591956688182053, "value2":5924417417665977435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245070, "value1":0.32465613049106207, "value2":8745282484552741587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719344, "value1":0.6149579128373653, "value2":7721284234183306230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095318, "value1":0.7783448314627689, "value2":7496490910707147022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696761, "value1":0.23959350451136963, "value2":7667698021180616606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170976, "value1":0.2107634553446009, "value2":1067019224393043111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177955, "value1":0.15559883601716398, "value2":570151235008619291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.334876, "value1":0.05250568880425774, "value2":313709473500400875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.645622, "value1":0.5874495193049836, "value2":4998753194711581798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403716, "value1":0.681377437500311, "value2":6083653541111639303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.511374, "value1":0.3173656336054262, "value2":3280525464397494846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.218153, "value1":0.6649425163171951, "value2":984695368325234886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_5":"g","key_6":"j","key_4":"a", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424332, "value1":0.13915015029096536, "value2":177058255857848790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882802, "value1":0.9604830357087591, "value2":3462432728417263080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_9":"f","key_0":"a","key_5":"d", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933587, "value1":0.05218287981155221, "value2":9126870231879692354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426076, "value1":0.7165203610998703, "value2":4392854854028036091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.302645, "value1":0.34553849779237905, "value2":2907279110591723700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269877, "value1":0.8711177705575849, "value2":8366161311313542515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.995800, "value1":0.6906708270307872, "value2":2179499360020827214, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433217, "value1":0.612549221250184, "value2":3062667501076675896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550193, "value1":0.8112010714019721, "value2":3511775054758615153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947917, "value1":0.4309103807271251, "value2":5927887337207634099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874180, "value1":0.8528260534422161, "value2":6693253358548032140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.714464, "value1":0.7660112509524851, "value2":4667923922359952270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.088061, "value1":0.2662708805339307, "value2":2956847029720911185, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960166, "value1":0.6603171866146907, "value2":4062360360972662646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.771991, "value1":0.06535928533875911, "value2":3275575695469134507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.907738, "value1":0.4729789504578976, "value2":6665898542309531788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_6":"c","key_9":"e","key_5":"f", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.446527, "value1":0.054360638009933175, "value2":8447197342747029062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871013, "value1":0.9325815658063547, "value2":3295816637987069156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322468, "value1":0.7896334184172146, "value2":2062716273940484317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145073, "value1":0.15908044288327283, "value2":31202777072956548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.450280, "value1":0.20707514852594208, "value2":3916360901630366923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.834638, "value1":0.6993008706531769, "value2":5937180712489291188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116654, "value1":0.23637601643034672, "value2":4626933419743360623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_5":"h","key_9":"b","key_3":"a", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.265735, "value1":0.37825344249371357, "value2":1259576346841819002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683748, "value1":0.8969615470937745, "value2":3878280364099631128, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219470, "value1":0.2735521439606725, "value2":2004029025316087157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476407, "value1":0.3060298444596037, "value2":2752192683000499763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973916, "value1":0.6236548692215601, "value2":1218902262886276264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777866, "value1":0.4605467848114755, "value2":1483419366258255115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045114, "value1":0.6713933571025698, "value2":7389349750224761078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.570321, "value1":0.6477885368549638, "value2":952734344221236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821043, "value1":0.6987507461356852, "value2":7961454653125032378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.586904, "value1":0.6114733442241732, "value2":3313794499633786815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705481, "value1":0.05021439091843144, "value2":1692908847970065889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497544, "value1":0.7911397038857368, "value2":7192760038857907146, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.932189, "value1":0.9148929522533059, "value2":9058585624863979355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769986, "value1":0.08834899410804685, "value2":1734900992084257549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695397, "value1":0.9679316547671841, "value2":7641677535001923086, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084900, "value1":0.7534718430023655, "value2":5595569809726413737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.562250, "value1":0.19691330948497934, "value2":5079334807276275801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.843384, "value1":0.13037385236459426, "value2":1781212766913480310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.102533, "value1":0.8945673902737089, "value2":4413715911002881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350432, "value1":0.9789410844857529, "value2":8061203484333522616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857648, "value1":0.17405749140176002, "value2":9069075917121269047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559262, "value1":0.2097671258596821, "value2":613585525667388168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.712886, "value1":0.7017574550139256, "value2":807895944720896464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.936984, "value1":0.8849159113766228, "value2":6360061394425352910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545107, "value1":0.7492436378397768, "value2":1930556756703562417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.681593, "value1":0.044080654493611975, "value2":4672445330328029132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388330, "value1":0.4856178458787539, "value2":6153421365310799825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.879197, "value1":0.16996161009540478, "value2":260583885651182711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.770981, "value1":0.7283458398178451, "value2":6187029331538785539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101721, "value1":0.5814924092013966, "value2":3246021123474652609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665903, "value1":0.2698824852319885, "value2":1593984595070302417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.014945, "value1":0.41108393803888965, "value2":849125531814643716, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455276, "value1":0.875114673636132, "value2":5155846223745306265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.044985, "value1":0.9997830302808556, "value2":2517744318699317082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236657, "value1":0.10301472150509888, "value2":3916175947622962913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.595845, "value1":0.681705831836613, "value2":4919968063926505445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909129, "value1":0.10658700130911442, "value2":9017518189462648176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994913, "value1":0.5742718119176038, "value2":2926652004213539521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350277, "value1":0.15279738754509417, "value2":5229335911692726266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.022399, "value1":0.3330863087425972, "value2":4953311412687090451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268930, "value1":0.7193122390898454, "value2":4256538916269794836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.983431, "value1":0.0510995090351011, "value2":1768238660724776680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672570, "value1":0.255109482234144, "value2":5911656502600207006, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478340, "value1":0.7629467314333804, "value2":2248461028173940598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954691, "value1":0.780819131964203, "value2":4988575109325316188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.854956, "value1":0.888056149862958, "value2":8089385147769653445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028845, "value1":0.6170371731466705, "value2":4427914831964658823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639652, "value1":0.8230090544258113, "value2":6386454055832759048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.810891, "value1":0.8133492204735011, "value2":6725307713283048525, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761452, "value1":0.9287463566978789, "value2":1984332858107282238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318552, "value1":0.12867977830402738, "value2":4258557511687812956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316280, "value1":0.5022437385431537, "value2":995773241737628619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954122, "value1":0.5240772572203263, "value2":6037304727626529988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177671, "value1":0.76376619698564, "value2":2077146903364533171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731650, "value1":0.7327573768170443, "value2":3115299633570203304, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.050578, "value1":0.7589539042984136, "value2":8472839195693038957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.606514, "value1":0.9364993124842877, "value2":5612839525396767856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369368, "value1":0.048054238834199556, "value2":5282267936079713586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_9":"i","key_8":"d", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619618, "value1":0.3228482622950357, "value2":8295983975137448078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783482, "value1":0.20802671279789772, "value2":3029896725457460111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_7":"j","key_8":"d","key_5":"k", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332994, "value1":0.9302305981560293, "value2":1560834164036895150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320192, "value1":0.22351800921367507, "value2":367222968940142358, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881266, "value1":0.08414682717481595, "value2":4156359981061160049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.708929, "value1":0.5859616740437592, "value2":4218658472873761504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.974600, "value1":0.18236565991254491, "value2":7158153525091699583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760005, "value1":0.2499852109211892, "value2":3103407843415565026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994395, "value1":0.6346134433025367, "value2":8895656809884955572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482016, "value1":0.09096777794371665, "value2":6086112719790576068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442570, "value1":0.8266959930861106, "value2":2275327626493666884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416700, "value1":0.36518175488705273, "value2":5793835688591222111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.488107, "value1":0.8111992750792233, "value2":3432067066553462639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.296157, "value1":0.748182736526456, "value2":5908492840734386304, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655322, "value1":0.77012733378921, "value2":2328983933411827460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299380, "value1":0.8617587030482643, "value2":4249851149905507488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.569550, "value1":0.8799767655587674, "value2":114484019272599760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.737420, "value1":0.660133828675834, "value2":47092260178100634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961538, "value1":0.5805975021285331, "value2":3098519932913684476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197731, "value1":0.7355297940768138, "value2":5347385484676578771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653672, "value1":0.9639932888139914, "value2":4154150822821006863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209735, "value1":0.25867188901232296, "value2":7573003393945357593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.243923, "value1":0.34698853560785725, "value2":4046809595104442791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055125, "value1":0.9118136199691569, "value2":6132859986145115351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.129146, "value1":0.737958001899574, "value2":3639000020359385593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439824, "value1":0.7612995291197047, "value2":9120669622233440416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222157, "value1":0.2453738047252645, "value2":5040757825498916672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916336, "value1":0.7380620101187605, "value2":4223451018219910420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365543, "value1":0.035239810434389264, "value2":7558213648465294836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.860762, "value1":0.4102850563256095, "value2":1429493420542448489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388872, "value1":0.5076851349389522, "value2":5984142931688899418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.324767, "value1":0.32798077886519317, "value2":7010471358073816774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605843, "value1":0.9054516940429083, "value2":5196636487460887513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653438, "value1":0.9546349180436571, "value2":1964265110886490351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576178, "value1":0.9072096436167739, "value2":4650061523788539671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.304613, "value1":0.07126080366571305, "value2":2739761425628315393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399413, "value1":0.9339282528280148, "value2":2738842204765000811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943959, "value1":0.08784682744168541, "value2":7785212390244201422, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_7":"g","key_8":"h","key_1":"j", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.155440, "value1":0.06596281114416563, "value2":5614935012679054029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_4":"d","key_2":"f", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.585232, "value1":0.38735104883356297, "value2":5357846878220935711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.231739, "value1":0.7284804092280743, "value2":1167665308911410443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555892, "value1":0.5915854017921878, "value2":1567147347614329809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.855303, "value1":0.09176207020704319, "value2":3808984845296778232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.938710, "value1":0.4819531123010938, "value2":4697805519732493178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476902, "value1":0.22082616880251751, "value2":2129509252592428060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744743, "value1":0.33257150382979367, "value2":7648865469799769588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306532, "value1":0.04481922646422544, "value2":5772762433855832524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364825, "value1":0.3587026951251621, "value2":5278057080002614026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_4":"c","key_3":"j", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.484995, "value1":0.39478883780380347, "value2":3136752353967205980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.359118, "value1":0.08414989767840077, "value2":1942845542819450866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095178, "value1":0.16106032134150491, "value2":2585752892335863488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268896, "value1":0.613902523132248, "value2":687319106737333189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716029, "value1":0.4960997765937663, "value2":2263474829486803965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748556, "value1":0.30380613001187684, "value2":569759032424044386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_3":"c","key_5":"b","key_1":"c", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819011, "value1":0.9766133829878352, "value2":4041486474077430360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_4":"i","key_8":"j","key_3":"i", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109307, "value1":0.7831795468260019, "value2":6028647505036494731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.179692, "value1":0.36834450950873376, "value2":5143099232657374567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408811, "value1":0.37776607558956526, "value2":6856480231263656565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.006253, "value1":0.5270930124398272, "value2":1411653061914296136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776912, "value1":0.4551811310760191, "value2":1536148212309064628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447453, "value1":0.8174960750892899, "value2":4331073111578365688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696722, "value1":0.2434463944121451, "value2":3006348927793308525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.747552, "value1":0.46652870663413926, "value2":8487775958608976630, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618043, "value1":0.5667967701695303, "value2":445864738400581228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162568, "value1":0.6628301293329222, "value2":8197909955604355013, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.861425, "value1":0.3485000903134516, "value2":8816338849324478138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.806516, "value1":0.4953223448201175, "value2":7711648028791565560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970539, "value1":0.2083396416590356, "value2":5815736098137722214, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108983, "value1":0.42614208367663026, "value2":5403929578330914973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095108, "value1":0.4675226715836234, "value2":4103322823365161263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.279008, "value1":0.5189838609534998, "value2":5583248834742899795, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080151, "value1":0.04025953736590447, "value2":1709967085975216083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960603, "value1":0.48558007294067956, "value2":147087377780565755, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522699, "value1":0.8996419465199031, "value2":8383086671138869204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812906, "value1":0.42145746065428685, "value2":3807545219581503501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973610, "value1":0.9578107981118065, "value2":4941505183486571577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972491, "value1":0.7644375474791567, "value2":5826757011862504756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048527, "value1":0.2281920875592009, "value2":4718173727183162630, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515627, "value1":0.3746899214010938, "value2":5127946555951269169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.346004, "value1":0.5308363801498025, "value2":1141799226947758562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_7":"d","key_3":"k","key_5":"c", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391384, "value1":0.5665502681985652, "value2":5138410786658734048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857410, "value1":0.1451852744317182, "value2":6603434642077187944, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.414061, "value1":0.265028330167258, "value2":8861513360159934632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653333, "value1":0.28437623266560746, "value2":5007130114350069495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.123846, "value1":0.29659698191393213, "value2":8540905938700351347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477802, "value1":0.6141130483574686, "value2":4145514544028354410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531351, "value1":0.3974291747311908, "value2":7891249160701823099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197949, "value1":0.9490013407410278, "value2":8218574898456721251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411857, "value1":0.8458181580037749, "value2":3063835093954499296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.877510, "value1":0.828501312919876, "value2":5054318282608028643, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665063, "value1":0.5885292293078199, "value2":7952781418541268636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041521, "value1":0.9878113457449454, "value2":7450867973872472368, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073061, "value1":0.939649958460976, "value2":7875175737198226822, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_5":"k","key_9":"d","key_4":"i", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781888, "value1":0.5031750456506159, "value2":7116386269513023773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160017, "value1":0.23130907810117704, "value2":2438724480357189127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835359, "value1":0.4415459106308202, "value2":8077656610946018544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251312, "value1":0.10148089002006272, "value2":1896400908090076616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.632483, "value1":0.7757884759608441, "value2":4168124167987668028, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.772638, "value1":0.14296917471420942, "value2":5892426599914094550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426554, "value1":0.4152913794309927, "value2":3491478566670263783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672520, "value1":0.5730477814369939, "value2":7734177692857714726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025475, "value1":0.8422765595744146, "value2":7729206509934976065, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384117, "value1":0.370521152586741, "value2":5964592513649221172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666556, "value1":0.5944594406776452, "value2":6664609919274167291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424484, "value1":0.6898557357450742, "value2":7620342586495006501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313298, "value1":0.7769705418438511, "value2":5904168920374633694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556524, "value1":0.7197157815883978, "value2":1785992040018522529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360907, "value1":0.5368763819006307, "value2":1221035619431068900, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576297, "value1":0.2976581980063287, "value2":6069726578064867551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.425007, "value1":0.05760812531545847, "value2":3218040486200104849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442385, "value1":0.7140138290502642, "value2":5982710133241839409, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364950, "value1":0.7133219094796532, "value2":2621884575274392083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211537, "value1":0.9662389413186122, "value2":6837048628615994370, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.161776, "value1":0.9162528731861198, "value2":98225928051258831, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811493, "value1":0.5668121931771126, "value2":4899385784647025957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822383, "value1":0.016988952685027904, "value2":7343988970036816255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865454, "value1":0.48779466867997384, "value2":3996429126028951858, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751902, "value1":0.517057218253837, "value2":4621244619940358234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.525579, "value1":0.7270638229553396, "value2":4556294068287353316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822049, "value1":0.8487037862810487, "value2":8759327482669918657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483767, "value1":0.795243973695518, "value2":5137269638112922032, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042738, "value1":0.7385935337493647, "value2":1091563145939490509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.521929, "value1":0.10598411546030329, "value2":1018022188446431653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792757, "value1":0.4495950673323956, "value2":1856520078086460155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695775, "value1":0.8122734246000456, "value2":2520061166650080735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105722, "value1":0.9820835252391706, "value2":7455782937968387088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820568, "value1":0.22530315350938107, "value2":3276960573086645000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.140320, "value1":0.5664165038340241, "value2":7444501896936670459, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.501828, "value1":0.6731262889242471, "value2":1425113709817927232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930172, "value1":0.48625040372483846, "value2":5327468289350467201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.706858, "value1":0.07914368053314678, "value2":1800402440657604521, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134725, "value1":0.4678852928796053, "value2":2325913710025246340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876578, "value1":0.9424035369996965, "value2":8055626132574707503, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108826, "value1":0.29002692913973105, "value2":774202804060462158, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.031892, "value1":0.9251224968475468, "value2":6954662396929356191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539689, "value1":0.24245379006948578, "value2":460788187782305876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.858085, "value1":0.02056945040393724, "value2":4590009984548317252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.202540, "value1":0.37546125893661825, "value2":6399435061817101457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384494, "value1":0.010928897877400335, "value2":2783102565309398205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_1":"f","key_9":"g","key_0":"k", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061578, "value1":0.259570974455371, "value2":8569411528487848312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_8":"h","key_9":"e","key_2":"h", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027239, "value1":0.09519589010929196, "value2":6611488629400003141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210858, "value1":0.6912816460992459, "value2":7725980859343417825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.825726, "value1":0.48050045655332785, "value2":6843673580007675724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430241, "value1":0.8940246106870715, "value2":7204593499636831454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_5":"d","key_9":"f","key_1":"h", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332710, "value1":0.05615399939144247, "value2":4573382128185893311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374418, "value1":0.5152317418659397, "value2":7908425193534726276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519252, "value1":0.2299372870124918, "value2":6913359063794068428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.405529, "value1":0.49571648923268996, "value2":1675726536905469427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_8":"i","key_2":"j", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605031, "value1":0.09917638686306547, "value2":3890367351649717445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194280, "value1":0.5756913899097403, "value2":6546196099308660172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763643, "value1":0.049961431289849915, "value2":2156338695320389229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755741, "value1":0.4853393193247385, "value2":5380709479032006182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434015, "value1":0.6730243170500183, "value2":4072172223941247851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.188499, "value1":0.8036991963254426, "value2":2042780350133201142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.788778, "value1":0.6590319721348468, "value2":6545712378323399560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431899, "value1":0.10111044541197162, "value2":3033625496942345343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846633, "value1":0.4624813135357754, "value2":5961232902625455903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649911, "value1":0.42335939078880974, "value2":9206873087714105578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532557, "value1":0.8374513257646443, "value2":4063679098902368460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.993435, "value1":0.026158890681344544, "value2":8561365511354827010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808220, "value1":0.6771449672369052, "value2":2392568978099066809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_2":"h","key_9":"g","key_0":"b", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357037, "value1":0.7840791560680648, "value2":1293280054604430799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686467, "value1":0.009361168431443715, "value2":1158105653011230048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.971166, "value1":0.5749373357687312, "value2":5056280036307903399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.544825, "value1":0.6259231852657728, "value2":1572064409805832355, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_9":"c","key_5":"i", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889816, "value1":0.6691815472587526, "value2":5159786074932669495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751169, "value1":0.5117344773320296, "value2":2709445168494389044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.358316, "value1":0.5060937970199147, "value2":4487218678538317276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.577717, "value1":0.8145066136721544, "value2":3597604992290980548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.990084, "value1":0.031203549542102884, "value2":6033709675731250939, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613856, "value1":0.5443798055071071, "value2":1023967738912955990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795434, "value1":0.2935457393386216, "value2":1392092134182260244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233100, "value1":0.7087138744541079, "value2":6901046651435105011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.215643, "value1":0.301381153698089, "value2":5724617386824483093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.536387, "value1":0.050992650300612716, "value2":1926309233688447762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_9":"c","key_3":"e", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.661031, "value1":0.5259350399353443, "value2":3757516621940142781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.883217, "value1":0.9897587697648693, "value2":166453122026887271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482631, "value1":0.7903296358928328, "value2":6917975765021057279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301142, "value1":0.03214802664084894, "value2":1209824808667363268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618630, "value1":0.34174203368906997, "value2":687273686120110296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872422, "value1":0.7312694201106051, "value2":1954246364933112681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675094, "value1":0.02386805295420225, "value2":2582799143372120508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.397537, "value1":0.7375439234748727, "value2":3761195316124557282, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998621, "value1":0.44001824362826686, "value2":2298817484668315029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471460, "value1":0.5040752524973563, "value2":7440124910918057262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303559, "value1":0.34760281561681783, "value2":3856193801537489239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477238, "value1":0.10358452410298057, "value2":3433455763442399890, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_5":"g","key_6":"a","key_4":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974082, "value1":0.9936842430072798, "value2":7647779573135010712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.679212, "value1":0.33597414933520936, "value2":7871330604163867562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.420730, "value1":0.4119033435452023, "value2":7398827678901559124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.546764, "value1":0.0476713858704466, "value2":112659874965942961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730784, "value1":0.2868023177018098, "value2":7913444002207179200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862440, "value1":0.5312714100525405, "value2":6175515648240114931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769163, "value1":0.9602531490116414, "value2":5345461780217485343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855124, "value1":0.553411156209368, "value2":4062088865979243439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.210418, "value1":0.8711908622408605, "value2":8107060141858943082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436733, "value1":0.7746479416891845, "value2":3021240871572817126, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.692065, "value1":0.594498398968098, "value2":988266694346712016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.441926, "value1":0.8776750344471822, "value2":6190746113032965957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013512, "value1":0.14091811490255743, "value2":4960449076483043020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548236, "value1":0.32376195419487824, "value2":4889372139161861463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.828595, "value1":0.6520731819283484, "value2":508152344585097791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759392, "value1":0.11332264235829892, "value2":8149117049560006848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901588, "value1":0.1665732927247099, "value2":1166517989622232910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035913, "value1":0.07988414283718974, "value2":7894355898795371390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.583982, "value1":0.25757663417932825, "value2":6007061793954380382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636209, "value1":0.38383005316111324, "value2":4618851528086053316, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672395, "value1":0.002191687957779578, "value2":1207882083575373188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_5":"c","key_7":"f","key_0":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462535, "value1":0.3192805953827191, "value2":365124979437384544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508318, "value1":0.6798889100739353, "value2":255815407107212152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905933, "value1":0.882876839434679, "value2":1335222091221270902, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.721902, "value1":0.3457876641727297, "value2":7448197074268866329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.837057, "value1":0.03599562374649498, "value2":1792244431983896309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056476, "value1":0.22903239483804314, "value2":5638633766102759583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_7":"j","key_2":"e", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.457531, "value1":0.05016525338775747, "value2":2976936137139451754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617621, "value1":0.16600638886403743, "value2":3903308221116403976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415798, "value1":0.5206207545088288, "value2":7426460172854544634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.470076, "value1":0.02758169975135837, "value2":3279193513719457677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.847049, "value1":0.818759672974311, "value2":5946222659493106047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367937, "value1":0.4074258090017152, "value2":6858775210252472573, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973712, "value1":0.13887537149474657, "value2":2800343207358775074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403585, "value1":0.2978164609006992, "value2":6234312472150253679, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904490, "value1":0.7280336451114688, "value2":2026079147622689137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.541237, "value1":0.1368259441713387, "value2":6424894922861577254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.783065, "value1":0.5704544299311838, "value2":4180282487041693175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.890800, "value1":0.7757728483786852, "value2":8132981856885227577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901025, "value1":0.22908718180210688, "value2":4747104448031391939, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180212, "value1":0.5476402281995001, "value2":8223100966387528196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626159, "value1":0.7996650121856338, "value2":64003761107808118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_9":"g","key_3":"k","key_4":"f", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545550, "value1":0.1740742638651991, "value2":5824563041193046282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026911, "value1":0.9915609719154322, "value2":3839917984332114189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970605, "value1":0.049651840615534626, "value2":3663818871674092196, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.795861, "value1":0.6046374060545386, "value2":6467693625848837489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741291, "value1":0.4689122935380642, "value2":8701125505618349182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_6":"h","key_2":"k","key_4":"e", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.877913, "value1":0.596765963129067, "value2":8094571743010403516, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.790394, "value1":0.06093826578205693, "value2":733010313976904838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_9":"b","key_1":"e", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603208, "value1":0.8364659181351286, "value2":1088146971265159206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823621, "value1":0.47775163306986124, "value2":7344279199603749996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171666, "value1":0.02782471041039711, "value2":1659046247294927834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.899246, "value1":0.23664326650032677, "value2":764233009249486091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658885, "value1":0.13203154011115373, "value2":5571262153472749697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_4":"a","key_7":"k","key_1":"j", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.711786, "value1":0.6314799048378197, "value2":8955079539774261393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.358690, "value1":0.4626190815088963, "value2":5349880367436474865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436669, "value1":0.7604788504004301, "value2":2830736413446294733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283630, "value1":0.11435547311453685, "value2":5764151651661369748, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550602, "value1":0.9394300496620045, "value2":3385128630739961936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004479, "value1":0.08103496493647538, "value2":1096104398248692524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272179, "value1":0.007258870887977933, "value2":2565647835692463774, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.207218, "value1":0.8139650845703748, "value2":905077995346686402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_3":"g","key_9":"k","key_0":"g", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217842, "value1":0.48125835395143424, "value2":6789722720604854599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.155093, "value1":0.7189778774843847, "value2":6292044296017930854, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723501, "value1":0.3703438091023322, "value2":3955334986917106318, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_9":"f","key_3":"j","key_5":"i", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216869, "value1":0.12053628422917362, "value2":636444633907082482, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.568591, "value1":0.23735656022841886, "value2":4679225743620568386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.792289, "value1":0.8769118352474913, "value2":8705903369099981004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.377367, "value1":0.9028525772471804, "value2":8394233771809162273, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_3":"d","key_6":"c","key_0":"d", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511370, "value1":0.06603888812647277, "value2":1587071294143636433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060860, "value1":0.3291125358805135, "value2":7312301032222805401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816308, "value1":0.6388914782369252, "value2":1128303197769336870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.068805, "value1":0.319277617656504, "value2":6681946261878539245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.805301, "value1":0.009280749666995351, "value2":544578594920738627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493659, "value1":0.6717590147189912, "value2":2097325929268646339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.253857, "value1":0.2354585485783823, "value2":5182832084934966477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454134, "value1":0.7059786460183258, "value2":8734869026681606253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769480, "value1":0.16571806998598893, "value2":1451771174019239273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.043302, "value1":0.0805122859716286, "value2":8615098369512434691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493432, "value1":0.37562446579716846, "value2":447590635610718249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.340933, "value1":0.476702778910786, "value2":2962733058848207480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.351270, "value1":0.04893100925940487, "value2":3216639061913163460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819062, "value1":0.43774247002290106, "value2":6810500585509381104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.789988, "value1":0.13044652854726224, "value2":6447819044361282673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.137973, "value1":0.9280703615023543, "value2":2854967575973809701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359816, "value1":0.6661196608352099, "value2":7585154254761447569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.374877, "value1":0.2719104405916207, "value2":1097487272974828210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766465, "value1":0.8656426220974218, "value2":2886501368125017740, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267362, "value1":0.594401465949902, "value2":1715419869280476267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.689698, "value1":0.09904556283985433, "value2":361858677456228074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244000, "value1":0.4739214533638735, "value2":6722075717885814318, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957522, "value1":0.4793380026682715, "value2":8094832594089347058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578816, "value1":0.32497702056237604, "value2":8938729328568281395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_6":"h","key_1":"f","key_5":"e", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999599, "value1":0.703313156867648, "value2":5755903281901537381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829223, "value1":0.9601134377422823, "value2":4062518676536818057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.145200, "value1":0.7216961923243395, "value2":9217282407920967916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820803, "value1":0.3093137747462916, "value2":7468155061834676810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682889, "value1":0.04485861766843415, "value2":4270651219964958841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549256, "value1":0.7924629761166294, "value2":5462651594092502924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515774, "value1":0.1753099089980214, "value2":4245273970497284960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280472, "value1":0.19807313433421816, "value2":6932983129449398217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.183908, "value1":0.8803355871960044, "value2":3795185008893112660, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.250576, "value1":0.8766744958241532, "value2":8316217539045644702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811589, "value1":0.7673136679205708, "value2":1656000207332411296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617752, "value1":0.33750626542384204, "value2":2818737959409200195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.402465, "value1":0.5975891290632979, "value2":8117852232376404604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.779395, "value1":0.6126614135133973, "value2":1972761209604651335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171014, "value1":0.3147679970132471, "value2":3872394099215939892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.602514, "value1":0.9979964274456362, "value2":8280861801559743509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.870286, "value1":0.1346288239954595, "value2":2167488486464723184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.902696, "value1":0.53464205049288, "value2":8019370943564861440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050828, "value1":0.7038559591596831, "value2":6864265904757939564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.465826, "value1":0.7686595552419602, "value2":1927886873036321447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149509, "value1":0.9153544198103084, "value2":7814774915075726621, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_9":"f","key_0":"k","key_1":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.485222, "value1":0.9188620757546552, "value2":5192187368195488529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520165, "value1":0.1037499454739713, "value2":2861179515410191072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894795, "value1":0.6670064141722539, "value2":8570373623610344103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858375, "value1":0.9690125335533832, "value2":6598305524856472910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639161, "value1":0.07029962179108588, "value2":4198012464202851461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.422856, "value1":0.243592288273798, "value2":7060563137853890653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356376, "value1":0.12506395613411664, "value2":3055799774165215720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296059, "value1":0.7972570100142576, "value2":2611996617533897408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.461343, "value1":0.024502184861611945, "value2":1123093231220631621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529133, "value1":0.12083989402126777, "value2":7219868115171352433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110681, "value1":0.8049351220640796, "value2":4305443631669757518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.981566, "value1":0.19931635115078358, "value2":4376781646759017505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631835, "value1":0.4428488032269089, "value2":2962840158667509272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596817, "value1":0.01388179851872705, "value2":5526059113621970146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988044, "value1":0.08966434896908937, "value2":845094337456207269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032366, "value1":0.4632294657773622, "value2":2896178627081634846, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_9":"i","key_0":"i","key_3":"a", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.195349, "value1":0.3544490738754114, "value2":5485050023862879453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.488827, "value1":0.05815655955779658, "value2":5446494658153199948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723246, "value1":0.6834006388190103, "value2":4274252704044712919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141449, "value1":0.3967110267075465, "value2":2849522299224594762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.712863, "value1":0.2643526520952433, "value2":6055035031034459665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974542, "value1":0.07826746989883603, "value2":8013938940173093770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476496, "value1":0.08282820679282239, "value2":8740534065670450787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.265568, "value1":0.7141009753996177, "value2":8015024495118709084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163000, "value1":0.9787912408769939, "value2":1407010919457389562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560997, "value1":0.22142486556739396, "value2":2439658579639630159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325355, "value1":0.2775049821475937, "value2":2346830518660166745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491491, "value1":0.3135314864967855, "value2":1129771089143019455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.459317, "value1":0.2812010467625179, "value2":7830625013744386941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.684694, "value1":0.4935589110227349, "value2":2540823529765104488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.237886, "value1":0.07014579422912577, "value2":7022680572815993337, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.414046, "value1":0.06496797245021041, "value2":2711277679569543532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799639, "value1":0.8106870375099371, "value2":1007449970411192794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.264752, "value1":0.0960343886163872, "value2":3312684645465619564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035469, "value1":0.0008360965935936516, "value2":5539611781321403864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.306352, "value1":0.07245144250837937, "value2":3686038376310116060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.048629, "value1":0.3482697350949442, "value2":2011496763457379089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978673, "value1":0.9982728323868918, "value2":702860377347588565, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987564, "value1":0.21515438859453273, "value2":2583082073184523816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.842602, "value1":0.3444834642501587, "value2":4908475207072135082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966769, "value1":0.7858094998520652, "value2":6077992589658011123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.165450, "value1":0.6383111602434741, "value2":7226409376141032655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.575923, "value1":0.8562665414193692, "value2":2173652645149705334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010707, "value1":0.5955696347147411, "value2":5814682382951778842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271135, "value1":0.1229263700863702, "value2":6350918748538162755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143782, "value1":0.975371469047403, "value2":8279280670157614673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536001, "value1":0.2818838543932868, "value2":5548964524114515649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.274203, "value1":0.525661345793623, "value2":7632566212944122955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394070, "value1":0.7913688238824117, "value2":2991067692464176886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243187, "value1":0.9989819654882474, "value2":7814314371764134982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490869, "value1":0.30039262696070124, "value2":7927229014613646504, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_6":"i","key_0":"f","key_5":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.850916, "value1":0.8481086988896461, "value2":3502125731621593449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113680, "value1":0.32930706694057965, "value2":6445034393695608409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192297, "value1":0.3973423811708676, "value2":8167271156640822516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646981, "value1":0.6421917320177064, "value2":2441775949474583008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.482724, "value1":0.3152543309412001, "value2":8846147135578854122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777161, "value1":0.26611205622992573, "value2":4707069095348547526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529669, "value1":0.25143936257925226, "value2":9113227763618672958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708875, "value1":0.10515643642729285, "value2":7582828897771165151, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041864, "value1":0.7204022074713844, "value2":4505351249313209540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.517118, "value1":0.2778141202519239, "value2":7041089660646534661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255695, "value1":0.5252045968012302, "value2":1609597628350217466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.001450, "value1":0.15419422555422022, "value2":6306687575854728640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.230403, "value1":0.7229181740004053, "value2":1306679799234664681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035841, "value1":0.6895973487797646, "value2":2559143873675894370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.449904, "value1":0.7368166767257398, "value2":7052097343090100521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139346, "value1":0.807172748525886, "value2":4308240895194146306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977592, "value1":0.7560595694496225, "value2":3367349773561100464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.831129, "value1":0.43661553653957313, "value2":1514573148689105923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804508, "value1":0.7956852333376209, "value2":8820887809742814806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858280, "value1":0.8493727617212654, "value2":8684654170930913793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640988, "value1":0.2223414127619492, "value2":7250214226981174458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153838, "value1":0.9864893077863718, "value2":1366599113373589549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144317, "value1":0.034179582463187676, "value2":4811674883286124820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_4":"g","key_7":"h","key_1":"k", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.670072, "value1":0.3811265659697653, "value2":4090252794464650298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486186, "value1":0.9084897972797951, "value2":2292875585752622652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724703, "value1":0.04582892017411178, "value2":5556549536478579022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012322, "value1":0.6305358760062275, "value2":1288975268472701530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_6":"a","key_8":"f","key_0":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.218849, "value1":0.8926861107470235, "value2":7051558642534889749, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.577242, "value1":0.96150058991849, "value2":5205402300881087071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525938, "value1":0.010643579624627687, "value2":9179800346032814438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393624, "value1":0.5282994953423363, "value2":6629726373755372175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697100, "value1":0.328819156896584, "value2":7890032700245067940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.911016, "value1":0.23451404605523937, "value2":1559532623894065416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128593, "value1":0.958219222491235, "value2":5238496228333042095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704394, "value1":0.5513948920447201, "value2":9062299692444161349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255075, "value1":0.35898456496590414, "value2":5087989108776558277, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788201, "value1":0.7466905262312481, "value2":479894845243154201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565236, "value1":0.15707319443827472, "value2":93010754190809493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462851, "value1":0.1393959893683005, "value2":1383438862980713878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.587928, "value1":0.9257613742750218, "value2":461452136435423636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.204910, "value1":0.45361150342145223, "value2":8114896084179399798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970393, "value1":0.8262379198959926, "value2":1410864297783479414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_1":"e","key_3":"g","key_0":"k", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571697, "value1":0.05523474900350812, "value2":6912470816433257487, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900965, "value1":0.3994455375010903, "value2":5910800813383852824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533983, "value1":0.8439578978089138, "value2":6520871065252687028, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732970, "value1":0.4096847894313233, "value2":4961560961953820016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.512990, "value1":0.9925070463343235, "value2":4031040144184448394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_9":"j","key_3":"b","key_5":"a", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.875107, "value1":0.8529856732344743, "value2":8171410486624791720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.662542, "value1":0.7969197657043129, "value2":4707105172837538302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.132562, "value1":0.5095481874741737, "value2":8469586257322312509, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171675, "value1":0.7140266335782444, "value2":8998618655550288053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_9":"h","key_1":"h","key_4":"a", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829529, "value1":0.8556707112048024, "value2":84360751498479327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.383509, "value1":0.6267421890569228, "value2":3548620263321255935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_5":"k","key_7":"h","key_1":"a", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725412, "value1":0.6792289410958199, "value2":8743030074795737682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217960, "value1":0.06523873388277363, "value2":4237433473740985929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.052828, "value1":0.39596522726411565, "value2":5268673916985408378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243583, "value1":0.24855400987123744, "value2":7145331513033105803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.643757, "value1":0.4621121441928861, "value2":4607779943623171423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777085, "value1":0.9713754435345697, "value2":3872286492266737542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741256, "value1":0.33772530129863026, "value2":2720260557973633476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040901, "value1":0.5366678442503715, "value2":5043767454211598671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.566517, "value1":0.3199753043754913, "value2":1787736737657465265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871703, "value1":0.40150570806322944, "value2":7426113550614212915, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.063390, "value1":0.8093881999536079, "value2":6062591517984503571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630695, "value1":0.5987756911109672, "value2":4564890959914284940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089316, "value1":0.9756718556017356, "value2":4422554680156925050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433679, "value1":0.6504367128725771, "value2":7179863028410040853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418340, "value1":0.5167928405286337, "value2":796797443554380087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148063, "value1":0.2982285613686749, "value2":3455828331199929793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.562273, "value1":0.1744043124258891, "value2":7056453619952557866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554680, "value1":0.9985428971035419, "value2":3134607000357482375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.498692, "value1":0.34351042666983833, "value2":9219919937586266788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983651, "value1":0.20667520837153053, "value2":8261113453188452260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900233, "value1":0.49430245377625404, "value2":1887699130768202184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.269041, "value1":0.24848195612656065, "value2":27297122896884729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.716521, "value1":0.703324321123364, "value2":5138993531610984864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.331287, "value1":0.6585119514499647, "value2":7805239120031598554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.898740, "value1":0.7897434254535518, "value2":8577413153173117857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366667, "value1":0.4615000106211479, "value2":4478388423093675456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060808, "value1":0.3611416577570406, "value2":1763611736817673526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987608, "value1":0.6511729153448047, "value2":7895118250604250511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.920802, "value1":0.8966340709602193, "value2":1619385988446426949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759468, "value1":0.9772199770736059, "value2":486666067155730811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697569, "value1":0.2215623962484913, "value2":998498599369301173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550984, "value1":0.23501917809435333, "value2":203018852406902455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865253, "value1":0.24390812385740993, "value2":2606627451901201346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050134, "value1":0.6480270130439445, "value2":6247279314207653751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614221, "value1":0.7050465104295126, "value2":3377246595104905199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166533, "value1":0.3877660847452995, "value2":7536153185490054501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644807, "value1":0.7190092729952982, "value2":5813482232408839023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739661, "value1":0.7257627180887302, "value2":3214351184373245408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552516, "value1":0.4939513402753451, "value2":465350226895561432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503358, "value1":0.14929982411048387, "value2":2610344885697195594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.765487, "value1":0.45537690586006585, "value2":4288868163982148480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.838240, "value1":0.7605183341936481, "value2":7774940282079152902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646978, "value1":0.988755613370451, "value2":4713140451733570872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746410, "value1":0.8174040579458727, "value2":7395122770602669759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823505, "value1":0.758453176804426, "value2":7518087779323040688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020233, "value1":0.29899993635819666, "value2":5085917800846866623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561926, "value1":0.8862149028157266, "value2":3614075635724672575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_2":"i","key_6":"f","key_0":"j", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.437110, "value1":0.32845862992503655, "value2":2977427326936422575, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706546, "value1":0.15047181179887134, "value2":2307845458700050829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.619350, "value1":0.1117344044076149, "value2":4988357694886705006, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.808461, "value1":0.715214325361221, "value2":7172177171113874947, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154987, "value1":0.3436009437038216, "value2":35294647035701645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.293185, "value1":0.1982820741163276, "value2":4721034174581841393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.177501, "value1":0.24102187665043326, "value2":5183952097737049563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434009, "value1":0.4074440055222358, "value2":2740366314909969021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.962158, "value1":0.6945642879060417, "value2":8488893912024069218, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_3":"b","key_7":"e","key_0":"b", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037976, "value1":0.05977827983958813, "value2":8714018046983615411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491879, "value1":0.5875339799333666, "value2":3507531774420107282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708852, "value1":0.941564068848663, "value2":7441568589430319196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665427, "value1":0.15650940720497003, "value2":2725018303601038315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728790, "value1":0.16909603056430217, "value2":7913664106090409634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748451, "value1":0.9309585511638759, "value2":6423846277129286464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701551, "value1":0.2886585284950581, "value2":1577558950874330311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356768, "value1":0.42594704925771354, "value2":3150589955894640969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056847, "value1":0.8050681337140542, "value2":3504760629437797890, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_9":"k","key_4":"b","key_7":"e", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.797591, "value1":0.9381497767913094, "value2":8069233160965701213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.653188, "value1":0.21115616320062328, "value2":2091006864800018643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489289, "value1":0.6249865113250967, "value2":3636072468848176197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.426149, "value1":0.7525465264386705, "value2":2554715790387615481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629922, "value1":0.244780100518832, "value2":2525567513505106874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.652167, "value1":0.44363110594266053, "value2":2074305704102402281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111575, "value1":0.7142448949298847, "value2":2599619943612027614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.720934, "value1":0.46314177190410183, "value2":5473312180675819943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_8":"d","key_9":"c","key_4":"d", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.542023, "value1":0.17074635857117051, "value2":576994586783200158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862606, "value1":0.849600128173387, "value2":4044011490168844428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.581818, "value1":0.4717710471390932, "value2":2021524622009984217, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.634138, "value1":0.3045108371861026, "value2":769588972141174181, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554282, "value1":0.9061849342045651, "value2":5246859609213477778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.275353, "value1":0.12209379665466558, "value2":3903264969659125855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_8":"f","key_9":"i","key_5":"d", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931382, "value1":0.38822648110972485, "value2":4321311553224557998, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.703867, "value1":0.6637597201672076, "value2":5683430536850722079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958497, "value1":0.011335239730124937, "value2":2491874770397540377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569325, "value1":0.8472934144463635, "value2":6850134665353105473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564173, "value1":0.28056996036761495, "value2":1661710999114510072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.046264, "value1":0.7988719698656381, "value2":80936298017494610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.086050, "value1":0.7269357269059649, "value2":3703081511929708827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.691310, "value1":0.968367674569065, "value2":8685309070712113009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.734580, "value1":0.16065685872806734, "value2":8341255006788851479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110471, "value1":0.2857648075220212, "value2":7895547866112316696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.594791, "value1":0.6011580461407761, "value2":8667655502460513802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.074613, "value1":0.7518175733891237, "value2":6808912328433269494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827510, "value1":0.4635989764886008, "value2":8535571864109924880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020243, "value1":0.9625575743602495, "value2":3255772979573003294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508001, "value1":0.9246513785407203, "value2":6789019308045406067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_3":"e","key_2":"b", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.175770, "value1":0.6231844141584031, "value2":5604032128383909386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338187, "value1":0.7085089535016765, "value2":7296894563489025180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924177, "value1":0.9072759724228442, "value2":2056681116966727911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474372, "value1":0.09374734312421011, "value2":8184327150731414451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970300, "value1":0.9441105421259884, "value2":6689514820107172641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_4":"f","key_9":"f","key_0":"e", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035348, "value1":0.14347791413195674, "value2":2423771496925044659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.807051, "value1":0.1533684521364213, "value2":8351770123785179919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814282, "value1":0.48165153274600025, "value2":2348575893675627866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438370, "value1":0.4065426353146516, "value2":5566206066472585427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.321909, "value1":0.057863583124824335, "value2":2888660799506273189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.191383, "value1":0.5857228209038977, "value2":343682983842502232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.030645, "value1":0.054007147421613756, "value2":7408821068820948342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.022320, "value1":0.5082171149433462, "value2":708896583672613900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359147, "value1":0.10928795884420436, "value2":5509040341747881439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245107, "value1":0.08990020039433881, "value2":4415685661085482000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.685276, "value1":0.9327230470765273, "value2":7932626821430255186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160518, "value1":0.9697823127450904, "value2":9176665002489756811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851145, "value1":0.3835892105704814, "value2":5122340871731266104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390684, "value1":0.8638095835380812, "value2":2196309567306303445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197750, "value1":0.5321904528249127, "value2":7059427737086849843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.606374, "value1":0.1712913717696049, "value2":6671507460576496016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814189, "value1":0.21162810005419255, "value2":6089699189069603789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565029, "value1":0.651039299686122, "value2":2488735588911638142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310043, "value1":0.3341800696393593, "value2":1886591498798226069, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646773, "value1":0.587523277238801, "value2":3984875805092838666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487672, "value1":0.4085632878496485, "value2":4006763171436697564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180539, "value1":0.614724172390471, "value2":3474361726777387017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413423, "value1":0.8015313942257714, "value2":3699130733479555941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375412, "value1":0.6194400337512462, "value2":89560215401334488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739736, "value1":0.01565923183582112, "value2":2040823465506133480, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164236, "value1":0.05609752284300993, "value2":459144572970603788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_5":"a","key_8":"g","key_3":"b", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857605, "value1":0.3918518578057805, "value2":2660076743726482223, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182324, "value1":0.19737269797519194, "value2":4090312209234045823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413638, "value1":0.23128603835680528, "value2":2432247666638343859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029664, "value1":0.7948810691175306, "value2":1090049786919167701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953471, "value1":0.7020043042253941, "value2":4064796540960008955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925306, "value1":0.8273248639036314, "value2":7383533353614933778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.353560, "value1":0.7154582581173291, "value2":7672433526090470392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.440133, "value1":0.8443212771497766, "value2":4871633339564000823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079309, "value1":0.9841787259191465, "value2":5158435517537975833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687146, "value1":0.547850782348478, "value2":107890618456621161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013887, "value1":0.9829126596506367, "value2":2398801344797864345, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142865, "value1":0.17685569327146947, "value2":2003576222991068603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503089, "value1":0.2089707954081848, "value2":5329263674364077685, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055628, "value1":0.9427569700789281, "value2":8376349363379609336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663522, "value1":0.5447970553906242, "value2":7034169322760923664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569193, "value1":0.4748210869235274, "value2":5009934188959179459, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.964851, "value1":0.1480433642451733, "value2":5780759776802899813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.674266, "value1":0.01926887357642196, "value2":5539462413096535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397018, "value1":0.3343175727423989, "value2":8363367362679630761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_8":"c","key_2":"e","key_7":"k", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665761, "value1":0.9909509158985629, "value2":598831903509406706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225685, "value1":0.9692539977538356, "value2":6753560478082272468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857888, "value1":0.3089073345023022, "value2":469238271712070571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296144, "value1":0.8953425439665634, "value2":5808544938153527415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641028, "value1":0.26770927578250153, "value2":4804718717106479393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865055, "value1":0.21277992187222125, "value2":7652322888419294762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.147216, "value1":0.4361812114988848, "value2":5453489060842625813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931652, "value1":0.9338772117034952, "value2":7926554101044226499, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451647, "value1":0.49187863740688065, "value2":626417169671954650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552836, "value1":0.08117641553234622, "value2":6479037192065714806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644667, "value1":0.8782871355247011, "value2":7618066153253340565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486533, "value1":0.34858818937194813, "value2":8450118240549444684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080371, "value1":0.1720187423008763, "value2":7683320479544120606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527916, "value1":0.09000309934383972, "value2":5784629514237788639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682327, "value1":0.9194439212395367, "value2":4269189023261883692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.961106, "value1":0.4405062734385544, "value2":1799260749356167755, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640959, "value1":0.04934157301568955, "value2":4101326969656042416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827544, "value1":0.6093692966542871, "value2":5408051744545184279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.916148, "value1":0.37486434605250213, "value2":8005097965237384093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477104, "value1":0.7823899457952176, "value2":2425125801220437344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631094, "value1":0.38182215652170554, "value2":6885679048992226816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242126, "value1":0.897478447775271, "value2":5819274494025981203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_4":"g","key_8":"b","key_2":"c", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134175, "value1":0.8364024700095963, "value2":2669419474556636937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.785456, "value1":0.47900969825766393, "value2":427990087950056586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.702759, "value1":0.4509768057551493, "value2":6899018348994248896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.844804, "value1":0.4506687041971474, "value2":2704465019491437568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841097, "value1":0.08259262088200636, "value2":3254311166592288697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.579894, "value1":0.634370403808335, "value2":8239103463468673989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.671756, "value1":0.3978266397618443, "value2":2011463990612690211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639712, "value1":0.5843126119887666, "value2":8514657269434627024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.090535, "value1":0.47166906237349404, "value2":2986627777164864344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079733, "value1":0.49607189078610653, "value2":7935174201590062011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484442, "value1":0.10087228739324149, "value2":5758223217302730640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.881297, "value1":0.743802063178793, "value2":7666746241262123009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189244, "value1":0.8471771982112483, "value2":8717600770763450338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958729, "value1":0.24046081337833752, "value2":2429080107883352722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904544, "value1":0.5967867401359257, "value2":5915623307556355066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.016356, "value1":0.2664341494092446, "value2":6082384923182522956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906737, "value1":0.8646356549293164, "value2":6093331013624350146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.650026, "value1":0.17820070872685148, "value2":2506978860590163255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.380897, "value1":0.559051486368018, "value2":959051280528283072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.645954, "value1":0.44113348983757733, "value2":3640682441395107897, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.705006, "value1":0.7603563234560595, "value2":185556188647790850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.946294, "value1":0.6649942159160177, "value2":8061815176820463265, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.007871, "value1":0.472486274082402, "value2":8110170220979486982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.817193, "value1":0.01062083501470273, "value2":8652411971395108438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.203058, "value1":0.2545105965864827, "value2":3692372485982923495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724358, "value1":0.49614729928160556, "value2":794673815448737267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029363, "value1":0.8128873282396352, "value2":5575910506626889012, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_3":"b","key_6":"b","key_1":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.388594, "value1":0.37922666609989497, "value2":5857319265944344227, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111761, "value1":0.07280095456744823, "value2":7190870932028618354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556559, "value1":0.011144718316818122, "value2":5637958779343170502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900852, "value1":0.2436048429102491, "value2":3178328441413468775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746269, "value1":0.44159443384096786, "value2":1064726188435771697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089137, "value1":0.5802300524767475, "value2":1179475447484216893, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045149, "value1":0.8412834417368987, "value2":2843566571645769718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032148, "value1":0.9106893921082526, "value2":2375027858901255873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180832, "value1":0.8856494080629095, "value2":2259505020949178568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571194, "value1":0.2087472777529694, "value2":3787711915274839208, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113569, "value1":0.6253289147788459, "value2":2290822587909135561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708878, "value1":0.3051231954056551, "value2":536614295748659218, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397834, "value1":0.004032765819560939, "value2":2143090210108317054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.279982, "value1":0.6698352738967139, "value2":8265876412027179825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.117595, "value1":0.36898116171165335, "value2":7917937119372643007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490128, "value1":0.7119026496334794, "value2":4217226709077888871, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.337373, "value1":0.24700927278867024, "value2":8633933010561287594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_4":"c","key_9":"j","key_1":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393403, "value1":0.3187858750570018, "value2":7155530342572901954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787897, "value1":0.731243787879042, "value2":3162978630167698275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394636, "value1":0.15744716640466827, "value2":6804804340000012886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589125, "value1":0.9206991440936233, "value2":3240974047558242833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122199, "value1":0.9027140609185327, "value2":4793574504420000662, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739368, "value1":0.0296490298815606, "value2":7450276243203379932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839046, "value1":0.885358696687765, "value2":2580404634255699486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432366, "value1":0.6379661863831838, "value2":4996786498907777190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129751, "value1":0.5622916100360914, "value2":8341777138936467511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202469, "value1":0.24574613351572672, "value2":7146206121672605367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.832142, "value1":0.5178470928944944, "value2":4133801221430187756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080311, "value1":0.486175008187209, "value2":5039371713095786937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723875, "value1":0.552001350718302, "value2":7101322585254858336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434143, "value1":0.6525700542145968, "value2":2081065923746015157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.410329, "value1":0.15418313782350743, "value2":2121964379701341165, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423463, "value1":0.984744888870281, "value2":7148485292923472325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658931, "value1":0.9086990564903296, "value2":4792038160363904869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.891060, "value1":0.6990632896148243, "value2":8537384774472959390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154122, "value1":0.10983855840529697, "value2":3710113544658117053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.543143, "value1":0.3045282199429521, "value2":6184622652371484263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174103, "value1":0.1622180223685842, "value2":1046120514820502820, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393987, "value1":0.12110279784132964, "value2":5830532384578071588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907289, "value1":0.9509521630571898, "value2":4933206466498891083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987070, "value1":0.5242278192300958, "value2":7374919878503376521, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641701, "value1":0.3760479049716257, "value2":4296425671360614419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.460082, "value1":0.7715932085121491, "value2":4216486260610422297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.308698, "value1":0.6733032497636853, "value2":8314310095556752952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356337, "value1":0.9180371571261539, "value2":7442497884658250592, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407046, "value1":0.8826200541831951, "value2":6272206334208867752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263547, "value1":0.5493018603625203, "value2":5332141292811821371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.690903, "value1":0.6510082736025081, "value2":2282336418703477702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599129, "value1":0.21529129751322154, "value2":5758745763053757888, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.294727, "value1":0.17537508292575937, "value2":1562987372702839556, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905165, "value1":0.5372817599107224, "value2":5788710138851546437, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714618, "value1":0.5147887871360798, "value2":2734992135513585397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.845879, "value1":0.03659856815217442, "value2":717088561981717268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.172178, "value1":0.5465356547159791, "value2":6019732615424773946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263201, "value1":0.7472793901983326, "value2":6735352007287694901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_4":"i","key_8":"a","key_2":"d", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418982, "value1":0.029853021112705932, "value2":7962917578850181628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.866946, "value1":0.10874512532228968, "value2":4523576734362004451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556178, "value1":0.3725090999387309, "value2":8591394434164967395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827178, "value1":0.564605338181153, "value2":1565791376780985293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_9":"d","key_5":"b","key_8":"j", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.427240, "value1":0.5937818761029834, "value2":725738161862814755, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153851, "value1":0.6812022447305227, "value2":2627803719401441193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982516, "value1":0.5197201488713766, "value2":5245967664424169262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.922654, "value1":0.8797142069534722, "value2":2538441830233727435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036732, "value1":0.8882312994752156, "value2":2860925793915692434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.848915, "value1":0.5650391916990911, "value2":8021099293155564160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189564, "value1":0.5152547802767392, "value2":1569067174314315770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.802112, "value1":0.9536154419324024, "value2":6565733445920939612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983922, "value1":0.4632188030903474, "value2":3429972812413807249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477177, "value1":0.333324845660299, "value2":4203450320031165809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097938, "value1":0.9171714255895188, "value2":2925569760522108797, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.551984, "value1":0.23531625096597034, "value2":8336676757418711666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.774461, "value1":0.26483886652245114, "value2":1317358716847210463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354793, "value1":0.850445247905183, "value2":3123908181356206943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.409594, "value1":0.7075083567078372, "value2":7315484295756660732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407267, "value1":0.6525751933510557, "value2":5480727731388217494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.572642, "value1":0.5899578957097297, "value2":5245828360681206097, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.731119, "value1":0.6986058457616005, "value2":4059549341372221062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.627725, "value1":0.5600488168022775, "value2":2564410170230629806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050520, "value1":0.8920464631997315, "value2":9043850999622300098, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.500683, "value1":0.004883654125166552, "value2":8338418914599161440, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970722, "value1":0.01152412783847713, "value2":4977793912981679064, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.860222, "value1":0.796875300130385, "value2":126152372728632215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629467, "value1":0.658754492873807, "value2":2189248151613602825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.098126, "value1":0.3838755732008427, "value2":8710340059607681652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812187, "value1":0.7527055571509035, "value2":4522503161830090202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303237, "value1":0.4170647409134182, "value2":1359570910802075414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794250, "value1":0.581379594095589, "value2":6842995986442641402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247703, "value1":0.7617287060418184, "value2":8955434883327208405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.815768, "value1":0.5618395871174356, "value2":6646583475463132751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_2":"i","key_5":"b","key_0":"d", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.248823, "value1":0.1954800995898102, "value2":1363474452253194433, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.531845, "value1":0.018792519031917622, "value2":2287281788624224406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966028, "value1":0.08281088225013297, "value2":1000202598795297845, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.935610, "value1":0.3880941622449169, "value2":3231927844022227730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.268000, "value1":0.37669283923759284, "value2":1820767283408305591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.229141, "value1":0.6757864654636687, "value2":850049194951914043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.889660, "value1":0.14317979698283573, "value2":4562037574783866851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836795, "value1":0.7422552273700601, "value2":1164639099163690154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.311980, "value1":0.6163196714525997, "value2":5427592556050607568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584578, "value1":0.38212773421861185, "value2":1414397003551807223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065172, "value1":0.5125177091208547, "value2":2833074963007555228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564271, "value1":0.5135791893267198, "value2":3074560175179976918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894567, "value1":0.8429539964906779, "value2":3097821505618288757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.733587, "value1":0.9459948105676399, "value2":3812465080486684733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.053881, "value1":0.6018467371683773, "value2":5589588667277003729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599521, "value1":0.5951991237753406, "value2":4485462521384926953, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354099, "value1":0.15573795731250434, "value2":8080731261812523696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969724, "value1":0.21276604577267808, "value2":8483165896229109105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263088, "value1":0.8705529546951637, "value2":1868660470516930885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.212304, "value1":0.28383507730046376, "value2":3678145555670791199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965823, "value1":0.23336598309503237, "value2":1387918772122086546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536800, "value1":0.6576906678976834, "value2":6988018932226580717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.884180, "value1":0.9109732193854059, "value2":2950864149550098491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448663, "value1":0.37035460819883087, "value2":6933572641209900634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965580, "value1":0.9403308475710539, "value2":5776171572361995550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.349421, "value1":0.19847751056585392, "value2":3418305989652553355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.469447, "value1":0.13572479625794354, "value2":4687020001075041801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393705, "value1":0.5764820211787193, "value2":296609812894901072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862549, "value1":0.3023964248018198, "value2":7566431551834336763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415447, "value1":0.07086082925570908, "value2":6961625726898421777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495354, "value1":0.5778296524778606, "value2":2860533724845138756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.510139, "value1":0.9905369504866598, "value2":7400245650080116394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129326, "value1":0.3322893702950888, "value2":273820673011485924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.793922, "value1":0.5795215776373468, "value2":7565118610448753571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114305, "value1":0.4965968551084071, "value2":3076343444745895229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539515, "value1":0.7424425979048557, "value2":3570552565123324839, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_8":"b","key_9":"b","key_0":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714983, "value1":0.4711244304607692, "value2":5134628196697329789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182520, "value1":0.11684538125954054, "value2":4642866567682796714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701117, "value1":0.46816724142010296, "value2":8359014347252866849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135967, "value1":0.6405455196141, "value2":6186468831622665956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.424454, "value1":0.5144728145894553, "value2":1160618526301999790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476344, "value1":0.8109846569048221, "value2":1049565304185759462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571700, "value1":0.5504531727903846, "value2":133387170146495310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324604, "value1":0.16411596790106633, "value2":4350527762007764055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_9":"g","key_1":"e","key_5":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272811, "value1":0.06793234743561845, "value2":3491914082935836765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496778, "value1":0.2139795243463974, "value2":9108500124467919738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855922, "value1":0.8034347088304469, "value2":1547578413909373466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.516953, "value1":0.46095603855138756, "value2":5334901216429463877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.941610, "value1":0.9546476007247299, "value2":1561805534534904078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157907, "value1":0.9790803369585773, "value2":7490355100154440772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.667071, "value1":0.23582538943103912, "value2":6462516448359960159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_3":"d","key_9":"f","key_2":"b", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.447846, "value1":0.08703603490291748, "value2":6137779903742056640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.201754, "value1":0.703586761350304, "value2":5895955356127986300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.214135, "value1":0.28977641931297293, "value2":2670270804816119209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171493, "value1":0.09189861336902101, "value2":3835094797774579940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.507098, "value1":0.8469316808229229, "value2":4394084036001197157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_2":"j","key_4":"d","key_1":"f", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.567706, "value1":0.4315878020065951, "value2":3026063068553888025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_9":"f","key_3":"i","key_5":"c", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.369862, "value1":0.25873173058422444, "value2":2694491932093728275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393535, "value1":0.2941316364368656, "value2":3384752395363156914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354927, "value1":0.13513861393265603, "value2":8481431502383347556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.468119, "value1":0.5679212228606271, "value2":2713405380262941377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163006, "value1":0.4227063637930681, "value2":6321042013913967311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.656131, "value1":0.5736248371417144, "value2":2472629476600653129, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511635, "value1":0.7728428788355186, "value2":7276523252477639153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080037, "value1":0.9949587251210614, "value2":1148201472548354758, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725064, "value1":0.05030431554866517, "value2":4579601693593819193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.342713, "value1":0.6797941637549237, "value2":2282393859292368923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012227, "value1":0.25217885012179175, "value2":2820047039844714200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.121409, "value1":0.08353369182495488, "value2":6648793005050735517, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.944843, "value1":0.7935143781883287, "value2":1141746305838051122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_7":"b","key_9":"b","key_1":"k", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216784, "value1":0.687358900553268, "value2":5413083464946109711, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.744704, "value1":0.26762670212162987, "value2":4557617436876999047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.105367, "value1":0.7098778066529884, "value2":2505389065681570054, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_3":"k","key_2":"f", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164496, "value1":0.375746421410976, "value2":874151975168015904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648692, "value1":0.26898807878557085, "value2":8174372151408578377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160502, "value1":0.8995428170685682, "value2":3027568952484727683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149122, "value1":0.0907853096527507, "value2":4677811628653537246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296780, "value1":0.3995862207774202, "value2":3598801053628990992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.464060, "value1":0.9713793366221031, "value2":4865123999270835737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288334, "value1":0.6512177032146146, "value2":1792245924965210603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.869907, "value1":0.007513793036623767, "value2":7995353402339020473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_8":"i","key_1":"i","key_5":"e", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527381, "value1":0.5920004855495694, "value2":1386169961861528891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.928125, "value1":0.7418903709677874, "value2":5060751330275548305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131800, "value1":0.9351703924333601, "value2":827562310794270736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_8":"g","key_0":"b","key_2":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367161, "value1":0.9314616031794654, "value2":8545717112597189310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704807, "value1":0.7062626357522712, "value2":1882734585108600742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270485, "value1":0.942646876486616, "value2":4665280302852454779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.521647, "value1":0.7127769947534454, "value2":7002036018179718342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.011614, "value1":0.2150115012001749, "value2":2368106908969391947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160161, "value1":0.5544338078106065, "value2":6068552071959300958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545350, "value1":0.5865458699587862, "value2":1122097202466314910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.948757, "value1":0.9353310253197936, "value2":9083948862293457994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.003970, "value1":0.13057333820161732, "value2":4887307186667923394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811016, "value1":0.0597992472783255, "value2":7198325554706277943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687715, "value1":0.029450254550187717, "value2":7347375517848816816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892901, "value1":0.5081883055870532, "value2":3289845879859033841, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006320, "value1":0.35956731353734483, "value2":6174658822769262086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.088057, "value1":0.5893474112602632, "value2":2264301146979603153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366406, "value1":0.09531376481437555, "value2":8460504532640850958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.742879, "value1":0.21104814940031102, "value2":8239563761654323619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014364, "value1":0.27433866046434513, "value2":1123249940897490575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442350, "value1":0.5779165272183289, "value2":7076967122005445282, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_6":"e","key_2":"c","key_3":"k", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.363242, "value1":0.7223710776768617, "value2":4947532477450255339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794231, "value1":0.9595692700611697, "value2":7086029131344930712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.696096, "value1":0.2902282025796308, "value2":2826023125040762860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366795, "value1":0.8338186933189231, "value2":7469909471986494672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.642061, "value1":0.025310231945638022, "value2":5571346287819468860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673449, "value1":0.9385869454324781, "value2":1722013371242991173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375120, "value1":0.09609963357456859, "value2":5591579449982830960, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391308, "value1":0.5308398826498623, "value2":2744980541531859716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658264, "value1":0.08305585159828374, "value2":3868051792086152227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.540598, "value1":0.9729141879466757, "value2":2886891615332464723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.077420, "value1":0.756799108978319, "value2":3709648144634270824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.762914, "value1":0.7697479099606849, "value2":3475167187400734782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_5":"f","key_6":"g","key_0":"c", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907718, "value1":0.44276263620911915, "value2":8007229889762742191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.677735, "value1":0.7114610811603074, "value2":420499484804274745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097065, "value1":0.9614980550005483, "value2":401687894704730769, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107273, "value1":0.7588043489806351, "value2":5258772379410637950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907641, "value1":0.16046492238486323, "value2":5755077471601284208, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_7":"f","key_9":"b","key_2":"j", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407605, "value1":0.5733175508809538, "value2":2858689191661453591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412001, "value1":0.6163004444656547, "value2":3938161718625625501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174438, "value1":0.7741718765670191, "value2":8151281186076102637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433895, "value1":0.6190720762613371, "value2":4753613664907841729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.754894, "value1":0.6861936508507025, "value2":4982660479879807579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135440, "value1":0.8484346139801648, "value2":4916410761830427120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901526, "value1":0.7029579471688302, "value2":7091187814115860335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969298, "value1":0.9971487141151381, "value2":4678178370126159004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.302462, "value1":0.5451450337065363, "value2":4647862155949132966, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174504, "value1":0.8364678736703143, "value2":4662336943284880044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724429, "value1":0.7605319213856574, "value2":5775369825299529065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196992, "value1":0.485221307178674, "value2":8161727305469743810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673298, "value1":0.12884478288346873, "value2":3433606731713363675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.008175, "value1":0.6854875474834051, "value2":7177608068823530117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_8":"b","key_1":"j","key_5":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.152906, "value1":0.937989165728203, "value2":2388297772896467042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354518, "value1":0.5251867146486633, "value2":2890885067302311396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.872640, "value1":0.7546063425064781, "value2":8461706694506205274, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_8":"f","key_3":"h","key_4":"j", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403113, "value1":0.7711761108505557, "value2":8776285961045082566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.713712, "value1":0.3686652392717231, "value2":3561820735837135599, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.115073, "value1":0.21496854825964173, "value2":1666306538045744009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.127362, "value1":0.0348593634697097, "value2":3277092601539134615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.042887, "value1":0.9391085748107487, "value2":3037900421018347064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957170, "value1":0.2742109117364031, "value2":7538881467268645993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.284083, "value1":0.6552095129564431, "value2":8324596282478983070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139776, "value1":0.32060963733799414, "value2":4387891090637798442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.446847, "value1":0.7514268853453937, "value2":1042496829752491158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.429327, "value1":0.7027895753887083, "value2":4302894157835600757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496272, "value1":0.2459555820489922, "value2":4743527397302686896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599745, "value1":0.3085143386003063, "value2":8510884550594616394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233545, "value1":0.4081732430516885, "value2":4780887317944713264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.396869, "value1":0.3834820243913789, "value2":2867737400343386587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.000965, "value1":0.7645272489414043, "value2":8292218247137907442, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390310, "value1":0.9922190988066435, "value2":6520801004207332848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.408746, "value1":0.11967416242419873, "value2":1289404280671317844, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.513154, "value1":0.43853228030860203, "value2":3816263914269322442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.425564, "value1":0.9721541652281699, "value2":1199225910826614784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.896976, "value1":0.5841175850919846, "value2":6380661791453585128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.385017, "value1":0.3642021068005677, "value2":7460310647019127617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.861063, "value1":0.18830600982566692, "value2":6167598179189147828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.942353, "value1":0.4661402168565476, "value2":8018642185562026949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.750347, "value1":0.3217686952651615, "value2":830941682724984711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.926420, "value1":0.0451768298368594, "value2":3858422103614938711, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451255, "value1":0.012250475428480134, "value2":468924453611892408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708013, "value1":0.5360411826543746, "value2":7399971881816481719, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.661279, "value1":0.5535815660872848, "value2":4401269288089200510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.710860, "value1":0.6074402471823324, "value2":7073880640711856002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245087, "value1":0.7738521153135302, "value2":5302937747460405991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724822, "value1":0.06719994230964553, "value2":1984993846520154478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841255, "value1":0.7199212526961583, "value2":3580548125031735289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130289, "value1":0.14169983508615547, "value2":7980800511434824689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_6":"i","key_8":"d","key_5":"k", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.215185, "value1":0.05456621441388223, "value2":3313875019669857059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730585, "value1":0.1938148933748891, "value2":2133259010192691032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603048, "value1":0.27374938601691284, "value2":6401354620294279338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148949, "value1":0.5947828484485513, "value2":8147515290794947255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.610152, "value1":0.4085609201069522, "value2":9002523177156729302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.607808, "value1":0.9266705539974761, "value2":4953852228607795114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197811, "value1":0.7637421466978226, "value2":7727001922124797631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672455, "value1":0.2140320622344972, "value2":8504587633123200194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781600, "value1":0.18861407062595717, "value2":2041192855707568703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_4":"i","key_5":"e","key_0":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648320, "value1":0.9980339865005591, "value2":6740770170971470746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481193, "value1":0.5203406695297639, "value2":6425306083284764527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954444, "value1":0.5949050635321682, "value2":2690790583476351571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971678, "value1":0.3007304152123297, "value2":4562717443975915242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026021, "value1":0.27575882598297025, "value2":978861683515042668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.840073, "value1":0.15154871188435406, "value2":7561212279807822666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393063, "value1":0.5006737130316841, "value2":835200053720218776, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589637, "value1":0.29002140408035565, "value2":5208166554872726624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525315, "value1":0.7084919237025886, "value2":4888712288784480920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026489, "value1":0.9459638977891762, "value2":6799511668473867834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356460, "value1":0.8482487483215856, "value2":4902836930964609800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871117, "value1":0.35037006883754834, "value2":6550768382747712274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.062992, "value1":0.3329675649141359, "value2":8563631384946292163, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.751101, "value1":0.9811341531679872, "value2":621345745999214417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.535295, "value1":0.922840944798324, "value2":586063777073635225, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.258523, "value1":0.7865906481221376, "value2":1141638202189461929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.10833711533264617, "value2":5739011556983431812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407251, "value1":0.4266309513517482, "value2":4708724256538796193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_6":"h","key_2":"e","key_4":"d", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931023, "value1":0.6949078276162761, "value2":6914579995867144592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.882968, "value1":0.27938573922269233, "value2":2134628658651324357, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599828, "value1":0.31593636777024753, "value2":2026576607314660982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166118, "value1":0.30221394298164866, "value2":7042763501275182896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114195, "value1":0.16993347985800208, "value2":356239511173356550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.676940, "value1":0.8142744662164688, "value2":1615963097558644501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525949, "value1":0.9287689609270591, "value2":8144174091016595622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852923, "value1":0.6682421979821775, "value2":1660916359233146706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714742, "value1":0.6940562364820247, "value2":8279633113622088778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.069728, "value1":0.023054188899827854, "value2":7432273276574908949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397060, "value1":0.6029753734289958, "value2":5859678381429923175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788868, "value1":0.4080940744104717, "value2":2775616148615876742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.497778, "value1":0.0933297131976096, "value2":2710674316640074263, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.251386, "value1":0.5835060543515784, "value2":1694819851220806994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144280, "value1":0.7762012742074322, "value2":6551654579563921110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778177, "value1":0.8529872059895475, "value2":4843965544345174856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_7":"c","key_6":"k", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495383, "value1":0.8821308188431486, "value2":7181474101563747015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421122, "value1":0.04021723515347381, "value2":8758814764030728391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055252, "value1":0.5894200751471586, "value2":3745371455267104958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343067, "value1":0.2157486959489704, "value2":3077439566978720910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489306, "value1":0.4542119549735926, "value2":5846254702440735178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233627, "value1":0.2946889464915457, "value2":2465181057584653557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.523076, "value1":0.579630183513931, "value2":4633358909899504280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.317577, "value1":0.49633251300853154, "value2":7243885459219565500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142016, "value1":0.41458124596836643, "value2":87263970443989795, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014459, "value1":0.3889314073670658, "value2":6642542110052864260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.249899, "value1":0.5981564529561014, "value2":4725189280029367657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.087078, "value1":0.48311123248467286, "value2":7915582091365577059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.681336, "value1":0.3569069050336655, "value2":7513851204641167627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266570, "value1":0.6724875939535222, "value2":4615308903415712327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474729, "value1":0.3876423090885996, "value2":319900338975917805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.328471, "value1":0.061053039369872907, "value2":9085903408798527191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010016, "value1":0.7472091713600674, "value2":8976037153524824381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412206, "value1":0.7796477061197584, "value2":6842085213034340664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288276, "value1":0.26992787569132787, "value2":1806846857113271486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.501201, "value1":0.781398815935773, "value2":4251812051340782659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270221, "value1":0.051986823543989535, "value2":4993776637838984787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.986643, "value1":0.7021309080168643, "value2":703842817955022555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.453898, "value1":0.4599169268098947, "value2":7963554429950594113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202118, "value1":0.94451324780601, "value2":7780362140982384262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.801389, "value1":0.7867483731478175, "value2":7192213512007631986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.228749, "value1":0.4182819773434862, "value2":3027632485243175966, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.184080, "value1":0.9567901938339193, "value2":5313717148514876661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.707330, "value1":0.826196713820751, "value2":8078395604816162581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130983, "value1":0.31932740597678144, "value2":8325619009408095997, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_6":"j","key_7":"j","key_5":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477577, "value1":0.8037112800341328, "value2":4641052188020943712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.835784, "value1":0.981119547136869, "value2":5830864768532244942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.951325, "value1":0.9017984159519533, "value2":7172321278356171509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949416, "value1":0.9735936936041008, "value2":2397210877296618753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.227491, "value1":0.056278467666299646, "value2":3619861078542842990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614325, "value1":0.9338179733958348, "value2":1856588427373456649, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.167780, "value1":0.6264999203061654, "value2":1887390160426943194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421682, "value1":0.3034767704367279, "value2":1117719742198046331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131869, "value1":0.7226359930389739, "value2":6009067442340606148, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.576451, "value1":0.4077319058968275, "value2":5262129379842658951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487400, "value1":0.49878540971656044, "value2":378267425259112843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417401, "value1":0.7695532802557139, "value2":6353721086899765325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.597343, "value1":0.9334570369057625, "value2":8106460523980946423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947755, "value1":0.599677921450711, "value2":1518293404194401856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533432, "value1":0.4134611723516883, "value2":4789921078575497056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271038, "value1":0.7693055378733823, "value2":2557684070989427251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.209928, "value1":0.5501502486470365, "value2":6523836811607831530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.514479, "value1":0.742552198141705, "value2":5448311945921729044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097323, "value1":0.342716151747015, "value2":5333351730397550620, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036400, "value1":0.9364465322438232, "value2":3515155010826544091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.161709, "value1":0.4801914817306215, "value2":4167056653387877843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128367, "value1":0.42861309279829746, "value2":6367124429829008192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.104233, "value1":0.5604661779053219, "value2":8908744031459681115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814224, "value1":0.8505352643130756, "value2":3611537515395296790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.257817, "value1":0.9534356262275627, "value2":6358735114691060805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_3":"i","key_5":"j","key_2":"e", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590971, "value1":0.8532839122834444, "value2":4683754442672481085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_9":"f","key_7":"b", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110492, "value1":0.563891691575366, "value2":4631985245890497320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953652, "value1":0.20700443841383587, "value2":1925175802270628257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012273, "value1":0.5960529408710756, "value2":6611884946690949474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107781, "value1":0.5528403788424425, "value2":2332669698469083563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289974, "value1":0.4420135407607212, "value2":3690892585669795676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.193764, "value1":0.5159643736419933, "value2":6218735166584063092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.386095, "value1":0.777297131904327, "value2":636830504959980122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977928, "value1":0.01676315692753733, "value2":4060533084263704194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.066993, "value1":0.001865435996499948, "value2":4739969679441502931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.910616, "value1":0.057836079222670464, "value2":5276870038639917478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971825, "value1":0.18153964718314727, "value2":2807508624852997267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.782626, "value1":0.41404427259634824, "value2":6643422113997639050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892943, "value1":0.5830967595104455, "value2":499806168567753593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_9":"b","key_7":"b","key_8":"e", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423402, "value1":0.9103545774481498, "value2":911868702803131336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006069, "value1":0.7748758341609289, "value2":6539224747903805609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143311, "value1":0.3741630815676959, "value2":375500557355639144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267012, "value1":0.06234671061027635, "value2":5595306219658860658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548719, "value1":0.7992113362076428, "value2":7214856807865350002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.919970, "value1":0.973745629473067, "value2":4169123266267227326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.071680, "value1":0.08778281877161515, "value2":7668684104220358875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.222513, "value1":0.4425030874471083, "value2":5614432156299415760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.031556, "value1":0.6971180191067721, "value2":6128893201277772027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839957, "value1":0.03151274581462018, "value2":8360563655204493448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924623, "value1":0.5790628849747803, "value2":6190166511034220622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.647381, "value1":0.18478113887851288, "value2":2971279323948578904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.818067, "value1":0.36190113937740354, "value2":1714448127957842853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.715974, "value1":0.23843892303460118, "value2":2439937704889068215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192672, "value1":0.6080907905788115, "value2":976012584384444918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157561, "value1":0.6894676876528992, "value2":7135259052981755980, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134773, "value1":0.09678670230466922, "value2":8180183833096777286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.616057, "value1":0.8520049469696344, "value2":2950033021270268791, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255317, "value1":0.7477685612049465, "value2":179441442795111864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906502, "value1":0.8256982980631076, "value2":4367074024600682775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_4":"d","key_6":"e","key_3":"j", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.116793, "value1":0.8891425230837835, "value2":1719946264236570624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324866, "value1":0.5386590615308351, "value2":8011171897264590756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.352403, "value1":0.7153316706866707, "value2":4379546221964794474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836874, "value1":0.14443484962198383, "value2":1482104610634689373, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_9":"b","key_5":"c","key_8":"a", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403735, "value1":0.8924124150178864, "value2":9156970764441343023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141142, "value1":0.434266569307909, "value2":4978801873133265222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.234292, "value1":0.7164352385365066, "value2":8038096300931135389, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973287, "value1":0.6600392290118359, "value2":2638933878817864633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728882, "value1":0.476991229582368, "value2":6510855349407866452, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787371, "value1":0.7330600795491425, "value2":6570294519251889382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903661, "value1":0.5944117402324142, "value2":6524268081747181442, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.659983, "value1":0.26592399871964695, "value2":1212972365053449050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723204, "value1":0.31589384939280846, "value2":2041435063818693781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701487, "value1":0.5974401671491981, "value2":625551200860992484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.893866, "value1":0.38643143057076507, "value2":6815713871810191176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561678, "value1":0.4057036812724311, "value2":4919580315129580518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004391, "value1":0.4801368737089002, "value2":7547608457928538573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.923757, "value1":0.7739736243035474, "value2":6832671633674661606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819284, "value1":0.7572793995611461, "value2":68290908865890434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.524308, "value1":0.44777317605800043, "value2":4272485517844220240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804762, "value1":0.20222965946470728, "value2":2164767398150388054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060732, "value1":0.060291087703427844, "value2":8373529279962935628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325849, "value1":0.2785168531780935, "value2":8895250356037639155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.378439, "value1":0.43808690997464833, "value2":7750734163790876517, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.341809, "value1":0.21201598129686366, "value2":5439024219774015639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.027383, "value1":0.519384455440142, "value2":2830105304694072087, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578370, "value1":0.4103102602537048, "value2":3491530677431875174, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.588914, "value1":0.6688147984910581, "value2":7129525479125432993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781407, "value1":0.7254015801455519, "value2":2135519716242173777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852755, "value1":0.6592390034228756, "value2":5378675889994915559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.996821, "value1":0.5491613669334785, "value2":3858296464786926451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.364027, "value1":0.6056949341817615, "value2":2956076236179631886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.605233, "value1":0.2433469771258824, "value2":8502972839119365612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.092828, "value1":0.29453399849858286, "value2":5192582279143186491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.179799, "value1":0.19121582692459024, "value2":9103159579935481171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.054748, "value1":0.7255577017707996, "value2":8476394128272498701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.959235, "value1":0.5345272240165994, "value2":8621793756396494146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.735109, "value1":0.7362204691534435, "value2":942332422397834734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778088, "value1":0.8485308761635748, "value2":8355286652172671018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026529, "value1":0.058403258262061046, "value2":3881865567315099972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.405396, "value1":0.78515016074467, "value2":3124806945169635921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325780, "value1":0.8484383369933163, "value2":5639577297430380059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.125526, "value1":0.9437452957692816, "value2":1600583134494091930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040638, "value1":0.8412837985326597, "value2":1891897645252751874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225539, "value1":0.7607755326390468, "value2":4494740517836964298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417989, "value1":0.8351179906534547, "value2":6640751019261660918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.930391, "value1":0.7797566384384467, "value2":5344540873017186839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130784, "value1":0.9081520679550602, "value2":2673655563019876645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065288, "value1":0.37878147094860193, "value2":40939122026803562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.023954, "value1":0.7421706325870447, "value2":3936838989809781841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639378, "value1":0.04809332390342072, "value2":138564553177528595, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.968036, "value1":0.4229941922357839, "value2":3252691734036720501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_6":"k","key_9":"j","key_4":"e", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925601, "value1":0.5862382194484029, "value2":8809171247244862983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.615163, "value1":0.41538444706998723, "value2":7806072407872661917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432657, "value1":0.2054174608330251, "value2":523721991362366859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584157, "value1":0.38115353295991117, "value2":5695252506949414616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969537, "value1":0.31115941644555445, "value2":3813252754578465682, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.755713, "value1":0.6078558755538082, "value2":4745658402471707078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.299770, "value1":0.39494561763395397, "value2":7454472376009363943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549446, "value1":0.9276674467108077, "value2":7450051253648573468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_4":"c","key_6":"c","key_3":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.339276, "value1":0.16417640235809516, "value2":376449946500069421, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_7":"j","key_4":"j","key_6":"c", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.044679, "value1":0.022401336029968563, "value2":7584563374980649526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978082, "value1":0.634805138120068, "value2":2509593268797839298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020832, "value1":0.2523976489068679, "value2":5330781529709694724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.874340, "value1":0.5447943369952332, "value2":753638192956179932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.532731, "value1":0.3415364500896806, "value2":4617908501249248943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.888667, "value1":0.03025871907918381, "value2":389467035709739331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630653, "value1":0.3815158272162314, "value2":1365697000438851836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947344, "value1":0.018503516339994433, "value2":6964047457370572921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.635335, "value1":0.7735503313212675, "value2":7863214250826862536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.323808, "value1":0.9277969228346735, "value2":7104799245414939756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527571, "value1":0.19552679967510578, "value2":2150832405165980506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.076891, "value1":0.45024886897327476, "value2":7461894110864047862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433797, "value1":0.03730319472343055, "value2":8041320080812623904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745634, "value1":0.6889162665973937, "value2":7455824590286680395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310508, "value1":0.4992657909321023, "value2":5571385578668134720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343053, "value1":0.21251046051646363, "value2":6470523853548900348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903927, "value1":0.9720310949900105, "value2":7380705280316406577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.200927, "value1":0.42664784751514406, "value2":5730113255528003277, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.563845, "value1":0.8055339411145567, "value2":7403476714487686904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.329254, "value1":0.8021973639133464, "value2":3080745955617166362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.879352, "value1":0.4277546544111605, "value2":1478811657161573091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_3":"a","key_9":"h","key_0":"i", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442846, "value1":0.006346529763383217, "value2":5543697338259725446, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.094781, "value1":0.40728088001797547, "value2":6613387129832293245, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.975150, "value1":0.3661351435962595, "value2":830819320558121686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245141, "value1":0.02569157141735858, "value2":2603499185616374516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949515, "value1":0.3643114110362523, "value2":832829152625075749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_6":"k","key_2":"f","key_4":"i", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741852, "value1":0.28794172682449687, "value2":453442634675086185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.738181, "value1":0.6087693219046798, "value2":7047508567290984350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.492510, "value1":0.5382269802655012, "value2":2689541564459522343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_3":"d","key_7":"i","key_2":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988702, "value1":0.0728816330863758, "value2":5556532985784164765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.655057, "value1":0.7734197867503856, "value2":5244006855203800595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520001, "value1":0.4376795320695461, "value2":1470774311764721388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164473, "value1":0.6572062696131487, "value2":1949710197196336093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.190159, "value1":0.9525604598487357, "value2":5478966602860717950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829372, "value1":0.6923157559412111, "value2":5023489111088251295, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391391, "value1":0.6340217327197775, "value2":8000248111604879168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883302, "value1":0.8070620002939071, "value2":653446263691796732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745426, "value1":0.45672801954182946, "value2":6893619651580083925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982512, "value1":0.5060051247468136, "value2":7419982333827237040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827729, "value1":0.15692262245568137, "value2":3804375123078218665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244618, "value1":0.6655062510702947, "value2":1607640639377335930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.945643, "value1":0.3793044572114196, "value2":8429298239346478761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626649, "value1":0.22068614209697351, "value2":8899816776188911022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045981, "value1":0.34313373362766186, "value2":1556434949234629007, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.381111, "value1":0.4930245395823471, "value2":3486502897378084847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596053, "value1":0.23068160326995202, "value2":1094706285002463668, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454071, "value1":0.4016206555788349, "value2":7766949472696777624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122082, "value1":0.12814082114630934, "value2":6860595913103836152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267563, "value1":0.915043661363235, "value2":7117774485449523197, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883324, "value1":0.48609131055046695, "value2":168158041910482971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.798512, "value1":0.9160093333280128, "value2":5957101416277255647, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_6":"c","key_9":"a","key_0":"k", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.956973, "value1":0.08410692933360829, "value2":3670111299929079308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799611, "value1":0.006983554187354543, "value2":560914434528784616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539372, "value1":0.1527069613506525, "value2":5396674816694339651, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.538750, "value1":0.21348159925068225, "value2":4409731296615206554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.854320, "value1":0.921941847389472, "value2":4199458567691684987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451148, "value1":0.5527973855585238, "value2":5090625521103681352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.585722, "value1":0.08217397453744285, "value2":5891174882356923897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_8":"k","key_9":"g","key_2":"c", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280675, "value1":0.17159417706244065, "value2":1155748852941674950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673546, "value1":0.9064641866629127, "value2":5826431744512876414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748734, "value1":0.7184282675490825, "value2":465504921239629350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448890, "value1":0.21947167931430242, "value2":97944850538911159, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.034158, "value1":0.6471204512621925, "value2":6508843210808028097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495253, "value1":0.7273452290068572, "value2":2698451345131648958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.362458, "value1":0.6109728168729301, "value2":8601029350565666815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263467, "value1":0.5089131780165215, "value2":7125458167541328714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373946, "value1":0.4318385725650386, "value2":2714678475704530055, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714267, "value1":0.9575832102733836, "value2":6392945512202004757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448342, "value1":0.22459353529927165, "value2":3260297440141888471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636333, "value1":0.5831279192445404, "value2":4643961286576988019, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.544083, "value1":0.5292455018046274, "value2":2947716990317042035, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.834093, "value1":0.03492084160563978, "value2":317252294366234894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266819, "value1":0.3363934052516993, "value2":7398703592096323308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006504, "value1":0.7543858008870775, "value2":1775651744055633759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114763, "value1":0.5343311798406293, "value2":3440557676505536535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338231, "value1":0.46269924741830853, "value2":2414529329564577894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.613570, "value1":0.5208853551038555, "value2":1476820686457140000, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954830, "value1":0.41362302765893333, "value2":5510842372132552038, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.009033, "value1":0.39228566619392624, "value2":4427054644662596059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182689, "value1":0.6549868089884497, "value2":4035004692027824929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037729, "value1":0.7486872272646202, "value2":4847504507990849152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.025112, "value1":0.34907292952301494, "value2":6027470898165077200, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438904, "value1":0.6318755547819239, "value2":7111135668532014691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812145, "value1":0.3675412312936663, "value2":5692745323903444622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.445020, "value1":0.3643115537301237, "value2":1216854539448528229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149138, "value1":0.5909969143971484, "value2":7044155605397820069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841137, "value1":0.5031802947436413, "value2":4756712241095891395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766360, "value1":0.42396402951055434, "value2":5375665261877407507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663263, "value1":0.21109789310310256, "value2":7791325581861318665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.623457, "value1":0.10997466706442727, "value2":4022405196702857232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687642, "value1":0.5890580006230834, "value2":4798397386267850627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_2":"b","key_0":"i", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041534, "value1":0.7259891148801754, "value2":7530272514921895889, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.8243889839817137, "value2":2465746906164654846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.282429, "value1":0.6519583818560468, "value2":6092402468810251742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816484, "value1":0.18010854645368016, "value2":8191892113724148654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851506, "value1":0.5899554257608193, "value2":4617440579117228711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.382630, "value1":0.5412707148385829, "value2":3548532358946210519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.473179, "value1":0.6228095400098033, "value2":5572845251799335628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266666, "value1":0.5868701622069099, "value2":6991916315373855609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_9":"d","key_3":"d","key_7":"f", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.952001, "value1":0.6189910604042005, "value2":6928874617046934238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107532, "value1":0.28068061790544596, "value2":7672880301382817972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.688840, "value1":0.6035662636797529, "value2":5891970996329552554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283889, "value1":0.9151251791608558, "value2":2952612587882160016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629939, "value1":0.5930273861647344, "value2":7387132779647308781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418325, "value1":0.9507448442764749, "value2":167010402424657095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_9":"f","key_1":"h","key_5":"d", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373327, "value1":0.6218852126400313, "value2":8371256546452205404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id":853455729293892, "__mgmt_id":853455729293892} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515481, "value1":0.0854866845303678, "value2":6987828829468502321, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820817, "value1":0.698882043745769, "value2":8235587889784697376, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.365626, "value1":0.4531560215251981, "value2":4722270617283077560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484276, "value1":0.6659326817404172, "value2":8721021931100680532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.528708, "value1":0.03487880924213162, "value2":6041728758379789938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590377, "value1":0.34239911489608105, "value2":7295038165138773264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481162, "value1":0.8346535177821227, "value2":4835378003160835691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.664721, "value1":0.9670197543347829, "value2":7077761062907993448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242914, "value1":0.7732131934363016, "value2":6013582492758534765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.185221, "value1":0.10448347281111905, "value2":4626218721873500711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.126356, "value1":0.6396318028070207, "value2":7845764314900334021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673777, "value1":0.7918565155988029, "value2":7808431007480445571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706238, "value1":0.49404656958089044, "value2":8867399074074176905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216638, "value1":0.2610211982854062, "value2":5851177857276860153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247760, "value1":0.720377370743505, "value2":284915355108242656, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484098, "value1":0.20488798005491424, "value2":8924950640215344491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.345583, "value1":0.26657891861913446, "value2":1138178968371346598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.315052, "value1":0.7472751608991773, "value2":5324337837076333029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247789, "value1":0.35415032152422526, "value2":3640906326280302592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787688, "value1":0.6162646188121896, "value2":8883014095767101323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477237, "value1":0.04016123924628, "value2":9093318721099578903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560140, "value1":0.30318020971980014, "value2":2686556711005804654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.002924, "value1":0.5022585667052755, "value2":3145515076422963133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732614, "value1":0.3545238139152862, "value2":1059232874175794196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.332613, "value1":0.32230993437491456, "value2":2724091454501972237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012206, "value1":0.931677320101252, "value2":5370261932149351692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196655, "value1":0.8224901135235602, "value2":9208484728253262649, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.326865, "value1":0.5922171068495821, "value2":7224402625385076257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289542, "value1":0.9512116686992638, "value2":7523857296190024264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.170535, "value1":0.5872728280225836, "value2":6413447856713598093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434673, "value1":0.18109149352889556, "value2":1218373397895276791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} diff --git a/docker/test_prom_metric.hjson b/docker/test_prom_metric.hjson index 0aa7b45c..ce9c7650 100644 --- a/docker/test_prom_metric.hjson +++ b/docker/test_prom_metric.hjson @@ -29,6 +29,8 @@ PrometheusSchema: true DynamicSchema: { Enable: true + whiteList: ^(key_[0-9]?|value[1-4]?|__mgmt_id|__name__|timestamp|__series_id)$ + blackList: ^(key_3|value4)$ } } logLevel: info diff --git a/go.metrictest.sh b/go.metrictest.sh index dcd3a94c..e87045c0 100755 --- a/go.metrictest.sh +++ b/go.metrictest.sh @@ -32,6 +32,7 @@ curl "localhost:8123" -d "CREATE TABLE dist_test_prom_series ON CLUSTER abc AS t echo "send messages to kafka" echo "cat /tmp/test_prom_metric.data | kafka-console-producer --topic test_metric_topic --broker-list localhost:9092" > send.sh +# data generated by ""./kafka_gen_prom 192.168.110.10:19092 TestMetric" sudo docker cp ./docker/test_prom_metric.data kafka:/tmp/ sudo docker cp send.sh kafka:/tmp/ sudo docker exec kafka kafka-topics --bootstrap-server localhost:9093 --topic test_metric_topic --delete @@ -47,7 +48,7 @@ echo "Got test_prom_metric schema => $schema" schema=`curl "localhost:8123" -d 'DESC test_prom_series' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` echo "Got test_prom_series schema => $schema" -[ $schema = "key_0Nullable(String),key_1Nullable(String),key_2Nullable(String),key_3Nullable(String),key_4Nullable(String),key_5Nullable(String),key_6Nullable(String),key_7Nullable(String),key_8Nullable(String),key_9Nullable(String),labelsString,__mgmt_idInt64,__name__String,__series_idInt64," ] || exit 1 +[ $schema = "key_0Nullable(String),key_1Nullable(String),key_2Nullable(String),key_4Nullable(String),key_5Nullable(String),key_6Nullable(String),key_7Nullable(String),key_8Nullable(String),key_9Nullable(String),labelsString,__mgmt_idInt64,__name__String,__series_idInt64," ] || exit 1 echo "check result 1" count=`curl "localhost:8123" -d 'select count() from dist_test_prom_metric'` From 45afaf3a188aefedc27d2bea596606c3c35e47f0 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 10 Mar 2023 14:57:08 +0800 Subject: [PATCH 326/404] make lint happy --- input/kafka_franz.go | 3 +-- task/consumer.go | 9 +++------ task/sinker.go | 14 +++++++++----- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index c695b227..1c9b0205 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -208,7 +208,7 @@ func (k *KafkaFranz) CommitMessages(msg *model.InputMessage) error { } // Stop kafka consumer and close all connections -func (k *KafkaFranz) Stop() error { +func (k *KafkaFranz) Stop() { k.cancel() // prevent the block of k.Run @@ -225,7 +225,6 @@ func (k *KafkaFranz) Stop() error { case quit <- struct{}{}: default: } - return nil } // Description of this kafka consumer, consumer group name diff --git a/task/consumer.go b/task/consumer.go index 58d85334..e5a71ec6 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -90,7 +90,7 @@ func (c *Consumer) start() { } } -func (c *Consumer) stop() (err error) { +func (c *Consumer) stop() { if c.state.Load() == util.StateStopped { return } @@ -99,14 +99,11 @@ func (c *Consumer) stop() (err error) { // stop the processFetch routine, make sure no more input to the commit chan & writing pool c.stopCh <- struct{}{} c.processWg.Wait() - err = c.inputer.Stop() - return err + c.inputer.Stop() } func (c *Consumer) restart() { - if err := c.stop(); err != nil { - util.Logger.Fatal("failed to restart consumer group", zap.String("group", c.grpConfig.Name), zap.Error(err)) - } + c.stop() c.start() } diff --git a/task/sinker.go b/task/sinker.go index ce795a85..0068429f 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -601,12 +601,16 @@ func loadBmSeries(conn clickhouse.Conn, sqKey string, tasks []*Service) (result dbname := strings.Split(sqKey, ".")[0] query := fmt.Sprintf(createTableSQL, mergetable, dbname, tasks[0].clickhouse.GetMetricTable(), dbname, reg[:len(reg)-1]) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) - conn.Exec(context.Background(), query) + if err = conn.Exec(context.Background(), query); err != nil { + return + } - var count int + var count uint64 query = fmt.Sprintf(countSeriesSQL, mergetable, sqKey, mergetable) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) - conn.QueryRow(context.Background(), query).Scan(&count) + if err = conn.QueryRow(context.Background(), query).Scan(&count); err != nil { + return + } seriesMap := make(map[int64]int64, count) query = fmt.Sprintf(loadSeriesSQL, mergetable, sqKey, mergetable) @@ -626,7 +630,7 @@ func loadBmSeries(conn clickhouse.Conn, sqKey string, tasks []*Service) (result } query = fmt.Sprintf(dropTableSQL, mergetable) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) - conn.Exec(context.Background(), query) + err = conn.Exec(context.Background(), query) - return seriesMap, nil + return seriesMap, err } From 685f94c019372636357678996ed2489031b8dc6b Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Sat, 11 Mar 2023 00:04:19 +0800 Subject: [PATCH 327/404] fix clickhouse acquire conn timeout issue --- CHANGELOG.md | 2 +- clickhouse_sinker-dashboard.json | 87 +++++++++++++++++++----- go.mod | 43 ++++++------ go.sum | 110 +++++++++++++++---------------- output/clickhouse.go | 60 ++++++++++------- output/clickhouse_util.go | 19 ++++-- util/common.go | 22 ------- 7 files changed, 198 insertions(+), 145 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 98f5b1f8..8da7de7c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,6 @@ # Changelog -#### Version 3.0.2 (2023-03-10) +#### Version 3.0.2 (2023-03-13) Improvements: - update sinker grafana dashboard diff --git a/clickhouse_sinker-dashboard.json b/clickhouse_sinker-dashboard.json index d231efd5..054d33d4 100644 --- a/clickhouse_sinker-dashboard.json +++ b/clickhouse_sinker-dashboard.json @@ -24,7 +24,7 @@ "editable": true, "fiscalYearStartMonth": 0, "graphTooltip": 0, - "id": 32, + "id": 41, "links": [], "liveNow": false, "panels": [ @@ -669,7 +669,7 @@ "uid": "${datasource}" }, "editorMode": "code", - "expr": " {instance=~\"${instance}\",job=~\"${job}\",topic=~\"${topic}\"}", + "expr": " {instance=~\"${instance}\",job=~\"${job}\",topic=~\"${topic}\",consumer=~\"${consumer}\"}", "legendFormat": "__auto", "range": true, "refId": "A" @@ -686,7 +686,37 @@ "fieldConfig": { "defaults": { "color": { - "mode": "thresholds" + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } }, "mappings": [], "thresholds": { @@ -713,18 +743,16 @@ }, "id": 22, "options": { - "colorMode": "value", - "graphMode": "area", - "justifyMode": "auto", - "orientation": "auto", - "reduceOptions": { - "calcs": [ - "lastNotNull" - ], - "fields": "", - "values": false + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true }, - "textMode": "auto" + "tooltip": { + "mode": "single", + "sort": "none" + } }, "pluginVersion": "9.1.2", "targets": [ @@ -734,14 +762,14 @@ "uid": "${datasource}" }, "editorMode": "code", - "expr": "clickhouse_sinker_consume_lags{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\"}", + "expr": "clickhouse_sinker_consume_lags{instance=~\"${instance}\",job=~\"${job}\",task=~\"${task}\",consumer=~\"${consumer}\"}", "legendFormat": "__auto", "range": true, "refId": "A" } ], "title": "clickhouse_sinker_consume_lags", - "type": "stat" + "type": "timeseries" }, { "datasource": { @@ -2121,6 +2149,33 @@ "skipUrlSync": false, "sort": 0, "type": "query" + }, + { + "current": { + "selected": false, + "text": "metric-012-new-1", + "value": "metric-012-new-1" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values(consumer)", + "hide": 0, + "includeAll": false, + "label": "consumer", + "multi": false, + "name": "consumer", + "options": [], + "query": { + "query": "label_values(consumer)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 0, + "type": "query" } ] }, diff --git a/go.mod b/go.mod index ba4049ea..d32b01b4 100644 --- a/go.mod +++ b/go.mod @@ -3,37 +3,38 @@ module github.com/housepower/clickhouse_sinker go 1.19 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.6.2 + github.com/ClickHouse/clickhouse-go/v2 v2.7.0 github.com/RoaringBitmap/roaring v1.2.3 - github.com/bytedance/sonic v1.8.1 + github.com/avast/retry-go/v4 v4.3.3 + github.com/bytedance/sonic v1.8.3 github.com/cespare/xxhash/v2 v2.2.0 - github.com/google/gops v0.3.26 + github.com/google/gops v0.3.27 github.com/google/uuid v1.3.0 github.com/hjson/hjson-go/v4 v4.3.0 - github.com/jcmturner/gokrb5/v8 v8.4.3 + github.com/jcmturner/gokrb5/v8 v8.4.4 github.com/jinzhu/copier v0.3.5 github.com/nacos-group/nacos-sdk-go v1.1.4 github.com/prometheus/client_golang v1.14.0 - github.com/prometheus/common v0.40.0 + github.com/prometheus/common v0.42.0 github.com/shopspring/decimal v1.3.1 - github.com/stretchr/testify v1.8.1 + github.com/stretchr/testify v1.8.2 github.com/thanos-io/thanos v0.30.2 github.com/tidwall/gjson v1.14.4 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/twmb/franz-go v1.12.1 - github.com/twmb/franz-go/pkg/kadm v1.6.0 + github.com/twmb/franz-go/pkg/kadm v1.7.0 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.1 github.com/valyala/fastjson v1.6.4 go.uber.org/zap v1.24.0 - golang.org/x/exp v0.0.0-20230213192124-5e25df0256eb + golang.org/x/exp v0.0.0-20230307190834-24139beb5833 golang.org/x/time v0.3.0 gopkg.in/natefinch/lumberjack.v2 v2.2.1 ) require ( - github.com/ClickHouse/ch-go v0.52.1 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.174 // indirect + github.com/ClickHouse/ch-go v0.53.0 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.234 // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.5.0 // indirect @@ -43,7 +44,7 @@ require ( github.com/go-errors/errors v1.4.2 // indirect github.com/go-faster/city v1.0.1 // indirect github.com/go-faster/errors v0.6.1 // indirect - github.com/golang/protobuf v1.5.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/jcmturner/aescts/v2 v2.0.0 // indirect github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect @@ -51,8 +52,8 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.15.15 // indirect - github.com/klauspost/cpuid/v2 v2.2.3 // indirect + github.com/klauspost/compress v1.16.0 // indirect + github.com/klauspost/cpuid/v2 v2.2.4 // indirect github.com/kr/text v0.2.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect @@ -72,16 +73,16 @@ require ( github.com/tidwall/pretty v1.2.1 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect github.com/twmb/franz-go/pkg/kmsg v1.4.0 // indirect - go.opentelemetry.io/otel v1.13.0 // indirect - go.opentelemetry.io/otel/trace v1.13.0 // indirect + go.opentelemetry.io/otel v1.14.0 // indirect + go.opentelemetry.io/otel/trace v1.14.0 // indirect go.uber.org/atomic v1.10.0 // indirect - go.uber.org/multierr v1.9.0 // indirect - golang.org/x/arch v0.2.0 // indirect - golang.org/x/crypto v0.6.0 // indirect - golang.org/x/net v0.7.0 // indirect + go.uber.org/multierr v1.10.0 // indirect + golang.org/x/arch v0.3.0 // indirect + golang.org/x/crypto v0.7.0 // indirect + golang.org/x/net v0.8.0 // indirect golang.org/x/sync v0.1.0 // indirect - golang.org/x/sys v0.5.0 // indirect - google.golang.org/protobuf v1.28.1 // indirect + golang.org/x/sys v0.6.0 // indirect + google.golang.org/protobuf v1.29.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index 94362291..9ab8ff95 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,8 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.52.1 h1:nucdgfD1BDSHjbNaG3VNebonxJzD8fX8jbuBpfo5VY0= -github.com/ClickHouse/ch-go v0.52.1/go.mod h1:B9htMJ0hii/zrC2hljUKdnagRBuLqtRG/GrU3jqCwRk= -github.com/ClickHouse/clickhouse-go/v2 v2.6.2 h1:Gw6hUHDgwtDqLOjRZFFxqkwdWdYp/Vw1F+F4eAiaMtI= -github.com/ClickHouse/clickhouse-go/v2 v2.6.2/go.mod h1:GcNAg9SniIu+BqzOxRsTmXAGvhlSaUm/Y9GFdWUCbX8= +github.com/ClickHouse/ch-go v0.53.0 h1:gD9oP15FW+1oTTYyVzmuVfM+bk5cB5wqdscBIIw/mRA= +github.com/ClickHouse/ch-go v0.53.0/go.mod h1:B9htMJ0hii/zrC2hljUKdnagRBuLqtRG/GrU3jqCwRk= +github.com/ClickHouse/clickhouse-go/v2 v2.7.0 h1:KFRvFjnewYkJBwkfBvDYESwZtZmQipz/xRuaBz0oVNA= +github.com/ClickHouse/clickhouse-go/v2 v2.7.0/go.mod h1:6I79Gj2EPbV/DdlDShfCaxrja/pxLVSfDrvEEQp77VE= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.2.3 h1:yqreLINqIrX22ErkKI0vY47/ivtJr6n+kMhVOVmhWBY= @@ -12,10 +12,12 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.174 h1:K5561f5qv64GmtQuY0OtkY0H2x1bTXBeYwwWFPzc4D8= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.174/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.234 h1:rS3mCvKv0boCM/8i2mpX/JUcOTMp/lMiGpC4cykL3EI= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.234/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= +github.com/avast/retry-go/v4 v4.3.3 h1:G56Bp6mU0b5HE1SkaoVjscZjlQb0oy4mezwY/cGH19w= +github.com/avast/retry-go/v4 v4.3.3/go.mod h1:rg6XFaiuFYII0Xu3RDbZQkxCofFwruZKW8oEF1jpWiU= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -28,15 +30,14 @@ github.com/bits-and-blooms/bitset v1.5.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edY github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.8.1 h1:NqAHCaGaTzro0xMmnTCLUyRlbEP6r8MCA1cJUrH3Pu4= -github.com/bytedance/sonic v1.8.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/bytedance/sonic v1.8.3 h1:pf6fGl5eqWYKkx1RcD4qpuX+BIUaduv/wTm5ekWJ80M= +github.com/bytedance/sonic v1.8.3/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 h1:qSGYFH7+jGhDF8vLC+iwCD4WpbV1EBDSzWkJODFLams= github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= -github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -53,7 +54,6 @@ github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2 github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= @@ -65,18 +65,16 @@ github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= -github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/gops v0.3.26 h1:Ziyfd8sEhWVbrCIy59c1WOKodI63Jzojwm0JSZbBPS4= -github.com/google/gops v0.3.26/go.mod h1:vZ68aOXu2zJoybPyGpaHMmrCyd51DCxJoex4cO3ht/o= +github.com/google/gops v0.3.27 h1:BDdWfedShsBbeatZ820oA4DbVOC8yJ4NI8xAlDFWfgI= +github.com/google/gops v0.3.27/go.mod h1:lYqabmfnq4Q6UumWNx96Hjup5BDAVc8zmfIy0SkNCSk= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -88,7 +86,6 @@ github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/C github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hjson/hjson-go/v4 v4.3.0 h1:dyrzJdqqFGhHt+FSrs5n9s6b0fPM8oSJdWo+oS3YnJw= github.com/hjson/hjson-go/v4 v4.3.0/go.mod h1:KaYt3bTw3zhBjYqnXkYywcYctk0A2nxeEFTse3rH13E= -github.com/inconshreveable/mousetrap v1.0.1/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= @@ -97,8 +94,9 @@ github.com/jcmturner/gofork v1.7.6 h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVET github.com/jcmturner/gofork v1.7.6/go.mod h1:1622LH6i/EZqLloHfE7IeZ0uEJwMSUyQ/nDd82IeqRo= github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= -github.com/jcmturner/gokrb5/v8 v8.4.3 h1:iTonLeSJOn7MVUtyMT+arAn5AKAPrkilzhGw8wE/Tq8= github.com/jcmturner/gokrb5/v8 v8.4.3/go.mod h1:dqRwJGXznQrzw6cWmyo6kH+E7jksEQG/CyVWsJEsJO0= +github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh687T8= +github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/jinzhu/copier v0.3.5 h1:GlvfUwHk62RokgqVNvYsku0TATCF7bAHVwEXoBh3iJg= @@ -121,11 +119,11 @@ github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47e github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.4/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.15.15 h1:EF27CXIuDsYJ6mmvtBRlEuB2UVOqHG1tAXgZ7yIO+lw= -github.com/klauspost/compress v1.15.15/go.mod h1:ZcK2JAFqKOpnBlxcLsJzYfrS9X1akm9fHZNnD9+Vo/4= +github.com/klauspost/compress v1.16.0 h1:iULayQNOReoYUe+1qtKOqw9CwJv3aNQu8ivo7lw1HU4= +github.com/klauspost/compress v1.16.0/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= -github.com/klauspost/cpuid/v2 v2.2.3 h1:sxCkb+qR91z4vsqw4vGGZlDgPz3G7gjaLyK3V8y70BU= -github.com/klauspost/cpuid/v2 v2.2.3/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= +github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= +github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -134,7 +132,6 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= @@ -167,7 +164,6 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= @@ -180,8 +176,8 @@ github.com/prometheus/client_model v0.3.0 h1:UBgGFHqYdG/TPFD1B1ogZywDqEkwp3fBMvq github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= -github.com/prometheus/common v0.40.0 h1:Afz7EVRqGg2Mqqf4JuF9vdvp1pi220m55Pi9T2JnO4Q= -github.com/prometheus/common v0.40.0/go.mod h1:L65ZJPSmfn/UBWLQIHV7dBrKFidB/wPlF1y5TlSt9OE= +github.com/prometheus/common v0.42.0 h1:EKsfXEYo4JpWMHH5cg+KOUWeuJSov1Id8zGR8eeI1YM= +github.com/prometheus/common v0.42.0/go.mod h1:xBwqVerjNdUDjgODMpudtOMwlOwf2SaTr1yjz4b7Zbc= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= @@ -190,18 +186,14 @@ github.com/prometheus/procfs v0.9.0/go.mod h1:+pB4zwohETzFnmlpe6yd2lSc+0/46IYZRB github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= -github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= -github.com/shirou/gopsutil/v3 v3.22.10/go.mod h1:QNza6r4YQoydyCfo6rH0blGfKahgibh4dQmV5xdFkQk= github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/spf13/cobra v1.6.1/go.mod h1:IOw/AERYS7UzyrGinqmz6HLUo219MORXGxhbaJUqzrY= -github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/amqp v1.0.0 h1:kuuDrUJFZL1QYL9hUNuCxNObNzB0bV/ZG5jV3RWAQgo= github.com/streadway/amqp v1.0.0/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= @@ -217,8 +209,9 @@ github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= +github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/thanos-io/thanos v0.30.2 h1:JcazfBlpX/i747uRxBWHJMopX0g+eSaZYGjqMeQIkPo= github.com/thanos-io/thanos v0.30.2/go.mod h1:k23ZcbQSK0Jd/siuEti6J6/3DeezIk+YLRBcYikzXl8= github.com/tidwall/gjson v1.14.4 h1:uo0p8EbA09J7RQaflQ1aBRffTR7xedD2bcIVSYxLnkM= @@ -229,8 +222,6 @@ github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhV github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= github.com/tidwall/pretty v1.2.1 h1:qjsOFOWWQl+N3RsoF5/ssm1pHmJJwhjlSbZ51I6wMl4= github.com/tidwall/pretty v1.2.1/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= -github.com/tklauser/go-sysconf v0.3.10/go.mod h1:C8XykCvCb+Gn0oNCWPIlcb0RuglQTYaQ2hGm7jmxEFk= -github.com/tklauser/numcpus v0.4.0/go.mod h1:1+UI3pD8NW14VMwdgJNJ1ESk2UnwhAnz5hMwiKKqXCQ= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNijVoeZCBqCXG1AWpWv2IylInTXeSjhdjWGc= github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= @@ -239,8 +230,8 @@ github.com/twmb/franz-go v1.5.3/go.mod h1:eqHYpAuvlTArOdZ1XtPYyOQ1uUb40CSZwbpL3c github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= github.com/twmb/franz-go v1.12.1 h1:8lWT8q0spL40Nfw6eonJ8OoPGLvF9arvadRRmcSiu9Y= github.com/twmb/franz-go v1.12.1/go.mod h1:Ofc5tSSUJKLmpRNUYSejUsAZKYAHDHywTS322KWdChQ= -github.com/twmb/franz-go/pkg/kadm v1.6.0 h1:jfbpdneFgwO8wcvkMnu670+qYmOI4A9USHR/VTsAqrA= -github.com/twmb/franz-go/pkg/kadm v1.6.0/go.mod h1:1FifItwSffE++249YqRooeEfDnKRgvki6GqTJezTVAM= +github.com/twmb/franz-go/pkg/kadm v1.7.0 h1:TAgcS+t5q+9jnm8INCD2OJ1MD9y4Ij6pD5CYfZ3tkbg= +github.com/twmb/franz-go/pkg/kadm v1.7.0/go.mod h1:sI9BjVkpjyYssIlVa+WIwseaUjJqPsR/8gmJi6aDyEk= github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= @@ -259,17 +250,16 @@ github.com/valyala/fastjson v1.6.4/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLr github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= github.com/xdg-go/scram v1.1.1/go.mod h1:RaEWvsqvNKKvBPvcKeFjrG2cJqOkHTiyTpzz23ni57g= github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgklLGvcBnW8= -github.com/xlab/treeprint v1.1.0/go.mod h1:gj5Gd3gPdKtR1ikdDK6fnFLdmIS0X30kTTuNd/WEJu0= github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d/go.mod h1:rHwXgn7JulP+udvsHwJoVG1YGAP6VLg4y9I5dyZdqmA= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.mongodb.org/mongo-driver v1.11.1/go.mod h1:s7p5vEtfbeR1gYi6pnj3c3/urpbLv2T5Sfd6Rp2HBB8= -go.opentelemetry.io/otel v1.13.0 h1:1ZAKnNQKwBBxFtww/GwxNUyTf0AxkZzrukO8MeXqe4Y= -go.opentelemetry.io/otel v1.13.0/go.mod h1:FH3RtdZCzRkJYFTCsAKDy9l/XYjMdNv6QrkFFB8DvVg= -go.opentelemetry.io/otel/trace v1.13.0 h1:CBgRZ6ntv+Amuj1jDsMhZtlAPT6gbyIRdaIzFhfBSdY= -go.opentelemetry.io/otel/trace v1.13.0/go.mod h1:muCvmmO9KKpvuXSf3KKAXXB2ygNYHQ+ZfI5X08d3tds= +go.opentelemetry.io/otel v1.14.0 h1:/79Huy8wbf5DnIPhemGB+zEPVwnN6fuQybr/SRXa6hM= +go.opentelemetry.io/otel v1.14.0/go.mod h1:o4buv+dJzx8rohcUeRmWUZhqupFvzWis188WlggnNeU= +go.opentelemetry.io/otel/trace v1.14.0 h1:wp2Mmvj41tDsyAJXiWDWpfNsOiIyd38fy85pyKcFq/M= +go.opentelemetry.io/otel/trace v1.14.0/go.mod h1:8avnQLK+CG77yNLUae4ea2JDQ6iT+gozhnZjy/rw9G8= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -280,34 +270,37 @@ go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= -go.uber.org/multierr v1.9.0 h1:7fIwc/ZtS0q++VgcfqFDxSBZVv/Xo49/SYnDFupUwlI= -go.uber.org/multierr v1.9.0/go.mod h1:X2jQV1h+kxSjClGpnseKVIxpmcjrj7MNnI0bnlfKTVQ= +go.uber.org/multierr v1.10.0 h1:S0h4aNzvfcFsC3dRF1jLoaov7oRaKqRGC/pUEJ2yvPQ= +go.uber.org/multierr v1.10.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.2.0 h1:W1sUEHXiJTfjaFJ5SLo0N6lZn+0eO5gWD1MFeTGqQEY= -golang.org/x/arch v0.2.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.3.0 h1:02VY4/ZcO/gBOH6PUaoiptASxtXU10jazRCP865E97k= +golang.org/x/arch v0.3.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.6.0 h1:qfktjS5LUO+fFKeJXZ+ikTRijMmljikvG68fpMMruSc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/exp v0.0.0-20230213192124-5e25df0256eb h1:PaBZQdo+iSDyHT053FjUCgZQ/9uqVwPOcl7KSWhKn6w= -golang.org/x/exp v0.0.0-20230213192124-5e25df0256eb/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= +golang.org/x/crypto v0.7.0 h1:AvwMYaRytfdeVt3u6mLaxYtErKYjxA2OXjJ1HHq6t3A= +golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= +golang.org/x/exp v0.0.0-20230307190834-24139beb5833 h1:SChBja7BCQewoTAU7IgvucQKMIXrEpFxNMs0spT3/5s= +golang.org/x/exp v0.0.0-20230307190834-24139beb5833/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -316,18 +309,23 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= -golang.org/x/net v0.7.0 h1:rJrUqqhjsgNp7KqAIc25s9pZnjU7TUcSY7HcVZjdn1g= +golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.8.0 h1:Zrh2ngAOFYneWTAIAPethzeaQLuHwhuBkuV6ZiRnUaQ= +golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0 h1:wsuoTGHzEhffawBOhz5CYhcrV4IdKZbEyZjBMuTp12o= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -335,28 +333,28 @@ golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220128215802-99c3d69c2c27/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.5.0 h1:MUK/U/4lj1t1oPg0HfuXDN/Z1wv31ZJ/YcPiGccS4DU= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ= +golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -370,6 +368,7 @@ golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -377,8 +376,8 @@ golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8T google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w= -google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.29.0 h1:44S3JjaKmLEE4YIkjzexaP+NzZsudE3Zin5Njn/pYX0= +google.golang.org/protobuf v1.29.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -401,5 +400,4 @@ gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -rsc.io/goversion v1.2.0/go.mod h1:Eih9y/uIBS3ulggl7KNJ09xGSLcuNaLgmvvqa07sgfo= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= diff --git a/output/clickhouse.go b/output/clickhouse.go index b8813d71..faf37bf2 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -26,6 +26,7 @@ import ( "github.com/ClickHouse/clickhouse-go/v2" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + "github.com/avast/retry-go/v4" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" @@ -223,29 +224,29 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( // LoopWrite will dead loop to write the records func (c *ClickHouse) loopWrite(batch *model.Batch) { - var err error - var times int - var reconnect bool + var retrycount int var dbVer int sc := pool.GetShardConn(batch.BatchIdx) - for { - if err = c.write(batch, sc, &dbVer); err == nil { - return - } - if errors.Is(err, context.Canceled) { - util.Logger.Info("ClickHouse.write failed due to the context has been cancelled", zap.String("task", c.taskCfg.Name)) - return - } - util.Logger.Error("flush batch failed", zap.String("task", c.taskCfg.Name), zap.Int("try", times), zap.Error(err)) - statistics.FlushMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) - times++ - reconnect = shouldReconnect(err, sc) - if reconnect && (c.cfg.Clickhouse.RetryTimes <= 0 || times < c.cfg.Clickhouse.RetryTimes) { - time.Sleep(10 * time.Second) - } else { - util.Logger.Fatal("ClickHouse.loopWrite failed", zap.String("task", c.taskCfg.Name)) - } + times := c.cfg.Clickhouse.RetryTimes + if times <= 0 { + times = 0 + } + if err := retry.Do( + func() error { return c.write(batch, sc, &dbVer) }, + retry.LastErrorOnly(true), + retry.Attempts(uint(times)), + retry.Delay(10*time.Second), + retry.RetryIf(func(err error) bool { return shouldReconnect(err, sc) }), + retry.OnRetry(func(n uint, err error) { + retrycount++ + if !errors.Is(err, clickhouse.ErrAcquireConnTimeout) { + util.Logger.Error("flush batch failed", zap.String("task", c.taskCfg.Name), zap.Int("try", int(retrycount)), zap.Error(err)) + statistics.FlushMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) + } + }), + ); err != nil { + util.Logger.Fatal("ClickHouse.loopWrite failed", zap.String("task", c.taskCfg.Name), zap.Error(err)) } } @@ -478,7 +479,13 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { alterTable := func(tbl, col string) error { query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, tbl, onCluster, col) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) - return util.RetryOperation(func() error { return conn.Exec(context.Background(), query) }, -1, []error{clickhouse.ErrAcquireConnTimeout}) + return retry.Do( + func() error { return conn.Exec(context.Background(), query) }, + retry.Attempts(0), + retry.LastErrorOnly(true), + retry.Delay(10*time.Second), + retry.RetryIf(func(err error) bool { return errors.Is(err, clickhouse.ErrAcquireConnTimeout) }), + ) } if len(alterSeries) != 0 { @@ -521,7 +528,16 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { c.dbName, chCfg.Cluster, c.dbName, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows driver.Rows - if rows, err = conn.Query(context.Background(), query); err != nil { + if err = retry.Do( + func() error { + rows, err = conn.Query(context.Background(), query) + return err + }, + retry.Attempts(0), + retry.LastErrorOnly(true), + retry.Delay(10*time.Second), + retry.RetryIf(func(err error) bool { return errors.Is(err, clickhouse.ErrAcquireConnTimeout) }), + ); err != nil { err = errors.Wrapf(err, "") return } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 44899e4f..f4481ed9 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -3,10 +3,12 @@ package output import ( "context" "fmt" + "time" "github.com/ClickHouse/clickhouse-go/v2" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/RoaringBitmap/roaring" + "github.com/avast/retry-go/v4" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/util" @@ -80,7 +82,16 @@ func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn cl func getDims(database, table string, excludedColumns []string, parser string, conn clickhouse.Conn) (dims []*model.ColumnWithType, err error) { var rs driver.Rows - if rs, err = conn.Query(context.Background(), fmt.Sprintf(selectSQLTemplate, database, table)); err != nil { + if err = retry.Do( + func() error { + rs, err = conn.Query(context.Background(), fmt.Sprintf(selectSQLTemplate, database, table)) + return err + }, + retry.Attempts(0), + retry.LastErrorOnly(true), + retry.Delay(10*time.Second), + retry.RetryIf(func(err error) bool { return errors.Is(err, clickhouse.ErrAcquireConnTimeout) }), + ); err != nil { err = errors.Wrapf(err, "") return } @@ -103,9 +114,3 @@ func getDims(database, table string, excludedColumns []string, parser string, co } return } - -const ( - EXEC = iota - QUERYROW - QUERY -) diff --git a/util/common.go b/util/common.go index ccc2cd7f..c2ab1d85 100644 --- a/util/common.go +++ b/util/common.go @@ -17,7 +17,6 @@ package util import ( "bytes" - "context" "crypto/tls" "crypto/x509" "fmt" @@ -28,7 +27,6 @@ import ( "reflect" "strconv" "strings" - "time" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -285,23 +283,3 @@ func SetLogLevel(newLogLevel string) { logAtomLevel.SetLevel(lvl) } } - -// dead loop to execute the operation -func RetryOperation(operation func() error, times int, tolerant []error) (err error) { -LOOP: - for times != 0 { - if err = operation(); err == nil || errors.Is(err, context.Canceled) { - return - } - for _, e := range tolerant { - if errors.Is(err, e) { - time.Sleep(10 * time.Second) - times-- - continue LOOP - } - } - return - } - - return -} From dc4d0b40103e812ddedabe3f72625485d8b353d1 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 23 Mar 2023 13:55:33 +0800 Subject: [PATCH 328/404] Have writingpool per shard to avoid ErrAcquireConnTimeout --- CHANGELOG.md | 5 ++++ Makefile | 1 + config/config.go | 2 +- go.mod | 20 +++++++-------- go.sum | 58 +++++++++++++++----------------------------- output/clickhouse.go | 9 +++---- pool/conn.go | 25 +++++++++++++------ task/sinker.go | 7 +----- util/common.go | 17 +++---------- 9 files changed, 62 insertions(+), 82 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8da7de7c..82e9fae3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,10 @@ # Changelog +#### Version 3.0.3 (not released yet) + +Improvements: +- Have writingpool per shard to avoid ErrAcquireConnTimeout + #### Version 3.0.2 (2023-03-13) Improvements: diff --git a/Makefile b/Makefile index 029413ee..3367e417 100644 --- a/Makefile +++ b/Makefile @@ -8,6 +8,7 @@ GOBUILD := $(GO) build $(BUILD_FLAG) pre: go mod tidy + go get -u ./... build: pre $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o . ./... debug: pre diff --git a/config/config.go b/config/config.go index acb4b960..6dcccf1b 100644 --- a/config/config.go +++ b/config/config.go @@ -179,7 +179,7 @@ const ( defaultLogLevel = "info" defaultKerberosConfigPath = "/etc/krb5.conf" defaultMaxOpenConns = 1 - defaultDialTimeout = 2 + defaultDialTimeout = 5 ) func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { diff --git a/go.mod b/go.mod index d32b01b4..708e8ab7 100644 --- a/go.mod +++ b/go.mod @@ -1,12 +1,12 @@ module github.com/housepower/clickhouse_sinker -go 1.19 +go 1.20 require ( github.com/ClickHouse/clickhouse-go/v2 v2.7.0 github.com/RoaringBitmap/roaring v1.2.3 github.com/avast/retry-go/v4 v4.3.3 - github.com/bytedance/sonic v1.8.3 + github.com/bytedance/sonic v1.8.5 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.27 github.com/google/uuid v1.3.0 @@ -21,20 +21,20 @@ require ( github.com/thanos-io/thanos v0.30.2 github.com/tidwall/gjson v1.14.4 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.12.1 - github.com/twmb/franz-go/pkg/kadm v1.7.0 + github.com/twmb/franz-go v1.13.1 + github.com/twmb/franz-go/pkg/kadm v1.8.0 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 - github.com/twmb/franz-go/plugin/kzap v1.1.1 + github.com/twmb/franz-go/plugin/kzap v1.1.2 github.com/valyala/fastjson v1.6.4 go.uber.org/zap v1.24.0 - golang.org/x/exp v0.0.0-20230307190834-24139beb5833 + golang.org/x/exp v0.0.0-20230321023759-10a507213a29 golang.org/x/time v0.3.0 gopkg.in/natefinch/lumberjack.v2 v2.2.1 ) require ( - github.com/ClickHouse/ch-go v0.53.0 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.234 // indirect + github.com/ClickHouse/ch-go v0.53.1 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.250 // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.5.0 // indirect @@ -52,7 +52,7 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.16.0 // indirect + github.com/klauspost/compress v1.16.3 // indirect github.com/klauspost/cpuid/v2 v2.2.4 // indirect github.com/kr/text v0.2.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect @@ -82,7 +82,7 @@ require ( golang.org/x/net v0.8.0 // indirect golang.org/x/sync v0.1.0 // indirect golang.org/x/sys v0.6.0 // indirect - google.golang.org/protobuf v1.29.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index 9ab8ff95..dd66a815 100644 --- a/go.sum +++ b/go.sum @@ -1,6 +1,7 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.53.0 h1:gD9oP15FW+1oTTYyVzmuVfM+bk5cB5wqdscBIIw/mRA= -github.com/ClickHouse/ch-go v0.53.0/go.mod h1:B9htMJ0hii/zrC2hljUKdnagRBuLqtRG/GrU3jqCwRk= +github.com/ClickHouse/ch-go v0.53.1 h1:7Mxn/e0ZUzC/iZUyuHQIpi5WRLB5wzxKbsAG8h02bNk= +github.com/ClickHouse/ch-go v0.53.1/go.mod h1:2jvyjBRb5zhzFvcOBxPelzpbB9qsS47vwJssurJe2OA= +github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= github.com/ClickHouse/clickhouse-go/v2 v2.7.0 h1:KFRvFjnewYkJBwkfBvDYESwZtZmQipz/xRuaBz0oVNA= github.com/ClickHouse/clickhouse-go/v2 v2.7.0/go.mod h1:6I79Gj2EPbV/DdlDShfCaxrja/pxLVSfDrvEEQp77VE= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= @@ -12,14 +13,13 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.234 h1:rS3mCvKv0boCM/8i2mpX/JUcOTMp/lMiGpC4cykL3EI= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.234/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.250 h1:hkF8iofQ8lEd/uXVg75JkWjjCcWhcY1DUg4fmLpKLD4= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.250/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/avast/retry-go/v4 v4.3.3 h1:G56Bp6mU0b5HE1SkaoVjscZjlQb0oy4mezwY/cGH19w= github.com/avast/retry-go/v4 v4.3.3/go.mod h1:rg6XFaiuFYII0Xu3RDbZQkxCofFwruZKW8oEF1jpWiU= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -30,8 +30,9 @@ github.com/bits-and-blooms/bitset v1.5.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edY github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.8.3 h1:pf6fGl5eqWYKkx1RcD4qpuX+BIUaduv/wTm5ekWJ80M= -github.com/bytedance/sonic v1.8.3/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/bytedance/sonic v1.8.5 h1:kjX0/vo5acEQ/sinD/18SkA/lDDUk23F0RcaHvI7omc= +github.com/bytedance/sonic v1.8.5/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -116,11 +117,9 @@ github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7V github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.15.4/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.16.0 h1:iULayQNOReoYUe+1qtKOqw9CwJv3aNQu8ivo7lw1HU4= -github.com/klauspost/compress v1.16.0/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.16.3 h1:XuJt9zzcnaz6a16/OU53ZjWp/v7/42WcR5t2a0PcNQY= +github.com/klauspost/compress v1.16.3/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= @@ -153,7 +152,6 @@ github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b/go.mo github.com/paulmach/orb v0.9.0 h1:MwA1DqOKtvCgm7u9RZ/pnYejTeDJPnr0+0oFajBbJqk= github.com/paulmach/orb v0.9.0/go.mod h1:SudmOk85SXtmXAB3sLGyJ6tZy/8pdfrV0o6ef98Xc30= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= -github.com/pierrec/lz4/v4 v4.1.14/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.17 h1:kV4Ip+/hUBC+8T6+2EgburRtkE9ef4nbY3f4dFhGjMc= github.com/pierrec/lz4/v4 v4.1.17/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -226,21 +224,18 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNij github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= -github.com/twmb/franz-go v1.5.3/go.mod h1:eqHYpAuvlTArOdZ1XtPYyOQ1uUb40CSZwbpL3ccjibI= github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go v1.12.1 h1:8lWT8q0spL40Nfw6eonJ8OoPGLvF9arvadRRmcSiu9Y= -github.com/twmb/franz-go v1.12.1/go.mod h1:Ofc5tSSUJKLmpRNUYSejUsAZKYAHDHywTS322KWdChQ= -github.com/twmb/franz-go/pkg/kadm v1.7.0 h1:TAgcS+t5q+9jnm8INCD2OJ1MD9y4Ij6pD5CYfZ3tkbg= -github.com/twmb/franz-go/pkg/kadm v1.7.0/go.mod h1:sI9BjVkpjyYssIlVa+WIwseaUjJqPsR/8gmJi6aDyEk= -github.com/twmb/franz-go/pkg/kmsg v1.0.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/kmsg v1.1.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go v1.13.1 h1:7dDuAjJItbZRjgmMzYzCfJtRMpVKfQ57yIG8SkZz24M= +github.com/twmb/franz-go v1.13.1/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= +github.com/twmb/franz-go/pkg/kadm v1.8.0 h1:vvKwZpxYn+VmM32v4mKkecHLKavZW+HcYLRKKxly5ZY= +github.com/twmb/franz-go/pkg/kadm v1.8.0/go.mod h1:qUSM7pxoMCU1UNu5H4USE64ODcVmeG9LS96mysv1nu8= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.4.0 h1:tbp9hxU6m8qZhQTlpGiaIJOm4BXix5lsuEZ7K00dF0s= github.com/twmb/franz-go/pkg/kmsg v1.4.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 h1:alKdbddkPw3rDh+AwmUEwh6HNYgTvDSFIe/GWYRR9RM= github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0/go.mod h1:k8BoBjyUbFj34f0rRbn+Ky12sZFAPbmShrg0karAIMo= -github.com/twmb/franz-go/plugin/kzap v1.1.1 h1:ae8Z2JXn8y9ceZ2AFnwPm5U1A8d6cBvvZlygF46F2N4= -github.com/twmb/franz-go/plugin/kzap v1.1.1/go.mod h1:TUlWYqucIyz6U7xLo++gkHBDiJmed5FpYe42rCL2YG0= +github.com/twmb/franz-go/plugin/kzap v1.1.2 h1:0arX5xJ0soUPX1LlDay6ZZoxuWkWk1lggQ5M/IgRXAE= +github.com/twmb/franz-go/plugin/kzap v1.1.2/go.mod h1:53Cl9Uz1pbdOPDvUISIxLrZIWSa2jCuY1bTMauRMBmo= github.com/uber/jaeger-client-go v2.30.0+incompatible h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o= github.com/uber/jaeger-client-go v2.30.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg= @@ -253,7 +248,6 @@ github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgk github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d/go.mod h1:rHwXgn7JulP+udvsHwJoVG1YGAP6VLg4y9I5dyZdqmA= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.mongodb.org/mongo-driver v1.11.1/go.mod h1:s7p5vEtfbeR1gYi6pnj3c3/urpbLv2T5Sfd6Rp2HBB8= go.opentelemetry.io/otel v1.14.0 h1:/79Huy8wbf5DnIPhemGB+zEPVwnN6fuQybr/SRXa6hM= @@ -261,20 +255,15 @@ go.opentelemetry.io/otel v1.14.0/go.mod h1:o4buv+dJzx8rohcUeRmWUZhqupFvzWis188Wl go.opentelemetry.io/otel/trace v1.14.0 h1:wp2Mmvj41tDsyAJXiWDWpfNsOiIyd38fy85pyKcFq/M= go.opentelemetry.io/otel/trace v1.14.0/go.mod h1:8avnQLK+CG77yNLUae4ea2JDQ6iT+gozhnZjy/rw9G8= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.10.0 h1:9qC72Qh0+3MqyJbAn8YU5xVq1frD8bn3JtD2oXtafVQ= go.uber.org/atomic v1.10.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= -go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= go.uber.org/multierr v1.10.0 h1:S0h4aNzvfcFsC3dRF1jLoaov7oRaKqRGC/pUEJ2yvPQ= go.uber.org/multierr v1.10.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= @@ -286,20 +275,18 @@ golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220427172511-eb4f295cb31f/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.7.0 h1:AvwMYaRytfdeVt3u6mLaxYtErKYjxA2OXjJ1HHq6t3A= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= -golang.org/x/exp v0.0.0-20230307190834-24139beb5833 h1:SChBja7BCQewoTAU7IgvucQKMIXrEpFxNMs0spT3/5s= -golang.org/x/exp v0.0.0-20230307190834-24139beb5833/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= +golang.org/x/exp v0.0.0-20230321023759-10a507213a29 h1:ooxPy7fPvB4kwsA2h+iBNHkAbp/4JxTSwCmvdjEYmug= +golang.org/x/exp v0.0.0-20230321023759-10a507213a29/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -310,7 +297,6 @@ golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= @@ -336,9 +322,7 @@ golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -367,7 +351,6 @@ golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -376,8 +359,8 @@ golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8T google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.29.0 h1:44S3JjaKmLEE4YIkjzexaP+NzZsudE3Zin5Njn/pYX0= -google.golang.org/protobuf v1.29.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= +google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -396,7 +379,6 @@ gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= diff --git a/output/clickhouse.go b/output/clickhouse.go index faf37bf2..aa05aaed 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -104,8 +104,9 @@ func (c *ClickHouse) Drain() { // Send a batch to clickhouse func (c *ClickHouse) Send(batch *model.Batch) { - if err := util.GlobalWritingPool.Submit(func() { - c.loopWrite(batch) + sc := pool.GetShardConn(batch.BatchIdx) + if err := sc.SubmitTask(func() { + c.loopWrite(batch, sc) batch.Wg.Done() c.mux.Lock() c.numFlying-- @@ -223,11 +224,9 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( } // LoopWrite will dead loop to write the records -func (c *ClickHouse) loopWrite(batch *model.Batch) { +func (c *ClickHouse) loopWrite(batch *model.Batch, sc *pool.ShardConn) { var retrycount int var dbVer int - sc := pool.GetShardConn(batch.BatchIdx) - times := c.cfg.Clickhouse.RetryTimes if times <= 0 { times = 0 diff --git a/pool/conn.go b/pool/conn.go index 6c76d3ca..928578c8 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -37,12 +37,17 @@ var ( // ShardConn a datastructure for storing the clickhouse connection type ShardConn struct { - lock sync.Mutex - db clickhouse.Conn - dbVer int - opts clickhouse.Options - replicas []string //ip:port list of replicas - nextRep int //index of next replica + lock sync.Mutex + db clickhouse.Conn + dbVer int + opts clickhouse.Options + replicas []string //ip:port list of replicas + nextRep int //index of next replica + writingPool *util.WorkerPool //the all tasks' writing ClickHouse, cpu-net balance +} + +func (sc *ShardConn) SubmitTask(fn func()) (err error) { + return sc.writingPool.Submit(fn) } // GetReplica returns the replica to which db connects @@ -64,6 +69,9 @@ func (sc *ShardConn) Close() { sc.db.Close() sc.db = nil } + if sc.writingPool != nil { + sc.writingPool.StopWait() + } } // NextGoodReplica connects to next good replica @@ -132,9 +140,10 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara }, DialTimeout: time.Second * time.Duration(dialTimeout), MaxOpenConns: maxOpenConns, - MaxIdleConns: 1, - ConnMaxLifetime: time.Hour, + MaxIdleConns: 5, // TODO - update this property to maxOpenConns when the lifetime of an idle connection honours the ConnMaxLifetime + ConnMaxLifetime: time.Minute * 10, }, + writingPool: util.NewWorkerPool(maxOpenConns, 1), } if secure { tlsConfig := &tls.Config{} diff --git a/task/sinker.go b/task/sinker.go index 0068429f..26d12b40 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -263,9 +263,7 @@ func (s *Sinker) stopAllTasks() { delete(s.consumers, name) } - if util.GlobalWritingPool != nil { - util.GlobalWritingPool.StopWait() - } + pool.CloseAll() util.Logger.Debug("stopped writing pool") } @@ -297,7 +295,6 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { } // 2. Start goroutine pools. - util.InitGlobalWritingPool(len(chCfg.Hosts) * chCfg.MaxOpenConns) go s.commitFn() // 3. Generate, initialize and run task @@ -339,8 +336,6 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 3. Restart goroutine pools. maxWorkers := len(newCfg.Clickhouse.Hosts) * newCfg.Clickhouse.MaxOpenConns - util.GlobalWritingPool.Resize(maxWorkers) - util.GlobalWritingPool.Restart() util.Logger.Info("resized writing pool", zap.Int("maxWorkers", maxWorkers)) go s.commitFn() diff --git a/util/common.go b/util/common.go index c2ab1d85..9c934788 100644 --- a/util/common.go +++ b/util/common.go @@ -36,10 +36,9 @@ import ( ) var ( - GlobalWritingPool *WorkerPool //the all tasks' writing ClickHouse, cpu-net balance - Logger *zap.Logger - logAtomLevel zap.AtomicLevel - logPaths []string + Logger *zap.Logger + logAtomLevel zap.AtomicLevel + logPaths []string ) type CmdOptions struct { @@ -59,16 +58,6 @@ type CmdOptions struct { NacosServiceName string // participate in assignment management if not empty } -// InitGlobalWritingPool initialize GlobalWritingPool -func InitGlobalWritingPool(maxWorkers int) { - if GlobalWritingPool != nil { - return - } - queueSize := 1 - GlobalWritingPool = NewWorkerPool(maxWorkers, queueSize) - Logger.Info("initialized writing pool", zap.Int("maxWorkers", maxWorkers), zap.Int("queueSize", queueSize)) -} - // StringContains check if contains string in array func StringContains(arr []string, str string) bool { for _, s := range arr { From 8cf39cedcb78f062b595cd22b74a9df7324142d7 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 24 Mar 2023 21:50:43 +0800 Subject: [PATCH 329/404] added PLAIN support --- config/config.go | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/config/config.go b/config/config.go index 6dcccf1b..0d755945 100644 --- a/config/config.go +++ b/config/config.go @@ -392,15 +392,6 @@ func (cfg *Config) convertKfkSecurity() { } if config, ok := cfg.Kafka.Security["sasl.jaas.config"]; ok { configMap := readConfig(config) - if strings.Contains(cfg.Kafka.Sasl.Mechanism, "SCRAM") { - // SCRAM-SHA-256 or SCRAM-SHA-512 - if username, ok := configMap["username"]; ok { - cfg.Kafka.Sasl.Username = username - } - if password, ok := configMap["password"]; ok { - cfg.Kafka.Sasl.Password = password - } - } if strings.Contains(cfg.Kafka.Sasl.Mechanism, "GSSAPI") { // GSSAPI if useKeyTab, ok := configMap["useKeyTab"]; ok { @@ -436,6 +427,14 @@ func (cfg *Config) convertKfkSecurity() { cfg.Kafka.Sasl.GSSAPI.KerberosConfigPath = defaultKerberosConfigPath } } + } else { + // PLAIN, SCRAM-SHA-256 or SCRAM-SHA-512 + if username, ok := configMap["username"]; ok { + cfg.Kafka.Sasl.Username = username + } + if password, ok := configMap["password"]; ok { + cfg.Kafka.Sasl.Password = password + } } } } From 2c371fc1748fe980d5148794aad57f55cf1a9ba6 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Mon, 27 Mar 2023 11:28:49 +0800 Subject: [PATCH 330/404] simplify convertKfkSecurity --- config/config.go | 76 +++++++++++++++--------------------------------- go.mod | 10 +++---- go.sum | 24 ++++++++------- 3 files changed, 41 insertions(+), 69 deletions(-) diff --git a/config/config.go b/config/config.go index 0d755945..acec16e7 100644 --- a/config/config.go +++ b/config/config.go @@ -360,81 +360,51 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { // convert java client style configuration into sinker func (cfg *Config) convertKfkSecurity() { - if protocol, ok := cfg.Kafka.Security["security.protocol"]; ok { - if strings.Contains(protocol, "SASL") { - cfg.Kafka.Sasl.Enable = true - } - if strings.Contains(protocol, "SSL") { - cfg.Kafka.TLS.Enable = true - } + protocol := cfg.Kafka.Security["security.protocol"] + if protocol == "" { + return } - if cfg.Kafka.TLS.Enable { - if endpIdentAlgo, ok := cfg.Kafka.Security["ssl.endpoint.identification.algorithm"]; ok { - cfg.Kafka.TLS.EndpIdentAlgo = endpIdentAlgo - } - if trustStoreLocation, ok := cfg.Kafka.Security["ssl.truststore.location"]; ok { - cfg.Kafka.TLS.TrustStoreLocation = trustStoreLocation - } - if trustStorePassword, ok := cfg.Kafka.Security["ssl.truststore.password"]; ok { - cfg.Kafka.TLS.TrustStorePassword = trustStorePassword - } - if keyStoreLocation, ok := cfg.Kafka.Security["ssl.keystore.location"]; ok { - cfg.Kafka.TLS.KeystoreLocation = keyStoreLocation - } - if keyStorePassword, ok := cfg.Kafka.Security["ssl.keystore.password"]; ok { - cfg.Kafka.TLS.KeystorePassword = keyStorePassword - } + if strings.Contains(protocol, "SSL") { + cfg.Kafka.TLS.Enable = true + cfg.Kafka.TLS.EndpIdentAlgo = cfg.Kafka.Security["ssl.endpoint.identification.algorithm"] + cfg.Kafka.TLS.TrustStoreLocation = cfg.Kafka.Security["ssl.truststore.location"] + cfg.Kafka.TLS.TrustStorePassword = cfg.Kafka.Security["ssl.truststore.password"] + cfg.Kafka.TLS.KeystoreLocation = cfg.Kafka.Security["ssl.keystore.location"] + cfg.Kafka.TLS.KeystorePassword = cfg.Kafka.Security["ssl.keystore.password"] } - if cfg.Kafka.Sasl.Enable { - if mechanism, ok := cfg.Kafka.Security["sasl.mechanism"]; ok { - cfg.Kafka.Sasl.Mechanism = mechanism - } + + if strings.Contains(protocol, "SASL") { + cfg.Kafka.Sasl.Enable = true + cfg.Kafka.Sasl.Mechanism = cfg.Kafka.Security["sasl.mechanism"] if config, ok := cfg.Kafka.Security["sasl.jaas.config"]; ok { configMap := readConfig(config) if strings.Contains(cfg.Kafka.Sasl.Mechanism, "GSSAPI") { // GSSAPI - if useKeyTab, ok := configMap["useKeyTab"]; ok { - if useKeyTab == "true" { - cfg.Kafka.Sasl.GSSAPI.AuthType = 2 - } else { - cfg.Kafka.Sasl.GSSAPI.AuthType = 1 - } - } - if cfg.Kafka.Sasl.GSSAPI.AuthType == 1 { + if configMap["useKeyTab"] != "true" { //Username and password - if username, ok := configMap["username"]; ok { - cfg.Kafka.Sasl.GSSAPI.Username = username - } - if password, ok := configMap["password"]; ok { - cfg.Kafka.Sasl.GSSAPI.Password = password - } + cfg.Kafka.Sasl.GSSAPI.AuthType = 1 + cfg.Kafka.Sasl.GSSAPI.Username = configMap["username"] + cfg.Kafka.Sasl.GSSAPI.Password = configMap["password"] } else { //Keytab - if keyTab, ok := configMap["keyTab"]; ok { - cfg.Kafka.Sasl.GSSAPI.KeyTabPath = keyTab - } + cfg.Kafka.Sasl.GSSAPI.AuthType = 2 + cfg.Kafka.Sasl.GSSAPI.KeyTabPath = configMap["keyTab"] if principal, ok := configMap["principal"]; ok { username := strings.Split(principal, "@")[0] realm := strings.Split(principal, "@")[1] cfg.Kafka.Sasl.GSSAPI.Username = username cfg.Kafka.Sasl.GSSAPI.Realm = realm } - if servicename, ok := cfg.Kafka.Security["sasl.kerberos.service.name"]; ok { - cfg.Kafka.Sasl.GSSAPI.ServiceName = servicename - } + cfg.Kafka.Sasl.GSSAPI.ServiceName = cfg.Kafka.Security["sasl.kerberos.service.name"] if cfg.Kafka.Sasl.GSSAPI.KerberosConfigPath == "" { cfg.Kafka.Sasl.GSSAPI.KerberosConfigPath = defaultKerberosConfigPath } } } else { // PLAIN, SCRAM-SHA-256 or SCRAM-SHA-512 - if username, ok := configMap["username"]; ok { - cfg.Kafka.Sasl.Username = username - } - if password, ok := configMap["password"]; ok { - cfg.Kafka.Sasl.Password = password - } + cfg.Kafka.Sasl.Username = configMap["username"] + cfg.Kafka.Sasl.Password = configMap["password"] } } } diff --git a/go.mod b/go.mod index 708e8ab7..28192e35 100644 --- a/go.mod +++ b/go.mod @@ -6,7 +6,7 @@ require ( github.com/ClickHouse/clickhouse-go/v2 v2.7.0 github.com/RoaringBitmap/roaring v1.2.3 github.com/avast/retry-go/v4 v4.3.3 - github.com/bytedance/sonic v1.8.5 + github.com/bytedance/sonic v1.8.6 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.27 github.com/google/uuid v1.3.0 @@ -18,7 +18,7 @@ require ( github.com/prometheus/common v0.42.0 github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.8.2 - github.com/thanos-io/thanos v0.30.2 + github.com/thanos-io/thanos v0.31.0 github.com/tidwall/gjson v1.14.4 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/twmb/franz-go v1.13.1 @@ -33,8 +33,8 @@ require ( ) require ( - github.com/ClickHouse/ch-go v0.53.1 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.250 // indirect + github.com/ClickHouse/ch-go v0.54.0 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.254 // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.5.0 // indirect @@ -72,7 +72,7 @@ require ( github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.1 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect - github.com/twmb/franz-go/pkg/kmsg v1.4.0 // indirect + github.com/twmb/franz-go/pkg/kmsg v1.5.0 // indirect go.opentelemetry.io/otel v1.14.0 // indirect go.opentelemetry.io/otel/trace v1.14.0 // indirect go.uber.org/atomic v1.10.0 // indirect diff --git a/go.sum b/go.sum index dd66a815..656f21b9 100644 --- a/go.sum +++ b/go.sum @@ -1,6 +1,6 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.53.1 h1:7Mxn/e0ZUzC/iZUyuHQIpi5WRLB5wzxKbsAG8h02bNk= -github.com/ClickHouse/ch-go v0.53.1/go.mod h1:2jvyjBRb5zhzFvcOBxPelzpbB9qsS47vwJssurJe2OA= +github.com/ClickHouse/ch-go v0.54.0 h1:WzPo/iZ8Gchb9Ze30TywMheOzKW5FkhTeZPxvz/iE6o= +github.com/ClickHouse/ch-go v0.54.0/go.mod h1:2jvyjBRb5zhzFvcOBxPelzpbB9qsS47vwJssurJe2OA= github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= github.com/ClickHouse/clickhouse-go/v2 v2.7.0 h1:KFRvFjnewYkJBwkfBvDYESwZtZmQipz/xRuaBz0oVNA= github.com/ClickHouse/clickhouse-go/v2 v2.7.0/go.mod h1:6I79Gj2EPbV/DdlDShfCaxrja/pxLVSfDrvEEQp77VE= @@ -13,8 +13,10 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.250 h1:hkF8iofQ8lEd/uXVg75JkWjjCcWhcY1DUg4fmLpKLD4= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.250/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.253 h1:hm1L9yYlK1uUy+/UgawSkuecMCCtXJsGvOJN/psM94M= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.253/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.254 h1:JyMWp7SrNpLjjWlRAY7WhSdAbwpg+4ia4jaegf1aD4U= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.254/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/avast/retry-go/v4 v4.3.3 h1:G56Bp6mU0b5HE1SkaoVjscZjlQb0oy4mezwY/cGH19w= @@ -30,8 +32,8 @@ github.com/bits-and-blooms/bitset v1.5.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edY github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.8.5 h1:kjX0/vo5acEQ/sinD/18SkA/lDDUk23F0RcaHvI7omc= -github.com/bytedance/sonic v1.8.5/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/bytedance/sonic v1.8.6 h1:aUgO9S8gvdN6SyW2EhIpAw5E4ChworywIEndZCkCVXk= +github.com/bytedance/sonic v1.8.6/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= @@ -43,7 +45,7 @@ github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ3 github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/efficientgo/core v1.0.0-rc.0.0.20221201130417-ba593f67d2a4 h1:rydBwnBoywKQMjWF0z8SriYtQ+uUcaFsxuijMjJr5PI= +github.com/efficientgo/core v1.0.0-rc.2 h1:7j62qHLnrZqO3V3UA0AqOGd5d5aXV3AX6m/NZBHp78I= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= @@ -210,8 +212,8 @@ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/thanos-io/thanos v0.30.2 h1:JcazfBlpX/i747uRxBWHJMopX0g+eSaZYGjqMeQIkPo= -github.com/thanos-io/thanos v0.30.2/go.mod h1:k23ZcbQSK0Jd/siuEti6J6/3DeezIk+YLRBcYikzXl8= +github.com/thanos-io/thanos v0.31.0 h1:jOCmaiIXwpByWXoVtHnktLm3YDB9xDQQzmZvd1XG5oY= +github.com/thanos-io/thanos v0.31.0/go.mod h1:5ux+jb2oKr59+3XsCC0mX+JuAbPGJEMijjhcmnL/PMo= github.com/tidwall/gjson v1.14.4 h1:uo0p8EbA09J7RQaflQ1aBRffTR7xedD2bcIVSYxLnkM= github.com/tidwall/gjson v1.14.4/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= @@ -230,8 +232,8 @@ github.com/twmb/franz-go v1.13.1/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR github.com/twmb/franz-go/pkg/kadm v1.8.0 h1:vvKwZpxYn+VmM32v4mKkecHLKavZW+HcYLRKKxly5ZY= github.com/twmb/franz-go/pkg/kadm v1.8.0/go.mod h1:qUSM7pxoMCU1UNu5H4USE64ODcVmeG9LS96mysv1nu8= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/kmsg v1.4.0 h1:tbp9hxU6m8qZhQTlpGiaIJOm4BXix5lsuEZ7K00dF0s= -github.com/twmb/franz-go/pkg/kmsg v1.4.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= +github.com/twmb/franz-go/pkg/kmsg v1.5.0 h1:eqVJquFQLdBNLrRMWX03pPDPpngn6PTjGZLlZnagouk= +github.com/twmb/franz-go/pkg/kmsg v1.5.0/go.mod h1:se9Mjdt0Nwzc9lnjJ0HyDtLyBnaBDAd7pCje47OhSyw= github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 h1:alKdbddkPw3rDh+AwmUEwh6HNYgTvDSFIe/GWYRR9RM= github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0/go.mod h1:k8BoBjyUbFj34f0rRbn+Ky12sZFAPbmShrg0karAIMo= github.com/twmb/franz-go/plugin/kzap v1.1.2 h1:0arX5xJ0soUPX1LlDay6ZZoxuWkWk1lggQ5M/IgRXAE= From 40a5f672ba2f0d421a963eb4e01673b00b2b5499 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 23 Mar 2023 15:32:03 +0800 Subject: [PATCH 331/404] perf - avoid creating kafka client everytime when caculate lags --- config_manager/lags.go | 59 ++++++++++++++++++----------------------- config_manager/nacos.go | 2 ++ go.sum | 2 -- 3 files changed, 28 insertions(+), 35 deletions(-) diff --git a/config_manager/lags.go b/config_manager/lags.go index 266b2fd9..89d883c7 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -2,6 +2,7 @@ package rcm import ( "context" + "reflect" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/input" @@ -11,6 +12,12 @@ import ( "github.com/twmb/franz-go/pkg/kgo" ) +var ( + theCl *kgo.Client + theAdm *kadm.Client + kafkaConfig *config.KafkaConfig +) + type StateLag struct { State string Lag int64 @@ -18,13 +25,10 @@ type StateLag struct { // GetTaskStateAndLags get state and lag of all tasks. func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err error) { - var cl *kgo.Client - var adm *kadm.Client - if cl, adm, err = newClient(cfg); err != nil { + _, adm, err := newClient(cfg.Kafka) + if err != nil { return } - defer adm.Close() - defer cl.Close() stateLags = make(map[string]StateLag, len(cfg.Tasks)) for _, taskCfg := range cfg.Tasks { @@ -39,45 +43,34 @@ func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err return } -// GetTaskStateAndLag get state and lag of a task. -func GetTaskStateAndLag(cfg *config.Config, taskName string) (stateLag StateLag, err error) { - var cl *kgo.Client - var adm *kadm.Client - if cl, adm, err = newClient(cfg); err != nil { - return - } - defer adm.Close() - defer cl.Close() - - var taskCfg *config.TaskConfig - for _, tskCfg := range cfg.Tasks { - if tskCfg.Name == taskName { - taskCfg = tskCfg - break - } - } - if taskCfg == nil { - err = errors.Newf("task %q doesn't exist", taskName) - return +func cleanupKafkaClient() { + if theCl != nil { + theCl.Close() } - if stateLag.State, stateLag.Lag, err = getStateAndLag(adm, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { - return + if theAdm != nil { + theAdm.Close() } - return } -func newClient(cfg *config.Config) (cl *kgo.Client, adm *kadm.Client, err error) { +func newClient(cfg config.KafkaConfig) (cl *kgo.Client, adm *kadm.Client, err error) { var opts []kgo.Opt - if opts, err = input.GetFranzConfig(&cfg.Kafka); err != nil { + if reflect.DeepEqual(&cfg, kafkaConfig) { + return theCl, theAdm, nil + } + + cleanupKafkaClient() + + kafkaConfig = &cfg + if opts, err = input.GetFranzConfig(&cfg); err != nil { return } // franz.config.go 379 - invalid autocommit options specified when a group was not specified - if cl, err = kgo.NewClient(opts...); err != nil { + if theCl, err = kgo.NewClient(opts...); err != nil { err = errors.Wrapf(err, "") return } - adm = kadm.NewClient(cl) - return + theAdm = kadm.NewClient(theCl) + return theCl, theAdm, err } // getStateAndLag is inspired by https://github.com/cloudhut/kminion/blob/1ffd02ba94a5edc26d4f11e57191ed3479d8a111/prometheus/collect_consumer_group_lags.go diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 88087f9a..0e0805c5 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -270,6 +270,8 @@ func (ncm *NacosConfManager) Stop() { if err = ncm.namingClient.Unsubscribe(&subParam); err != nil { util.Logger.Error("ncm.namingClient.Unsubscribe failed", zap.Error(err)) } + cleanupKafkaClient() + util.Logger.Info("stopped nacos config manager") } diff --git a/go.sum b/go.sum index 656f21b9..a7e1b5b5 100644 --- a/go.sum +++ b/go.sum @@ -13,8 +13,6 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.253 h1:hm1L9yYlK1uUy+/UgawSkuecMCCtXJsGvOJN/psM94M= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.253/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/aliyun/alibaba-cloud-sdk-go v1.62.254 h1:JyMWp7SrNpLjjWlRAY7WhSdAbwpg+4ia4jaegf1aD4U= github.com/aliyun/alibaba-cloud-sdk-go v1.62.254/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= From 0b28e445286bcec8d8c67877ad75f8cecb167823 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 23 Mar 2023 18:35:04 +0800 Subject: [PATCH 332/404] fix - avoid program running into stuck when facing fatalpanic --- CHANGELOG.md | 6 ++++++ cmd/kafka_gen_log/main.go | 4 +--- config_manager/lags.go | 25 ++++++++++++------------- docs/configuration/config.md | 2 -- go.sum | 2 -- task/sinker.go | 14 ++++++++------ 6 files changed, 27 insertions(+), 26 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 82e9fae3..ab0e9d40 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,6 +4,12 @@ Improvements: - Have writingpool per shard to avoid ErrAcquireConnTimeout +- Do not create kafka client everytime when caculate lags +- Support configuring PlainloginModule in kafka.security section + +Bug Fixes: +- Avoid program running into stuck when facing fatalpanic + #### Version 3.0.2 (2023-03-13) diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 34c8677b..8bb86be5 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -151,9 +151,7 @@ func (g *LogGenerator) Init() error { g.lineno = 0 fnPatt := regexp.MustCompile(LogfilePattern) d, err := os.Open(LogfileDir) - defer func() { - d.Close() - }() + defer d.Close() if err != nil { err = errors.Wrapf(err, "") return err diff --git a/config_manager/lags.go b/config_manager/lags.go index 89d883c7..d5881116 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -25,16 +25,20 @@ type StateLag struct { // GetTaskStateAndLags get state and lag of all tasks. func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err error) { - _, adm, err := newClient(cfg.Kafka) - if err != nil { - return + kconf := cfg.Kafka + if !reflect.DeepEqual(&kconf, kafkaConfig) { + cleanupKafkaClient() + if err = newClient(cfg.Kafka); err != nil { + return + } + kafkaConfig = &kconf } stateLags = make(map[string]StateLag, len(cfg.Tasks)) for _, taskCfg := range cfg.Tasks { var state string var totalLags int64 - if state, totalLags, err = getStateAndLag(adm, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { + if state, totalLags, err = getStateAndLag(theAdm, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { return } stateLags[taskCfg.Name] = StateLag{State: state, Lag: totalLags} @@ -46,21 +50,16 @@ func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err func cleanupKafkaClient() { if theCl != nil { theCl.Close() + theCl = nil } if theAdm != nil { theAdm.Close() + theAdm = nil } } -func newClient(cfg config.KafkaConfig) (cl *kgo.Client, adm *kadm.Client, err error) { +func newClient(cfg config.KafkaConfig) (err error) { var opts []kgo.Opt - if reflect.DeepEqual(&cfg, kafkaConfig) { - return theCl, theAdm, nil - } - - cleanupKafkaClient() - - kafkaConfig = &cfg if opts, err = input.GetFranzConfig(&cfg); err != nil { return } @@ -70,7 +69,7 @@ func newClient(cfg config.KafkaConfig) (cl *kgo.Client, adm *kadm.Client, err er return } theAdm = kadm.NewClient(theCl) - return theCl, theAdm, err + return } // getStateAndLag is inspired by https://github.com/cloudhut/kminion/blob/1ffd02ba94a5edc26d4f11e57191ed3479d8a111/prometheus/collect_consumer_group_lags.go diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 268fc310..e0ef501a 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -89,8 +89,6 @@ "task": { "name": "test_dynamic_schema", - // kafka client, possible values: sarama, kafka-go. (defaults to sarama) - "kafkaClient": "sarama", // kafka topic "topic": "topic", // kafka consume from earliest or latest diff --git a/go.sum b/go.sum index a7e1b5b5..d3909743 100644 --- a/go.sum +++ b/go.sum @@ -1,7 +1,6 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/ClickHouse/ch-go v0.54.0 h1:WzPo/iZ8Gchb9Ze30TywMheOzKW5FkhTeZPxvz/iE6o= github.com/ClickHouse/ch-go v0.54.0/go.mod h1:2jvyjBRb5zhzFvcOBxPelzpbB9qsS47vwJssurJe2OA= -github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= github.com/ClickHouse/clickhouse-go/v2 v2.7.0 h1:KFRvFjnewYkJBwkfBvDYESwZtZmQipz/xRuaBz0oVNA= github.com/ClickHouse/clickhouse-go/v2 v2.7.0/go.mod h1:6I79Gj2EPbV/DdlDShfCaxrja/pxLVSfDrvEEQp77VE= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= @@ -32,7 +31,6 @@ github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx2 github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= github.com/bytedance/sonic v1.8.6 h1:aUgO9S8gvdN6SyW2EhIpAw5E4ChworywIEndZCkCVXk= github.com/bytedance/sonic v1.8.6/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= -github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= diff --git a/task/sinker.go b/task/sinker.go index 26d12b40..6c32f736 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -99,14 +99,12 @@ func (s *Sinker) GetCurrentConfig() *config.Config { func (s *Sinker) Run() { var err error var newCfg *config.Config - defer func() { - s.exitCh <- struct{}{} - }() + if s.cmdOps.PushGatewayAddrs != "" { addrs := strings.Split(s.cmdOps.PushGatewayAddrs, ",") s.pusher = statistics.NewPusher(addrs, s.cmdOps.PushInterval, s.httpAddr) if err = s.pusher.Init(); err != nil { - util.Logger.Error("failed to initialize connection to the specified push gateway address", zap.Error(err)) + util.Logger.Fatal("failed to initialize connection to the specified push gateway address", zap.Error(err)) return } go s.pusher.Run() @@ -136,11 +134,12 @@ func (s *Sinker) Run() { util.Logger.Fatal("s.applyConfig failed", zap.Error(err)) return } + LOOP: for { select { case <-s.ctx.Done(): util.Logger.Info("Sinker.Run quit due to context has been canceled") - return + break LOOP case c := <-s.consumerRestartCh: // only restart the consumer which was not changed in applyAnotherConfig if c == s.consumers[c.grpConfig.Name] { @@ -171,11 +170,12 @@ func (s *Sinker) Run() { // Golang <-time.After() is not garbage collected before expiry. ticker := time.NewTicker(10 * time.Second) defer ticker.Stop() + WORKLOOP: for { select { case <-s.ctx.Done(): util.Logger.Info("Sinker.Run quit due to context has been canceled") - return + break WORKLOOP case <-ticker.C: if newCfg, err = s.rcm.GetConfig(); err != nil { util.Logger.Error("s.rcm.GetConfig failed", zap.Error(err)) @@ -217,6 +217,8 @@ func (s *Sinker) Run() { } } } + + s.exitCh <- struct{}{} } // Close shutdown task From 05af994fa99a56ed14798012daca47078fe4ac73 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 7 Apr 2023 15:44:04 +0800 Subject: [PATCH 333/404] Automatically end sinker if it remains inactive in last 10 mins --- CHANGELOG.md | 8 +++++++- config/config.go | 1 + go.mod | 16 ++++++++-------- go.sum | 32 ++++++++++++++++++++++++++------ input/kafka_franz.go | 11 ++++++++++- model/value.go | 2 +- 6 files changed, 53 insertions(+), 17 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ab0e9d40..cbc7e521 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,12 @@ # Changelog -#### Version 3.0.3 (not released yet) +#### Version 3.0.4 (TBD) + +Improvements: +- Automatically end sinker if it remains inactive in last 10 mins + + +#### Version 3.0.3 (2023-03-30) Improvements: - Have writingpool per shard to avoid ErrAcquireConnTimeout diff --git a/config/config.go b/config/config.go index acec16e7..fb465a56 100644 --- a/config/config.go +++ b/config/config.go @@ -319,6 +319,7 @@ func (cfg *Config) normallizeTask(taskCfg *TaskConfig) (err error) { taskCfg.PromLabelsBlackList = "" } if taskCfg.DynamicSchema.Enable { + taskCfg.AutoSchema = true if taskCfg.Parser != "fastjson" && taskCfg.Parser != "gjson" { err = errors.Newf("Parser %s doesn't support DynamicSchema", taskCfg.Parser) return diff --git a/go.mod b/go.mod index 28192e35..638f5b2a 100644 --- a/go.mod +++ b/go.mod @@ -3,10 +3,10 @@ module github.com/housepower/clickhouse_sinker go 1.20 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.7.0 + github.com/ClickHouse/clickhouse-go/v2 v2.8.3 github.com/RoaringBitmap/roaring v1.2.3 github.com/avast/retry-go/v4 v4.3.3 - github.com/bytedance/sonic v1.8.6 + github.com/bytedance/sonic v1.8.7 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.27 github.com/google/uuid v1.3.0 @@ -21,7 +21,7 @@ require ( github.com/thanos-io/thanos v0.31.0 github.com/tidwall/gjson v1.14.4 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.13.1 + github.com/twmb/franz-go v1.13.2 github.com/twmb/franz-go/pkg/kadm v1.8.0 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.2 @@ -34,7 +34,7 @@ require ( require ( github.com/ClickHouse/ch-go v0.54.0 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.254 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.272 // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.5.0 // indirect @@ -52,7 +52,7 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.16.3 // indirect + github.com/klauspost/compress v1.16.4 // indirect github.com/klauspost/cpuid/v2 v2.2.4 // indirect github.com/kr/text v0.2.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect @@ -76,12 +76,12 @@ require ( go.opentelemetry.io/otel v1.14.0 // indirect go.opentelemetry.io/otel/trace v1.14.0 // indirect go.uber.org/atomic v1.10.0 // indirect - go.uber.org/multierr v1.10.0 // indirect + go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.3.0 // indirect golang.org/x/crypto v0.7.0 // indirect - golang.org/x/net v0.8.0 // indirect + golang.org/x/net v0.9.0 // indirect golang.org/x/sync v0.1.0 // indirect - golang.org/x/sys v0.6.0 // indirect + golang.org/x/sys v0.7.0 // indirect google.golang.org/protobuf v1.30.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index d3909743..29cbd7ee 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,11 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/ClickHouse/ch-go v0.54.0 h1:WzPo/iZ8Gchb9Ze30TywMheOzKW5FkhTeZPxvz/iE6o= github.com/ClickHouse/ch-go v0.54.0/go.mod h1:2jvyjBRb5zhzFvcOBxPelzpbB9qsS47vwJssurJe2OA= -github.com/ClickHouse/clickhouse-go/v2 v2.7.0 h1:KFRvFjnewYkJBwkfBvDYESwZtZmQipz/xRuaBz0oVNA= -github.com/ClickHouse/clickhouse-go/v2 v2.7.0/go.mod h1:6I79Gj2EPbV/DdlDShfCaxrja/pxLVSfDrvEEQp77VE= +github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= +github.com/ClickHouse/clickhouse-go/v2 v2.8.0 h1:NkqldIAAhYMVwPZIMtpdNQbgqB1eWDP5YjpsMDQ83bU= +github.com/ClickHouse/clickhouse-go/v2 v2.8.0/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= +github.com/ClickHouse/clickhouse-go/v2 v2.8.3 h1:R6na3RNq/4vEEwfwkxQYrWOf21T9HMhGmE8mhkhq7TI= +github.com/ClickHouse/clickhouse-go/v2 v2.8.3/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.2.3 h1:yqreLINqIrX22ErkKI0vY47/ivtJr6n+kMhVOVmhWBY= @@ -12,8 +15,14 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.254 h1:JyMWp7SrNpLjjWlRAY7WhSdAbwpg+4ia4jaegf1aD4U= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.254/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.264 h1:CJFO65adjRjcvl1Uo84EKj8tlWqDpn0pSRdUrX/+uRw= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.264/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.267 h1:p0SMV+o/NLPeaIJYTnIwEvNQdLrOxQ97EFfo7pYkQpM= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.267/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.271 h1:0QmSDMovuCyUbYp70MZHoTi/GYnHb/wYEIIBqoVsCjs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.271/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.272 h1:uebsAO24XxrtZWqK3fwyqaq4WZfdAAVftjxOUVpJRHc= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.272/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/avast/retry-go/v4 v4.3.3 h1:G56Bp6mU0b5HE1SkaoVjscZjlQb0oy4mezwY/cGH19w= @@ -31,6 +40,9 @@ github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx2 github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= github.com/bytedance/sonic v1.8.6 h1:aUgO9S8gvdN6SyW2EhIpAw5E4ChworywIEndZCkCVXk= github.com/bytedance/sonic v1.8.6/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/bytedance/sonic v1.8.7 h1:d3sry5vGgVq/OpgozRUNP6xBsSo0mtNdwliApw+SAMQ= +github.com/bytedance/sonic v1.8.7/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -118,6 +130,8 @@ github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47e github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.16.3 h1:XuJt9zzcnaz6a16/OU53ZjWp/v7/42WcR5t2a0PcNQY= github.com/klauspost/compress v1.16.3/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.16.4 h1:91KN02FnsOYhuunwU4ssRe8lc2JosWmizWa91B5v1PU= +github.com/klauspost/compress v1.16.4/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= @@ -225,6 +239,8 @@ github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2 github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= github.com/twmb/franz-go v1.13.1 h1:7dDuAjJItbZRjgmMzYzCfJtRMpVKfQ57yIG8SkZz24M= github.com/twmb/franz-go v1.13.1/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= +github.com/twmb/franz-go v1.13.2 h1:jIdDoFiq8uP3Zrx6TZZTXpaXrv3bh1w3tV5mn/B+Gw8= +github.com/twmb/franz-go v1.13.2/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= github.com/twmb/franz-go/pkg/kadm v1.8.0 h1:vvKwZpxYn+VmM32v4mKkecHLKavZW+HcYLRKKxly5ZY= github.com/twmb/franz-go/pkg/kadm v1.8.0/go.mod h1:qUSM7pxoMCU1UNu5H4USE64ODcVmeG9LS96mysv1nu8= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= @@ -258,8 +274,8 @@ go.uber.org/atomic v1.10.0 h1:9qC72Qh0+3MqyJbAn8YU5xVq1frD8bn3JtD2oXtafVQ= go.uber.org/atomic v1.10.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= -go.uber.org/multierr v1.10.0 h1:S0h4aNzvfcFsC3dRF1jLoaov7oRaKqRGC/pUEJ2yvPQ= -go.uber.org/multierr v1.10.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= +go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= +go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= @@ -303,6 +319,8 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0 h1:Zrh2ngAOFYneWTAIAPethzeaQLuHwhuBkuV6ZiRnUaQ= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= +golang.org/x/net v0.9.0 h1:aWJ/m6xSmxWBx+V0XRHTlrYrPG56jKsLdTFmsSsCzOM= +golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -329,6 +347,8 @@ golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.7.0 h1:3jlCCIQZPdOYu1h8BkNvLz8Kgwtae2cagcG/VamtZRU= +golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 1c9b0205..378e98e1 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -45,6 +45,7 @@ const ( Krb5KeytabAuth = 2 CommitRetries = 6 RetryBackoff = 5 * time.Second + processTimeOut = 10 ) // KafkaFranz implements input.Inputer @@ -184,7 +185,15 @@ func (k *KafkaFranz) Run() { } util.Logger.Debug("Records fetched", zap.String("records", strconv.Itoa(fetches.NumRecords())), zap.String("consumer group", k.grpConfig.Name)) - k.fetch <- &fetches + + // Automatically end the program if it remains inactive for a specific duration of time. + t := time.NewTimer(processTimeOut * time.Minute) + select { + case k.fetch <- &fetches: + t.Stop() + case <-t.C: + util.Logger.Fatal(fmt.Sprintf("Sinker abort because group %s was not processing in last %d minutes", k.grpConfig.Name, processTimeOut)) + } } k.cl.Close() // will trigger k.onPartitionRevoked util.Logger.Info("KafkaFranz.Run quit due to context has been canceled", zap.String("consumer group", k.grpConfig.Name)) diff --git a/model/value.go b/model/value.go index 09bd7b7f..2eda1ab2 100644 --- a/model/value.go +++ b/model/value.go @@ -181,7 +181,7 @@ func WhichType(typ string) (ti *TypeInfo) { typeInfo[origTyp] = ti return ti } else { - util.Logger.Fatal(fmt.Sprintf("ClickHouse column type %v is not inside supported ones: %v", origTyp, typeInfo)) + util.Logger.Fatal(fmt.Sprintf("ClickHouse column type %v is not inside supported ones(case-sensitive): %v", origTyp, typeInfo)) } ti = &TypeInfo{Type: dataType, Nullable: nullable, Array: array} typeInfo[origTyp] = ti From 7b0e8331e61800cd1169852d19baacbee52ea6dd Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Mon, 10 Apr 2023 10:16:54 +0800 Subject: [PATCH 334/404] chore - no more input when context canceled --- input/kafka_franz.go | 4 ++++ output/clickhouse.go | 25 ++++--------------------- output/clickhouse_util.go | 13 +------------ 3 files changed, 9 insertions(+), 33 deletions(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 378e98e1..87d4c4a7 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -173,6 +173,7 @@ func GetFranzConfig(kfkCfg *config.KafkaConfig) (opts []kgo.Opt, err error) { func (k *KafkaFranz) Run() { k.wgRun.Add(1) defer k.wgRun.Done() +LOOP: for { fetches := k.cl.PollRecords(k.ctx, k.grpConfig.BufferSize) err := fetches.Err() @@ -191,6 +192,9 @@ func (k *KafkaFranz) Run() { select { case k.fetch <- &fetches: t.Stop() + case <-k.ctx.Done(): + t.Stop() + break LOOP case <-t.C: util.Logger.Fatal(fmt.Sprintf("Sinker abort because group %s was not processing in last %d minutes", k.grpConfig.Name, processTimeOut)) } diff --git a/output/clickhouse.go b/output/clickhouse.go index aa05aaed..013cb743 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -239,10 +239,8 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, sc *pool.ShardConn) { retry.RetryIf(func(err error) bool { return shouldReconnect(err, sc) }), retry.OnRetry(func(n uint, err error) { retrycount++ - if !errors.Is(err, clickhouse.ErrAcquireConnTimeout) { - util.Logger.Error("flush batch failed", zap.String("task", c.taskCfg.Name), zap.Int("try", int(retrycount)), zap.Error(err)) - statistics.FlushMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) - } + util.Logger.Error("flush batch failed", zap.String("task", c.taskCfg.Name), zap.Int("try", int(retrycount)), zap.Error(err)) + statistics.FlushMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) }), ); err != nil { util.Logger.Fatal("ClickHouse.loopWrite failed", zap.String("task", c.taskCfg.Name), zap.Error(err)) @@ -478,13 +476,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { alterTable := func(tbl, col string) error { query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, tbl, onCluster, col) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) - return retry.Do( - func() error { return conn.Exec(context.Background(), query) }, - retry.Attempts(0), - retry.LastErrorOnly(true), - retry.Delay(10*time.Second), - retry.RetryIf(func(err error) bool { return errors.Is(err, clickhouse.ErrAcquireConnTimeout) }), - ) + return conn.Exec(context.Background(), query) } if len(alterSeries) != 0 { @@ -527,16 +519,7 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { c.dbName, chCfg.Cluster, c.dbName, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows driver.Rows - if err = retry.Do( - func() error { - rows, err = conn.Query(context.Background(), query) - return err - }, - retry.Attempts(0), - retry.LastErrorOnly(true), - retry.Delay(10*time.Second), - retry.RetryIf(func(err error) bool { return errors.Is(err, clickhouse.ErrAcquireConnTimeout) }), - ); err != nil { + if rows, err = conn.Query(context.Background(), query); err != nil { err = errors.Wrapf(err, "") return } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index f4481ed9..8280257a 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -3,12 +3,10 @@ package output import ( "context" "fmt" - "time" "github.com/ClickHouse/clickhouse-go/v2" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/RoaringBitmap/roaring" - "github.com/avast/retry-go/v4" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/util" @@ -82,16 +80,7 @@ func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn cl func getDims(database, table string, excludedColumns []string, parser string, conn clickhouse.Conn) (dims []*model.ColumnWithType, err error) { var rs driver.Rows - if err = retry.Do( - func() error { - rs, err = conn.Query(context.Background(), fmt.Sprintf(selectSQLTemplate, database, table)) - return err - }, - retry.Attempts(0), - retry.LastErrorOnly(true), - retry.Delay(10*time.Second), - retry.RetryIf(func(err error) bool { return errors.Is(err, clickhouse.ErrAcquireConnTimeout) }), - ); err != nil { + if rs, err = conn.Query(context.Background(), fmt.Sprintf(selectSQLTemplate, database, table)); err != nil { err = errors.Wrapf(err, "") return } From 8125e26c9aba4cbc5807175a45de9d1b8b3edb18 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 21 Apr 2023 14:47:51 +0800 Subject: [PATCH 335/404] chore - update the makefile to calculate case coverage --- CHANGELOG.md | 2 +- Dockerfile | 2 +- Dockerfile_goreleaser | 6 ++--- Makefile | 51 +++++++++++++++++++++++++++++++++++--- cmd/kafka_gen_log/main.go | 7 +++++- cmd/kafka_gen_prom/main.go | 2 +- docs/configuration/flag.md | 2 +- go.mod | 10 ++++---- go.sum | 42 ++++++++----------------------- 9 files changed, 77 insertions(+), 47 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cbc7e521..2d808478 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,6 @@ # Changelog -#### Version 3.0.4 (TBD) +#### Version 3.0.4 (2023-04-18) Improvements: - Automatically end sinker if it remains inactive in last 10 mins diff --git a/Dockerfile b/Dockerfile index 96ce6662..a54d5293 100644 --- a/Dockerfile +++ b/Dockerfile @@ -13,7 +13,7 @@ COPY --from=builder /app/nacos_publish_config /usr/local/bin/nacos_publish_confi COPY --from=builder /app/kafka_gen_log /usr/local/bin/kafka_gen_log COPY --from=builder /app/kafka_gen_metric /usr/local/bin/kafka_gen_metric -# clickhouse_sinker gets config from local file "/etc/clickhouse_sinker.json" by default. +# clickhouse_sinker gets config from local file "/etc/clickhouse_sinker.hjson" by default. # Customize behavior with following env variables: # - V # - LOG_LEVEL diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index 8c4b6160..d00ac98b 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -1,10 +1,10 @@ FROM alpine:latest RUN apk --no-cache add ca-certificates tzdata RUN echo "UTC" > /etc/timezone -ADD dist/clickhouse_sinker_linux_amd64/clickhouse_sinker /usr/local/bin/clickhouse_sinker -ADD dist/nacos_publish_config_linux_amd64/nacos_publish_config /usr/local/bin/nacos_publish_config +ADD dist/clickhouse_sinker_linux_amd64_v1/clickhouse_sinker /usr/local/bin/clickhouse_sinker +ADD dist/nacos_publish_config_linux_amd64_v1/nacos_publish_config /usr/local/bin/nacos_publish_config -# clickhouse_sinker gets config from local file "/etc/clickhouse_sinker.json" by default. +# clickhouse_sinker gets config from local file "/etc/clickhouse_sinker.hjson" by default. # Customize behavior with following env variables: # - V # - LOG_LEVEL diff --git a/Makefile b/Makefile index 3367e417..78b4718c 100644 --- a/Makefile +++ b/Makefile @@ -1,26 +1,71 @@ -SINKER_LDFLAGS += -X "main.version=$(shell git describe --tags --dirty)" +VERSION=$(shell git describe --tags --dirty) +SINKER_LDFLAGS += -X "main.version=$(VERSION)" SINKER_LDFLAGS += -X "main.date=$(shell date --iso-8601=s)" SINKER_LDFLAGS += -X "main.commit=$(shell git rev-parse HEAD)" SINKER_LDFLAGS += -X "main.builtBy=$(shell echo `whoami`@`hostname`)" +DEFAULT_CFG_PATH = /etc/clickhouse_sinker.hjson +IMG_TAGGED = hub.eoitek.net/storage/clickhouse_sinker:${VERSION} +IMG_LATEST = hub.eoitek.net/storage/clickhouse_sinker:latest +export GOPROXY=https://goproxy.cn,direct GO := CGO_ENABLED=0 go GOBUILD := $(GO) build $(BUILD_FLAG) + +.PHONY: pre pre: go mod tidy - go get -u ./... + +.PHONY: build build: pre $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o . ./... + +.PHONY: debug debug: pre $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o . ./... + +.PHONY: unittest unittest: pre go test -v ./... + +.PHONY: benchtest benchtest: pre go test -bench=. ./... + +.PHONY: systest systest: build bash go.test.sh bash go.metrictest.sh + +.PHONY: coverage +coverage: + go test ./... -coverprofile=coverage.txt -covermode count + go tool cover -func coverage.txt + +.PHONY: lint lint: - golangci-lint run --timeout=3m + golangci-lint run -D errcheck,govet,gosimple + +.PHONY: run run: pre go run cmd/clickhouse_sinker/main.go --local-cfg-file docker/test_dynamic_schema.hjson + +.PHONY: release +release: + goreleaser release --skip-publish --clean + +.PHONY: docker-build +docker-build: release + docker build . -t clickhouse_sinker:${VERSION} -f Dockerfile_goreleaser + docker tag clickhouse_sinker:${VERSION} ${IMG_TAGGED} + docker tag clickhouse_sinker:${VERSION} ${IMG_LATEST} + docker rmi clickhouse_sinker:${VERSION} + +.PHONY: docker-push +docker-push: + docker push ${IMG_TAGGED} + docker push ${IMG_LATEST} + +.PHONY: docker-run +docker-run: + docker run -d -v ${DEFAULT_CFG_PATH}:${DEFAULT_CFG_PATH} ${IMG_LATEST} diff --git a/cmd/kafka_gen_log/main.go b/cmd/kafka_gen_log/main.go index 8bb86be5..3654d4a3 100644 --- a/cmd/kafka_gen_log/main.go +++ b/cmd/kafka_gen_log/main.go @@ -41,6 +41,7 @@ import ( "context" "flag" "fmt" + "log" "math/rand" "os" "os/signal" @@ -151,11 +152,15 @@ func (g *LogGenerator) Init() error { g.lineno = 0 fnPatt := regexp.MustCompile(LogfilePattern) d, err := os.Open(LogfileDir) - defer d.Close() if err != nil { err = errors.Wrapf(err, "") return err } + defer func() { + if err := d.Close(); err != nil { + log.Printf("error closing file: %v", err) + } + }() fis, err := d.Readdir(0) if err != nil { err = errors.Wrapf(err, "") diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index 649748d3..b96540aa 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -165,7 +165,7 @@ func randValue() (val string) { } func randBool() bool { - rand.Seed(time.Now().UnixNano()) + rand.New(rand.NewSource(time.Now().UnixNano())) return rand.Intn(2) == 1 } diff --git a/docs/configuration/flag.md b/docs/configuration/flag.md index 7c92417e..9221fde3 100644 --- a/docs/configuration/flag.md +++ b/docs/configuration/flag.md @@ -7,7 +7,7 @@ Usage of ./clickhouse_sinker: -http-port int http listen port (default 2112) -local-cfg-file string - local config file (default "/etc/clickhouse_sinker.json") + local config file (default "/etc/clickhouse_sinker.hjson") -metric-push-gateway-addrs string a list of comma-separated prometheus push gatway address -nacos-addr string diff --git a/go.mod b/go.mod index 638f5b2a..64e08492 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module github.com/housepower/clickhouse_sinker go 1.20 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.8.3 + github.com/ClickHouse/clickhouse-go/v2 v2.9.0 github.com/RoaringBitmap/roaring v1.2.3 github.com/avast/retry-go/v4 v4.3.3 github.com/bytedance/sonic v1.8.7 @@ -14,7 +14,7 @@ require ( github.com/jcmturner/gokrb5/v8 v8.4.4 github.com/jinzhu/copier v0.3.5 github.com/nacos-group/nacos-sdk-go v1.1.4 - github.com/prometheus/client_golang v1.14.0 + github.com/prometheus/client_golang v1.15.0 github.com/prometheus/common v0.42.0 github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.8.2 @@ -34,7 +34,7 @@ require ( require ( github.com/ClickHouse/ch-go v0.54.0 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.272 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.282 // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.5.0 // indirect @@ -52,7 +52,7 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.16.4 // indirect + github.com/klauspost/compress v1.16.5 // indirect github.com/klauspost/cpuid/v2 v2.2.4 // indirect github.com/kr/text v0.2.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect @@ -78,7 +78,7 @@ require ( go.uber.org/atomic v1.10.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.3.0 // indirect - golang.org/x/crypto v0.7.0 // indirect + golang.org/x/crypto v0.8.0 // indirect golang.org/x/net v0.9.0 // indirect golang.org/x/sync v0.1.0 // indirect golang.org/x/sys v0.7.0 // indirect diff --git a/go.sum b/go.sum index 29cbd7ee..4ec02851 100644 --- a/go.sum +++ b/go.sum @@ -1,11 +1,8 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/ClickHouse/ch-go v0.54.0 h1:WzPo/iZ8Gchb9Ze30TywMheOzKW5FkhTeZPxvz/iE6o= github.com/ClickHouse/ch-go v0.54.0/go.mod h1:2jvyjBRb5zhzFvcOBxPelzpbB9qsS47vwJssurJe2OA= -github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= -github.com/ClickHouse/clickhouse-go/v2 v2.8.0 h1:NkqldIAAhYMVwPZIMtpdNQbgqB1eWDP5YjpsMDQ83bU= -github.com/ClickHouse/clickhouse-go/v2 v2.8.0/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= -github.com/ClickHouse/clickhouse-go/v2 v2.8.3 h1:R6na3RNq/4vEEwfwkxQYrWOf21T9HMhGmE8mhkhq7TI= -github.com/ClickHouse/clickhouse-go/v2 v2.8.3/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= +github.com/ClickHouse/clickhouse-go/v2 v2.9.0 h1:vh0D+9p5hKWLYqCfU8Hd7/GMXNfHC84isUf7sNlJrZk= +github.com/ClickHouse/clickhouse-go/v2 v2.9.0/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.2.3 h1:yqreLINqIrX22ErkKI0vY47/ivtJr6n+kMhVOVmhWBY= @@ -15,14 +12,8 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.264 h1:CJFO65adjRjcvl1Uo84EKj8tlWqDpn0pSRdUrX/+uRw= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.264/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.267 h1:p0SMV+o/NLPeaIJYTnIwEvNQdLrOxQ97EFfo7pYkQpM= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.267/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.271 h1:0QmSDMovuCyUbYp70MZHoTi/GYnHb/wYEIIBqoVsCjs= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.271/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.272 h1:uebsAO24XxrtZWqK3fwyqaq4WZfdAAVftjxOUVpJRHc= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.272/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.282 h1:UK4Eq2Xs1PfdI5w9CeNAHbZUGOT81oNAp+ahUtnjP1g= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.282/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/avast/retry-go/v4 v4.3.3 h1:G56Bp6mU0b5HE1SkaoVjscZjlQb0oy4mezwY/cGH19w= @@ -38,11 +29,8 @@ github.com/bits-and-blooms/bitset v1.5.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edY github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.8.6 h1:aUgO9S8gvdN6SyW2EhIpAw5E4ChworywIEndZCkCVXk= -github.com/bytedance/sonic v1.8.6/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/bytedance/sonic v1.8.7 h1:d3sry5vGgVq/OpgozRUNP6xBsSo0mtNdwliApw+SAMQ= github.com/bytedance/sonic v1.8.7/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= -github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -128,17 +116,15 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.16.3 h1:XuJt9zzcnaz6a16/OU53ZjWp/v7/42WcR5t2a0PcNQY= -github.com/klauspost/compress v1.16.3/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= -github.com/klauspost/compress v1.16.4 h1:91KN02FnsOYhuunwU4ssRe8lc2JosWmizWa91B5v1PU= -github.com/klauspost/compress v1.16.4/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.16.5 h1:IFV2oUNUzZaz+XyusxpLzpzS8Pt5rh0Z16For/djlyI= +github.com/klauspost/compress v1.16.5/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -177,8 +163,8 @@ github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= -github.com/prometheus/client_golang v1.14.0 h1:nJdhIvne2eSX/XRAFV9PcvFFRbrjbcTUj0VP62TMhnw= -github.com/prometheus/client_golang v1.14.0/go.mod h1:8vpkKitgIVNcqrRBWh1C4TIUQgYNtG/XQE4E/Zae36Y= +github.com/prometheus/client_golang v1.15.0 h1:5fCgGYogn0hFdhyhLbw7hEsWxufKtY9klyvdNfFlFhM= +github.com/prometheus/client_golang v1.15.0/go.mod h1:e9yaBhRPU2pPNsZwE+JdQl0KEt1N9XgF6zxWmaC0xOk= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -237,8 +223,6 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0 github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go v1.13.1 h1:7dDuAjJItbZRjgmMzYzCfJtRMpVKfQ57yIG8SkZz24M= -github.com/twmb/franz-go v1.13.1/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= github.com/twmb/franz-go v1.13.2 h1:jIdDoFiq8uP3Zrx6TZZTXpaXrv3bh1w3tV5mn/B+Gw8= github.com/twmb/franz-go v1.13.2/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= github.com/twmb/franz-go/pkg/kadm v1.8.0 h1:vvKwZpxYn+VmM32v4mKkecHLKavZW+HcYLRKKxly5ZY= @@ -293,8 +277,8 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.7.0 h1:AvwMYaRytfdeVt3u6mLaxYtErKYjxA2OXjJ1HHq6t3A= -golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= +golang.org/x/crypto v0.8.0 h1:pd9TJtTueMTVQXzk8E2XESSMQDj/U7OUu0PqJqPXQjQ= +golang.org/x/crypto v0.8.0/go.mod h1:mRqEX+O9/h5TFCrQhkgjo2yKi0yYA+9ecGkdQoHrywE= golang.org/x/exp v0.0.0-20230321023759-10a507213a29 h1:ooxPy7fPvB4kwsA2h+iBNHkAbp/4JxTSwCmvdjEYmug= golang.org/x/exp v0.0.0-20230321023759-10a507213a29/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -317,8 +301,6 @@ golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.8.0 h1:Zrh2ngAOFYneWTAIAPethzeaQLuHwhuBkuV6ZiRnUaQ= -golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.9.0 h1:aWJ/m6xSmxWBx+V0XRHTlrYrPG56jKsLdTFmsSsCzOM= golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -345,8 +327,6 @@ golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ= -golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.7.0 h1:3jlCCIQZPdOYu1h8BkNvLz8Kgwtae2cagcG/VamtZRU= golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= From 605533fba4002c5bb450735ca8898dccec6048e4 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Sun, 23 Apr 2023 12:32:01 +0800 Subject: [PATCH 336/404] chore - add test file for TestJksToPem --- .gitignore | 3 +++ Makefile | 12 ++++-------- test/kafka.client.keystore.jks | Bin 0 -> 2239 bytes test/kafka.client.truststore.jks | Bin 0 -> 948 bytes util/common.go | 8 ++++++-- util/common_test.go | 9 ++++++--- 6 files changed, 19 insertions(+), 13 deletions(-) create mode 100644 test/kafka.client.keystore.jks create mode 100644 test/kafka.client.truststore.jks diff --git a/.gitignore b/.gitignore index 804e3dd9..13e536ba 100644 --- a/.gitignore +++ b/.gitignore @@ -8,6 +8,9 @@ package-lock.json *.log .idea *.bin +*.out +*.pem +*.p12 dist vendor diff --git a/Makefile b/Makefile index 78b4718c..d837a02c 100644 --- a/Makefile +++ b/Makefile @@ -24,10 +24,6 @@ build: pre debug: pre $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o . ./... -.PHONY: unittest -unittest: pre - go test -v ./... - .PHONY: benchtest benchtest: pre go test -bench=. ./... @@ -37,10 +33,10 @@ systest: build bash go.test.sh bash go.metrictest.sh -.PHONY: coverage -coverage: - go test ./... -coverprofile=coverage.txt -covermode count - go tool cover -func coverage.txt +.PHONY: gotest +gotest: pre + go test -v ./... -coverprofile=coverage.out -covermode count + go tool cover -func coverage.out .PHONY: lint lint: diff --git a/test/kafka.client.keystore.jks b/test/kafka.client.keystore.jks new file mode 100644 index 0000000000000000000000000000000000000000..80417c9a5a7da78f42e38e79ed58e8f6b2d75022 GIT binary patch literal 2239 zcmchY`9IVP7sqF{3?mH6E@kZXwPj0|kVGOh6(USC!jwHEjmc6Nk202FXvo$Ovg9KB z-peu`TVj+E(n!{B*K&2A`@CNF_51k2E#!xB5YaDg!grPnC2hP z(#A3|1lo1H71b0v>L;YTUfN(-GJmdsP4XvYc6O(d!|v@2nbGIgsV1h6FFig^SF655 z_oqLtZ=Z5ci_gp8^>x6;S|2qj{+_7o`jd-d zWhD|Kl8j*P9$gFcmbN;G4gXZqqGmTdP})=nR~>TW*SM-&p3B5UE%QZ}`n}d=!Al|Z z_JpMZAznmwH~3mo^ylrx#pYyrGiOD$q$QaaIrWZk^hS=ecCwYH>$+Pc zZ{}0k?#{p!b9p;R4tkvuMt{AdQ(RO`X#1M1xLh(=rr;(S;7TSQQ8g^4df0LJ@9v$m zJj2W>WA8SR)~S=_muk4~iZDvr4TR&yAN>#qDt>wzcD=9j(6&X#sX={ivAWhIFz}KK z7Cq6nBU713DOj>o-J075-{Rp=UF12%O?AR?b>6};P2P!CARcxN#%)(uSzbjM)Auo z%r;3o=66M~lZ?$XtY>p4d>#*M2nWTi;BX$rlZ=~L-j>tp;8pCUGM zy5vE`9R29ti~B{5$tg+$P8d4Fo#PGN z%n=|p$>6__-O7>i{8~XDW*W%V-yP?q-PsVfPyc@H_zX$zigQ$_;X*$=htMGC`s#24 zH~YS5ksLa84utu}a!HUJl{;n@x&BuCRz#7}Ngj)R5f@f%0c_du0`Q8|)}iymqKnJ> z{s*n1{u@k*pEXa0T4!$~hHT2G^Ji5m$}%kzTe5!FJAb1)y*tq4;coW^O-NRG*0a55 zoT!8rN;v9R8un?mvdqj_VIC!NZkjy<8B~;(&)E@ELv@XXC5aY1q7pbr8;{Qx2q4-g^tc~A%#0)c^w(ER|Ci&rJqDs~$Q2J^u|fVT!8z{>@7f^nfB zXD@090f&A84~i!c=iz?~=YjvP#r+S9A3*%dLZG1D1iZ%I#P=T(0fc|WpeR0^x34cj zE|d`9>jlUhMrvvS+6U6o($vzqb{MJk3xU7!|5XMNEdS3296Sgp5qt;)5Wxrt5ex?P zfH9R$H(8nz%@Azn%Lncif2(>@u)#`^@J-?uXBV$o(v#fsNH@=I(`< zGf$BHrKmumuZRsps6n`)r%hbpQaw+PjdN$^&tv|vhg&0-My5T&S-Zl04xI+|W#^457aq=JIO+9Oy_ZeOB;`wy z1>l1!Q>piT9KTp5c1Dv-rP66ZT@zo|mH$27k(GXVD0YJ4bH)EUfoJf?z;+}{JOUTn zL5H+-<%YCTnyMP6)KhT3z3W7jyYVG1rjQkjdxOm@HqU$!P>v$Cjrv{Hn6B!U{UcAKf4#zgHqXv$;-x16O{qGhXG!YESKi1i%-=`X(S$oe$ zT4g;|ycVbQ{PBa-@h5d|UxKeU*2p@zPP$?>{Hkq=p2ed}8UwbXEx{!w-W5I8_Ij4W z*DPyiKDTD8bl#Mt1V63cJUx1{vmkF=vg7;vJ$X~C;_l<(n((w0@ux38I>vk!>qRWU zC9pbU!JsgdG28k|&Vrr$eq*9Twhh9E3Xwapq*WbE`&ex)U6uS+a=-hezQQTv1r`78 zSE3>0jo#J}t2)P;2;tGPo7?A0V^;{q`r-7H9^pIm=-s-|IcpD!q%wL@G9M*TObeV> zNFB>zkxHqJwxVcdy)8^}g}?5e%#MZ^nq$g%xr99uGaP9RSpJ+#mEE6Z$kcA`DHzZ$ I-93T+6Uk%lPXGV_ literal 0 HcmV?d00001 diff --git a/test/kafka.client.truststore.jks b/test/kafka.client.truststore.jks new file mode 100644 index 0000000000000000000000000000000000000000..2ef7e7f35e92a642cd3994ed4236771ab5168be3 GIT binary patch literal 948 zcmezO_TO6u1_mY|W(3oG$*DypnQ58Hi6yDosg*$C_BHy-rx;ix^h^yb85o#z4VsuU z4Vst&7cet1GBL3*ELS^iz{|#|)#lOmotKf3o0Y*J)sWkOlZ`o)g-w{r+0RhWfFH!+ z5auY(NX$#mNX&!@u_Fob7;u49aS1c0=4Tp0dE5}5h=CADj#-#HBQ+-{U!gp|C?~~0 zPMp`s*uVscjg5_rO{2tljX_)kD3@N&Xkt`C4gf}02IeM4eg=akMlPl%Mn;Acj1e0{ zlkOWy?e~s7Ahlc~C8gW8f8XO>^RI{Im&Tm??>i-{e*T?j=Xw9n5&*2o}&#djxy8dl^nbA=CkIN>qlly55CZLV_H^tUUWXk`5))L z*W8z^&S*Qbi0Qzw`DKTu?cKEdw(f+CU1yGRt&8LCc`>11eZ|RAmknR-Uh!$xwH&&d z8)?MxVm1>qBLm}NMFV*QSzsi}^0A1qh@2DL|000nOvV$J?X!!oCKL$FXFyLyz(8bV zSgd^XkmV`uYNKtHKC)V$CrNJ0&{)`2Kk0h+_PAH2(R;Sa2gTluO*72h;r2ik+w^TK=f3B7HeZ6Q||Cf^o%RIJ+Y^@f$x;E*%=c=}M`SzAoizb{9 zDp=I;cYFC`uf%E6nrbhk7Yj?Rm{ Date: Fri, 28 Apr 2023 09:56:08 +0800 Subject: [PATCH 337/404] feat - two new properties to manage reloading of bmseriesMap --- .gitignore | 2 +- CHANGELOG.md | 38 +++++++++++++++------- config/config.go | 61 +++++++++++++++++++----------------- docs/configuration/config.md | 22 ++++++++++--- pool/conn.go | 6 ++-- task/sinker.go | 49 +++++++++++++++++++++-------- 6 files changed, 117 insertions(+), 61 deletions(-) diff --git a/.gitignore b/.gitignore index 13e536ba..2a03a7d7 100644 --- a/.gitignore +++ b/.gitignore @@ -3,7 +3,7 @@ sh .vscode node_modules/ package-lock.json - +nacos_cache .DS_Store *.log .idea diff --git a/CHANGELOG.md b/CHANGELOG.md index 2d808478..3aabdf55 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,100 +1,114 @@ # Changelog +#### Version 3.0.5 (TBD) + +New Features: + +- Introduced two new configuration properties to Sinker: + - "ReloadSeriesMapInterval", allows you to manage how frequently the cached seriesMap is reloaded. + - "ActiveSeriesRange", allows you to manage the amount of cached information. + Refer to the "config.md" file for more info. + #### Version 3.0.4 (2023-04-18) Improvements: -- Automatically end sinker if it remains inactive in last 10 mins +- Automatically end sinker if it remains inactive in last 10 mins #### Version 3.0.3 (2023-03-30) Improvements: + - Have writingpool per shard to avoid ErrAcquireConnTimeout - Do not create kafka client everytime when caculate lags - Support configuring PlainloginModule in kafka.security section Bug Fixes: -- Avoid program running into stuck when facing fatalpanic +- Avoid program running into stuck when facing fatalpanic #### Version 3.0.2 (2023-03-13) Improvements: + - update sinker grafana dashboard - combine nacos log into sinker log - update dmseries map when applying new config, reload the records from series table every single day - avoid recreating dist tables, alter the table schema instead - update clickhouse_sinker_consume_lags metric every 10 secs - #### Version 3.0.1 (2023-03-03) Bug Fixes: + - Fix nacos publish config error "BUG: got different config" - Fix changing "TimeUnit" config property does not trigger config reload - Fix illegal "TimeZone" value result in sinker crash - Fix wrong parsing result of Decimal type [909](https://github.com/ClickHouse/clickhouse-go/pull/909) Improvements: + - Metrics from GoCollector and ProcessCollector are now being pushed to metric-push-gateway-addrs - Terminate program immediately when receiving one more exit signal - Limit the fetch size and poll size based on the BufferSize config property - #### Version 3.0.0 (2023-02-07) New Features: + - Add support of ingesting multi-value metrics, the metric table will be expanded accordingly - Allow specifying the series table name - Allow customization of DatabaseName in task level Improvements: + - Group the tasks by consumerGroup property to reduce number of kafka client, see design.md for details Deprecation: + - Kafka-go and Sarama are no longer internal options for sinker - - #### Version 2.6.9 (2023-02-07) Improvements: + - Ignore SIGHUP signal, so that fire up sinker with nohup could work correctly - Stop retrying when facing offsets commit error, leave it to the future commitment to sync the offsets - Offsets commit error should not result in a process abort - #### Version 2.6.8 (2022-12-10) New Features: + - Add clickhouse Map type support -- Small updates to allow TLS connections for AWS MSK, etc. +- Small updates to allow TLS connections for AWS MSK, etc. ([169](https://github.com/housepower/clickhouse_sinker/pull/169)) Bug Fixes: -- Fix ClickHouse.Init goroutine leak +- Fix ClickHouse.Init goroutine leak #### Version 2.6.7 (2022-12-07) Improvements: + - Add new sinker metrics to show the wrSeriesQuota status - Always allow writing new series to avoid data mismatch between series and metrics table - #### Version 2.6.6 (2022-12-05) Bug Fixes: -- reset wrSeries timely to avoid failure of writing metric data to clickhouse +- reset wrSeries timely to avoid failure of writing metric data to clickhouse #### Version 2.6.5 (2022-11-30) Bug Fixes: + - Fix the 'segmentation violation' in ch-go package - Fix the create table error 'table already exists' when trying to create a distribution table - #### Previous releases -See https://github.com/housepower/clickhouse_sinker/releases \ No newline at end of file +See https://github.com/housepower/clickhouse_sinker/releases diff --git a/config/config.go b/config/config.go index fb465a56..090762f1 100644 --- a/config/config.go +++ b/config/config.go @@ -30,13 +30,16 @@ import ( // Config struct used for different configurations use type Config struct { - Kafka KafkaConfig - Clickhouse ClickHouseConfig - Task *TaskConfig - Tasks []*TaskConfig - Assignment Assignment - LogLevel string - Groups map[string]*GroupConfig `json:"-"` + Kafka KafkaConfig + Clickhouse ClickHouseConfig + Task *TaskConfig + Tasks []*TaskConfig + Assignment Assignment + LogLevel string + ReloadSeriesMapInterval int + ActiveSeriesRange int + + Groups map[string]*GroupConfig `json:"-"` } // KafkaConfig configuration parameters @@ -83,13 +86,12 @@ type KafkaConfig struct { // ClickHouseConfig configuration parameters type ClickHouseConfig struct { - Cluster string - DB string - Hosts [][]string - Port int - Username string - Password string - DsnParams string + Cluster string + DB string + Hosts [][]string + Port int + Username string + Password string // Whether enable TLS encryption with clickhouse-server Secure bool @@ -98,7 +100,6 @@ type ClickHouseConfig struct { RetryTimes int //<=0 means retry infinitely MaxOpenConns int - DialTimeout int // Connection dial timeout in seconds } // TaskConfig parameters @@ -171,15 +172,16 @@ type Assignment struct { } const ( - MaxBufferSize = 1 << 20 //1048576 - defaultBufferSize = 1 << 18 //262144 - maxFlushInterval = 600 - defaultFlushInterval = 5 - defaultTimeZone = "Local" - defaultLogLevel = "info" - defaultKerberosConfigPath = "/etc/krb5.conf" - defaultMaxOpenConns = 1 - defaultDialTimeout = 5 + MaxBufferSize = 1 << 20 //1048576 + defaultBufferSize = 1 << 18 //262144 + maxFlushInterval = 600 + defaultFlushInterval = 10 + defaultTimeZone = "Local" + defaultLogLevel = "info" + defaultKerberosConfigPath = "/etc/krb5.conf" + defaultMaxOpenConns = 1 + defaultReloadSeriesMapInterval = 3600 // 1 hour + defaultActiveSeriesRange = 86400 // 1 day ) func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { @@ -233,10 +235,6 @@ func (cfg *Config) Normallize(constructGroup bool, httpAddr string) (err error) cfg.Clickhouse.MaxOpenConns = defaultMaxOpenConns } - if cfg.Clickhouse.DialTimeout <= 0 { - cfg.Clickhouse.DialTimeout = defaultDialTimeout - } - if cfg.Task != nil { cfg.Tasks = append(cfg.Tasks, cfg.Task) cfg.Task = nil @@ -280,6 +278,13 @@ func (cfg *Config) Normallize(constructGroup bool, httpAddr string) (err error) default: cfg.LogLevel = defaultLogLevel } + if cfg.ReloadSeriesMapInterval <= 0 { + cfg.ReloadSeriesMapInterval = defaultReloadSeriesMapInterval + } + if cfg.ActiveSeriesRange <= 0 { + cfg.ActiveSeriesRange = defaultActiveSeriesRange + } + return } diff --git a/docs/configuration/config.md b/docs/configuration/config.md index e0ef501a..601f75be 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -1,9 +1,10 @@ # Config Items -> Here we use json with comments for documentation, config file in hjson format is also supported +> Here we use json with comments for documentation, config file in hjson format is also supported ```json -{ + ReloadSeriesMapInterval int + ActiveSeriesRange int{ // ClickHouse config "clickhouse": { // cluster the ClickHouse node belongs @@ -42,7 +43,7 @@ // Kafka config "kafka": { "brokers": "127.0.0.1:9093", - + // jave client style security authentication "security":{ "security.protocol": "SASL_PLAINTEXT", @@ -148,6 +149,12 @@ "blackList": "@" }, + // PrometheusSchema expects each message is a Prometheus metric(timestamp, value, metric name and a list of labels). + "prometheusSchema": true, + // the regexp of labels black list, fields match promLabelsBlackList are not considered as part of labels column in series table + // Requires PrometheusSchema be true. + "promLabelsBlackList": "", + // shardingKey is the column name to which sharding against "shardingKey": "", // shardingStripe take effect if the sharding key is numerical @@ -167,6 +174,13 @@ }, // log level, possible value: "debug", "info", "warn", "error", "dpanic", "panic", "fatal". Default to "info". - "logLevel": "debug" + "logLevel": "debug", + // The Series table may contain hundreds of columns, and writing this table every time a datapoint is persisted can result in significant + // performance overhead. This should be unnecessary since the lables from the same timeseries usually do not change(mid could be an exception). + // Therefore, it would be reasonable to keep the map between "sid" and "mid" in cache to avoid frequent write operations. To optimize the memory + // utilization, only active series from the last "activeSeriesRange" seconds will be cached, and the map in the cache will be updated every + // "reloadSeriesMapInterval" seconds. By default, series from the last 24 hours will be cached, and the cache will be updated every hour. + "reloadSeriesMapInterval": 3600, + "activeSeriesRange": 86400 } ``` diff --git a/pool/conn.go b/pool/conn.go index 928578c8..db3df3bf 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -112,8 +112,8 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db clickhouse.Conn, dbVer i // Each shard has a clickhouse.Conn which connects to one replica inside the shard. // We need more control than replica single-point-failure. -func InitClusterConn(hosts [][]string, port int, db, username, password, dsnParams string, secure, skipVerify bool, - maxOpenConns int, dialTimeout int) (err error) { +func InitClusterConn(hosts [][]string, port int, db, username, password string, secure, skipVerify bool, + maxOpenConns int) (err error) { lock.Lock() defer lock.Unlock() freeClusterConn() @@ -138,7 +138,7 @@ func InitClusterConn(hosts [][]string, port int, db, username, password, dsnPara Username: username, Password: password, }, - DialTimeout: time.Second * time.Duration(dialTimeout), + DialTimeout: time.Minute * 10, MaxOpenConns: maxOpenConns, MaxIdleConns: 5, // TODO - update this property to maxOpenConns when the lifetime of an idle connection honours the ConnMaxLifetime ConnMaxLifetime: time.Minute * 10, diff --git a/task/sinker.go b/task/sinker.go index 6c32f736..259af38f 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -44,10 +44,10 @@ var ( dropTableSQL = `DROP TABLE IF EXISTS %s ` countSeriesSQL = `WITH (SELECT max(timestamp) FROM %s) AS m SELECT count() FROM %s FINAL WHERE __series_id GLOBAL IN ( - SELECT DISTINCT __series_id FROM %s WHERE timestamp >= addDays(m, -1));` + SELECT DISTINCT __series_id FROM %s WHERE timestamp >= addSeconds(m, -%d));` loadSeriesSQL = `WITH (SELECT max(timestamp) FROM %s) AS m SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s FINAL WHERE sid GLOBAL IN ( - SELECT DISTINCT toInt64(__series_id) FROM %s WHERE timestamp >= addDays(m, -1) + SELECT DISTINCT toInt64(__series_id) FROM %s WHERE timestamp >= addSeconds(m, -%d) ) ORDER BY sid;` ) @@ -110,8 +110,6 @@ func (s *Sinker) Run() { go s.pusher.Run() } - reloadBmSeriesTicker := time.NewTicker(time.Hour) - defer reloadBmSeriesTicker.Stop() if s.rcm == nil { if _, err = os.Stat(s.cmdOps.LocalCfgFile); err == nil { if newCfg, err = config.ParseLocalCfgFile(s.cmdOps.LocalCfgFile); err != nil { @@ -134,6 +132,9 @@ func (s *Sinker) Run() { util.Logger.Fatal("s.applyConfig failed", zap.Error(err)) return } + + reloadBmSeriesTicker := time.NewTicker(time.Duration(s.curCfg.ReloadSeriesMapInterval) * time.Second) + defer reloadBmSeriesTicker.Stop() LOOP: for { select { @@ -170,6 +171,11 @@ func (s *Sinker) Run() { // Golang <-time.After() is not garbage collected before expiry. ticker := time.NewTicker(10 * time.Second) defer ticker.Stop() + + // start the ticker with default value 1 hour + curInterval := 3600 + reloadBmSeriesTicker := time.NewTicker(time.Second * time.Duration(curInterval)) + defer reloadBmSeriesTicker.Stop() WORKLOOP: for { select { @@ -189,10 +195,16 @@ func (s *Sinker) Run() { util.Logger.Error("newCfg.Normallize failed", zap.Error(err)) continue } + if s.curCfg != nil { + curInterval = s.curCfg.ReloadSeriesMapInterval + } if err = s.applyConfig(newCfg); err != nil { util.Logger.Error("s.applyConfig failed", zap.Error(err)) continue } + if curInterval != newCfg.ReloadSeriesMapInterval { + reloadBmSeriesTicker.Reset(time.Duration(newCfg.ReloadSeriesMapInterval) * time.Second) + } case c := <-s.consumerRestartCh: // only restart the consumer which was not changed in applyAnotherConfig if c == s.consumers[c.grpConfig.Name] { @@ -280,6 +292,8 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { !reflect.DeepEqual(newCfg.Assignment.Map, s.curCfg.Assignment.Map) { err = s.applyAnotherConfig(newCfg) } + s.curCfg.ActiveSeriesRange = newCfg.ActiveSeriesRange + s.curCfg.ReloadSeriesMapInterval = newCfg.ReloadSeriesMapInterval if len(s.consumers) == 0 && s.cmdOps.NacosServiceName != "" { util.Logger.Warn("No task fetched from Nacos, make sure the program is running with correct commandline option!") @@ -292,7 +306,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { // 1. Initialize clickhouse connections chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.DialTimeout); err != nil { + chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { return } @@ -332,7 +346,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 2. Initialize clickhouse connections. chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.DsnParams, chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.DialTimeout); err != nil { + chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { return } @@ -520,7 +534,7 @@ func (s *Sinker) initBmSeries() (err error) { } } - seriesMap, err := loadBmSeries(conn, k, v) + seriesMap, err := loadBmSeries(conn, k, v, s.curCfg.ActiveSeriesRange) if err != nil { return err } @@ -542,13 +556,18 @@ func (s *Sinker) reloadBmSeries() (err error) { k := key.(string) v := value.(*model.SeriesQuota) v.Lock() - if now.Sub(v.Birth) > time.Hour*24 { + if now.Sub(v.Birth).Seconds() > float64(s.curCfg.ReloadSeriesMapInterval) { sqMap[k] = v } v.Unlock() return true }) + if len(sqMap) == 0 { + util.Logger.Info("SeriesMap cache is up to date!") + return + } + // series table could be shared between multiple tasks tables := make(map[string][]*Service) for _, c := range s.consumers { @@ -561,6 +580,10 @@ func (s *Sinker) reloadBmSeries() (err error) { }) } + if len(tables) == 0 { + return + } + var conn clickhouse.Conn if conn, _, err = pool.GetShardConn(0).NextGoodReplica(0); err != nil { return @@ -568,7 +591,7 @@ func (s *Sinker) reloadBmSeries() (err error) { // reload seriesQuotas which is out-of-date for k, v := range tables { - seriesMap, err := loadBmSeries(conn, k, v) + seriesMap, err := loadBmSeries(conn, k, v, s.curCfg.ActiveSeriesRange) if err != nil { return err } @@ -578,13 +601,13 @@ func (s *Sinker) reloadBmSeries() (err error) { sq.BmSeries = seriesMap sq.Birth = time.Now() sq.Unlock() - util.Logger.Info(fmt.Sprintf("reloaded %d series from %v", len(seriesMap), k)) + util.Logger.Info(fmt.Sprintf("reloaded %d series from %v, SeriesMap cache is up to date!", len(seriesMap), k)) } return } -func loadBmSeries(conn clickhouse.Conn, sqKey string, tasks []*Service) (result map[int64]int64, err error) { +func loadBmSeries(conn clickhouse.Conn, sqKey string, tasks []*Service, activeSeriesRange int) (result map[int64]int64, err error) { // merge all metric tables to get the latest timestamp // old bmseries record won't be loaded into memory to avoid OOM var reg string @@ -603,14 +626,14 @@ func loadBmSeries(conn clickhouse.Conn, sqKey string, tasks []*Service) (result } var count uint64 - query = fmt.Sprintf(countSeriesSQL, mergetable, sqKey, mergetable) + query = fmt.Sprintf(countSeriesSQL, mergetable, sqKey, mergetable, activeSeriesRange) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) if err = conn.QueryRow(context.Background(), query).Scan(&count); err != nil { return } seriesMap := make(map[int64]int64, count) - query = fmt.Sprintf(loadSeriesSQL, mergetable, sqKey, mergetable) + query = fmt.Sprintf(loadSeriesSQL, mergetable, sqKey, mergetable, activeSeriesRange) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) rs, err := conn.Query(context.Background(), query) if err != nil { From b7869868780e12ea5f7cce36aa4f9756cb83f879 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 12 May 2023 10:13:00 +0800 Subject: [PATCH 338/404] Add a new http endpoint /debug/vars for viewing contents of seriesMap --- CHANGELOG.md | 3 ++- cmd/clickhouse_sinker/main.go | 1 + docs/configuration/config.md | 3 +-- model/metric.go | 4 ++-- output/clickhouse.go | 19 +++++++++++++++++++ 5 files changed, 25 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3aabdf55..28e292d0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,6 @@ # Changelog -#### Version 3.0.5 (TBD) +#### Version 3.0.5 (2023-05-12) New Features: @@ -8,6 +8,7 @@ New Features: - "ReloadSeriesMapInterval", allows you to manage how frequently the cached seriesMap is reloaded. - "ActiveSeriesRange", allows you to manage the amount of cached information. Refer to the "config.md" file for more info. +- Add a new http endpoint "/debug/vars" for viewing contents of seriesMap #### Version 3.0.4 (2023-04-18) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 04aefa9b..09c1cccc 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -169,6 +169,7 @@ func main() { mux.HandleFunc("/debug/pprof/profile", pprof.Profile) mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) mux.HandleFunc("/debug/pprof/trace", pprof.Trace) + mux.Handle("/debug/vars", http.DefaultServeMux) // cmdOps.HTTPPort=0: let OS choose the listen port, and record the exact metrics URL to log. httpPort := cmdOps.HTTPPort diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 601f75be..d770d5e8 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -3,8 +3,7 @@ > Here we use json with comments for documentation, config file in hjson format is also supported ```json - ReloadSeriesMapInterval int - ActiveSeriesRange int{ +{ // ClickHouse config "clickhouse": { // cluster the ClickHouse node belongs diff --git a/model/metric.go b/model/metric.go index 6ef3e20d..cab37451 100644 --- a/model/metric.go +++ b/model/metric.go @@ -101,9 +101,9 @@ func NewOrderedMap() *OrderedMap { } type SeriesQuota struct { - sync.Mutex + sync.Mutex `json:"-"` NextResetQuota time.Time - BmSeries map[int64]int64 + BmSeries map[int64]int64 // sid:mid WrSeries int Birth time.Time } diff --git a/output/clickhouse.go b/output/clickhouse.go index 013cb743..4bcd7754 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -17,6 +17,8 @@ package output import ( "context" + "encoding/json" + "expvar" "fmt" "math" "sort" @@ -78,6 +80,23 @@ type ClickHouse struct { taskDone *sync.Cond } +func init() { + expvar.Publish("SeriesMap", expvar.Func(func() interface{} { + var result = make(map[string]string) + SeriesQuotas.Range(func(key, value interface{}) bool { + if sq, ok := value.(*model.SeriesQuota); ok { + sq.Lock() + if bs, err := json.Marshal(sq); err == nil { + result[key.(string)] = string(bs) + } + sq.Unlock() + } + return true + }) + return result + })) +} + // NewClickHouse new a clickhouse instance func NewClickHouse(cfg *config.Config, taskCfg *config.TaskConfig) *ClickHouse { ck := &ClickHouse{cfg: cfg, taskCfg: taskCfg} From 57bf7e7f41cbc7b11a8bcb7b9fe53072ceb73ac2 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 12 May 2023 13:20:47 +0800 Subject: [PATCH 339/404] update logical cluster table schema when idenfying new columns --- CHANGELOG.md | 10 ++++++++++ output/clickhouse.go | 47 ++++++++++++++++++++++++++++++++------------ 2 files changed, 44 insertions(+), 13 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 28e292d0..88b42c3b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,11 @@ # Changelog +#### Version 3.0.6 (TBD) + +Improvements: + +- When idenfying new columns, also update the schema of tables that are part of a logical cluster + #### Version 3.0.5 (2023-05-12) New Features: @@ -10,6 +16,10 @@ New Features: Refer to the "config.md" file for more info. - Add a new http endpoint "/debug/vars" for viewing contents of seriesMap +Deprecations: + +- DialTimeout option is deprecated. + #### Version 3.0.4 (2023-04-18) Improvements: diff --git a/output/clickhouse.go b/output/clickhouse.go index 4bcd7754..ab1a379e 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -80,6 +80,11 @@ type ClickHouse struct { taskDone *sync.Cond } +type DistTblInfo struct { + name string + cluster string +} + func init() { expvar.Publish("SeriesMap", expvar.Func(func() interface{} { var result = make(map[string]string) @@ -347,10 +352,18 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { // Check distributed series table if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { - if c.distSeriesTbls, err = c.getDistTbls(c.seriesTbl); err != nil { - return + withDistTable := false + info, e := c.getDistTbls(c.seriesTbl) + if e != nil { + return e + } + for _, i := range info { + c.distSeriesTbls = append(c.distSeriesTbls, i.name) + if i.cluster == c.cfg.Clickhouse.Cluster { + withDistTable = true + } } - if c.distSeriesTbls == nil { + if !withDistTable { err = errors.Newf("Please create distributed table for %s in cluster '%s'.", c.seriesTbl, c.cfg.Clickhouse.Cluster) return } @@ -416,10 +429,18 @@ func (c *ClickHouse) initSchema() (err error) { // Check distributed metric table if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { - if c.distMetricTbls, err = c.getDistTbls(c.TableName); err != nil { - return + withDistTable := false + info, e := c.getDistTbls(c.TableName) + if e != nil { + return e + } + for _, i := range info { + c.distMetricTbls = append(c.distMetricTbls, i.name) + if i.cluster == c.cfg.Clickhouse.Cluster { + withDistTable = true + } } - if c.distMetricTbls == nil { + if !withDistTable { err = errors.Newf("Please create distributed table for %s in cluster '%s'.", c.seriesTbl, c.cfg.Clickhouse.Cluster) return } @@ -526,16 +547,16 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return } -func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { +func (c *ClickHouse) getDistTbls(table string) (distTbls []DistTblInfo, err error) { taskCfg := c.taskCfg - chCfg := &c.cfg.Clickhouse sc := pool.GetShardConn(0) var conn clickhouse.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } - query := fmt.Sprintf(`SELECT name FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(create_table_query, 'Distributed\(\'%s\', \'%s\', \'%s\'.*\)')`, - c.dbName, chCfg.Cluster, c.dbName, table) + query := fmt.Sprintf(`SELECT name, (extractAllGroups(engine_full, '(Distributed\\(\')(.*)\',\\s+\'(.*)\',\\s+\'(.*)\'(.*)')[1])[2] AS cluster + FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(engine_full, 'Distributed\(\'.*\', \'%s\', \'%s\'.*\)')`, + c.dbName, c.dbName, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows driver.Rows if rows, err = conn.Query(context.Background(), query); err != nil { @@ -544,12 +565,12 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []string, err error) { } defer rows.Close() for rows.Next() { - var name string - if err = rows.Scan(&name); err != nil { + var name, cluster string + if err = rows.Scan(&name, &cluster); err != nil { err = errors.Wrapf(err, "") return } - distTbls = append(distTbls, name) + distTbls = append(distTbls, DistTblInfo{name: name, cluster: cluster}) } return } From be49fcfa5b4ee99ade9e0f02f30eea2988db259a Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Fri, 19 May 2023 16:52:56 +0800 Subject: [PATCH 340/404] Allow writing non-numeric metric --- CHANGELOG.md | 5 +++-- output/clickhouse.go | 5 +---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 88b42c3b..0bcb408c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,11 @@ # Changelog -#### Version 3.0.6 (TBD) +#### Version 3.0.6 (2023-05-19) Improvements: -- When idenfying new columns, also update the schema of tables that are part of a logical cluster +- When idenfying new columns, also update the schema of tables that are part of a logical cluster +- Allow writing non-numeric type metric #### Version 3.0.5 (2023-05-12) diff --git a/output/clickhouse.go b/output/clickhouse.go index ab1a379e..50042b3e 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -283,9 +283,6 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { if dim.Name == "__series_id" && dim.Type.Type == model.Int64 { dimSerID = dim c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) - } else if dim.Type.Type == model.String { - c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) - util.Logger.Warn("non-numeric type metric ignored", zap.String("metric name", dim.Name)) } else { i++ } @@ -441,7 +438,7 @@ func (c *ClickHouse) initSchema() (err error) { } } if !withDistTable { - err = errors.Newf("Please create distributed table for %s in cluster '%s'.", c.seriesTbl, c.cfg.Clickhouse.Cluster) + err = errors.Newf("Please create distributed table for %s in cluster '%s'.", c.TableName, c.cfg.Clickhouse.Cluster) return } } From 07f0d13d4b1e83cf92b146428b143602d0e6c4be Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 11:24:13 +0300 Subject: [PATCH 341/404] Separate tcp port test in a func Signed-off-by: sashayakovtseva --- util/common.go | 31 +++++++++++++++++++------------ 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/util/common.go b/util/common.go index 25cdcbc9..155d769c 100644 --- a/util/common.go +++ b/util/common.go @@ -20,6 +20,7 @@ import ( "crypto/tls" "crypto/x509" "fmt" + "math" "net" "os" "os/exec" @@ -85,8 +86,8 @@ func GetShift(s int) (shift uint) { return } -// GetOutboundIP get preferred outbound ip of this machine -// https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go +// GetOutboundIP gets preferred outbound ip of this machine +// https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go. func GetOutboundIP() (ip net.IP, err error) { var conn net.Conn if conn, err = net.Dial("udp", "8.8.8.8:80"); err != nil { @@ -99,18 +100,24 @@ func GetOutboundIP() (ip net.IP, err error) { return } -// GetSpareTCPPort find a spare TCP port -func GetSpareTCPPort(portBegin int) (port int) { -LOOP: - for port = portBegin; ; port++ { - addr := fmt.Sprintf(":%d", port) - ln, err := net.Listen("tcp", addr) - if err == nil { - ln.Close() - break LOOP +// GetSpareTCPPort finds a spare TCP port. +func GetSpareTCPPort(portBegin int) int { + for port := portBegin; port < math.MaxInt; port++ { + if err := testListenOnPort(port); err == nil { + return port } } - return + return 0 +} + +func testListenOnPort(port int) error { + addr := fmt.Sprintf(":%d", port) + ln, err := net.Listen("tcp", addr) + if err != nil { + return err + } + ln.Close() //nolint:errcheck + return nil } // https://stackoverflow.com/questions/50428176/how-to-get-ip-and-port-from-net-addr-when-it-could-be-a-net-udpaddr-or-net-tcpad From 0abf312ff3cc8bd5a4da69605b15c14b2c5c8aff Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 11:34:05 +0300 Subject: [PATCH 342/404] Allow to specify http host to bound to Signed-off-by: sashayakovtseva --- cmd/clickhouse_sinker/main.go | 55 +++++++++++++++++------------------ util/common.go | 13 ++++++--- 2 files changed, 35 insertions(+), 33 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 09c1cccc..23a7f2dd 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -19,7 +19,6 @@ import ( "encoding/json" "flag" "fmt" - "log" "net" "net/http" "net/http/pprof" @@ -38,14 +37,13 @@ import ( ) var ( - //goreleaser fill following info per https://goreleaser.com/customization/build/. + // goreleaser fills the following info per https://goreleaser.com/customization/build/. version = "None" commit = "None" date = "None" builtBy = "None" cmdOps util.CmdOptions - selfIP string httpAddr string httpMetrics = promhttp.Handler() runner *task.Sinker @@ -54,20 +52,14 @@ var ( func initCmdOptions() { // 1. Set options to default value. cmdOps = util.CmdOptions{ - ShowVer: false, - LogLevel: "info", - LogPaths: "stdout,clickhouse_sinker.log", - HTTPPort: 0, - PushGatewayAddrs: "", - PushInterval: 10, - LocalCfgFile: "/etc/clickhouse_sinker.hjson", - NacosAddr: "127.0.0.1:8848", - NacosNamespaceID: "", - NacosGroup: "DEFAULT_GROUP", - NacosUsername: "nacos", - NacosPassword: "nacos", - NacosDataID: "", - NacosServiceName: "", + LogLevel: "info", + LogPaths: "stdout,clickhouse_sinker.log", + PushInterval: 10, + LocalCfgFile: "/etc/clickhouse_sinker.hjson", + NacosAddr: "127.0.0.1:8848", + NacosGroup: "DEFAULT_GROUP", + NacosUsername: "nacos", + NacosPassword: "nacos", } // 2. Replace options with the corresponding env variable if present. @@ -75,6 +67,7 @@ func initCmdOptions() { util.EnvStringVar(&cmdOps.LogLevel, "log-level") util.EnvStringVar(&cmdOps.LogPaths, "log-paths") util.EnvIntVar(&cmdOps.HTTPPort, "http-port") + util.EnvStringVar(&cmdOps.HTTPHost, "http-host") util.EnvStringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs") util.EnvIntVar(&cmdOps.PushInterval, "push-interval") util.EnvStringVar(&cmdOps.LocalCfgFile, "local-cfg-file") @@ -92,6 +85,7 @@ func initCmdOptions() { flag.StringVar(&cmdOps.LogLevel, "log-level", cmdOps.LogLevel, "one of debug, info, warn, error, dpanic, panic, fatal") flag.StringVar(&cmdOps.LogPaths, "log-paths", cmdOps.LogPaths, "a list of comma-separated log file path. stdout means the console stdout") flag.IntVar(&cmdOps.HTTPPort, "http-port", cmdOps.HTTPPort, "http listen port") + flag.StringVar(&cmdOps.HTTPHost, "http-host", cmdOps.HTTPHost, "http host to bind to") flag.StringVar(&cmdOps.PushGatewayAddrs, "metric-push-gateway-addrs", cmdOps.PushGatewayAddrs, "a list of comma-separated prometheus push gatway address") flag.IntVar(&cmdOps.PushInterval, "push-interval", cmdOps.PushInterval, "push interval in seconds") flag.StringVar(&cmdOps.LocalCfgFile, "local-cfg-file", cmdOps.LocalCfgFile, "local config file") @@ -120,12 +114,6 @@ func init() { if cmdOps.ShowVer { os.Exit(0) } - var err error - var ip net.IP - if ip, err = util.GetOutboundIP(); err != nil { - log.Fatal("unable to determine self ip", err) - } - selfIP = ip.String() util.Logger.Info("parsed command options:", zap.Reflect("opts", cmdOps)) } @@ -173,16 +161,25 @@ func main() { // cmdOps.HTTPPort=0: let OS choose the listen port, and record the exact metrics URL to log. httpPort := cmdOps.HTTPPort - if httpPort != 0 { + if httpPort == 0 { httpPort = util.GetSpareTCPPort(httpPort) } - httpAddr = fmt.Sprintf(":%d", httpPort) + + httpHost := cmdOps.HTTPHost + if httpHost == "" { + ip, err := util.GetOutboundIP() + if err != nil { + return fmt.Errorf("failed to determine outbound ip: %w", err) + } + httpHost = ip.String() + } + + httpAddr = fmt.Sprintf("%s:%d", httpHost, httpPort) listener, err := net.Listen("tcp", httpAddr) if err != nil { - util.Logger.Fatal("net.Listen failed", zap.String("httpAddr", httpAddr), zap.Error(err)) + return fmt.Errorf("failed to listen on %q: %w", httpAddr, err) } - httpPort = util.GetNetAddrPort(listener.Addr()) - httpAddr = fmt.Sprintf("%s:%d", selfIP, httpPort) + util.Logger.Info(fmt.Sprintf("Run http server at http://%s/", httpAddr)) go func() { @@ -222,7 +219,7 @@ func main() { util.Logger.Fatal("rcm.Init failed", zap.Error(err)) } if cmdOps.NacosServiceName != "" { - if err := rcm.Register(selfIP, httpPort); err != nil { + if err := rcm.Register(httpHost, httpPort); err != nil { util.Logger.Fatal("rcm.Init failed", zap.Error(err)) } } diff --git a/util/common.go b/util/common.go index 155d769c..424ae87e 100644 --- a/util/common.go +++ b/util/common.go @@ -43,10 +43,15 @@ var ( ) type CmdOptions struct { - ShowVer bool - LogLevel string // "debug", "info", "warn", "error", "dpanic", "panic", "fatal" - LogPaths string // comma-separated paths. "stdout" means the console stdout - HTTPPort int // 0 menas a randomly OS chosen port + ShowVer bool + LogLevel string // "debug", "info", "warn", "error", "dpanic", "panic", "fatal" + LogPaths string // comma-separated paths. "stdout" means the console stdout + + // HTTPHost to bind to. If empty, outbound ip of machine + // is automatically determined and used. + HTTPHost string + HTTPPort int // 0 means a randomly chosen port. + PushGatewayAddrs string PushInterval int LocalCfgFile string From bdbee381367bc0a99b6d954662241cb97a9dc040 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 15:21:11 +0300 Subject: [PATCH 343/404] Add new envs in comment Signed-off-by: sashayakovtseva --- Dockerfile | 5 +++++ Dockerfile_goreleaser | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/Dockerfile b/Dockerfile index a54d5293..feb73e73 100644 --- a/Dockerfile +++ b/Dockerfile @@ -19,6 +19,7 @@ COPY --from=builder /app/kafka_gen_metric /usr/local/bin/kafka_gen_metric # - LOG_LEVEL # - LOG_PATHS # - HTTP_PORT +# - HTTP_HOST # - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL # - LOCAL_CFG_FILE @@ -29,6 +30,10 @@ COPY --from=builder /app/kafka_gen_metric /usr/local/bin/kafka_gen_metric # - NACOS_GROUP # - NACOS_DATAID # - NACOS_SERVICE_NAME +# - CLICKHOUSE_USERNAME +# - CLICKHOUSE_PASSWORD +# - KAFKA_USERNAME +# - KAFKA_PASSWORD # See cmd/clickhouse_sinker/main.go for details. ENTRYPOINT ["/usr/local/bin/clickhouse_sinker"] diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index d00ac98b..0340cafc 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -10,6 +10,7 @@ ADD dist/nacos_publish_config_linux_amd64_v1/nacos_publish_config /usr/local/bin # - LOG_LEVEL # - LOG_PATHS # - HTTP_PORT +# - HTTP_HOST # - METRIC_PUSH_GATEWAY_ADDRS # - PUSH_INTERVAL # - LOCAL_CFG_FILE @@ -20,6 +21,10 @@ ADD dist/nacos_publish_config_linux_amd64_v1/nacos_publish_config /usr/local/bin # - NACOS_GROUP # - NACOS_DATAID # - NACOS_SERVICE_NAME +# - CLICKHOUSE_USERNAME +# - CLICKHOUSE_PASSWORD +# - KAFKA_USERNAME +# - KAFKA_PASSWORD # See cmd/clickhouse_sinker/main.go for details. ENTRYPOINT ["/usr/local/bin/clickhouse_sinker"] From b3d9d2a554fb4f06ec67695468bea9e506da50d4 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 15:29:39 +0300 Subject: [PATCH 344/404] Read credentials from envs or flags Signed-off-by: sashayakovtseva --- cmd/clickhouse_sinker/main.go | 11 +++++++++++ util/common.go | 9 +++++++++ 2 files changed, 20 insertions(+) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 23a7f2dd..a011fe4c 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -80,6 +80,11 @@ func initCmdOptions() { util.EnvStringVar(&cmdOps.NacosDataID, "nacos-dataid") util.EnvStringVar(&cmdOps.NacosServiceName, "nacos-service-name") + util.EnvStringVar(&cmdOps.ClickhouseUsername, "clickhouse-username") + util.EnvStringVar(&cmdOps.ClickhousePassword, "clickhouse-password") + util.EnvStringVar(&cmdOps.KafkaUsername, "kafka-username") + util.EnvStringVar(&cmdOps.KafkaPassword, "kafka-password") + // 3. Replace options with the corresponding CLI parameter if present. flag.BoolVar(&cmdOps.ShowVer, "v", cmdOps.ShowVer, "show build version and quit") flag.StringVar(&cmdOps.LogLevel, "log-level", cmdOps.LogLevel, "one of debug, info, warn, error, dpanic, panic, fatal") @@ -98,6 +103,12 @@ func initCmdOptions() { flag.StringVar(&cmdOps.NacosGroup, "nacos-group", cmdOps.NacosGroup, `nacos group name. Empty string doesn't work!`) flag.StringVar(&cmdOps.NacosDataID, "nacos-dataid", cmdOps.NacosDataID, "nacos dataid") flag.StringVar(&cmdOps.NacosServiceName, "nacos-service-name", cmdOps.NacosServiceName, "nacos service name") + + flag.StringVar(&cmdOps.ClickhouseUsername, "clickhouse-username", cmdOps.ClickhouseUsername, "clickhouse username") + flag.StringVar(&cmdOps.ClickhousePassword, "clickhouse-password", cmdOps.ClickhousePassword, "clickhouse password") + flag.StringVar(&cmdOps.KafkaUsername, "kafka-username", cmdOps.KafkaUsername, "kafka username") + flag.StringVar(&cmdOps.KafkaPassword, "kafka-password", cmdOps.KafkaPassword, "kafka password") + flag.Parse() } diff --git a/util/common.go b/util/common.go index 424ae87e..ccfdeacb 100644 --- a/util/common.go +++ b/util/common.go @@ -62,6 +62,15 @@ type CmdOptions struct { NacosPassword string NacosDataID string NacosServiceName string // participate in assignment management if not empty + + Credentials +} + +type Credentials struct { + ClickhouseUsername string + ClickhousePassword string + KafkaUsername string + KafkaPassword string } // StringContains check if contains string in array From b34051a2364e8bb8bc0e82691a75fc0bf0afef76 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 15:30:46 +0300 Subject: [PATCH 345/404] Apply credentials from envs to final config Signed-off-by: sashayakovtseva --- task/sinker.go | 35 ++++++++++++++++++++++++++--------- 1 file changed, 26 insertions(+), 9 deletions(-) diff --git a/task/sinker.go b/task/sinker.go index 259af38f..8105c7b5 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -40,7 +40,7 @@ import ( ) var ( - createTableSQL = `CREATE TABLE IF NOT EXISTS %s as %s.%s ENGINE=Merge('%s', '%s')` + createTableSQL = `CREATE TABLE IF NOT EXISTS %s AS %s.%s ENGINE=Merge('%s', '%s')` dropTableSQL = `DROP TABLE IF EXISTS %s ` countSeriesSQL = `WITH (SELECT max(timestamp) FROM %s) AS m SELECT count() FROM %s FINAL WHERE __series_id GLOBAL IN ( @@ -111,15 +111,17 @@ func (s *Sinker) Run() { } if s.rcm == nil { - if _, err = os.Stat(s.cmdOps.LocalCfgFile); err == nil { - if newCfg, err = config.ParseLocalCfgFile(s.cmdOps.LocalCfgFile); err != nil { - util.Logger.Fatal("config.ParseLocalCfgFile failed", zap.Error(err)) - return - } - } else { + if _, err := os.Stat(s.cmdOps.LocalCfgFile); err != nil { util.Logger.Fatal("expect --local-cfg-file or --nacos-dataid") return } + + if newCfg, err = config.ParseLocalCfgFile(s.cmdOps.LocalCfgFile); err != nil { + util.Logger.Fatal("config.ParseLocalCfgFile failed", zap.Error(err)) + return + } + applyCredentials(newCfg, s.cmdOps.Credentials) + ha := "" if s.cmdOps.NacosServiceName != "" { ha = s.httpAddr @@ -302,7 +304,7 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { } func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { - util.Logger.Info("going to apply the first config", zap.Reflect("config", newCfg)) + util.Logger.Info("going to apply the first config") // 1. Initialize clickhouse connections chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, @@ -338,7 +340,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { } func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { - util.Logger.Info("going to apply another config", zap.Int("number", s.numCfg), zap.Reflect("config", newCfg)) + util.Logger.Info("going to apply another config", zap.Int("number", s.numCfg)) if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) { // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). s.stopAllTasks() @@ -453,6 +455,21 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { return } +func applyCredentials(newCfg *config.Config, cred util.Credentials) { + if cred.ClickhouseUsername != "" { + newCfg.Clickhouse.Username = cred.ClickhouseUsername + } + if cred.ClickhousePassword != "" { + newCfg.Clickhouse.Password = cred.ClickhousePassword + } + if cred.KafkaUsername != "" { + newCfg.Kafka.Sasl.Username = cred.KafkaUsername + } + if cred.KafkaPassword != "" { + newCfg.Kafka.Sasl.Password = cred.KafkaPassword + } +} + func (s *Sinker) commitFn() { for { select { From 9e345c4e1c18b00eaeed44418a4fe6655e112fb9 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 15:42:06 +0300 Subject: [PATCH 346/404] Fix build Signed-off-by: sashayakovtseva --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index d837a02c..ed0d2967 100644 --- a/Makefile +++ b/Makefile @@ -14,7 +14,7 @@ GOBUILD := $(GO) build $(BUILD_FLAG) .PHONY: pre pre: - go mod tidy + go mod tidy -compat="1.20" .PHONY: build build: pre From 491707d2dbefb1da345505b81d84730042dd1f30 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 15:45:19 +0300 Subject: [PATCH 347/404] Update go version in CI Signed-off-by: sashayakovtseva --- .github/workflows/lint.yml | 6 +++--- .github/workflows/tests.yml | 4 ++-- Makefile | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index 53980c5e..74df4a5d 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -14,15 +14,15 @@ jobs: golangci-lint: runs-on: ubuntu-latest steps: - - uses: actions/setup-go@v3 + - uses: actions/setup-go@v4 with: - go-version: 1.19 + go-version: '1.20' - uses: actions/checkout@v3 - name: golangci-lint uses: golangci/golangci-lint-action@v3 with: # Optional: version of golangci-lint to use in form of v1.2 or v1.2.3 or `latest` to use the latest version - version: v1.48.0 + version: v1.51.2 # Optional: working directory, useful for monorepos # working-directory: somedir diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index a1d2d186..73c464b7 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -14,9 +14,9 @@ jobs: steps: - name: Set up Go 1.x - uses: actions/setup-go@v2 + uses: actions/setup-go@v4 with: - go-version: 1.19 + go-version: '1.20' id: go - name: Check out code into the Go module directory diff --git a/Makefile b/Makefile index ed0d2967..d837a02c 100644 --- a/Makefile +++ b/Makefile @@ -14,7 +14,7 @@ GOBUILD := $(GO) build $(BUILD_FLAG) .PHONY: pre pre: - go mod tidy -compat="1.20" + go mod tidy .PHONY: build build: pre From 523bedc46a11f5af56f91fd868aa4cc5117c3050 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 17:10:33 +0300 Subject: [PATCH 348/404] Fix tests in CI Signed-off-by: sashayakovtseva --- .github/workflows/tests.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index 73c464b7..285dbcfc 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -31,7 +31,7 @@ jobs: run: make build - name: unittest - run: make unittest + run: make gotest - name: benchtest run: make benchtest From 5ffaf07abd09b2a73a7ad2536eb7e1deacfc86bd Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Wed, 2 Aug 2023 13:57:55 +0300 Subject: [PATCH 349/404] Apply credentials in config.Normalize Signed-off-by: sashayakovtseva --- cmd/nacos_publish_config/main.go | 12 +++++++++++- config/config.go | 33 ++++++++++++++++++++++---------- task/sinker.go | 20 ++----------------- 3 files changed, 36 insertions(+), 29 deletions(-) diff --git a/cmd/nacos_publish_config/main.go b/cmd/nacos_publish_config/main.go index 58273663..0cc1ad77 100644 --- a/cmd/nacos_publish_config/main.go +++ b/cmd/nacos_publish_config/main.go @@ -47,6 +47,11 @@ var ( localCfgFile = flag.String("local-cfg-file", "/etc/clickhouse_sinker.hjson", "local config file") replicas = flag.Int("replicas", 1, "replicate each task to multiple ones with the same config except task name, consumer group and table name") maxOpenConns = flag.Int("max-open-conns", 0, "max open connections per shard") + + clickhouseUsername = flag.String("clickhouse-username", "", "clickhouse username") + clickhousePassword = flag.String("clickhouse-password", "", "clickhouse password") + kafkaUsername = flag.String("kafka-username", "", "kafka username") + kafkaPassword = flag.String("kafka-password", "", "kafka password") ) // Empty is not valid namespaceID @@ -74,7 +79,12 @@ func PublishSinkerConfig() { return } - if err = cfg.Normallize(false, ""); err != nil { + if err = cfg.Normallize(false, "", util.Credentials{ + ClickhouseUsername: *clickhouseUsername, + ClickhousePassword: *clickhousePassword, + KafkaUsername: *kafkaUsername, + KafkaPassword: *kafkaPassword, + }); err != nil { util.Logger.Fatal("cfg.Normallize failed", zap.Error(err)) return } diff --git a/config/config.go b/config/config.go index 090762f1..9d1738ba 100644 --- a/config/config.go +++ b/config/config.go @@ -52,13 +52,13 @@ type KafkaConfig struct { ClientCertFile string // Required for client authentication. It's client cert.pem. ClientKeyFile string // Required if and only if ClientCertFile is present. It's client key.pem. - TrustStoreLocation string //JKS format of CA certificate, used to extract CA cert.pem. + TrustStoreLocation string // JKS format of CA certificate, used to extract CA cert.pem. TrustStorePassword string - KeystoreLocation string //JKS format of client certificate and key, used to extrace client cert.pem and key.pem. + KeystoreLocation string // JKS format of client certificate and key, used to extrace client cert.pem and key.pem. KeystorePassword string EndpIdentAlgo string } - //simplified sarama.Config.Net.SASL to only support SASL/PLAIN and SASL/GSSAPI(Kerberos) + // simplified sarama.Config.Net.SASL to only support SASL/PLAIN and SASL/GSSAPI(Kerberos) Sasl struct { // Whether or not to use SASL authentication when connecting to the broker // (defaults to false). @@ -72,7 +72,7 @@ type KafkaConfig struct { // Password for SASL/PLAIN or SASL/SCRAM authentication Password string GSSAPI struct { - AuthType int //1. KRB5_USER_AUTH, 2. KRB5_KEYTAB_AUTH + AuthType int // 1. KRB5_USER_AUTH, 2. KRB5_KEYTAB_AUTH KeyTabPath string KerberosConfigPath string ServiceName string @@ -98,7 +98,7 @@ type ClickHouseConfig struct { // Whether skip verify clickhouse-server cert InsecureSkipVerify bool - RetryTimes int //<=0 means retry infinitely + RetryTimes int // <=0 means retry infinitely MaxOpenConns int } @@ -172,8 +172,8 @@ type Assignment struct { } const ( - MaxBufferSize = 1 << 20 //1048576 - defaultBufferSize = 1 << 18 //262144 + MaxBufferSize = 1 << 20 // 1048576 + defaultBufferSize = 1 << 18 // 262144 maxFlushInterval = 600 defaultFlushInterval = 10 defaultTimeZone = "Local" @@ -202,7 +202,20 @@ func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { } // Normalize and validate configuration -func (cfg *Config) Normallize(constructGroup bool, httpAddr string) (err error) { +func (cfg *Config) Normallize(constructGroup bool, httpAddr string, cred util.Credentials) (err error) { + if cred.ClickhouseUsername != "" { + cfg.Clickhouse.Username = cred.ClickhouseUsername + } + if cred.ClickhousePassword != "" { + cfg.Clickhouse.Password = cred.ClickhousePassword + } + if cred.KafkaUsername != "" { + cfg.Kafka.Sasl.Username = cred.KafkaUsername + } + if cred.KafkaPassword != "" { + cfg.Kafka.Sasl.Password = cred.KafkaPassword + } + if len(cfg.Clickhouse.Hosts) == 0 || cfg.Kafka.Brokers == "" { err = errors.Newf("invalid configuration, Clickhouse or Kafka section is missing!") return @@ -388,12 +401,12 @@ func (cfg *Config) convertKfkSecurity() { if strings.Contains(cfg.Kafka.Sasl.Mechanism, "GSSAPI") { // GSSAPI if configMap["useKeyTab"] != "true" { - //Username and password + // Username and password cfg.Kafka.Sasl.GSSAPI.AuthType = 1 cfg.Kafka.Sasl.GSSAPI.Username = configMap["username"] cfg.Kafka.Sasl.GSSAPI.Password = configMap["password"] } else { - //Keytab + // Keytab cfg.Kafka.Sasl.GSSAPI.AuthType = 2 cfg.Kafka.Sasl.GSSAPI.KeyTabPath = configMap["keyTab"] if principal, ok := configMap["principal"]; ok { diff --git a/task/sinker.go b/task/sinker.go index 8105c7b5..df5e4e53 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -120,13 +120,12 @@ func (s *Sinker) Run() { util.Logger.Fatal("config.ParseLocalCfgFile failed", zap.Error(err)) return } - applyCredentials(newCfg, s.cmdOps.Credentials) ha := "" if s.cmdOps.NacosServiceName != "" { ha = s.httpAddr } - if err = newCfg.Normallize(true, ha); err != nil { + if err = newCfg.Normallize(true, ha, s.cmdOps.Credentials); err != nil { util.Logger.Fatal("newCfg.Normallize failed", zap.Error(err)) return } @@ -193,7 +192,7 @@ func (s *Sinker) Run() { if s.cmdOps.NacosServiceName != "" { ha = s.httpAddr } - if err = newCfg.Normallize(true, ha); err != nil { + if err = newCfg.Normallize(true, ha, s.cmdOps.Credentials); err != nil { util.Logger.Error("newCfg.Normallize failed", zap.Error(err)) continue } @@ -455,21 +454,6 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { return } -func applyCredentials(newCfg *config.Config, cred util.Credentials) { - if cred.ClickhouseUsername != "" { - newCfg.Clickhouse.Username = cred.ClickhouseUsername - } - if cred.ClickhousePassword != "" { - newCfg.Clickhouse.Password = cred.ClickhousePassword - } - if cred.KafkaUsername != "" { - newCfg.Kafka.Sasl.Username = cred.KafkaUsername - } - if cred.KafkaPassword != "" { - newCfg.Kafka.Sasl.Password = cred.KafkaPassword - } -} - func (s *Sinker) commitFn() { for { select { From b6e253e9df0801022710bba3c238c9ed95362bcd Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Wed, 2 Aug 2023 20:56:40 +0300 Subject: [PATCH 350/404] Log config without passwords Signed-off-by: sashayakovtseva --- config/config.go | 6 +++--- task/sinker.go | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/config/config.go b/config/config.go index 9d1738ba..f905a610 100644 --- a/config/config.go +++ b/config/config.go @@ -70,14 +70,14 @@ type KafkaConfig struct { // SASL/PLAIN or SASL/SCRAM authentication Username string // Password for SASL/PLAIN or SASL/SCRAM authentication - Password string + Password string `json:"-"` GSSAPI struct { AuthType int // 1. KRB5_USER_AUTH, 2. KRB5_KEYTAB_AUTH KeyTabPath string KerberosConfigPath string ServiceName string Username string - Password string + Password string `json:"-"` Realm string DisablePAFXFAST bool } @@ -91,7 +91,7 @@ type ClickHouseConfig struct { Hosts [][]string Port int Username string - Password string + Password string `json:"-"` // Whether enable TLS encryption with clickhouse-server Secure bool diff --git a/task/sinker.go b/task/sinker.go index df5e4e53..1f3fb117 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -303,7 +303,7 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { } func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { - util.Logger.Info("going to apply the first config") + util.Logger.Info("going to apply the first config", zap.Any("config", newCfg)) // 1. Initialize clickhouse connections chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, @@ -339,7 +339,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { } func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { - util.Logger.Info("going to apply another config", zap.Int("number", s.numCfg)) + util.Logger.Info("going to apply another config", zap.Int("number", s.numCfg), zap.Any("config", newCfg)) if !reflect.DeepEqual(newCfg.Kafka, s.curCfg.Kafka) || !reflect.DeepEqual(newCfg.Clickhouse, s.curCfg.Clickhouse) { // 1. Stop tasks gracefully. Wait until all flying data be processed (write to CH and commit to Kafka). s.stopAllTasks() From e31152e2ee4c9d4d8d0f5b31c750d082e3af8043 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Wed, 2 Aug 2023 21:19:02 +0300 Subject: [PATCH 351/404] Add kafka gssapi credentials as well Signed-off-by: sashayakovtseva --- Dockerfile | 2 ++ Dockerfile_goreleaser | 2 ++ cmd/clickhouse_sinker/main.go | 4 ++++ config/config.go | 6 ++++++ util/common.go | 10 ++++++---- 5 files changed, 20 insertions(+), 4 deletions(-) diff --git a/Dockerfile b/Dockerfile index feb73e73..a8d6c8cb 100644 --- a/Dockerfile +++ b/Dockerfile @@ -34,6 +34,8 @@ COPY --from=builder /app/kafka_gen_metric /usr/local/bin/kafka_gen_metric # - CLICKHOUSE_PASSWORD # - KAFKA_USERNAME # - KAFKA_PASSWORD +# - KAFKA_GSSAPI_USERNAME +# - KAFKA_GSSAPI_PASSWORD # See cmd/clickhouse_sinker/main.go for details. ENTRYPOINT ["/usr/local/bin/clickhouse_sinker"] diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index 0340cafc..fbc6662e 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -25,6 +25,8 @@ ADD dist/nacos_publish_config_linux_amd64_v1/nacos_publish_config /usr/local/bin # - CLICKHOUSE_PASSWORD # - KAFKA_USERNAME # - KAFKA_PASSWORD +# - KAFKA_GSSAPI_USERNAME +# - KAFKA_GSSAPI_PASSWORD # See cmd/clickhouse_sinker/main.go for details. ENTRYPOINT ["/usr/local/bin/clickhouse_sinker"] diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index a011fe4c..eb487501 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -84,6 +84,8 @@ func initCmdOptions() { util.EnvStringVar(&cmdOps.ClickhousePassword, "clickhouse-password") util.EnvStringVar(&cmdOps.KafkaUsername, "kafka-username") util.EnvStringVar(&cmdOps.KafkaPassword, "kafka-password") + util.EnvStringVar(&cmdOps.KafkaGSSAPIUsername, "kafka-gssapi-username") + util.EnvStringVar(&cmdOps.KafkaGSSAPIPassword, "kafka-gssapi-password") // 3. Replace options with the corresponding CLI parameter if present. flag.BoolVar(&cmdOps.ShowVer, "v", cmdOps.ShowVer, "show build version and quit") @@ -108,6 +110,8 @@ func initCmdOptions() { flag.StringVar(&cmdOps.ClickhousePassword, "clickhouse-password", cmdOps.ClickhousePassword, "clickhouse password") flag.StringVar(&cmdOps.KafkaUsername, "kafka-username", cmdOps.KafkaUsername, "kafka username") flag.StringVar(&cmdOps.KafkaPassword, "kafka-password", cmdOps.KafkaPassword, "kafka password") + flag.StringVar(&cmdOps.KafkaGSSAPIUsername, "kafka-gssapi-username", cmdOps.KafkaGSSAPIUsername, "kafka GSSAPI username") + flag.StringVar(&cmdOps.KafkaGSSAPIPassword, "kafka-gssapi-password", cmdOps.KafkaGSSAPIPassword, "kafka GSSAPI password") flag.Parse() } diff --git a/config/config.go b/config/config.go index f905a610..b98e76a6 100644 --- a/config/config.go +++ b/config/config.go @@ -215,6 +215,12 @@ func (cfg *Config) Normallize(constructGroup bool, httpAddr string, cred util.Cr if cred.KafkaPassword != "" { cfg.Kafka.Sasl.Password = cred.KafkaPassword } + if cred.KafkaGSSAPIUsername != "" { + cfg.Kafka.Sasl.GSSAPI.Username = cred.KafkaGSSAPIUsername + } + if cred.KafkaGSSAPIPassword != "" { + cfg.Kafka.Sasl.GSSAPI.Password = cred.KafkaGSSAPIPassword + } if len(cfg.Clickhouse.Hosts) == 0 || cfg.Kafka.Brokers == "" { err = errors.Newf("invalid configuration, Clickhouse or Kafka section is missing!") diff --git a/util/common.go b/util/common.go index ccfdeacb..3032d776 100644 --- a/util/common.go +++ b/util/common.go @@ -67,10 +67,12 @@ type CmdOptions struct { } type Credentials struct { - ClickhouseUsername string - ClickhousePassword string - KafkaUsername string - KafkaPassword string + ClickhouseUsername string + ClickhousePassword string + KafkaUsername string + KafkaPassword string + KafkaGSSAPIUsername string + KafkaGSSAPIPassword string } // StringContains check if contains string in array From d69680db70eddb50c8c71b364b476b9cd6581c39 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Thu, 3 Aug 2023 11:27:05 +0300 Subject: [PATCH 352/404] Revert removing passwords from logs Signed-off-by: sashayakovtseva --- config/config.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/config/config.go b/config/config.go index b98e76a6..cb84c7fe 100644 --- a/config/config.go +++ b/config/config.go @@ -70,14 +70,14 @@ type KafkaConfig struct { // SASL/PLAIN or SASL/SCRAM authentication Username string // Password for SASL/PLAIN or SASL/SCRAM authentication - Password string `json:"-"` + Password string GSSAPI struct { AuthType int // 1. KRB5_USER_AUTH, 2. KRB5_KEYTAB_AUTH KeyTabPath string KerberosConfigPath string ServiceName string Username string - Password string `json:"-"` + Password string Realm string DisablePAFXFAST bool } @@ -91,7 +91,7 @@ type ClickHouseConfig struct { Hosts [][]string Port int Username string - Password string `json:"-"` + Password string // Whether enable TLS encryption with clickhouse-server Secure bool From 3101c6c1fcd61d055f247a6b31e844b52b55044f Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 15 Jun 2023 14:24:54 +0800 Subject: [PATCH 353/404] update MaxIdleConns to maxOpenConns --- CHANGELOG.md | 6 +++ go.mod | 47 ++++++++++--------- go.sum | 99 ++++++++++++++++++++-------------------- output/clickhouse.go | 4 +- pool/conn.go | 2 +- statistics/statistics.go | 2 +- 6 files changed, 83 insertions(+), 77 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0bcb408c..63022cf4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,11 @@ # Changelog +#### Version 3.0.7 (2023-06-05) + +New Features: + +- Introduced a new configuration property "OverrideSeries", which allows you to override the existing records in series table. + #### Version 3.0.6 (2023-05-19) Improvements: diff --git a/go.mod b/go.mod index 64e08492..c2649e12 100644 --- a/go.mod +++ b/go.mod @@ -3,10 +3,10 @@ module github.com/housepower/clickhouse_sinker go 1.20 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.9.0 - github.com/RoaringBitmap/roaring v1.2.3 - github.com/avast/retry-go/v4 v4.3.3 - github.com/bytedance/sonic v1.8.7 + github.com/ClickHouse/clickhouse-go/v2 v2.10.0 + github.com/RoaringBitmap/roaring v1.3.0 + github.com/avast/retry-go/v4 v4.3.4 + github.com/bytedance/sonic v1.8.10 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.27 github.com/google/uuid v1.3.0 @@ -14,30 +14,30 @@ require ( github.com/jcmturner/gokrb5/v8 v8.4.4 github.com/jinzhu/copier v0.3.5 github.com/nacos-group/nacos-sdk-go v1.1.4 - github.com/prometheus/client_golang v1.15.0 - github.com/prometheus/common v0.42.0 + github.com/prometheus/client_golang v1.15.1 + github.com/prometheus/common v0.44.0 github.com/shopspring/decimal v1.3.1 - github.com/stretchr/testify v1.8.2 + github.com/stretchr/testify v1.8.3 github.com/thanos-io/thanos v0.31.0 github.com/tidwall/gjson v1.14.4 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.13.2 - github.com/twmb/franz-go/pkg/kadm v1.8.0 + github.com/twmb/franz-go v1.13.4 + github.com/twmb/franz-go/pkg/kadm v1.8.1 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.2 github.com/valyala/fastjson v1.6.4 go.uber.org/zap v1.24.0 - golang.org/x/exp v0.0.0-20230321023759-10a507213a29 + golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1 golang.org/x/time v0.3.0 gopkg.in/natefinch/lumberjack.v2 v2.2.1 ) require ( - github.com/ClickHouse/ch-go v0.54.0 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.282 // indirect + github.com/ClickHouse/ch-go v0.56.0 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.345 // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/bits-and-blooms/bitset v1.5.0 // indirect + github.com/bits-and-blooms/bitset v1.7.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect github.com/davecgh/go-spew v1.1.1 // indirect @@ -60,28 +60,27 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b // indirect - github.com/paulmach/orb v0.9.0 // indirect + github.com/paulmach/orb v0.9.2 // indirect github.com/pierrec/lz4/v4 v4.1.17 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect - github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect - github.com/rogpeppe/go-internal v1.6.1 // indirect + github.com/prometheus/client_model v0.4.0 // indirect + github.com/prometheus/procfs v0.10.0 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.1 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect github.com/twmb/franz-go/pkg/kmsg v1.5.0 // indirect - go.opentelemetry.io/otel v1.14.0 // indirect - go.opentelemetry.io/otel/trace v1.14.0 // indirect - go.uber.org/atomic v1.10.0 // indirect + go.opentelemetry.io/otel v1.16.0 // indirect + go.opentelemetry.io/otel/trace v1.16.0 // indirect + go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.3.0 // indirect - golang.org/x/crypto v0.8.0 // indirect - golang.org/x/net v0.9.0 // indirect - golang.org/x/sync v0.1.0 // indirect - golang.org/x/sys v0.7.0 // indirect + golang.org/x/crypto v0.9.0 // indirect + golang.org/x/net v0.10.0 // indirect + golang.org/x/sync v0.2.0 // indirect + golang.org/x/sys v0.8.0 // indirect google.golang.org/protobuf v1.30.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 4ec02851..c0e76753 100644 --- a/go.sum +++ b/go.sum @@ -1,36 +1,36 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.54.0 h1:WzPo/iZ8Gchb9Ze30TywMheOzKW5FkhTeZPxvz/iE6o= -github.com/ClickHouse/ch-go v0.54.0/go.mod h1:2jvyjBRb5zhzFvcOBxPelzpbB9qsS47vwJssurJe2OA= -github.com/ClickHouse/clickhouse-go/v2 v2.9.0 h1:vh0D+9p5hKWLYqCfU8Hd7/GMXNfHC84isUf7sNlJrZk= -github.com/ClickHouse/clickhouse-go/v2 v2.9.0/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= +github.com/ClickHouse/ch-go v0.56.0 h1:Uli/jOQl7+pYyA2E+/VvOSQRX3jHZ4FjESCRReAeFuM= +github.com/ClickHouse/ch-go v0.56.0/go.mod h1:nKtI66Lz8pRvbO7NZDrgbRaC9ldsD4HMeWKb1hpbpBI= +github.com/ClickHouse/clickhouse-go/v2 v2.10.0 h1:0w/A50D5MfsRUYBaV6rLKwZ4LXWKLZKJ1u31QXjTIO4= +github.com/ClickHouse/clickhouse-go/v2 v2.10.0/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/RoaringBitmap/roaring v1.2.3 h1:yqreLINqIrX22ErkKI0vY47/ivtJr6n+kMhVOVmhWBY= -github.com/RoaringBitmap/roaring v1.2.3/go.mod h1:plvDsJQpxOC5bw8LRteu/MLWHsHez/3y6cubLI4/1yE= +github.com/RoaringBitmap/roaring v1.3.0 h1:aQmu9zQxDU0uhwR8SXOH/OrqEf+X8A0LQmwW3JX8Lcg= +github.com/RoaringBitmap/roaring v1.3.0/go.mod h1:plvDsJQpxOC5bw8LRteu/MLWHsHez/3y6cubLI4/1yE= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.282 h1:UK4Eq2Xs1PfdI5w9CeNAHbZUGOT81oNAp+ahUtnjP1g= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.282/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.345 h1:15ivO7Sc1cncAwQzbD3A7fk6U5Twfwww+yMfruABHmk= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.345/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= -github.com/avast/retry-go/v4 v4.3.3 h1:G56Bp6mU0b5HE1SkaoVjscZjlQb0oy4mezwY/cGH19w= -github.com/avast/retry-go/v4 v4.3.3/go.mod h1:rg6XFaiuFYII0Xu3RDbZQkxCofFwruZKW8oEF1jpWiU= +github.com/avast/retry-go/v4 v4.3.4 h1:pHLkL7jvCvP317I8Ge+Km2Yhntv3SdkJm7uekkqbKhM= +github.com/avast/retry-go/v4 v4.3.4/go.mod h1:rv+Nla6Vk3/ilU0H51VHddWHiwimzX66yZ0JT6T+UvE= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= -github.com/bits-and-blooms/bitset v1.5.0 h1:NpE8frKRLGHIcEzkR+gZhiioW1+WbYV6fKwD6ZIpQT8= -github.com/bits-and-blooms/bitset v1.5.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= +github.com/bits-and-blooms/bitset v1.7.0 h1:YjAGVd3XmtK9ktAbX8Zg2g2PwLIMjGREZJHlV4j7NEo= +github.com/bits-and-blooms/bitset v1.7.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.8.7 h1:d3sry5vGgVq/OpgozRUNP6xBsSo0mtNdwliApw+SAMQ= -github.com/bytedance/sonic v1.8.7/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/bytedance/sonic v1.8.10 h1:XFSQg4/rwpQnNWSybNDr8oz6QtQY9uRGfRKDVWVsvP8= +github.com/bytedance/sonic v1.8.10/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -62,7 +62,6 @@ github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFU github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= @@ -147,8 +146,8 @@ github.com/nacos-group/nacos-sdk-go v1.1.4 h1:qyrZ7HTWM4aeymFfqnbgNRERh7TWuER10p github.com/nacos-group/nacos-sdk-go v1.1.4/go.mod h1:cBv9wy5iObs7khOqov1ERFQrCuTR4ILpgaiaVMxEmGI= github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b h1:FfH+VrHHk6Lxt9HdVS0PXzSXFyS2NbZKXv33FYPol0A= github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b/go.mod h1:AC62GU6hc0BrNm+9RK9VSiwa/EUe1bkIeFORAMcHvJU= -github.com/paulmach/orb v0.9.0 h1:MwA1DqOKtvCgm7u9RZ/pnYejTeDJPnr0+0oFajBbJqk= -github.com/paulmach/orb v0.9.0/go.mod h1:SudmOk85SXtmXAB3sLGyJ6tZy/8pdfrV0o6ef98Xc30= +github.com/paulmach/orb v0.9.2 h1:p/YWV2uJwamAynnDOJGNbPBVtDHj3vG51k9tR1rFwJE= +github.com/paulmach/orb v0.9.2/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.17 h1:kV4Ip+/hUBC+8T6+2EgburRtkE9ef4nbY3f4dFhGjMc= @@ -163,25 +162,24 @@ github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= -github.com/prometheus/client_golang v1.15.0 h1:5fCgGYogn0hFdhyhLbw7hEsWxufKtY9klyvdNfFlFhM= -github.com/prometheus/client_golang v1.15.0/go.mod h1:e9yaBhRPU2pPNsZwE+JdQl0KEt1N9XgF6zxWmaC0xOk= +github.com/prometheus/client_golang v1.15.1 h1:8tXpTmJbyH5lydzFPoxSIJ0J46jdh3tylbvM1xCv0LI= +github.com/prometheus/client_golang v1.15.1/go.mod h1:e9yaBhRPU2pPNsZwE+JdQl0KEt1N9XgF6zxWmaC0xOk= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.3.0 h1:UBgGFHqYdG/TPFD1B1ogZywDqEkwp3fBMvqdiQ7Xew4= -github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w= +github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= +github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= -github.com/prometheus/common v0.42.0 h1:EKsfXEYo4JpWMHH5cg+KOUWeuJSov1Id8zGR8eeI1YM= -github.com/prometheus/common v0.42.0/go.mod h1:xBwqVerjNdUDjgODMpudtOMwlOwf2SaTr1yjz4b7Zbc= +github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= +github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/prometheus/procfs v0.9.0 h1:wzCHvIvM5SxWqYvwgVL7yJY8Lz3PKn49KQtpgMYJfhI= -github.com/prometheus/procfs v0.9.0/go.mod h1:+pB4zwohETzFnmlpe6yd2lSc+0/46IYZRB/chUwxUZY= +github.com/prometheus/procfs v0.10.0 h1:UkG7GPYkO4UZyLnyXjaWYcgOSONqwdBqFUT95ugmt6I= +github.com/prometheus/procfs v0.10.0/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= -github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= +github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= @@ -206,8 +204,9 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.3 h1:RP3t2pwF7cMEbC1dqtB6poj3niw/9gnV4Cjg5oW5gtY= +github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/thanos-io/thanos v0.31.0 h1:jOCmaiIXwpByWXoVtHnktLm3YDB9xDQQzmZvd1XG5oY= github.com/thanos-io/thanos v0.31.0/go.mod h1:5ux+jb2oKr59+3XsCC0mX+JuAbPGJEMijjhcmnL/PMo= github.com/tidwall/gjson v1.14.4 h1:uo0p8EbA09J7RQaflQ1aBRffTR7xedD2bcIVSYxLnkM= @@ -223,10 +222,10 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0 github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go v1.13.2 h1:jIdDoFiq8uP3Zrx6TZZTXpaXrv3bh1w3tV5mn/B+Gw8= -github.com/twmb/franz-go v1.13.2/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= -github.com/twmb/franz-go/pkg/kadm v1.8.0 h1:vvKwZpxYn+VmM32v4mKkecHLKavZW+HcYLRKKxly5ZY= -github.com/twmb/franz-go/pkg/kadm v1.8.0/go.mod h1:qUSM7pxoMCU1UNu5H4USE64ODcVmeG9LS96mysv1nu8= +github.com/twmb/franz-go v1.13.4 h1:xu+lTAJChFqDh6u+Edr1IR2sq81nS+XGtk6OETTxFbc= +github.com/twmb/franz-go v1.13.4/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= +github.com/twmb/franz-go/pkg/kadm v1.8.1 h1:SrzL855I7gQTGdMtOYGTHhebs7TPgPN29FPtjusqwlE= +github.com/twmb/franz-go/pkg/kadm v1.8.1/go.mod h1:qUSM7pxoMCU1UNu5H4USE64ODcVmeG9LS96mysv1nu8= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.5.0 h1:eqVJquFQLdBNLrRMWX03pPDPpngn6PTjGZLlZnagouk= github.com/twmb/franz-go/pkg/kmsg v1.5.0/go.mod h1:se9Mjdt0Nwzc9lnjJ0HyDtLyBnaBDAd7pCje47OhSyw= @@ -247,16 +246,16 @@ github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d/go.mod h1:rHwXgn7Jul github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.mongodb.org/mongo-driver v1.11.1/go.mod h1:s7p5vEtfbeR1gYi6pnj3c3/urpbLv2T5Sfd6Rp2HBB8= -go.opentelemetry.io/otel v1.14.0 h1:/79Huy8wbf5DnIPhemGB+zEPVwnN6fuQybr/SRXa6hM= -go.opentelemetry.io/otel v1.14.0/go.mod h1:o4buv+dJzx8rohcUeRmWUZhqupFvzWis188WlggnNeU= -go.opentelemetry.io/otel/trace v1.14.0 h1:wp2Mmvj41tDsyAJXiWDWpfNsOiIyd38fy85pyKcFq/M= -go.opentelemetry.io/otel/trace v1.14.0/go.mod h1:8avnQLK+CG77yNLUae4ea2JDQ6iT+gozhnZjy/rw9G8= +go.mongodb.org/mongo-driver v1.11.4/go.mod h1:PTSz5yu21bkT/wXpkS7WR5f0ddqw5quethTUn9WM+2g= +go.opentelemetry.io/otel v1.16.0 h1:Z7GVAX/UkAXPKsy94IU+i6thsQS4nb7LviLpnaNeW8s= +go.opentelemetry.io/otel v1.16.0/go.mod h1:vl0h9NUa1D5s1nv3A5vZOYWn8av4K8Ml6JDeHrT/bx4= +go.opentelemetry.io/otel/trace v1.16.0 h1:8JRpaObFoW0pxuVPapkgH8UhHQj+bJW8jJsCZEu5MQs= +go.opentelemetry.io/otel/trace v1.16.0/go.mod h1:Yt9vYq1SdNz3xdjZZK7wcXv1qv2pwLkqr2QVwea0ef0= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/atomic v1.10.0 h1:9qC72Qh0+3MqyJbAn8YU5xVq1frD8bn3JtD2oXtafVQ= -go.uber.org/atomic v1.10.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= -go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= +go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= +go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= +go.uber.org/goleak v1.2.1 h1:NBol2c7O1ZokfZ0LEU9K6Whx/KnwvepVetCUhtKja4A= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= @@ -277,10 +276,10 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.8.0 h1:pd9TJtTueMTVQXzk8E2XESSMQDj/U7OUu0PqJqPXQjQ= -golang.org/x/crypto v0.8.0/go.mod h1:mRqEX+O9/h5TFCrQhkgjo2yKi0yYA+9ecGkdQoHrywE= -golang.org/x/exp v0.0.0-20230321023759-10a507213a29 h1:ooxPy7fPvB4kwsA2h+iBNHkAbp/4JxTSwCmvdjEYmug= -golang.org/x/exp v0.0.0-20230321023759-10a507213a29/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= +golang.org/x/crypto v0.9.0 h1:LF6fAI+IutBocDJ2OT0Q1g8plpYljMZ4+lty+dsqw3g= +golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= +golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1 h1:k/i9J1pBpvlfR+9QsetwPyERsqu1GIbi967PQMq3Ivc= +golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1/go.mod h1:V1LtkGg67GoY2N1AnLN78QLrzxkLyJw7RJb1gzOOz9w= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -301,8 +300,8 @@ golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.9.0 h1:aWJ/m6xSmxWBx+V0XRHTlrYrPG56jKsLdTFmsSsCzOM= -golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= +golang.org/x/net v0.10.0 h1:X2//UzNDwYmtCLn7To6G58Wr6f5ahEAQgKNzv9Y951M= +golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -310,8 +309,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.1.0 h1:wsuoTGHzEhffawBOhz5CYhcrV4IdKZbEyZjBMuTp12o= -golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.2.0 h1:PUR+T4wwASmuSTYdKjYHI5TD22Wy5ogLU5qZCOLxBrI= +golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -327,8 +326,8 @@ golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.7.0 h1:3jlCCIQZPdOYu1h8BkNvLz8Kgwtae2cagcG/VamtZRU= -golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.8.0 h1:EBmGv8NaZBZTWvrbjNoL6HVt+IVy3QDQpJs7VRIw3tU= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= diff --git a/output/clickhouse.go b/output/clickhouse.go index 50042b3e..e7e1dd30 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -276,13 +276,14 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { c.IdxSerID = -1 return } - // Strip away string columns from metric table, and move column "__series_id" to the last. + // Move column "__series_id" to the last. var dimSerID *model.ColumnWithType for i := 0; i < len(c.Dims); { dim := c.Dims[i] if dim.Name == "__series_id" && dim.Type.Type == model.Int64 { dimSerID = dim c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) + break } else { i++ } @@ -346,6 +347,7 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { c.dbName, c.seriesTbl, strings.Join(serDimsQuoted, ",")) + util.Logger.Info(fmt.Sprintf("promSer sql=> %s", c.promSerSQL), zap.String("task", c.taskCfg.Name)) // Check distributed series table if chCfg := &c.cfg.Clickhouse; chCfg.Cluster != "" { diff --git a/pool/conn.go b/pool/conn.go index db3df3bf..324499c8 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -140,7 +140,7 @@ func InitClusterConn(hosts [][]string, port int, db, username, password string, }, DialTimeout: time.Minute * 10, MaxOpenConns: maxOpenConns, - MaxIdleConns: 5, // TODO - update this property to maxOpenConns when the lifetime of an idle connection honours the ConnMaxLifetime + MaxIdleConns: maxOpenConns, ConnMaxLifetime: time.Minute * 10, }, writingPool: util.NewWorkerPool(maxOpenConns, 1), diff --git a/statistics/statistics.go b/statistics/statistics.go index 57e17e36..d9bbb3fc 100755 --- a/statistics/statistics.go +++ b/statistics/statistics.go @@ -32,7 +32,7 @@ import ( var ( prefix = "clickhouse_sinker_" - // ConsumeMsgsTotal = ParseMsgsErrorTotal + RingMsgsOffTooSmallErrorTotal + FlushMsgsTotal + FlushMsgsErrorTotal + // ConsumeMsgsTotal = ParseMsgsErrorTotal + FlushMsgsTotal + FlushMsgsErrorTotal ConsumeMsgsTotal = prometheus.NewCounterVec( prometheus.CounterOpts{ Name: prefix + "consume_msgs_total", From 28c3fcd9c642063f525c6df4e4268a6e25fcbc07 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 15 Jun 2023 15:35:51 +0800 Subject: [PATCH 354/404] caculate lag failure of certain task should not block the entire rebalance work --- CHANGELOG.md | 6 +++--- config_manager/lags.go | 7 ++++++- go.mod | 30 +++++++++++++++--------------- go.sum | 32 ++++++++++++++++++++++++++++++++ 4 files changed, 56 insertions(+), 19 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 63022cf4..a65e94d4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,10 @@ # Changelog -#### Version 3.0.7 (2023-06-05) +#### Version 3.0.7 (2023-0) -New Features: +Improvements: -- Introduced a new configuration property "OverrideSeries", which allows you to override the existing records in series table. +- when run sinker in cluster mode, caculate lag failure of certain task should not block the entire rebalance work. Instead, the current task should not be assigned to any running sinker instance. #### Version 3.0.6 (2023-05-19) diff --git a/config_manager/lags.go b/config_manager/lags.go index d5881116..9dc8afad 100644 --- a/config_manager/lags.go +++ b/config_manager/lags.go @@ -7,9 +7,11 @@ import ( "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/input" "github.com/housepower/clickhouse_sinker/statistics" + "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" "github.com/twmb/franz-go/pkg/kadm" "github.com/twmb/franz-go/pkg/kgo" + "go.uber.org/zap" ) var ( @@ -39,7 +41,10 @@ func GetTaskStateAndLags(cfg *config.Config) (stateLags map[string]StateLag, err var state string var totalLags int64 if state, totalLags, err = getStateAndLag(theAdm, taskCfg.Topic, taskCfg.ConsumerGroup); err != nil { - return + // skip this task for now, wait next assign cycle + util.Logger.Error("retrieve lag failed", zap.String("task", taskCfg.Name), zap.Error(err)) + statistics.ConsumeLags.WithLabelValues(taskCfg.ConsumerGroup, taskCfg.Topic, taskCfg.Name).Set(float64(-1)) + continue } stateLags[taskCfg.Name] = StateLag{State: state, Lag: totalLags} statistics.ConsumeLags.WithLabelValues(taskCfg.ConsumerGroup, taskCfg.Topic, taskCfg.Name).Set(float64(totalLags)) diff --git a/go.mod b/go.mod index c2649e12..966bc4c8 100644 --- a/go.mod +++ b/go.mod @@ -3,10 +3,10 @@ module github.com/housepower/clickhouse_sinker go 1.20 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.10.0 + github.com/ClickHouse/clickhouse-go/v2 v2.10.1 github.com/RoaringBitmap/roaring v1.3.0 github.com/avast/retry-go/v4 v4.3.4 - github.com/bytedance/sonic v1.8.10 + github.com/bytedance/sonic v1.9.1 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.27 github.com/google/uuid v1.3.0 @@ -17,11 +17,11 @@ require ( github.com/prometheus/client_golang v1.15.1 github.com/prometheus/common v0.44.0 github.com/shopspring/decimal v1.3.1 - github.com/stretchr/testify v1.8.3 + github.com/stretchr/testify v1.8.4 github.com/thanos-io/thanos v0.31.0 github.com/tidwall/gjson v1.14.4 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.13.4 + github.com/twmb/franz-go v1.13.5 github.com/twmb/franz-go/pkg/kadm v1.8.1 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.2 @@ -33,11 +33,11 @@ require ( ) require ( - github.com/ClickHouse/ch-go v0.56.0 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.345 // indirect + github.com/ClickHouse/ch-go v0.56.1 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.377 // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/bits-and-blooms/bitset v1.7.0 // indirect + github.com/bits-and-blooms/bitset v1.8.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect github.com/davecgh/go-spew v1.1.1 // indirect @@ -52,8 +52,8 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.16.5 // indirect - github.com/klauspost/cpuid/v2 v2.2.4 // indirect + github.com/klauspost/compress v1.16.6 // indirect + github.com/klauspost/cpuid/v2 v2.2.5 // indirect github.com/kr/text v0.2.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect @@ -61,11 +61,11 @@ require ( github.com/mschoch/smat v0.2.0 // indirect github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b // indirect github.com/paulmach/orb v0.9.2 // indirect - github.com/pierrec/lz4/v4 v4.1.17 // indirect + github.com/pierrec/lz4/v4 v4.1.18 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/procfs v0.10.0 // indirect + github.com/prometheus/procfs v0.10.1 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect @@ -77,10 +77,10 @@ require ( go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.3.0 // indirect - golang.org/x/crypto v0.9.0 // indirect - golang.org/x/net v0.10.0 // indirect - golang.org/x/sync v0.2.0 // indirect - golang.org/x/sys v0.8.0 // indirect + golang.org/x/crypto v0.10.0 // indirect + golang.org/x/net v0.11.0 // indirect + golang.org/x/sync v0.3.0 // indirect + golang.org/x/sys v0.9.0 // indirect google.golang.org/protobuf v1.30.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index c0e76753..fbeb7f79 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,13 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/ClickHouse/ch-go v0.56.0 h1:Uli/jOQl7+pYyA2E+/VvOSQRX3jHZ4FjESCRReAeFuM= github.com/ClickHouse/ch-go v0.56.0/go.mod h1:nKtI66Lz8pRvbO7NZDrgbRaC9ldsD4HMeWKb1hpbpBI= +github.com/ClickHouse/ch-go v0.56.1 h1:OshO1TWLpWGFFNFvOGDMtqRX38N7qjK9UwQwXzm20pA= +github.com/ClickHouse/ch-go v0.56.1/go.mod h1:6Mxu7RO1nuIC6/wBVzvAsUOSiLDMB0ZbTqMlBA3Gl/A= +github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= github.com/ClickHouse/clickhouse-go/v2 v2.10.0 h1:0w/A50D5MfsRUYBaV6rLKwZ4LXWKLZKJ1u31QXjTIO4= github.com/ClickHouse/clickhouse-go/v2 v2.10.0/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= +github.com/ClickHouse/clickhouse-go/v2 v2.10.1 h1:WCnusqEeCO/9sLFVIv57le/O1ydUb+x9+SYYhJ11fsY= +github.com/ClickHouse/clickhouse-go/v2 v2.10.1/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.3.0 h1:aQmu9zQxDU0uhwR8SXOH/OrqEf+X8A0LQmwW3JX8Lcg= @@ -14,6 +19,8 @@ github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRF github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= github.com/aliyun/alibaba-cloud-sdk-go v1.62.345 h1:15ivO7Sc1cncAwQzbD3A7fk6U5Twfwww+yMfruABHmk= github.com/aliyun/alibaba-cloud-sdk-go v1.62.345/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.377 h1:6czd9jysvWELv3Yiip7dZJhBVKojt+GdWIrddzokEgY= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.377/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/avast/retry-go/v4 v4.3.4 h1:pHLkL7jvCvP317I8Ge+Km2Yhntv3SdkJm7uekkqbKhM= @@ -26,11 +33,16 @@ github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6r github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.7.0 h1:YjAGVd3XmtK9ktAbX8Zg2g2PwLIMjGREZJHlV4j7NEo= github.com/bits-and-blooms/bitset v1.7.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= +github.com/bits-and-blooms/bitset v1.8.0 h1:FD+XqgOZDUxxZ8hzoBFuV9+cGWY9CslN6d5MS5JVb4c= +github.com/bits-and-blooms/bitset v1.8.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= github.com/bytedance/sonic v1.8.10 h1:XFSQg4/rwpQnNWSybNDr8oz6QtQY9uRGfRKDVWVsvP8= github.com/bytedance/sonic v1.8.10/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/bytedance/sonic v1.9.1 h1:6iJ6NqdoxCDr6mbY8h18oSO+cShGSMRGCEo7F2h0x8s= +github.com/bytedance/sonic v1.9.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -117,9 +129,13 @@ github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47e github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.16.5 h1:IFV2oUNUzZaz+XyusxpLzpzS8Pt5rh0Z16For/djlyI= github.com/klauspost/compress v1.16.5/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.16.6 h1:91SKEy4K37vkp255cJ8QesJhjyRO0hn9i9G0GoUwLsk= +github.com/klauspost/compress v1.16.6/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= +github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= +github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -152,6 +168,8 @@ github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKf github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.17 h1:kV4Ip+/hUBC+8T6+2EgburRtkE9ef4nbY3f4dFhGjMc= github.com/pierrec/lz4/v4 v4.1.17/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= +github.com/pierrec/lz4/v4 v4.1.18/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -178,6 +196,8 @@ github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsT github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/procfs v0.10.0 h1:UkG7GPYkO4UZyLnyXjaWYcgOSONqwdBqFUT95ugmt6I= github.com/prometheus/procfs v0.10.0/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= +github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= @@ -207,6 +227,8 @@ github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.3 h1:RP3t2pwF7cMEbC1dqtB6poj3niw/9gnV4Cjg5oW5gtY= github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/thanos-io/thanos v0.31.0 h1:jOCmaiIXwpByWXoVtHnktLm3YDB9xDQQzmZvd1XG5oY= github.com/thanos-io/thanos v0.31.0/go.mod h1:5ux+jb2oKr59+3XsCC0mX+JuAbPGJEMijjhcmnL/PMo= github.com/tidwall/gjson v1.14.4 h1:uo0p8EbA09J7RQaflQ1aBRffTR7xedD2bcIVSYxLnkM= @@ -224,6 +246,8 @@ github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2 github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= github.com/twmb/franz-go v1.13.4 h1:xu+lTAJChFqDh6u+Edr1IR2sq81nS+XGtk6OETTxFbc= github.com/twmb/franz-go v1.13.4/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= +github.com/twmb/franz-go v1.13.5 h1:7Hk47eZ7XRb4yWXQZk1GZU4BthkrKuZUfKOuP9Sgp24= +github.com/twmb/franz-go v1.13.5/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= github.com/twmb/franz-go/pkg/kadm v1.8.1 h1:SrzL855I7gQTGdMtOYGTHhebs7TPgPN29FPtjusqwlE= github.com/twmb/franz-go/pkg/kadm v1.8.1/go.mod h1:qUSM7pxoMCU1UNu5H4USE64ODcVmeG9LS96mysv1nu8= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= @@ -278,6 +302,8 @@ golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.9.0 h1:LF6fAI+IutBocDJ2OT0Q1g8plpYljMZ4+lty+dsqw3g= golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= +golang.org/x/crypto v0.10.0 h1:LKqV2xt9+kDzSTfOhx4FrkEBcMrAgHSYgzywV9zcGmM= +golang.org/x/crypto v0.10.0/go.mod h1:o4eNf7Ede1fv+hwOwZsTHl9EsPFO6q6ZvYR8vYfY45I= golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1 h1:k/i9J1pBpvlfR+9QsetwPyERsqu1GIbi967PQMq3Ivc= golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1/go.mod h1:V1LtkGg67GoY2N1AnLN78QLrzxkLyJw7RJb1gzOOz9w= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -302,6 +328,8 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.10.0 h1:X2//UzNDwYmtCLn7To6G58Wr6f5ahEAQgKNzv9Y951M= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.11.0 h1:Gi2tvZIJyBtO9SDr1q9h5hEQCp/4L2RQ+ar0qjx2oNU= +golang.org/x/net v0.11.0/go.mod h1:2L/ixqYpgIVXmeoSA/4Lu7BzTG4KIyPIryS4IsOd1oQ= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -311,6 +339,8 @@ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.2.0 h1:PUR+T4wwASmuSTYdKjYHI5TD22Wy5ogLU5qZCOLxBrI= golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= +golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -328,6 +358,8 @@ golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0 h1:EBmGv8NaZBZTWvrbjNoL6HVt+IVy3QDQpJs7VRIw3tU= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.9.0 h1:KS/R3tvhPqvJvwcKfnBHJwwthS11LRhmM5D59eEXa0s= +golang.org/x/sys v0.9.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= From 6dba494b7410883e1bb3aa18ce29e90553b922f0 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Mon, 19 Jun 2023 11:11:00 +0800 Subject: [PATCH 355/404] try to improve the ut coverage --- go.mod | 4 +- go.sum | 41 +--------- parser/csv.go | 7 +- parser/parser_test.go | 173 ++++++++++++++++++++++++++++++++++++++---- util/common.go | 47 ------------ util/common_test.go | 74 ++++++++++++++++++ util/net.go | 46 ++++++++++- 7 files changed, 287 insertions(+), 105 deletions(-) diff --git a/go.mod b/go.mod index 966bc4c8..879e398e 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/jcmturner/gokrb5/v8 v8.4.4 github.com/jinzhu/copier v0.3.5 github.com/nacos-group/nacos-sdk-go v1.1.4 - github.com/prometheus/client_golang v1.15.1 + github.com/prometheus/client_golang v1.16.0 github.com/prometheus/common v0.44.0 github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.8.4 @@ -65,7 +65,7 @@ require ( github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect + github.com/prometheus/procfs v0.11.0 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/streadway/amqp v1.0.0 // indirect github.com/tidwall/match v1.1.1 // indirect diff --git a/go.sum b/go.sum index fbeb7f79..60f29090 100644 --- a/go.sum +++ b/go.sum @@ -1,11 +1,6 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.56.0 h1:Uli/jOQl7+pYyA2E+/VvOSQRX3jHZ4FjESCRReAeFuM= -github.com/ClickHouse/ch-go v0.56.0/go.mod h1:nKtI66Lz8pRvbO7NZDrgbRaC9ldsD4HMeWKb1hpbpBI= github.com/ClickHouse/ch-go v0.56.1 h1:OshO1TWLpWGFFNFvOGDMtqRX38N7qjK9UwQwXzm20pA= github.com/ClickHouse/ch-go v0.56.1/go.mod h1:6Mxu7RO1nuIC6/wBVzvAsUOSiLDMB0ZbTqMlBA3Gl/A= -github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= -github.com/ClickHouse/clickhouse-go/v2 v2.10.0 h1:0w/A50D5MfsRUYBaV6rLKwZ4LXWKLZKJ1u31QXjTIO4= -github.com/ClickHouse/clickhouse-go/v2 v2.10.0/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= github.com/ClickHouse/clickhouse-go/v2 v2.10.1 h1:WCnusqEeCO/9sLFVIv57le/O1ydUb+x9+SYYhJ11fsY= github.com/ClickHouse/clickhouse-go/v2 v2.10.1/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= @@ -17,8 +12,6 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.345 h1:15ivO7Sc1cncAwQzbD3A7fk6U5Twfwww+yMfruABHmk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.345/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/aliyun/alibaba-cloud-sdk-go v1.62.377 h1:6czd9jysvWELv3Yiip7dZJhBVKojt+GdWIrddzokEgY= github.com/aliyun/alibaba-cloud-sdk-go v1.62.377/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= @@ -31,18 +24,13 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= -github.com/bits-and-blooms/bitset v1.7.0 h1:YjAGVd3XmtK9ktAbX8Zg2g2PwLIMjGREZJHlV4j7NEo= -github.com/bits-and-blooms/bitset v1.7.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.8.0 h1:FD+XqgOZDUxxZ8hzoBFuV9+cGWY9CslN6d5MS5JVb4c= github.com/bits-and-blooms/bitset v1.8.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.8.10 h1:XFSQg4/rwpQnNWSybNDr8oz6QtQY9uRGfRKDVWVsvP8= -github.com/bytedance/sonic v1.8.10/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/bytedance/sonic v1.9.1 h1:6iJ6NqdoxCDr6mbY8h18oSO+cShGSMRGCEo7F2h0x8s= github.com/bytedance/sonic v1.9.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= -github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -127,13 +115,9 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.16.5 h1:IFV2oUNUzZaz+XyusxpLzpzS8Pt5rh0Z16For/djlyI= -github.com/klauspost/compress v1.16.5/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/compress v1.16.6 h1:91SKEy4K37vkp255cJ8QesJhjyRO0hn9i9G0GoUwLsk= github.com/klauspost/compress v1.16.6/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= -github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= -github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -166,8 +150,6 @@ github.com/paulmach/orb v0.9.2 h1:p/YWV2uJwamAynnDOJGNbPBVtDHj3vG51k9tR1rFwJE= github.com/paulmach/orb v0.9.2/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= -github.com/pierrec/lz4/v4 v4.1.17 h1:kV4Ip+/hUBC+8T6+2EgburRtkE9ef4nbY3f4dFhGjMc= -github.com/pierrec/lz4/v4 v4.1.17/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= github.com/pierrec/lz4/v4 v4.1.18/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -180,8 +162,8 @@ github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= -github.com/prometheus/client_golang v1.15.1 h1:8tXpTmJbyH5lydzFPoxSIJ0J46jdh3tylbvM1xCv0LI= -github.com/prometheus/client_golang v1.15.1/go.mod h1:e9yaBhRPU2pPNsZwE+JdQl0KEt1N9XgF6zxWmaC0xOk= +github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= +github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -194,10 +176,8 @@ github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/prometheus/procfs v0.10.0 h1:UkG7GPYkO4UZyLnyXjaWYcgOSONqwdBqFUT95ugmt6I= -github.com/prometheus/procfs v0.10.0/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/procfs v0.11.0 h1:5EAgkfkMl659uZPbe9AS2N68a7Cc1TJbPEuGzFuRbyk= +github.com/prometheus/procfs v0.11.0/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= @@ -225,8 +205,6 @@ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.3 h1:RP3t2pwF7cMEbC1dqtB6poj3niw/9gnV4Cjg5oW5gtY= -github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/thanos-io/thanos v0.31.0 h1:jOCmaiIXwpByWXoVtHnktLm3YDB9xDQQzmZvd1XG5oY= @@ -244,8 +222,6 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0 github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go v1.13.4 h1:xu+lTAJChFqDh6u+Edr1IR2sq81nS+XGtk6OETTxFbc= -github.com/twmb/franz-go v1.13.4/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= github.com/twmb/franz-go v1.13.5 h1:7Hk47eZ7XRb4yWXQZk1GZU4BthkrKuZUfKOuP9Sgp24= github.com/twmb/franz-go v1.13.5/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= github.com/twmb/franz-go/pkg/kadm v1.8.1 h1:SrzL855I7gQTGdMtOYGTHhebs7TPgPN29FPtjusqwlE= @@ -300,8 +276,6 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.9.0 h1:LF6fAI+IutBocDJ2OT0Q1g8plpYljMZ4+lty+dsqw3g= -golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= golang.org/x/crypto v0.10.0 h1:LKqV2xt9+kDzSTfOhx4FrkEBcMrAgHSYgzywV9zcGmM= golang.org/x/crypto v0.10.0/go.mod h1:o4eNf7Ede1fv+hwOwZsTHl9EsPFO6q6ZvYR8vYfY45I= golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1 h1:k/i9J1pBpvlfR+9QsetwPyERsqu1GIbi967PQMq3Ivc= @@ -326,8 +300,6 @@ golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.10.0 h1:X2//UzNDwYmtCLn7To6G58Wr6f5ahEAQgKNzv9Y951M= -golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.11.0 h1:Gi2tvZIJyBtO9SDr1q9h5hEQCp/4L2RQ+ar0qjx2oNU= golang.org/x/net v0.11.0/go.mod h1:2L/ixqYpgIVXmeoSA/4Lu7BzTG4KIyPIryS4IsOd1oQ= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -337,8 +309,6 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.2.0 h1:PUR+T4wwASmuSTYdKjYHI5TD22Wy5ogLU5qZCOLxBrI= -golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -352,12 +322,9 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.8.0 h1:EBmGv8NaZBZTWvrbjNoL6HVt+IVy3QDQpJs7VRIw3tU= -golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.9.0 h1:KS/R3tvhPqvJvwcKfnBHJwwthS11LRhmM5D59eEXa0s= golang.org/x/sys v0.9.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= diff --git a/parser/csv.go b/parser/csv.go index 3dcffeaf..9ec4249b 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -93,7 +93,10 @@ func (c *CsvMetric) GetDecimal(key string, nullable bool) (val interface{}) { val = decimal.NewFromInt(0) return } - val, _ = decimal.NewFromString(c.values[idx]) + var err error + if val, err = decimal.NewFromString(c.values[idx]); err != nil { + val = decimal.NewFromInt(0) + } return } @@ -207,7 +210,7 @@ func CsvGetFloat[T constraints.Float](c *CsvMetric, key string, nullable bool, m if nullable { return } - val = float64(0.0) + val = T(0.0) return } val2 := fastfloat.ParseBestEffort(c.values[idx]) diff --git a/parser/parser_test.go b/parser/parser_test.go index fadf26fc..b05801d5 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -30,10 +30,12 @@ import ( "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" + "github.com/shopspring/decimal" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/tidwall/gjson" "github.com/valyala/fastjson" + "golang.org/x/exp/constraints" ) // https://golang.org/pkg/math/, Mathematical constants @@ -157,6 +159,8 @@ var csvSchema = []string{ "array_obj", } +var csvSpecificOp = []string{"GetBool", "GetInt8", "GetInt16", "GetInt32", "GetInt64", + "GetUint8", "GetUint16", "GetUint32", "GetUint64", "GetFloat32", "GetFloat64", "GetDateTime", "GetDecimal"} var ( bdUtcNs = time.Date(2009, 7, 13, 9, 7, 13, 123000000, time.UTC) bdUtcSec = bdUtcNs.Truncate(1 * time.Second) @@ -247,7 +251,10 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { for j := range testCases { var v interface{} desc := fmt.Sprintf(`%s.%s("%s", %s)`, name, method, testCases[j].Field, strconv.FormatBool(testCases[j].Nullable)) - if (name == "csv" && (sliceContains([]string{"GetBool", "GetInt64", "GetFloat64", "GetDateTime"}, method) && sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || testCases[j].Nullable)) || (name == "gjson" && strings.Contains(testCases[j].Field, ".")) { + if (name == "csv" && (sliceContains(csvSpecificOp, method) && + sliceContains([]string{"str_int", "str_float"}, testCases[j].Field) || + testCases[j].Nullable)) || + (name == "gjson" && strings.Contains(testCases[j].Field, ".")) { skipped = append(skipped, desc) continue } @@ -278,6 +285,8 @@ func doTestSimple(t *testing.T, method string, testCases []SimpleCase) { v = metric.GetDateTime(testCases[j].Field, testCases[j].Nullable) case "GetString": v = metric.GetString(testCases[j].Field, testCases[j].Nullable) + case "GetDecimal": + v = metric.GetDecimal(testCases[j].Field, testCases[j].Nullable) default: panic("error!") } @@ -354,27 +363,33 @@ func TestParserInt(t *testing.T) { } func TestParserFloat(t *testing.T) { + testFloatType[float32](t, "GetFloat32") + testFloatType[float64](t, "GetFloat64") +} + +func testFloatType[T constraints.Float](t *testing.T, method string) { + defaultVal := T(0.0) testCases := []SimpleCase{ // nullable: false - {"not_exist", false, 0.0}, - {"null", false, 0.0}, - {"bool_true", false, 0.0}, - {"bool_false", false, 0.0}, - {"num_int", false, 123.0}, - {"num_float", false, 123.321}, - {"str", false, 0.0}, - {"str_int", false, 0.0}, - {"str_float", false, 0.0}, - {"str_date_1", false, 0.0}, - {"obj", false, 0.0}, - {"array_empty", false, 0.0}, + {"not_exist", false, defaultVal}, + {"null", false, defaultVal}, + {"bool_true", false, defaultVal}, + {"bool_false", false, defaultVal}, + {"num_int", false, T(123.0)}, + {"num_float", false, T(123.321)}, + {"str", false, defaultVal}, + {"str_int", false, defaultVal}, + {"str_float", false, defaultVal}, + {"str_date_1", false, defaultVal}, + {"obj", false, defaultVal}, + {"array_empty", false, defaultVal}, // nullable: true {"not_exist", true, nil}, {"null", true, nil}, {"bool_true", true, nil}, {"bool_false", true, nil}, - {"num_int", true, 123.0}, - {"num_float", true, 123.321}, + {"num_int", true, T(123.0)}, + {"num_float", true, T(123.321)}, {"str", true, nil}, {"str_int", true, nil}, {"str_float", true, nil}, @@ -382,7 +397,7 @@ func TestParserFloat(t *testing.T) { {"obj", true, nil}, {"array_empty", true, nil}, } - doTestSimple(t, "GetFloat64", testCases) + doTestSimple(t, method, testCases) } func TestParserString(t *testing.T) { @@ -589,6 +604,8 @@ func TestParserMap(t *testing.T) { compareMap(t, orderMap, it.expVal, desc) } } + // GetMap is not supported for csv format + require.Equal(t, nil, metrics["csv"].GetMap("whatever", &model.TypeInfo{Type: model.String})) } func compareMap(t *testing.T, map1 interface{}, map2 interface{}, desc string) { @@ -650,6 +667,55 @@ func compareMap(t *testing.T, map1 interface{}, map2 interface{}, desc string) { } } +func TestParseObject(t *testing.T) { + compareObj := func(t *testing.T, map1 interface{}, map2 interface{}, desc string) { + value1 := reflect.ValueOf(map1) + value2 := reflect.ValueOf(map2) + assert.Equal(t, value1.Len(), value2.Len()) + + // v1 - map[interface{}]interface{}, v2 could be map[string][string] or map[string][float64] + var compareValueFunc func(v1, v2 reflect.Value) = func(v1, v2 reflect.Value) { + switch v2.Kind() { + case reflect.String: + assert.Equal(t, v1.Interface().(string), v2.String()) + case reflect.Float32, reflect.Float64: + assert.Equal(t, v1.Interface().(float64), v2.Float()) + default: + // Normal equality suffices + assert.Equal(t, v1.Interface(), v2.Interface()) + } + } + for _, key := range value2.MapKeys() { + compareValueFunc(value1.MapIndex(key), value2.MapIndex(key)) + } + } + + initialize.Do(initMetrics) + require.Nil(t, errInit) + testCases := []struct { + name string + expVal interface{} + }{ + // ParseObject only result a map[string][string] or map[string][float64] + {"map_str_str", map[string]string{"i": "first", "j": "second"}}, + {"map_str_uint", map[string]float64{"i": float64(1), "j": float64(2)}}, + {"map_str_int", map[string]float64{"i": float64(-1), "j": float64(-2)}}, + {"map_str_float", map[string]float64{"i": float64(3.1415), "j": float64(9.876)}}, + {"map_str_bool", map[string]interface{}{}}, // type with non fastjson.TypeNumber will be discarded + {"map_str_date", map[string]string{"i": "2008-08-08", "j": "2022-01-01"}}, + } + + for _, it := range testCases { + desc := fmt.Sprintf(`fastjson.GetObject("%s", false)`, it.name) + result := metrics["fastjson"].GetObject(it.name, false) + compareObj(t, result, it.expVal, desc) + } + + // GetObject is not supported for csv ang gjson parser + require.Equal(t, nil, metrics["csv"].GetObject("whatever", false)) + require.Equal(t, nil, metrics["gjson"].GetObject("whatever", false)) +} + func TestParseDateTime(t *testing.T) { // https://en.wikipedia.org/wiki/List_of_tz_database_time_zones // https://en.wikipedia.org/wiki/List_of_time_zone_abbreviations, "not part of the international time and date standard ISO 8601 and their use as sole designator for a time zone is discouraged." @@ -791,6 +857,81 @@ func TestParseInt(t *testing.T) { } } } + + testIntType[int8](t, "GetInt8") + testIntType[int16](t, "GetInt16") + testIntType[int32](t, "GetInt32") + testIntType[int64](t, "GetInt64") + testIntType[uint8](t, "GetUint8") + testIntType[uint16](t, "GetUint16") + testIntType[uint32](t, "GetUint32") + testIntType[uint64](t, "GetUint64") +} + +func testIntType[T constraints.Integer](t *testing.T, method string) { + defaultVal := T(0) + testCases := []SimpleCase{ + // nullable: false + {"not_exist", false, defaultVal}, + {"null", false, defaultVal}, + {"bool_true", false, T(1)}, + {"bool_false", false, defaultVal}, + {"num_int", false, T(123)}, + {"num_float", false, defaultVal}, + {"str", false, defaultVal}, + {"str_int", false, defaultVal}, + {"str_float", false, defaultVal}, + {"str_date_1", false, defaultVal}, + {"obj", false, defaultVal}, + {"array_empty", false, defaultVal}, + // nullable: true + {"not_exist", true, nil}, + {"null", true, nil}, + {"bool_true", true, T(1)}, + {"bool_false", true, defaultVal}, + {"num_int", true, T(123)}, + {"num_float", true, nil}, + {"str", true, nil}, + {"str_int", true, nil}, + {"str_float", true, nil}, + {"str_date_1", true, nil}, + {"obj", true, nil}, + {"array_empty", true, nil}, + } + doTestSimple(t, method, testCases) +} + +func TestParseDecimal(t *testing.T) { + expvar := decimal.NewFromInt(0) + testCases := []SimpleCase{ + // nullable: false + {"not_exist", false, expvar}, + {"null", false, expvar}, + {"bool_true", false, expvar}, + {"bool_false", false, expvar}, + {"num_int", false, decimal.NewFromInt(123)}, + {"num_float", false, decimal.NewFromFloat(123.321)}, + {"str", false, expvar}, + {"str_int", false, expvar}, + {"str_float", false, expvar}, + {"str_date_1", false, expvar}, + {"obj", false, expvar}, + {"array_empty", false, expvar}, + // // nullable: true + {"not_exist", true, nil}, + {"null", true, nil}, + {"bool_true", true, nil}, + {"bool_false", true, nil}, + {"num_int", true, decimal.NewFromInt(123)}, + {"num_float", true, decimal.NewFromFloat(123.321)}, + {"str", true, nil}, + {"str_int", true, nil}, + {"str_float", true, nil}, + {"str_date_1", true, nil}, + {"obj", true, nil}, + {"array_empty", true, nil}, + } + doTestSimple(t, "GetDecimal", testCases) } func TestFastjsonDetectSchema(t *testing.T) { diff --git a/util/common.go b/util/common.go index 3032d776..7777e0f5 100644 --- a/util/common.go +++ b/util/common.go @@ -20,8 +20,6 @@ import ( "crypto/tls" "crypto/x509" "fmt" - "math" - "net" "os" "os/exec" "path/filepath" @@ -102,51 +100,6 @@ func GetShift(s int) (shift uint) { return } -// GetOutboundIP gets preferred outbound ip of this machine -// https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go. -func GetOutboundIP() (ip net.IP, err error) { - var conn net.Conn - if conn, err = net.Dial("udp", "8.8.8.8:80"); err != nil { - err = errors.Wrapf(err, "") - return - } - defer conn.Close() - localAddr, _ := conn.LocalAddr().(*net.UDPAddr) - ip = localAddr.IP - return -} - -// GetSpareTCPPort finds a spare TCP port. -func GetSpareTCPPort(portBegin int) int { - for port := portBegin; port < math.MaxInt; port++ { - if err := testListenOnPort(port); err == nil { - return port - } - } - return 0 -} - -func testListenOnPort(port int) error { - addr := fmt.Sprintf(":%d", port) - ln, err := net.Listen("tcp", addr) - if err != nil { - return err - } - ln.Close() //nolint:errcheck - return nil -} - -// https://stackoverflow.com/questions/50428176/how-to-get-ip-and-port-from-net-addr-when-it-could-be-a-net-udpaddr-or-net-tcpad -func GetNetAddrPort(addr net.Addr) (port int) { - switch addr := addr.(type) { - case *net.UDPAddr: - port = addr.Port - case *net.TCPAddr: - port = addr.Port - } - return -} - // Refers to: // https://medium.com/processone/using-tls-authentication-for-your-go-kafka-client-3c5841f2a625 // https://github.com/denji/golang-tls diff --git a/util/common_test.go b/util/common_test.go index cc8fbc87..e16b7e72 100644 --- a/util/common_test.go +++ b/util/common_test.go @@ -24,3 +24,77 @@ func TestJksToPem(t *testing.T) { t.Logf("converted %s to %s, %s\n", jksPath, certPemPath, keyPemPath) } } + +func TestStringContains(t *testing.T) { + tests := []struct { + name string + array []string + result bool + }{ + { + name: "false", + array: []string{""}, + result: false, + }, + { + name: "true", + array: []string{"true", "hi"}, + result: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + result := StringContains(tt.array, tt.name) + require.Equal(t, result, tt.result) + }) + } +} + +func TestGetSourceName(t *testing.T) { + tests := []struct { + parser, name, result string + }{ + { + parser: "gjson", + name: "a.b.c", + result: "a\\.b\\.c", + }, + { + parser: "csv", + name: "a.b.c", + result: "a.b.c", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + result := GetSourceName(tt.parser, tt.name) + require.Equal(t, result, tt.result) + }) + } +} + +func TestMisc(t *testing.T) { + require.Equal(t, uint(6), GetShift(64)) + require.Equal(t, uint(7), GetShift(65)) + + tests := []struct { + parser, name, result string + }{ + { + parser: "gjson", + name: "a.b.c", + result: "a\\.b\\.c", + }, + { + parser: "csv", + name: "a.b.c", + result: "a.b.c", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + result := GetSourceName(tt.parser, tt.name) + require.Equal(t, result, tt.result) + }) + } +} diff --git a/util/net.go b/util/net.go index 884bd1af..e5410a12 100644 --- a/util/net.go +++ b/util/net.go @@ -15,7 +15,12 @@ limitations under the License. package util -import "net" +import ( + "fmt" + "net" + + "github.com/thanos-io/thanos/pkg/errors" +) func GetIP4Byname(host string) (ips []string, err error) { addrs, err := net.LookupIP(host) @@ -30,3 +35,42 @@ func GetIP4Byname(host string) (ips []string, err error) { } return } + +// GetOutboundIP get preferred outbound ip of this machine +// https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go +func GetOutboundIP() (ip net.IP, err error) { + var conn net.Conn + if conn, err = net.Dial("udp", "8.8.8.8:80"); err != nil { + err = errors.Wrapf(err, "") + return + } + defer conn.Close() + localAddr, _ := conn.LocalAddr().(*net.UDPAddr) + ip = localAddr.IP + return +} + +// GetSpareTCPPort find a spare TCP port +func GetSpareTCPPort(portBegin int) (port int) { +LOOP: + for port = portBegin; ; port++ { + addr := fmt.Sprintf(":%d", port) + ln, err := net.Listen("tcp", addr) + if err == nil { + ln.Close() + break LOOP + } + } + return +} + +// https://stackoverflow.com/questions/50428176/how-to-get-ip-and-port-from-net-addr-when-it-could-be-a-net-udpaddr-or-net-tcpad +func GetNetAddrPort(addr net.Addr) (port int) { + switch addr := addr.(type) { + case *net.UDPAddr: + port = addr.Port + case *net.TCPAddr: + port = addr.Port + } + return +} From 72bed83cf0a54304ea00f3ed563ccc52b036bdac Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Tue, 18 Jul 2023 13:36:12 +0800 Subject: [PATCH 356/404] Update TestParserMap, sinker should assign all tasks properly when rebalancing loads in cluster mode --- CHANGELOG.md | 3 +- config_manager/nacos.go | 6 ++-- go.mod | 33 ++++++++++---------- go.sum | 68 ++++++++++++++++++++++------------------- parser/fastjson.go | 10 +++++- parser/parser_test.go | 17 ++++++++++- task/sharding.go | 8 ++--- 7 files changed, 89 insertions(+), 56 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a65e94d4..cb5ee760 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,11 @@ # Changelog -#### Version 3.0.7 (2023-0) +#### Version 3.0.7 (2023-07-19) Improvements: - when run sinker in cluster mode, caculate lag failure of certain task should not block the entire rebalance work. Instead, the current task should not be assigned to any running sinker instance. +- sinker will not crash when handling incompatible map type. #### Version 3.0.6 (2023-05-19) diff --git a/config_manager/nacos.go b/config_manager/nacos.go index 0e0805c5..f26031e8 100644 --- a/config_manager/nacos.go +++ b/config_manager/nacos.go @@ -349,8 +349,10 @@ func (ncm *NacosConfManager) assign() (err error) { var validTasks []string for _, taskCfg := range newCfg.Tasks { - if _, ok := stateLags[taskCfg.Name]; ok { - validTasks = append(validTasks, taskCfg.Name) + // make sure all tasks get properly assigned + validTasks = append(validTasks, taskCfg.Name) + if _, ok := stateLags[taskCfg.Name]; !ok { + stateLags[taskCfg.Name] = StateLag{State: "NA", Lag: 0} } } sort.Slice(validTasks, func(i, j int) bool { diff --git a/go.mod b/go.mod index 879e398e..428ae0ba 100644 --- a/go.mod +++ b/go.mod @@ -6,7 +6,7 @@ require ( github.com/ClickHouse/clickhouse-go/v2 v2.10.1 github.com/RoaringBitmap/roaring v1.3.0 github.com/avast/retry-go/v4 v4.3.4 - github.com/bytedance/sonic v1.9.1 + github.com/bytedance/sonic v1.10.0-rc2 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.27 github.com/google/uuid v1.3.0 @@ -21,25 +21,26 @@ require ( github.com/thanos-io/thanos v0.31.0 github.com/tidwall/gjson v1.14.4 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.13.5 - github.com/twmb/franz-go/pkg/kadm v1.8.1 + github.com/twmb/franz-go v1.14.2 + github.com/twmb/franz-go/pkg/kadm v1.9.0 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.2 github.com/valyala/fastjson v1.6.4 go.uber.org/zap v1.24.0 - golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1 + golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 golang.org/x/time v0.3.0 gopkg.in/natefinch/lumberjack.v2 v2.2.1 ) require ( - github.com/ClickHouse/ch-go v0.56.1 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.377 // indirect + github.com/ClickHouse/ch-go v0.58.0 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.451 // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.8.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect - github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect + github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d // indirect + github.com/chenzhuoyu/iasm v0.9.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/go-errors/errors v1.4.2 // indirect github.com/go-faster/city v1.0.1 // indirect @@ -52,7 +53,7 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.16.6 // indirect + github.com/klauspost/compress v1.16.7 // indirect github.com/klauspost/cpuid/v2 v2.2.5 // indirect github.com/kr/text v0.2.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect @@ -60,28 +61,28 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b // indirect - github.com/paulmach/orb v0.9.2 // indirect + github.com/paulmach/orb v0.10.0 // indirect github.com/pierrec/lz4/v4 v4.1.18 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_model v0.4.0 // indirect github.com/prometheus/procfs v0.11.0 // indirect github.com/segmentio/asm v1.2.0 // indirect - github.com/streadway/amqp v1.0.0 // indirect + github.com/streadway/amqp v1.1.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.1 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect - github.com/twmb/franz-go/pkg/kmsg v1.5.0 // indirect + github.com/twmb/franz-go/pkg/kmsg v1.6.1 // indirect go.opentelemetry.io/otel v1.16.0 // indirect go.opentelemetry.io/otel/trace v1.16.0 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/arch v0.3.0 // indirect - golang.org/x/crypto v0.10.0 // indirect - golang.org/x/net v0.11.0 // indirect + golang.org/x/arch v0.4.0 // indirect + golang.org/x/crypto v0.11.0 // indirect + golang.org/x/net v0.12.0 // indirect golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.9.0 // indirect - google.golang.org/protobuf v1.30.0 // indirect + golang.org/x/sys v0.10.0 // indirect + google.golang.org/protobuf v1.31.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index 60f29090..945c4888 100644 --- a/go.sum +++ b/go.sum @@ -1,6 +1,6 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.56.1 h1:OshO1TWLpWGFFNFvOGDMtqRX38N7qjK9UwQwXzm20pA= -github.com/ClickHouse/ch-go v0.56.1/go.mod h1:6Mxu7RO1nuIC6/wBVzvAsUOSiLDMB0ZbTqMlBA3Gl/A= +github.com/ClickHouse/ch-go v0.58.0 h1:SwCH/WWJnwHnQdehiQsmraC13+uMpkexPR61c2Jj5Qo= +github.com/ClickHouse/ch-go v0.58.0/go.mod h1:P0YtetDdxICIC7IuRM9Qv/TMi83W0mRgBVQ9wcpt+4Y= github.com/ClickHouse/clickhouse-go/v2 v2.10.1 h1:WCnusqEeCO/9sLFVIv57le/O1ydUb+x9+SYYhJ11fsY= github.com/ClickHouse/clickhouse-go/v2 v2.10.1/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= @@ -12,8 +12,8 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.377 h1:6czd9jysvWELv3Yiip7dZJhBVKojt+GdWIrddzokEgY= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.377/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.451 h1:wCN/UCqeWifQZvoMBjbHwljEjzf/xkhh1vFZcKcTgI8= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.451/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/avast/retry-go/v4 v4.3.4 h1:pHLkL7jvCvP317I8Ge+Km2Yhntv3SdkJm7uekkqbKhM= @@ -29,14 +29,18 @@ github.com/bits-and-blooms/bitset v1.8.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.9.1 h1:6iJ6NqdoxCDr6mbY8h18oSO+cShGSMRGCEo7F2h0x8s= -github.com/bytedance/sonic v1.9.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/bytedance/sonic v1.10.0-rc/go.mod h1:ElCzW+ufi8qKqNW0FY314xriJhyJhuoJ3gFZdAHF7NM= +github.com/bytedance/sonic v1.10.0-rc2 h1:oDfRZ+4m6AYCOC0GFeOCeYqvBmucy1isvouS2K0cPzo= +github.com/bytedance/sonic v1.10.0-rc2/go.mod h1:iZcSUejdk5aukTND/Eu/ivjQuEL0Cu9/rf50Hi0u/g4= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= -github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 h1:qSGYFH7+jGhDF8vLC+iwCD4WpbV1EBDSzWkJODFLams= github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= +github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d h1:77cEq6EriyTZ0g/qfRdp61a3Uu/AWrgIq2s0ClJV1g0= +github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d/go.mod h1:8EPpVsBuRksnlj1mLy4AWzRNQYxauNi62uWcE3to6eA= +github.com/chenzhuoyu/iasm v0.9.0 h1:9fhXjVzq5hUy2gkhhgHl95zG2cEAhw9OSGs8toWWAwo= +github.com/chenzhuoyu/iasm v0.9.0/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -115,11 +119,12 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.16.6 h1:91SKEy4K37vkp255cJ8QesJhjyRO0hn9i9G0GoUwLsk= -github.com/klauspost/compress v1.16.6/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.16.7 h1:2mk3MPGNzKyxErAw8YaohYh69+pa4sIQSC0fPGCFR9I= +github.com/klauspost/compress v1.16.7/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= +github.com/knz/go-libedit v1.10.1/go.mod h1:MZTVkCWyz0oBc7JOWP3wNAzd002ZbM/5hgShxwh4x8M= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -146,8 +151,8 @@ github.com/nacos-group/nacos-sdk-go v1.1.4 h1:qyrZ7HTWM4aeymFfqnbgNRERh7TWuER10p github.com/nacos-group/nacos-sdk-go v1.1.4/go.mod h1:cBv9wy5iObs7khOqov1ERFQrCuTR4ILpgaiaVMxEmGI= github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b h1:FfH+VrHHk6Lxt9HdVS0PXzSXFyS2NbZKXv33FYPol0A= github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b/go.mod h1:AC62GU6hc0BrNm+9RK9VSiwa/EUe1bkIeFORAMcHvJU= -github.com/paulmach/orb v0.9.2 h1:p/YWV2uJwamAynnDOJGNbPBVtDHj3vG51k9tR1rFwJE= -github.com/paulmach/orb v0.9.2/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= +github.com/paulmach/orb v0.10.0 h1:guVYVqzxHE/CQ1KpfGO077TR0ATHSNjp4s6XGLn3W9s= +github.com/paulmach/orb v0.10.0/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= @@ -189,8 +194,8 @@ github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6Mwd github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/streadway/amqp v0.0.0-20200108173154-1c71cc93ed71/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= -github.com/streadway/amqp v1.0.0 h1:kuuDrUJFZL1QYL9hUNuCxNObNzB0bV/ZG5jV3RWAQgo= -github.com/streadway/amqp v1.0.0/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= +github.com/streadway/amqp v1.1.0 h1:py12iX8XSyI7aN/3dUT8DFIDJazNJsVJdxNVEpnQTZM= +github.com/streadway/amqp v1.1.0/go.mod h1:WYSrTEYHOXHd0nwFeUXAe2G2hRnQT+deZJJf88uS9Bg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= @@ -222,13 +227,13 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0 github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go v1.13.5 h1:7Hk47eZ7XRb4yWXQZk1GZU4BthkrKuZUfKOuP9Sgp24= -github.com/twmb/franz-go v1.13.5/go.mod h1:jm/FtYxmhxDTN0gNSb26XaJY0irdSVcsckLiR5tQNMk= -github.com/twmb/franz-go/pkg/kadm v1.8.1 h1:SrzL855I7gQTGdMtOYGTHhebs7TPgPN29FPtjusqwlE= -github.com/twmb/franz-go/pkg/kadm v1.8.1/go.mod h1:qUSM7pxoMCU1UNu5H4USE64ODcVmeG9LS96mysv1nu8= +github.com/twmb/franz-go v1.14.2 h1:5ONLhKdpztr4vKvMY0PvfG3iBkD7h0Sd8Goe1QQ6uzY= +github.com/twmb/franz-go v1.14.2/go.mod h1:nMAvTC2kHtK+ceaSHeHm4dlxC78389M/1DjpOswEgu4= +github.com/twmb/franz-go/pkg/kadm v1.9.0 h1:UgwBu0YCd6P8HLdg6ZRA4v9W6/zoI1042fOd2CvvLBE= +github.com/twmb/franz-go/pkg/kadm v1.9.0/go.mod h1:eG3f+GHUndq1CUSVvjp+WdNq5zePeJi3tEHzyTkao6g= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= -github.com/twmb/franz-go/pkg/kmsg v1.5.0 h1:eqVJquFQLdBNLrRMWX03pPDPpngn6PTjGZLlZnagouk= -github.com/twmb/franz-go/pkg/kmsg v1.5.0/go.mod h1:se9Mjdt0Nwzc9lnjJ0HyDtLyBnaBDAd7pCje47OhSyw= +github.com/twmb/franz-go/pkg/kmsg v1.6.1 h1:tm6hXPv5antMHLasTfKv9R+X03AjHSkSkXhQo2c5ALM= +github.com/twmb/franz-go/pkg/kmsg v1.6.1/go.mod h1:se9Mjdt0Nwzc9lnjJ0HyDtLyBnaBDAd7pCje47OhSyw= github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 h1:alKdbddkPw3rDh+AwmUEwh6HNYgTvDSFIe/GWYRR9RM= github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0/go.mod h1:k8BoBjyUbFj34f0rRbn+Ky12sZFAPbmShrg0karAIMo= github.com/twmb/franz-go/plugin/kzap v1.1.2 h1:0arX5xJ0soUPX1LlDay6ZZoxuWkWk1lggQ5M/IgRXAE= @@ -264,8 +269,8 @@ go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.3.0 h1:02VY4/ZcO/gBOH6PUaoiptASxtXU10jazRCP865E97k= -golang.org/x/arch v0.3.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.4.0 h1:A8WCeEWhLwPBKNbFi5Wv5UTCBx5zzubnXDlMOFAzFMc= +golang.org/x/arch v0.4.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -276,10 +281,10 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.10.0 h1:LKqV2xt9+kDzSTfOhx4FrkEBcMrAgHSYgzywV9zcGmM= -golang.org/x/crypto v0.10.0/go.mod h1:o4eNf7Ede1fv+hwOwZsTHl9EsPFO6q6ZvYR8vYfY45I= -golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1 h1:k/i9J1pBpvlfR+9QsetwPyERsqu1GIbi967PQMq3Ivc= -golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1/go.mod h1:V1LtkGg67GoY2N1AnLN78QLrzxkLyJw7RJb1gzOOz9w= +golang.org/x/crypto v0.11.0 h1:6Ewdq3tDic1mg5xRO4milcWCfMVQhI4NkqWWvqejpuA= +golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= +golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 h1:MGwJjxBy0HJshjDNfLsYO8xppfqWlA5ZT9OhtUUhTNw= +golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -300,8 +305,8 @@ golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.11.0 h1:Gi2tvZIJyBtO9SDr1q9h5hEQCp/4L2RQ+ar0qjx2oNU= -golang.org/x/net v0.11.0/go.mod h1:2L/ixqYpgIVXmeoSA/4Lu7BzTG4KIyPIryS4IsOd1oQ= +golang.org/x/net v0.12.0 h1:cfawfvKITfUsFCeJIHJrbSxpeu/E81khclypR0GVT50= +golang.org/x/net v0.12.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -325,8 +330,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.9.0 h1:KS/R3tvhPqvJvwcKfnBHJwwthS11LRhmM5D59eEXa0s= -golang.org/x/sys v0.9.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.10.0 h1:SqMFp9UcQJZa+pmYuAKjd9xq1f0j5rLcDIk0mj4qAsA= +golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= @@ -355,8 +360,8 @@ golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8T google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= -google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= +google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -378,4 +383,5 @@ gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +nullprogram.com/x/optparse v1.0.0/go.mod h1:KdyPE+Igbe0jQUrVfMqDMeJQIJZEuyV7pjYmp6pbG50= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= diff --git a/parser/fastjson.go b/parser/fastjson.go index 1c385738..9fdf335f 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -403,8 +403,10 @@ func getArray(c *FastjsonMetric, sourcename string, v *fastjson.Value, typ int) } func getMap(c *FastjsonMetric, v *fastjson.Value, typeinfo *model.TypeInfo) (val interface{}) { - if v != nil && v.Type() == fastjson.TypeObject { + if v == nil || v.Type() == fastjson.TypeObject { val = c.val2OrderedMap(v, typeinfo) + } else { + util.Logger.Fatal(fmt.Sprintf("SOURCE ERROR: unsupported map type: %v", v.Type())) } return } @@ -468,6 +470,12 @@ func (c *FastjsonMetric) castMapKeyByType(key []byte, typeinfo *model.TypeInfo) val = getDefaultDateTime(typeinfo.Nullable) case model.String: return string(key) + case model.Float32: + fallthrough + case model.Float64: + fallthrough + case model.Bool: + util.Logger.Fatal("unsupported map key type") default: util.Logger.Fatal("LOGIC ERROR: reached switch default condition") } diff --git a/parser/parser_test.go b/parser/parser_test.go index b05801d5..ff2191f7 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -78,7 +78,9 @@ var jsonSample = []byte(`{ "map_str_bool": {"i":true, "j":false}, "map_str_date": {"i":"2008-08-08", "j":"2022-01-01"}, "map_str_array": {"i":[1,2,3],"j":[4,5,6]}, - "map_str_map": {"i":{"i":1, "j":2}, "j":{"i":3, "j":4}} + "map_str_map": {"i":{"i":1, "j":2}, "j":{"i":3, "j":4}}, + "map_uint_uint": {"1":1, "2":2}, + "map_int_string": {"1":"3.1415", "2":"9.876"} }`) var jsonSchema = map[string]string{ @@ -121,6 +123,8 @@ var jsonSchema = map[string]string{ "map_str_date": "Object('json')", "map_str_array": "Object('json')", "map_str_map": "Object('json')", + "map_uint_uint": "Object('json')", + "map_int_string": "Object('json')", } var csvSample = []byte(`null,true,false,123,123.321,kube-state-metrics,"escaped_""ws",123,123.321,2009-07-13,13/07/2009,2009-07-13T09:07:13Z,2009-07-13T09:07:13.123+08:00,2009-07-13 09:07:13,2009-07-13 09:07:13.123,"{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",[],[null],"[true,false]","[0,255,256,65535,65536,4294967295,4294967296,18446744073709551615,18446744073709551616]","[-9223372036854775808,-2147483649,-2147483648,-32769,-32768,-129,-128,0,127,128,32767,32768,2147483647,2147483648,9223372036854775807]","[4.940656458412465441765687928682213723651e-324,1.401298464324817070923729583289916131280e-45,0.0,3.40282346638528859811704183484516925440e+38,1.797693134862315708145274237317043567981e+308]","[""aa"",""bb"",""cc""]","[""0"",""255"",""256"",""65535"",""65536"",""4294967295"",""4294967296"",""18446744073709551615"",""18446744073709551616""]","[""-9223372036854775808"",""-2147483649"",""-2147483648"",""-32769"",""-32768"",""-129"",""-128"",""0"",""127"",""128"",""32767"",""32768"",""2147483647"",""2147483648"",""9223372036854775807""]","[""4.940656458412465441765687928682213723651e-324"",""1.401298464324817070923729583289916131280e-45"",""0.0"",""3.40282346638528859811704183484516925440e+38"",""1.797693134862315708145274237317043567981e+308""]","[""2009-07-13"",""2009-07-14"",""2009-07-15""]","[""13/07/2009"",""14/07/2009"",""15/07/2009""]","[""2009-07-13T09:07:13Z"",""2009-07-13T09:07:13+08:00"",""2009-07-13T09:07:13.123Z"",""2009-07-13T09:07:13.123+08:00""]","[""2009-07-13 09:07:13"",""2009-07-13 09:07:13.123""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]"`) @@ -606,6 +610,17 @@ func TestParserMap(t *testing.T) { } // GetMap is not supported for csv format require.Equal(t, nil, metrics["csv"].GetMap("whatever", &model.TypeInfo{Type: model.String})) + + // map key as int type, only supported by fastjson parser + testCases = []MapCase{ + {"map_uint_uint", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.UInt64}, MapValue: &model.TypeInfo{Type: model.UInt64}}, map[uint64]uint64{1: 1, 2: 2}}, + {"map_int_string", &model.TypeInfo{Type: model.Map, MapKey: &model.TypeInfo{Type: model.Int64}, MapValue: &model.TypeInfo{Type: model.String}}, map[int64]string{1: "3.1415", 2: "9.876"}}, + } + for _, it := range testCases { + desc := fmt.Sprintf(`%s.GetMap("%s", %s)`, "fastjson", it.field, model.GetTypeName(it.typ.Type)) + orderMap := metrics["fastjson"].GetMap(it.field, it.typ) + compareMap(t, orderMap, it.expVal, desc) + } } func compareMap(t *testing.T, map1 interface{}, map2 interface{}, desc string) { diff --git a/task/sharding.go b/task/sharding.go index 2e78e8f9..f96515e3 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -26,7 +26,7 @@ func NewShardingPolicy(shardingKey string, shardingStripe uint64, dims []*model. for i, dim := range dims { if dim.Name == shardingKey { if dim.Type.Nullable || dim.Type.Array { - err = errors.Newf("invalid shardingKey %s, expect its type be numerical or string", shardingKey) + err = errors.Newf("invalid shardingKey '%s', expect its type be numerical or string", shardingKey) return } colSeq = i @@ -40,13 +40,13 @@ func NewShardingPolicy(shardingKey string, shardingStripe uint64, dims []*model. //string policy.stripe = 0 default: - err = errors.Newf("invalid shardingKey %s, expect its type be numerical or string", shardingKey) + err = errors.Newf("invalid shardingKey '%s', expect its type be numerical or string", shardingKey) return } } } if colSeq < 0 { - err = errors.Newf("invalid shardingKey %s, no such column", shardingKey) + err = errors.Newf("invalid shardingKey '%s', no such column", shardingKey) return } policy.colSeq = colSeq @@ -121,7 +121,7 @@ func NewSharder(service *Service) (sh *Sharder, err error) { taskCfg := service.taskCfg if taskCfg.ShardingKey != "" { if policy, err = NewShardingPolicy(taskCfg.ShardingKey, taskCfg.ShardingStripe, service.clickhouse.Dims, shards); err != nil { - return + return sh, errors.Wrapf(err, "error when creating sharding policy for task '%s'", service.taskCfg.Name) } } sh = &Sharder{ From 24bbf4cae68eb936d182c80d98613ac9a9e7b845 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 11:24:13 +0300 Subject: [PATCH 357/404] Separate tcp port test in a func Signed-off-by: sashayakovtseva --- util/common.go | 47 +++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/util/common.go b/util/common.go index 7777e0f5..3032d776 100644 --- a/util/common.go +++ b/util/common.go @@ -20,6 +20,8 @@ import ( "crypto/tls" "crypto/x509" "fmt" + "math" + "net" "os" "os/exec" "path/filepath" @@ -100,6 +102,51 @@ func GetShift(s int) (shift uint) { return } +// GetOutboundIP gets preferred outbound ip of this machine +// https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go. +func GetOutboundIP() (ip net.IP, err error) { + var conn net.Conn + if conn, err = net.Dial("udp", "8.8.8.8:80"); err != nil { + err = errors.Wrapf(err, "") + return + } + defer conn.Close() + localAddr, _ := conn.LocalAddr().(*net.UDPAddr) + ip = localAddr.IP + return +} + +// GetSpareTCPPort finds a spare TCP port. +func GetSpareTCPPort(portBegin int) int { + for port := portBegin; port < math.MaxInt; port++ { + if err := testListenOnPort(port); err == nil { + return port + } + } + return 0 +} + +func testListenOnPort(port int) error { + addr := fmt.Sprintf(":%d", port) + ln, err := net.Listen("tcp", addr) + if err != nil { + return err + } + ln.Close() //nolint:errcheck + return nil +} + +// https://stackoverflow.com/questions/50428176/how-to-get-ip-and-port-from-net-addr-when-it-could-be-a-net-udpaddr-or-net-tcpad +func GetNetAddrPort(addr net.Addr) (port int) { + switch addr := addr.(type) { + case *net.UDPAddr: + port = addr.Port + case *net.TCPAddr: + port = addr.Port + } + return +} + // Refers to: // https://medium.com/processone/using-tls-authentication-for-your-go-kafka-client-3c5841f2a625 // https://github.com/denji/golang-tls From b14ebfd2f2a9a7135cb4c8b037573a00416951a8 Mon Sep 17 00:00:00 2001 From: sashayakovtseva Date: Tue, 1 Aug 2023 11:34:05 +0300 Subject: [PATCH 358/404] Allow to specify http host to bound to Signed-off-by: sashayakovtseva --- util/common.go | 47 ----------------------------------------------- util/net.go | 31 +++++++++++++++++++------------ 2 files changed, 19 insertions(+), 59 deletions(-) diff --git a/util/common.go b/util/common.go index 3032d776..7777e0f5 100644 --- a/util/common.go +++ b/util/common.go @@ -20,8 +20,6 @@ import ( "crypto/tls" "crypto/x509" "fmt" - "math" - "net" "os" "os/exec" "path/filepath" @@ -102,51 +100,6 @@ func GetShift(s int) (shift uint) { return } -// GetOutboundIP gets preferred outbound ip of this machine -// https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go. -func GetOutboundIP() (ip net.IP, err error) { - var conn net.Conn - if conn, err = net.Dial("udp", "8.8.8.8:80"); err != nil { - err = errors.Wrapf(err, "") - return - } - defer conn.Close() - localAddr, _ := conn.LocalAddr().(*net.UDPAddr) - ip = localAddr.IP - return -} - -// GetSpareTCPPort finds a spare TCP port. -func GetSpareTCPPort(portBegin int) int { - for port := portBegin; port < math.MaxInt; port++ { - if err := testListenOnPort(port); err == nil { - return port - } - } - return 0 -} - -func testListenOnPort(port int) error { - addr := fmt.Sprintf(":%d", port) - ln, err := net.Listen("tcp", addr) - if err != nil { - return err - } - ln.Close() //nolint:errcheck - return nil -} - -// https://stackoverflow.com/questions/50428176/how-to-get-ip-and-port-from-net-addr-when-it-could-be-a-net-udpaddr-or-net-tcpad -func GetNetAddrPort(addr net.Addr) (port int) { - switch addr := addr.(type) { - case *net.UDPAddr: - port = addr.Port - case *net.TCPAddr: - port = addr.Port - } - return -} - // Refers to: // https://medium.com/processone/using-tls-authentication-for-your-go-kafka-client-3c5841f2a625 // https://github.com/denji/golang-tls diff --git a/util/net.go b/util/net.go index e5410a12..2a8c5be1 100644 --- a/util/net.go +++ b/util/net.go @@ -17,6 +17,7 @@ package util import ( "fmt" + "math" "net" "github.com/thanos-io/thanos/pkg/errors" @@ -36,8 +37,8 @@ func GetIP4Byname(host string) (ips []string, err error) { return } -// GetOutboundIP get preferred outbound ip of this machine -// https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go +// GetOutboundIP gets preferred outbound ip of this machine +// https://stackoverflow.com/questions/23558425/how-do-i-get-the-local-ip-address-in-go. func GetOutboundIP() (ip net.IP, err error) { var conn net.Conn if conn, err = net.Dial("udp", "8.8.8.8:80"); err != nil { @@ -50,18 +51,14 @@ func GetOutboundIP() (ip net.IP, err error) { return } -// GetSpareTCPPort find a spare TCP port -func GetSpareTCPPort(portBegin int) (port int) { -LOOP: - for port = portBegin; ; port++ { - addr := fmt.Sprintf(":%d", port) - ln, err := net.Listen("tcp", addr) - if err == nil { - ln.Close() - break LOOP +// GetSpareTCPPort finds a spare TCP port. +func GetSpareTCPPort(portBegin int) int { + for port := portBegin; port < math.MaxInt; port++ { + if err := testListenOnPort(port); err == nil { + return port } } - return + return 0 } // https://stackoverflow.com/questions/50428176/how-to-get-ip-and-port-from-net-addr-when-it-could-be-a-net-udpaddr-or-net-tcpad @@ -74,3 +71,13 @@ func GetNetAddrPort(addr net.Addr) (port int) { } return } + +func testListenOnPort(port int) error { + addr := fmt.Sprintf(":%d", port) + ln, err := net.Listen("tcp", addr) + if err != nil { + return err + } + ln.Close() //nolint:errcheck + return nil +} From 52d208e4ff27a2d65deab99721b7589fd6d94e00 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 10 Aug 2023 15:11:04 +0800 Subject: [PATCH 359/404] log the batch offset, and the batch group Id --- go.mod | 27 ++++++++++----------- go.sum | 57 ++++++++++++++++++++++---------------------- model/message.go | 8 +++++++ output/clickhouse.go | 6 ++++- task/consumer.go | 39 +++++++++++++++++++----------- task/sharding.go | 53 ++++++++++++++++++++++++---------------- task/sinker.go | 4 ++-- 7 files changed, 114 insertions(+), 80 deletions(-) diff --git a/go.mod b/go.mod index 428ae0ba..3e391d8c 100644 --- a/go.mod +++ b/go.mod @@ -3,38 +3,39 @@ module github.com/housepower/clickhouse_sinker go 1.20 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.10.1 + github.com/ClickHouse/clickhouse-go/v2 v2.12.1 github.com/RoaringBitmap/roaring v1.3.0 - github.com/avast/retry-go/v4 v4.3.4 - github.com/bytedance/sonic v1.10.0-rc2 + github.com/avast/retry-go/v4 v4.5.0 + github.com/bytedance/sonic v1.10.0-rc3 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.27 github.com/google/uuid v1.3.0 github.com/hjson/hjson-go/v4 v4.3.0 github.com/jcmturner/gokrb5/v8 v8.4.4 github.com/jinzhu/copier v0.3.5 + github.com/matoous/go-nanoid/v2 v2.0.0 github.com/nacos-group/nacos-sdk-go v1.1.4 github.com/prometheus/client_golang v1.16.0 github.com/prometheus/common v0.44.0 github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.8.4 github.com/thanos-io/thanos v0.31.0 - github.com/tidwall/gjson v1.14.4 + github.com/tidwall/gjson v1.16.0 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.14.2 + github.com/twmb/franz-go v1.14.3 github.com/twmb/franz-go/pkg/kadm v1.9.0 github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.2 github.com/valyala/fastjson v1.6.4 - go.uber.org/zap v1.24.0 - golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 + go.uber.org/zap v1.25.0 + golang.org/x/exp v0.0.0-20230810033253-352e893a4cad golang.org/x/time v0.3.0 gopkg.in/natefinch/lumberjack.v2 v2.2.1 ) require ( github.com/ClickHouse/ch-go v0.58.0 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.451 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.510 // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.8.0 // indirect @@ -55,7 +56,6 @@ require ( github.com/json-iterator/go v1.1.12 // indirect github.com/klauspost/compress v1.16.7 // indirect github.com/klauspost/cpuid/v2 v2.2.5 // indirect - github.com/kr/text v0.2.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect @@ -66,7 +66,7 @@ require ( github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/procfs v0.11.0 // indirect + github.com/prometheus/procfs v0.11.1 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/streadway/amqp v1.1.0 // indirect github.com/tidwall/match v1.1.1 // indirect @@ -75,13 +75,12 @@ require ( github.com/twmb/franz-go/pkg/kmsg v1.6.1 // indirect go.opentelemetry.io/otel v1.16.0 // indirect go.opentelemetry.io/otel/trace v1.16.0 // indirect - go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.4.0 // indirect - golang.org/x/crypto v0.11.0 // indirect - golang.org/x/net v0.12.0 // indirect + golang.org/x/crypto v0.12.0 // indirect + golang.org/x/net v0.14.0 // indirect golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.10.0 // indirect + golang.org/x/sys v0.11.0 // indirect google.golang.org/protobuf v1.31.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 945c4888..905ff06f 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,8 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/ClickHouse/ch-go v0.58.0 h1:SwCH/WWJnwHnQdehiQsmraC13+uMpkexPR61c2Jj5Qo= github.com/ClickHouse/ch-go v0.58.0/go.mod h1:P0YtetDdxICIC7IuRM9Qv/TMi83W0mRgBVQ9wcpt+4Y= -github.com/ClickHouse/clickhouse-go/v2 v2.10.1 h1:WCnusqEeCO/9sLFVIv57le/O1ydUb+x9+SYYhJ11fsY= -github.com/ClickHouse/clickhouse-go/v2 v2.10.1/go.mod h1:teXfZNM90iQ99Jnuht+dxQXCuhDZ8nvvMoTJOFrcmcg= +github.com/ClickHouse/clickhouse-go/v2 v2.12.1 h1:KzNUk4oLL3vo+EOSel+QmonF5GViG4B6/hVkTd9iZx8= +github.com/ClickHouse/clickhouse-go/v2 v2.12.1/go.mod h1:W/UQ/GchOF+Q0k5iv6ZanLKQNukA4Oiyt4sMFDsv8QY= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.3.0 h1:aQmu9zQxDU0uhwR8SXOH/OrqEf+X8A0LQmwW3JX8Lcg= @@ -12,13 +12,13 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.451 h1:wCN/UCqeWifQZvoMBjbHwljEjzf/xkhh1vFZcKcTgI8= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.451/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.510 h1:mvveZfYcJUOyj0jJqbYxWrM298JXt+ltj7dMbekjraI= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.510/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= -github.com/avast/retry-go/v4 v4.3.4 h1:pHLkL7jvCvP317I8Ge+Km2Yhntv3SdkJm7uekkqbKhM= -github.com/avast/retry-go/v4 v4.3.4/go.mod h1:rv+Nla6Vk3/ilU0H51VHddWHiwimzX66yZ0JT6T+UvE= -github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= +github.com/avast/retry-go/v4 v4.5.0 h1:QoRAZZ90cj5oni2Lsgl2GW8mNTnUCnmpx/iKpwVisHg= +github.com/avast/retry-go/v4 v4.5.0/go.mod h1:7hLEXp0oku2Nir2xBAsg0PTphp9z71bN5Aq1fboC3+I= +github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -30,8 +30,8 @@ github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMU github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= github.com/bytedance/sonic v1.10.0-rc/go.mod h1:ElCzW+ufi8qKqNW0FY314xriJhyJhuoJ3gFZdAHF7NM= -github.com/bytedance/sonic v1.10.0-rc2 h1:oDfRZ+4m6AYCOC0GFeOCeYqvBmucy1isvouS2K0cPzo= -github.com/bytedance/sonic v1.10.0-rc2/go.mod h1:iZcSUejdk5aukTND/Eu/ivjQuEL0Cu9/rf50Hi0u/g4= +github.com/bytedance/sonic v1.10.0-rc3 h1:uNSnscRapXTwUgTyOF0GVljYD08p9X/Lbr9MweSV3V0= +github.com/bytedance/sonic v1.10.0-rc3/go.mod h1:iZcSUejdk5aukTND/Eu/ivjQuEL0Cu9/rf50Hi0u/g4= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -41,7 +41,6 @@ github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d h1:77cEq6EriyTZ github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d/go.mod h1:8EPpVsBuRksnlj1mLy4AWzRNQYxauNi62uWcE3to6eA= github.com/chenzhuoyu/iasm v0.9.0 h1:9fhXjVzq5hUy2gkhhgHl95zG2cEAhw9OSGs8toWWAwo= github.com/chenzhuoyu/iasm v0.9.0/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= -github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -132,7 +131,9 @@ github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= -github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/matoous/go-nanoid v1.5.0/go.mod h1:zyD2a71IubI24efhpvkJz+ZwfwagzgSO6UNiFsZKN7U= +github.com/matoous/go-nanoid/v2 v2.0.0 h1:d19kur2QuLeHmJBkvYkFdhFBzLoo1XVm2GgTpL+9Tj0= +github.com/matoous/go-nanoid/v2 v2.0.0/go.mod h1:FtS4aGPVfEkxKxhdWPAspZpZSh1cOjtM7Ej/So3hR0g= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= @@ -181,8 +182,8 @@ github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/prometheus/procfs v0.11.0 h1:5EAgkfkMl659uZPbe9AS2N68a7Cc1TJbPEuGzFuRbyk= -github.com/prometheus/procfs v0.11.0/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/procfs v0.11.1 h1:xRC8Iq1yyca5ypa9n1EZnWZkt7dwcoRPQwX/5gwaUuI= +github.com/prometheus/procfs v0.11.1/go.mod h1:eesXgaPo1q7lBpVMoMy0ZOFTth9hBn4W/y0/p/ScXhY= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= @@ -209,13 +210,12 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/thanos-io/thanos v0.31.0 h1:jOCmaiIXwpByWXoVtHnktLm3YDB9xDQQzmZvd1XG5oY= github.com/thanos-io/thanos v0.31.0/go.mod h1:5ux+jb2oKr59+3XsCC0mX+JuAbPGJEMijjhcmnL/PMo= -github.com/tidwall/gjson v1.14.4 h1:uo0p8EbA09J7RQaflQ1aBRffTR7xedD2bcIVSYxLnkM= -github.com/tidwall/gjson v1.14.4/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= +github.com/tidwall/gjson v1.16.0 h1:SyXa+dsSPpUlcwEDuKuEBJEz5vzTvOea+9rjyYodQFg= +github.com/tidwall/gjson v1.16.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= @@ -227,8 +227,8 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0 github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= -github.com/twmb/franz-go v1.14.2 h1:5ONLhKdpztr4vKvMY0PvfG3iBkD7h0Sd8Goe1QQ6uzY= -github.com/twmb/franz-go v1.14.2/go.mod h1:nMAvTC2kHtK+ceaSHeHm4dlxC78389M/1DjpOswEgu4= +github.com/twmb/franz-go v1.14.3 h1:cq8rxAnVYU1uF3SRVn8eEaUf+AaXKWlB0Cl3Ca7JSa4= +github.com/twmb/franz-go v1.14.3/go.mod h1:nMAvTC2kHtK+ceaSHeHm4dlxC78389M/1DjpOswEgu4= github.com/twmb/franz-go/pkg/kadm v1.9.0 h1:UgwBu0YCd6P8HLdg6ZRA4v9W6/zoI1042fOd2CvvLBE= github.com/twmb/franz-go/pkg/kadm v1.9.0/go.mod h1:eG3f+GHUndq1CUSVvjp+WdNq5zePeJi3tEHzyTkao6g= github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= @@ -259,15 +259,14 @@ go.opentelemetry.io/otel/trace v1.16.0/go.mod h1:Yt9vYq1SdNz3xdjZZK7wcXv1qv2pwLk go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= -go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= go.uber.org/goleak v1.2.1 h1:NBol2c7O1ZokfZ0LEU9K6Whx/KnwvepVetCUhtKja4A= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= -go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= +go.uber.org/zap v1.25.0 h1:4Hvk6GtkucQ790dqmj7l1eEnRdKm3k3ZUrUMS2d5+5c= +go.uber.org/zap v1.25.0/go.mod h1:JIAUzQIH94IC4fOJQm7gMmBJP5k7wQfdcnYdPoEXJYk= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/arch v0.4.0 h1:A8WCeEWhLwPBKNbFi5Wv5UTCBx5zzubnXDlMOFAzFMc= golang.org/x/arch v0.4.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= @@ -281,10 +280,10 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.11.0 h1:6Ewdq3tDic1mg5xRO4milcWCfMVQhI4NkqWWvqejpuA= -golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= -golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 h1:MGwJjxBy0HJshjDNfLsYO8xppfqWlA5ZT9OhtUUhTNw= -golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= +golang.org/x/crypto v0.12.0 h1:tFM/ta59kqch6LlvYnPa0yx5a83cL2nHflFhYKvv9Yk= +golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= +golang.org/x/exp v0.0.0-20230810033253-352e893a4cad h1:g0bG7Z4uG+OgH2QDODnjp6ggkk1bJDsINcuWmJN1iJU= +golang.org/x/exp v0.0.0-20230810033253-352e893a4cad/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -305,8 +304,8 @@ golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.12.0 h1:cfawfvKITfUsFCeJIHJrbSxpeu/E81khclypR0GVT50= -golang.org/x/net v0.12.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= +golang.org/x/net v0.14.0 h1:BONx9s002vGdD9umnlX1Po8vOZmrgH34qlHcD1MfK14= +golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -330,8 +329,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.10.0 h1:SqMFp9UcQJZa+pmYuAKjd9xq1f0j5rLcDIk0mj4qAsA= -golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.11.0 h1:eG7RXZHdqOJ1i+0lgLgCpSXAp6M3LYlAo6osgSi0xOM= +golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= diff --git a/model/message.go b/model/message.go index e9ede079..fb08ee0f 100644 --- a/model/message.go +++ b/model/message.go @@ -28,6 +28,7 @@ type MsgRow struct { type Batch struct { Rows *Rows BatchIdx int64 + GroupId string RealSize int Wg *sync.WaitGroup @@ -36,3 +37,10 @@ type Batch struct { func (b *Batch) Size() int { return len(*b.Rows) } + +type BatchRange struct { + Begin int64 + End int64 +} + +type RecordMap = map[string]map[int32]*BatchRange diff --git a/output/clickhouse.go b/output/clickhouse.go index e7e1dd30..bd5ecd3a 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -263,7 +263,11 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, sc *pool.ShardConn) { retry.RetryIf(func(err error) bool { return shouldReconnect(err, sc) }), retry.OnRetry(func(n uint, err error) { retrycount++ - util.Logger.Error("flush batch failed", zap.String("task", c.taskCfg.Name), zap.Int("try", int(retrycount)), zap.Error(err)) + util.Logger.Error("flush batch failed", + zap.String("task", c.taskCfg.Name), + zap.String("group", batch.GroupId), + zap.Int("try", int(retrycount)), + zap.Error(err)) statistics.FlushMsgsErrorTotal.WithLabelValues(c.taskCfg.Name).Add(float64(batch.RealSize)) }), ); err != nil { diff --git a/task/consumer.go b/task/consumer.go index e5a71ec6..24638a11 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -16,6 +16,8 @@ limitations under the License. package task import ( + "context" + "math" "sync" "sync/atomic" "time" @@ -32,7 +34,7 @@ import ( type Commit struct { group string - offsets map[string]map[int32]int64 + offsets model.RecordMap wg *sync.WaitGroup consumer *Consumer } @@ -44,7 +46,8 @@ type Consumer struct { grpConfig *config.GroupConfig fetchesCh chan *kgo.Fetches processWg sync.WaitGroup - stopCh chan struct{} + ctx context.Context + cancel context.CancelFunc state atomic.Uint32 errCommit bool @@ -64,7 +67,6 @@ func newConsumer(s *Sinker, gCfg *config.GroupConfig) *Consumer { numFlying: 0, errCommit: false, grpConfig: gCfg, - stopCh: make(chan struct{}), fetchesCh: make(chan *kgo.Fetches), } c.state.Store(util.StateStopped) @@ -80,6 +82,7 @@ func (c *Consumer) start() { if c.state.Load() == util.StateRunning { return } + c.ctx, c.cancel = context.WithCancel(context.Background()) c.inputer = input.NewKafkaFranz() c.state.Store(util.StateRunning) if err := c.inputer.Init(c.sinker.curCfg, c.grpConfig, c.fetchesCh, c.cleanupFn); err == nil { @@ -97,7 +100,7 @@ func (c *Consumer) stop() { c.state.Store(util.StateStopped) // stop the processFetch routine, make sure no more input to the commit chan & writing pool - c.stopCh <- struct{}{} + c.cancel() c.processWg.Wait() c.inputer.Stop() } @@ -137,7 +140,7 @@ func (c *Consumer) updateGroupConfig(g *config.GroupConfig) { c.grpConfig = g // restart the processFetch routine because of potential BufferSize or FlushInterval change // make sure no more input to the commit chan & writing pool - c.stopCh <- struct{}{} + c.cancel() c.processWg.Wait() go c.processFetch() } @@ -145,7 +148,7 @@ func (c *Consumer) updateGroupConfig(g *config.GroupConfig) { func (c *Consumer) processFetch() { c.processWg.Add(1) defer c.processWg.Done() - recMap := make(map[string]map[int32]int64) + recMap := make(model.RecordMap) var bufLength int flushFn := func() { @@ -155,7 +158,8 @@ func (c *Consumer) processFetch() { var wg sync.WaitGroup c.tasks.Range(func(key, value any) bool { // flush to shard, ck - value.(*Service).sharder.Flush(&wg) + task := value.(*Service) + task.sharder.Flush(c.ctx, &wg, recMap[task.taskCfg.Topic]) return true }) bufLength = 0 @@ -164,7 +168,7 @@ func (c *Consumer) processFetch() { c.numFlying++ c.mux.Unlock() c.sinker.commitsCh <- &Commit{group: c.grpConfig.Name, offsets: recMap, wg: &wg, consumer: c} - recMap = make(map[string]map[int32]int64) + recMap = make(model.RecordMap) } bufThreshold := c.grpConfig.BufferSize * len(c.sinker.curCfg.Clickhouse.Hosts) * 4 / 5 @@ -241,7 +245,7 @@ func (c *Consumer) processFetch() { for i := range f.Topics { ft := &f.Topics[i] if recMap[ft.Topic] == nil { - recMap[ft.Topic] = make(map[int32]int64) + recMap[ft.Topic] = make(map[int32]*model.BatchRange) } for j := range ft.Partitions { fpr := ft.Partitions[j].Records @@ -249,10 +253,18 @@ func (c *Consumer) processFetch() { continue } lastOff := fpr[len(fpr)-1].Offset + firstOff := fpr[0].Offset - old, ok := recMap[ft.Topic][ft.Partitions[j].Partition] - if !ok || old < lastOff { - recMap[ft.Topic][ft.Partitions[j].Partition] = lastOff + or, ok := recMap[ft.Topic][ft.Partitions[j].Partition] + if !ok { + or = &model.BatchRange{Begin: math.MaxInt64, End: -1} + recMap[ft.Topic][ft.Partitions[j].Partition] = or + } + if or.End < lastOff { + or.End = lastOff + } + if or.Begin > firstOff { + or.Begin = firstOff } } } @@ -265,8 +277,7 @@ func (c *Consumer) processFetch() { } case <-ticker.C: flushFn() - case <-c.stopCh: - flushFn() + case <-c.ctx.Done(): util.Logger.Info("stopped processing loop", zap.String("group", c.grpConfig.Name)) return } diff --git a/task/sharding.go b/task/sharding.go index f96515e3..b89fef4d 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -1,6 +1,7 @@ package task import ( + "context" "fmt" "sync" "time" @@ -10,8 +11,10 @@ import ( "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/statistics" "github.com/housepower/clickhouse_sinker/util" + nanoid "github.com/matoous/go-nanoid/v2" "github.com/shopspring/decimal" "github.com/thanos-io/thanos/pkg/errors" + "go.uber.org/zap" ) type ShardingPolicy struct { @@ -149,29 +152,39 @@ func (sh *Sharder) PutElement(msgRow *model.MsgRow) { statistics.ShardMsgs.WithLabelValues(sh.service.taskCfg.Name).Inc() } -func (sh *Sharder) Flush(wg *sync.WaitGroup) { +func (sh *Sharder) Flush(c context.Context, wg *sync.WaitGroup, rmap map[int32]*model.BatchRange) { sh.mux.Lock() defer sh.mux.Unlock() - var msgCnt int - taskCfg := sh.service.taskCfg - for i, rows := range sh.msgBuf { - realSize := len(*rows) - if realSize > 0 { - msgCnt += realSize - batch := &model.Batch{ - Rows: rows, - BatchIdx: int64(i), - RealSize: realSize, - Wg: wg, + select { + case <-c.Done(): + util.Logger.Info("batch abandoned because of context canceled") + return + default: + var msgCnt int + taskCfg := sh.service.taskCfg + batchId, _ := nanoid.New() + for i, rows := range sh.msgBuf { + realSize := len(*rows) + if realSize > 0 { + msgCnt += realSize + batch := &model.Batch{ + Rows: rows, + BatchIdx: int64(i), + GroupId: batchId, + RealSize: realSize, + Wg: wg, + } + batch.Wg.Add(1) + sh.service.clickhouse.Send(batch) + rs := make(model.Rows, 0, realSize) + sh.msgBuf[i] = &rs } - batch.Wg.Add(1) - sh.service.clickhouse.Send(batch) - rs := make(model.Rows, 0, realSize) - sh.msgBuf[i] = &rs } - } - if msgCnt > 0 { - util.Logger.Info(fmt.Sprintf("created a batch group for task %v with %d shards, total messages %d", sh.service.taskCfg.Name, len(sh.msgBuf), msgCnt)) - statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) + if msgCnt > 0 { + util.Logger.Info(fmt.Sprintf("created a batch group for task %v with %d shards, total messages %d", sh.service.taskCfg.Name, len(sh.msgBuf), msgCnt), + zap.String("group", batchId), + zap.Reflect("offsets", rmap)) + statistics.ShardMsgs.WithLabelValues(taskCfg.Name).Sub(float64(msgCnt)) + } } } diff --git a/task/sinker.go b/task/sinker.go index 1f3fb117..fa486e19 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -465,7 +465,7 @@ func (s *Sinker) commitFn() { LOOP: for i, value := range com.offsets { for k, v := range value { - if err := c.inputer.CommitMessages(&model.InputMessage{Topic: i, Partition: int(k), Offset: v}); err != nil { + if err := c.inputer.CommitMessages(&model.InputMessage{Topic: i, Partition: int(k), Offset: v.End}); err != nil { c.errCommit = true // restart the consumer when facing commit error, avoid change the s.consumers outside of s.Run // error could be RebalanceInProgress, IllegalGeneration, UnknownMemberID @@ -476,7 +476,7 @@ func (s *Sinker) commitFn() { util.Logger.Warn("Batch.Commit failed, will restart later", zap.Error(err)) break LOOP } else { - statistics.ConsumeOffsets.WithLabelValues(com.consumer.grpConfig.Name, i, strconv.Itoa(int(k))).Set(float64(v)) + statistics.ConsumeOffsets.WithLabelValues(com.consumer.grpConfig.Name, i, strconv.Itoa(int(k))).Set(float64(v.End)) } } } From bfff89a7a7b1159691bf828e657878bf8847d79b Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Wed, 16 Aug 2023 10:54:36 +0800 Subject: [PATCH 360/404] sinker should retry flush operation when seeing TOO_MANY_SIMULTANEOUS_QUERIES error --- output/clickhouse.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index bd5ecd3a..25aad982 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -49,7 +49,8 @@ var ( // CKServer too many parts possibly reason: https://github.com/ClickHouse/ClickHouse/issues/6720#issuecomment-526045768 // zooKeeper Connection loss issue: https://cwiki.apache.org/confluence/display/ZOOKEEPER/FAQ#:~:text=How%20should%20I%20handle%20the%20CONNECTION_LOSS%20error%3F // zooKeeper Session expired issue: https://cwiki.apache.org/confluence/display/ZOOKEEPER/FAQ#:~:text=How%20should%20I%20handle%20SESSION_EXPIRED%3F - replicaSpecificErrorCodes = []int32{225, 242, 252, 319, 999, 1000} //NO_ZOOKEEPER, TABLE_IS_READ_ONLY, TOO_MANY_PARTS, UNKNOWN_STATUS_OF_INSERT, KEEPER_EXCEPTION, POCO_EXCEPTION + // TOO_MANY_SIMULTANEOUS_QUERIES, NO_ZOOKEEPER, TABLE_IS_READ_ONLY, TOO_MANY_PARTS, UNKNOWN_STATUS_OF_INSERT, KEEPER_EXCEPTION, POCO_EXCEPTION + replicaSpecificErrorCodes = []int32{202, 225, 242, 252, 319, 999, 1000} wrSeriesQuota = 16384 SeriesQuotas sync.Map From 7ec6dd400ac57670a28fb84015708ab0a85648e6 Mon Sep 17 00:00:00 2001 From: wangxinbo Date: Thu, 17 Aug 2023 14:01:12 +0800 Subject: [PATCH 361/404] introduce task level config property fields --- CHANGELOG.md | 12 +++ config/config.go | 2 + docs/configuration/config.md | 4 + parser/fastjson.go | 12 ++- parser/gjson.go | 48 +++++++----- parser/parser.go | 37 +++++++-- parser/parser_test.go | 141 ++++++++++++++++++++++++++++++++--- task/consumer.go | 1 + task/task.go | 7 +- 9 files changed, 225 insertions(+), 39 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cb5ee760..69cec72e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,17 @@ # Changelog +#### Version 3.0.8 (2023-08-xx) + +Improvements: + +- "too many simultaneous queries" from ClickHouse will no longer be a fatal error, instead sinker will keep retrying untill complete the write operation +- Output the offset range per batch + +New Features: + +- Introduced a task level configuration property "field", which accepts a valid json string and allows additional +fields to be appened to each input message. + #### Version 3.0.7 (2023-07-19) Improvements: diff --git a/config/config.go b/config/config.go index cb84c7fe..238dcf60 100644 --- a/config/config.go +++ b/config/config.go @@ -139,6 +139,8 @@ type TaskConfig struct { WhiteList string // the regexp of white list BlackList string // the regexp of black list } + // additional fields to be appended to each input message, should be a valid json string + Fields string `json:"fields,omitempty"` // PrometheusSchema expects each message is a Prometheus metric(timestamp, value, metric name and a list of labels). PrometheusSchema bool // fields match PromLabelsBlackList are not considered as labels. Requires PrometheusSchema be true. diff --git a/docs/configuration/config.md b/docs/configuration/config.md index d770d5e8..e1f50e65 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -148,6 +148,10 @@ "blackList": "@" }, + // additional fields to be appended to each input message, should be a valid json string + // e.g. fields: "{\"Enable\":true,\"MaxDims\":0,\"Earliest\":false,\"Parser\":\"fastjson\"}" + "fields": "", + // PrometheusSchema expects each message is a Prometheus metric(timestamp, value, metric name and a list of labels). "prometheusSchema": true, // the regexp of labels black list, fields match promLabelsBlackList are not considered as part of labels column in series table diff --git a/parser/fastjson.go b/parser/fastjson.go index 9fdf335f..d19516ae 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -38,8 +38,9 @@ var EmpytObject = make(map[string]interface{}) // FastjsonParser, parser for get data in json format type FastjsonParser struct { - pp *Pool - fjp fastjson.Parser + pp *Pool + fjp fastjson.Parser + fields *fastjson.Object } func (p *FastjsonParser) Parse(bs []byte) (metric model.Metric, err error) { @@ -48,6 +49,13 @@ func (p *FastjsonParser) Parse(bs []byte) (metric model.Metric, err error) { err = errors.Wrapf(err, "") return } + + if p.fields != nil { + p.fields.Visit(func(key []byte, v *fastjson.Value) { + value.Set(string(key), v) + }) + } + metric = &FastjsonMetric{pp: p.pp, value: value} return } diff --git a/parser/gjson.go b/parser/gjson.go index 73c4a32c..93883d9f 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -48,56 +48,64 @@ type GjsonMetric struct { raw string } +func (c *GjsonMetric) getField(key string) gjson.Result { + ret := gjson.Get(c.pp.fields, key) + if !ret.Exists() { + ret = gjson.Get(c.raw, key) + } + return ret +} + func (c *GjsonMetric) GetString(key string, nullable bool) (val interface{}) { - return getGJsonString(gjson.Get(c.raw, key), nullable) + return getGJsonString(c.getField(key), nullable) } func (c *GjsonMetric) GetBool(key string, nullable bool) (val interface{}) { - return getGJsonBool(gjson.Get(c.raw, key), nullable) + return getGJsonBool(c.getField(key), nullable) } func (c *GjsonMetric) GetDecimal(key string, nullable bool) (val interface{}) { - return getGJsonDecimal(gjson.Get(c.raw, key), nullable) + return getGJsonDecimal(c.getField(key), nullable) } func (c *GjsonMetric) GetInt8(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int8](c, gjson.Get(c.raw, key), nullable, math.MinInt8, math.MaxInt8) + return GjsonGetInt[int8](c, c.getField(key), nullable, math.MinInt8, math.MaxInt8) } func (c *GjsonMetric) GetInt16(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int16](c, gjson.Get(c.raw, key), nullable, math.MinInt16, math.MaxInt16) + return GjsonGetInt[int16](c, c.getField(key), nullable, math.MinInt16, math.MaxInt16) } func (c *GjsonMetric) GetInt32(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int32](c, gjson.Get(c.raw, key), nullable, math.MinInt32, math.MaxInt32) + return GjsonGetInt[int32](c, c.getField(key), nullable, math.MinInt32, math.MaxInt32) } func (c *GjsonMetric) GetInt64(key string, nullable bool) (val interface{}) { - return GjsonGetInt[int64](c, gjson.Get(c.raw, key), nullable, math.MinInt64, math.MaxInt64) + return GjsonGetInt[int64](c, c.getField(key), nullable, math.MinInt64, math.MaxInt64) } func (c *GjsonMetric) GetUint8(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint8](c, gjson.Get(c.raw, key), nullable, math.MaxUint8) + return GjsonGetUint[uint8](c, c.getField(key), nullable, math.MaxUint8) } func (c *GjsonMetric) GetUint16(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint16](c, gjson.Get(c.raw, key), nullable, math.MaxUint16) + return GjsonGetUint[uint16](c, c.getField(key), nullable, math.MaxUint16) } func (c *GjsonMetric) GetUint32(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint32](c, gjson.Get(c.raw, key), nullable, math.MaxUint32) + return GjsonGetUint[uint32](c, c.getField(key), nullable, math.MaxUint32) } func (c *GjsonMetric) GetUint64(key string, nullable bool) (val interface{}) { - return GjsonGetUint[uint64](c, gjson.Get(c.raw, key), nullable, math.MaxUint64) + return GjsonGetUint[uint64](c, c.getField(key), nullable, math.MaxUint64) } func (c *GjsonMetric) GetFloat32(key string, nullable bool) (val interface{}) { - return GjsonGetFloat[float32](c, gjson.Get(c.raw, key), nullable, math.MaxFloat32) + return GjsonGetFloat[float32](c, c.getField(key), nullable, math.MaxFloat32) } func (c *GjsonMetric) GetFloat64(key string, nullable bool) (val interface{}) { - return GjsonGetFloat[float64](c, gjson.Get(c.raw, key), nullable, math.MaxFloat64) + return GjsonGetFloat[float64](c, c.getField(key), nullable, math.MaxFloat64) } func GjsonGetInt[T constraints.Signed](c *GjsonMetric, r gjson.Result, nullable bool, min, max int64) (val interface{}) { @@ -169,7 +177,7 @@ func GjsonGetFloat[T constraints.Float](c *GjsonMetric, r gjson.Result, nullable } func (c *GjsonMetric) GetDateTime(key string, nullable bool) (val interface{}) { - return getGJsonDateTime(c, key, gjson.Get(c.raw, key), nullable) + return getGJsonDateTime(c, key, c.getField(key), nullable) } func (c *GjsonMetric) GetObject(key string, nullable bool) (val interface{}) { @@ -177,11 +185,11 @@ func (c *GjsonMetric) GetObject(key string, nullable bool) (val interface{}) { } func (c *GjsonMetric) GetArray(key string, typ int) (val interface{}) { - return getGJsonArray(c, key, gjson.Get(c.raw, key), typ) + return getGJsonArray(c, key, c.getField(key), typ) } func (c *GjsonMetric) GetMap(key string, typeinfo *model.TypeInfo) (val interface{}) { - return getGJsonMap(c, gjson.Get(c.raw, key), typeinfo) + return getGJsonMap(c, c.getField(key), typeinfo) } func (c *GjsonMetric) val2OrderedMap(v gjson.Result, typeinfo *model.TypeInfo) (m *model.OrderedMap) { @@ -263,7 +271,7 @@ func GjsonFloatArray[T constraints.Float](a []gjson.Result, max float64) (arr [] } func (c *GjsonMetric) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, black *regexp.Regexp, partition int, offset int64) (foundNew bool) { - gjson.Parse(c.raw).ForEach(func(k, v gjson.Result) bool { + ite := func(k, v gjson.Result) bool { strKey := k.Str if _, loaded := knownKeys.LoadOrStore(strKey, nil); !loaded { if (white == nil || white.MatchString(strKey)) && @@ -280,7 +288,11 @@ func (c *GjsonMetric) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, } } return true - }) + } + + c.pp.once.Do(func() { gjson.Parse(c.pp.fields).ForEach(ite) }) + gjson.Parse(c.raw).ForEach(ite) + return } diff --git a/parser/parser.go b/parser/parser.go index ea74ac16..9f38330c 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -21,7 +21,9 @@ import ( "time" "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" + "github.com/valyala/fastjson" ) var ( @@ -78,10 +80,12 @@ type Pool struct { timeUnit float64 knownLayouts sync.Map pool sync.Pool + once sync.Once // only need to detect new keys from fields once + fields string } // NewParserPool creates a parser pool -func NewParserPool(name string, csvFormat []string, delimiter string, timezone string, timeunit float64) (pp *Pool, err error) { +func NewParserPool(name string, csvFormat []string, delimiter string, timezone string, timeunit float64, fields string) (pp *Pool, err error) { var tz *time.Location if timezone == "" { tz = time.Local @@ -94,6 +98,7 @@ func NewParserPool(name string, csvFormat []string, delimiter string, timezone s delimiter: delimiter, timeZone: tz, timeUnit: timeunit, + fields: fields, } if csvFormat != nil { pp.csvFormat = make(map[string]int, len(csvFormat)) @@ -107,21 +112,37 @@ func NewParserPool(name string, csvFormat []string, delimiter string, timezone s // Get returns a Parser from pp. // // The Parser must be Put to pp after use. -func (pp *Pool) Get() Parser { +func (pp *Pool) Get() (Parser, error) { v := pp.pool.Get() if v == nil { switch pp.name { case "gjson": - return &GjsonParser{pp: pp} - case "fastjson": - return &FastjsonParser{pp: pp} + return &GjsonParser{pp: pp}, nil case "csv": - return &CsvParser{pp: pp} + if pp.fields != "" { + util.Logger.Warn("extra fields for csv parser is not supported, fields ignored") + } + return &CsvParser{pp: pp}, nil + case "fastjson": + fallthrough default: - return &FastjsonParser{pp: pp} + var obj *fastjson.Object + if pp.fields != "" { + value, err := fastjson.Parse(pp.fields) + if err != nil { + err = errors.Wrapf(err, "failed to parse fields as a valid json object") + return nil, err + } + obj, err = value.Object() + if err != nil { + err = errors.Wrapf(err, "failed to retrive fields member") + return nil, err + } + } + return &FastjsonParser{pp: pp, fields: obj}, nil } } - return v.(Parser) + return v.(Parser), nil } // Put returns p to pp. diff --git a/parser/parser_test.go b/parser/parser_test.go index ff2191f7..09c07a62 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -127,6 +127,50 @@ var jsonSchema = map[string]string{ "map_int_string": "Object('json')", } +var jsonFields = `{ + "fnull": null, + "fbool_true": true, + "fbool_false": false, + "fnum_int": 123, + "fnum_float": 123.321, + "fapp.kubernetes.io/name": "kube-state-metrics", + "fstr": "escaped_\"ws", + "fstr_int": "123", + "fstr_float": "123.321", + "fstr_date_1": "2009-07-13", + "fstr_date_2": "13/07/2009", + "fstr_time_rfc3339_1": "2009-07-13T09:07:13Z", + "fstr_time_rfc3339_2": "2009-07-13T09:07:13.123+08:00", + "fstr_time_clickhouse_1": "2009-07-13 09:07:13", + "fstr_time_clickhouse_2": "2009-07-13 09:07:13.123", + "fobj": {"i":[1,2,3],"f":[1.1,2.2,3.3],"s":["aa","bb","cc"],"e":[]}, + "farray_empty": [], + "farray_null": [null], + "farray_bool": [true,false], + "farray_num_int_1": [0, 255, 256, 65535, 65536, 4294967295, 4294967296, 18446744073709551615, 18446744073709551616], + "farray_num_int_2": [-9223372036854775808, -2147483649, -2147483648, -32769, -32768, -129, -128, 0, 127, 128, 32767, 32768, 2147483647, 2147483648, 9223372036854775807], + "farray_num_float": [4.940656458412465441765687928682213723651e-324, 1.401298464324817070923729583289916131280e-45, 0.0, 3.40282346638528859811704183484516925440e+38, 1.797693134862315708145274237317043567981e+308, -inf, +inf], + "farray_str": ["aa","bb","cc"], + "farray_str_int_1": ["0", "255", "256", "65535", "65536", "4294967295", "4294967296", "18446744073709551615", "18446744073709551616"], + "farray_str_int_2": ["-9223372036854775808", "-2147483649", "-2147483648", "-32769", "-32768", "-129", "-128", "0", "127", "128", "32767", "32768", "2147483647", "2147483648", "9223372036854775807"], + "farray_str_float": ["4.940656458412465441765687928682213723651e-324", "1.401298464324817070923729583289916131280e-45", "0.0", "3.40282346638528859811704183484516925440e+38", "1.797693134862315708145274237317043567981e+308", "-inf", "+inf"], + "farray_str_date_1": ["2009-07-13","2009-07-14","2009-07-15"], + "farray_str_date_2": ["13/07/2009","14/07/2009","15/07/2009"], + "farray_str_time_rfc3339": ["2009-07-13T09:07:13Z", "2009-07-13T09:07:13+08:00", "2009-07-13T09:07:13.123Z", "2009-07-13T09:07:13.123+08:00"], + "farray_str_time_clickhouse": ["2009-07-13 09:07:13", "2009-07-13 09:07:13.123"], + "farray_obj": [{"i":[1,2,3],"f":[1.1,2.2,3.3]},{"s":["aa","bb","cc"],"e":[]}], + "fmap_str_str": {"i":"first", "j":"second"}, + "fmap_str_uint": {"i":1, "j":2}, + "fmap_str_int": {"i":-1, "j":-2}, + "fmap_str_float": {"i":3.1415, "j":9.876}, + "fmap_str_bool": {"i":true, "j":false}, + "fmap_str_date": {"i":"2008-08-08", "j":"2022-01-01"}, + "fmap_str_array": {"i":[1,2,3],"j":[4,5,6]}, + "fmap_str_map": {"i":{"i":1, "j":2}, "j":{"i":3, "j":4}}, + "fmap_uint_uint": {"1":1, "2":2}, + "fmap_int_string": {"1":"3.1415", "2":"9.876"} +}` + var csvSample = []byte(`null,true,false,123,123.321,kube-state-metrics,"escaped_""ws",123,123.321,2009-07-13,13/07/2009,2009-07-13T09:07:13Z,2009-07-13T09:07:13.123+08:00,2009-07-13 09:07:13,2009-07-13 09:07:13.123,"{""i"":[1,2,3],""f"":[1.1,2.2,3.3],""s"":[""aa"",""bb"",""cc""],""e"":[]}",[],[null],"[true,false]","[0,255,256,65535,65536,4294967295,4294967296,18446744073709551615,18446744073709551616]","[-9223372036854775808,-2147483649,-2147483648,-32769,-32768,-129,-128,0,127,128,32767,32768,2147483647,2147483648,9223372036854775807]","[4.940656458412465441765687928682213723651e-324,1.401298464324817070923729583289916131280e-45,0.0,3.40282346638528859811704183484516925440e+38,1.797693134862315708145274237317043567981e+308]","[""aa"",""bb"",""cc""]","[""0"",""255"",""256"",""65535"",""65536"",""4294967295"",""4294967296"",""18446744073709551615"",""18446744073709551616""]","[""-9223372036854775808"",""-2147483649"",""-2147483648"",""-32769"",""-32768"",""-129"",""-128"",""0"",""127"",""128"",""32767"",""32768"",""2147483647"",""2147483648"",""9223372036854775807""]","[""4.940656458412465441765687928682213723651e-324"",""1.401298464324817070923729583289916131280e-45"",""0.0"",""3.40282346638528859811704183484516925440e+38"",""1.797693134862315708145274237317043567981e+308""]","[""2009-07-13"",""2009-07-14"",""2009-07-15""]","[""13/07/2009"",""14/07/2009"",""15/07/2009""]","[""2009-07-13T09:07:13Z"",""2009-07-13T09:07:13+08:00"",""2009-07-13T09:07:13.123Z"",""2009-07-13T09:07:13.123+08:00""]","[""2009-07-13 09:07:13"",""2009-07-13 09:07:13.123""]","[{""i"":[1,2,3],""f"":[1.1,2.2,3.3]},{""s"":[""aa"",""bb"",""cc""],""e"":[]}]"`) var csvSchema = []string{ @@ -216,16 +260,19 @@ func initMetrics() { for _, name := range names { switch name { case "csv": - pp, _ = NewParserPool("csv", csvSchema, ",", "", timeUnit) + pp, _ = NewParserPool("csv", csvSchema, ",", "", timeUnit, "") sample = csvSample case "fastjson": - pp, _ = NewParserPool("fastjson", nil, "", "", timeUnit) + pp, _ = NewParserPool("fastjson", nil, "", "", timeUnit, jsonFields) sample = jsonSample case "gjson": - pp, _ = NewParserPool("gjson", nil, "", "", timeUnit) + pp, _ = NewParserPool("gjson", nil, "", "", timeUnit, jsonFields) sample = jsonSample } - parser = pp.Get() + parser, errInit = pp.Get() + if errInit != nil { + panic(fmt.Sprintf("failed to initialize parser: %+v\n", errInit)) + } if metric, errInit = parser.Parse(sample); errInit != nil { msg := fmt.Sprintf("parser.Parse failed: %+v\n", errInit) panic(msg) @@ -949,9 +996,38 @@ func TestParseDecimal(t *testing.T) { doTestSimple(t, "GetDecimal", testCases) } +func TestFields(t *testing.T) { + testFunc := func(metric model.Metric, kind string) { + v := metric.GetBool("fbool_true", false) + require.Equal(t, true, v, "error calling GetBool") + v = metric.GetInt8("fnum_int", false) + require.Equal(t, int8(123), v, "error calling GetInt8") + v = metric.GetString("fstr_time_rfc3339_1", false) + require.Equal(t, "2009-07-13T09:07:13Z", v, "error calling GetString") + v = metric.GetDecimal("fnum_float", false) + require.Equal(t, decimal.NewFromFloat(123.321), v, "error calling GetDecimal") + if kind == "fastjson" { + v = metric.GetObject("fmap_str_str", false) + require.Equal(t, map[string]interface{}{"i": "first", "j": "second"}, v, "error calling GetObject") + } + } + + pp, _ := NewParserPool("fastjson", nil, "", "", timeUnit, jsonFields) + fparser, _ := pp.Get() + defer pp.Put(fparser) + fmetric, _ := fparser.Parse(jsonSample) + testFunc(fmetric, "fastjson") + + pp, _ = NewParserPool("gjson", nil, "", "", timeUnit, jsonFields) + gparser, _ := pp.Get() + defer pp.Put(gparser) + gmetric, _ := gparser.Parse(jsonSample) + testFunc(gmetric, "gjson") +} + func TestFastjsonDetectSchema(t *testing.T) { - pp, _ := NewParserPool("fastjson", nil, "", "", timeUnit) - parser := pp.Get() + pp, _ := NewParserPool("fastjson", nil, "", "", timeUnit, jsonFields) + parser, _ := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) @@ -970,12 +1046,59 @@ func TestFastjsonDetectSchema(t *testing.T) { } act[string(k)] = tn }) - require.Equal(t, jsonSchema, act) + + fastjsonSchema := map[string]string{ + "fnull": "Unknown", + "fbool_true": "Bool", + "fbool_false": "Bool", + "fnum_int": "Int64", + "fnum_float": "Float64", + "fapp.kubernetes.io/name": "String", + "fstr": "String", + "fstr_int": "String", + "fstr_float": "String", + "fstr_date_1": "DateTime", + "fstr_date_2": "DateTime", + "fstr_time_rfc3339_1": "DateTime", + "fstr_time_rfc3339_2": "DateTime", + "fstr_time_clickhouse_1": "DateTime", + "fstr_time_clickhouse_2": "DateTime", + "fobj": "Object('json')", + "farray_empty": "Unknown", + "farray_null": "Unknown", + "farray_bool": "BoolArray", + "farray_num_int_1": "Int64Array", + "farray_num_int_2": "Int64Array", + "farray_num_float": "Float64Array", + "farray_str": "StringArray", + "farray_str_int_1": "StringArray", + "farray_str_int_2": "StringArray", + "farray_str_float": "StringArray", + "farray_str_date_1": "DateTimeArray", + "farray_str_date_2": "DateTimeArray", + "farray_str_time_rfc3339": "DateTimeArray", + "farray_str_time_clickhouse": "DateTimeArray", + "farray_obj": "Object('json')Array", + "fmap_str_str": "Object('json')", + "fmap_str_uint": "Object('json')", + "fmap_str_int": "Object('json')", + "fmap_str_float": "Object('json')", + "fmap_str_bool": "Object('json')", + "fmap_str_date": "Object('json')", + "fmap_str_array": "Object('json')", + "fmap_str_map": "Object('json')", + "fmap_uint_uint": "Object('json')", + "fmap_int_string": "Object('json')"} + + for k, v := range jsonSchema { + fastjsonSchema[k] = v + } + require.Equal(t, fastjsonSchema, act) } func TestGjsonDetectSchema(t *testing.T) { - pp, _ := NewParserPool("gjson", nil, "", "", timeUnit) - parser := pp.Get() + pp, _ := NewParserPool("gjson", nil, "", "", timeUnit, "") + parser, _ := pp.Get() defer pp.Put(parser) metric, _ := parser.Parse(jsonSample) diff --git a/task/consumer.go b/task/consumer.go index 24638a11..fa63f770 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -142,6 +142,7 @@ func (c *Consumer) updateGroupConfig(g *config.GroupConfig) { // make sure no more input to the commit chan & writing pool c.cancel() c.processWg.Wait() + c.ctx, c.cancel = context.WithCancel(context.Background()) go c.processFetch() } diff --git a/task/task.go b/task/task.go index fc963a6a..9848ecd4 100644 --- a/task/task.go +++ b/task/task.go @@ -85,7 +85,7 @@ func cloneTask(s *Service, newGroup *Consumer) (service *Service) { // NewTaskService creates an instance of new tasks with kafka, clickhouse and paser instances func NewTaskService(cfg *config.Config, taskCfg *config.TaskConfig, c *Consumer) (service *Service) { ck := output.NewClickHouse(cfg, taskCfg) - pp, err := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone, taskCfg.TimeUnit) + pp, err := parser.NewParserPool(taskCfg.Parser, taskCfg.CsvFormat, taskCfg.Delimiter, taskCfg.TimeZone, taskCfg.TimeUnit, taskCfg.Fields) if err != nil { util.Logger.Fatal("failed to create task", zap.String("group", c.grpConfig.Name), zap.String("task", taskCfg.Name), zap.Error(err)) } @@ -160,7 +160,10 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { var foundNewKeys bool var metric model.Metric - p := service.pp.Get() + p, err := service.pp.Get() + if err != nil { + util.Logger.Fatal("error initializing json parser", zap.String("task", taskCfg.Name), zap.Error(err)) + } if metric, err = p.Parse(msg.Value); err != nil { // directly return, ignore the row with parsing errors statistics.ParseMsgsErrorTotal.WithLabelValues(taskCfg.Name).Inc() From d6b4a10daa76f0322e855cbdc4b0bc7e73747c0f Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Thu, 9 Nov 2023 15:46:01 +0800 Subject: [PATCH 362/404] feat: support http protocol --- config/config.go | 14 +++ go.mod | 22 ++-- go.sum | 44 +++---- output/clickhouse.go | 92 +++++++++----- output/clickhouse_util.go | 57 +-------- pool/ck_cli.go | 254 ++++++++++++++++++++++++++++++++++++++ pool/conn.go | 59 ++++++--- task/sinker.go | 20 ++- 8 files changed, 417 insertions(+), 145 deletions(-) create mode 100644 pool/ck_cli.go diff --git a/config/config.go b/config/config.go index 238dcf60..df001f76 100644 --- a/config/config.go +++ b/config/config.go @@ -20,6 +20,7 @@ import ( "regexp" "strings" + "github.com/ClickHouse/clickhouse-go/v2" "github.com/hjson/hjson-go/v4" "go.uber.org/zap" @@ -92,6 +93,7 @@ type ClickHouseConfig struct { Port int Username string Password string + Protocol string //native, http // Whether enable TLS encryption with clickhouse-server Secure bool @@ -256,6 +258,18 @@ func (cfg *Config) Normallize(constructGroup bool, httpAddr string, cred util.Cr cfg.Clickhouse.MaxOpenConns = defaultMaxOpenConns } + if cfg.Clickhouse.Protocol == "" { + cfg.Clickhouse.Protocol = clickhouse.Native.String() + } + + if cfg.Clickhouse.Port == 0 { + if cfg.Clickhouse.Protocol == clickhouse.HTTP.String() { + cfg.Clickhouse.Port = 8123 + } else { + cfg.Clickhouse.Port = 9000 + } + } + if cfg.Task != nil { cfg.Tasks = append(cfg.Tasks, cfg.Task) cfg.Task = nil diff --git a/go.mod b/go.mod index 3e391d8c..1d3e937f 100644 --- a/go.mod +++ b/go.mod @@ -3,13 +3,13 @@ module github.com/housepower/clickhouse_sinker go 1.20 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.12.1 + github.com/ClickHouse/clickhouse-go/v2 v2.15.0 github.com/RoaringBitmap/roaring v1.3.0 github.com/avast/retry-go/v4 v4.5.0 github.com/bytedance/sonic v1.10.0-rc3 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.27 - github.com/google/uuid v1.3.0 + github.com/google/uuid v1.4.0 github.com/hjson/hjson-go/v4 v4.3.0 github.com/jcmturner/gokrb5/v8 v8.4.4 github.com/jinzhu/copier v0.3.5 @@ -34,9 +34,9 @@ require ( ) require ( - github.com/ClickHouse/ch-go v0.58.0 // indirect + github.com/ClickHouse/ch-go v0.58.2 // indirect github.com/aliyun/alibaba-cloud-sdk-go v1.62.510 // indirect - github.com/andybalholm/brotli v1.0.5 // indirect + github.com/andybalholm/brotli v1.0.6 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.8.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect @@ -45,7 +45,7 @@ require ( github.com/davecgh/go-spew v1.1.1 // indirect github.com/go-errors/errors v1.4.2 // indirect github.com/go-faster/city v1.0.1 // indirect - github.com/go-faster/errors v0.6.1 // indirect + github.com/go-faster/errors v0.7.0 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/jcmturner/aescts/v2 v2.0.0 // indirect @@ -54,7 +54,7 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.16.7 // indirect + github.com/klauspost/compress v1.17.2 // indirect github.com/klauspost/cpuid/v2 v2.2.5 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect @@ -73,14 +73,14 @@ require ( github.com/tidwall/pretty v1.2.1 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect github.com/twmb/franz-go/pkg/kmsg v1.6.1 // indirect - go.opentelemetry.io/otel v1.16.0 // indirect - go.opentelemetry.io/otel/trace v1.16.0 // indirect + go.opentelemetry.io/otel v1.19.0 // indirect + go.opentelemetry.io/otel/trace v1.19.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.4.0 // indirect - golang.org/x/crypto v0.12.0 // indirect - golang.org/x/net v0.14.0 // indirect + golang.org/x/crypto v0.14.0 // indirect + golang.org/x/net v0.17.0 // indirect golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.11.0 // indirect + golang.org/x/sys v0.14.0 // indirect google.golang.org/protobuf v1.31.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 905ff06f..e6c48de4 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,8 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.58.0 h1:SwCH/WWJnwHnQdehiQsmraC13+uMpkexPR61c2Jj5Qo= -github.com/ClickHouse/ch-go v0.58.0/go.mod h1:P0YtetDdxICIC7IuRM9Qv/TMi83W0mRgBVQ9wcpt+4Y= -github.com/ClickHouse/clickhouse-go/v2 v2.12.1 h1:KzNUk4oLL3vo+EOSel+QmonF5GViG4B6/hVkTd9iZx8= -github.com/ClickHouse/clickhouse-go/v2 v2.12.1/go.mod h1:W/UQ/GchOF+Q0k5iv6ZanLKQNukA4Oiyt4sMFDsv8QY= +github.com/ClickHouse/ch-go v0.58.2 h1:jSm2szHbT9MCAB1rJ3WuCJqmGLi5UTjlNu+f530UTS0= +github.com/ClickHouse/ch-go v0.58.2/go.mod h1:Ap/0bEmiLa14gYjCiRkYGbXvbe8vwdrfTYWhsuQ99aw= +github.com/ClickHouse/clickhouse-go/v2 v2.15.0 h1:G0hTKyO8fXXR1bGnZ0DY3vTG01xYfOGW76zgjg5tmC4= +github.com/ClickHouse/clickhouse-go/v2 v2.15.0/go.mod h1:kXt1SRq0PIRa6aKZD7TnFnY9PQKmc2b13sHtOYcK6cQ= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.3.0 h1:aQmu9zQxDU0uhwR8SXOH/OrqEf+X8A0LQmwW3JX8Lcg= @@ -14,8 +14,8 @@ github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRF github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= github.com/aliyun/alibaba-cloud-sdk-go v1.62.510 h1:mvveZfYcJUOyj0jJqbYxWrM298JXt+ltj7dMbekjraI= github.com/aliyun/alibaba-cloud-sdk-go v1.62.510/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= -github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= -github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= +github.com/andybalholm/brotli v1.0.6 h1:Yf9fFpf49Zrxb9NlQaluyE92/+X7UVHlhMNJN2sxfOI= +github.com/andybalholm/brotli v1.0.6/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/avast/retry-go/v4 v4.5.0 h1:QoRAZZ90cj5oni2Lsgl2GW8mNTnUCnmpx/iKpwVisHg= github.com/avast/retry-go/v4 v4.5.0/go.mod h1:7hLEXp0oku2Nir2xBAsg0PTphp9z71bN5Aq1fboC3+I= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= @@ -50,8 +50,8 @@ github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxI github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= github.com/go-faster/city v1.0.1 h1:4WAxSZ3V2Ws4QRDrscLEDcibJY8uf41H6AhXDrNDcGw= github.com/go-faster/city v1.0.1/go.mod h1:jKcUJId49qdW3L1qKHH/3wPeUstCVpVSXTM6vO3VcTw= -github.com/go-faster/errors v0.6.1 h1:nNIPOBkprlKzkThvS/0YaX8Zs9KewLCOSFQS5BU06FI= -github.com/go-faster/errors v0.6.1/go.mod h1:5MGV2/2T9yvlrbhe9pD9LO5Z/2zCSq2T8j+Jpi2LAyY= +github.com/go-faster/errors v0.7.0 h1:UnD/xusnfUgtEYkgRZohqL2AfmPTwv13NAJwwFFaNYc= +github.com/go-faster/errors v0.7.0/go.mod h1:5ySTjWFiphBs07IKuiL69nxdfd5+fzh1u7FPGZP2quo= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= @@ -77,8 +77,8 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/ github.com/google/gops v0.3.27 h1:BDdWfedShsBbeatZ820oA4DbVOC8yJ4NI8xAlDFWfgI= github.com/google/gops v0.3.27/go.mod h1:lYqabmfnq4Q6UumWNx96Hjup5BDAVc8zmfIy0SkNCSk= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= -github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4= +github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= @@ -118,8 +118,8 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.16.7 h1:2mk3MPGNzKyxErAw8YaohYh69+pa4sIQSC0fPGCFR9I= -github.com/klauspost/compress v1.16.7/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.17.2 h1:RlWWUY/Dr4fL8qk9YG7DTZ7PDgME2V4csBXA8L/ixi4= +github.com/klauspost/compress v1.17.2/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= @@ -252,10 +252,10 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.mongodb.org/mongo-driver v1.11.4/go.mod h1:PTSz5yu21bkT/wXpkS7WR5f0ddqw5quethTUn9WM+2g= -go.opentelemetry.io/otel v1.16.0 h1:Z7GVAX/UkAXPKsy94IU+i6thsQS4nb7LviLpnaNeW8s= -go.opentelemetry.io/otel v1.16.0/go.mod h1:vl0h9NUa1D5s1nv3A5vZOYWn8av4K8Ml6JDeHrT/bx4= -go.opentelemetry.io/otel/trace v1.16.0 h1:8JRpaObFoW0pxuVPapkgH8UhHQj+bJW8jJsCZEu5MQs= -go.opentelemetry.io/otel/trace v1.16.0/go.mod h1:Yt9vYq1SdNz3xdjZZK7wcXv1qv2pwLkqr2QVwea0ef0= +go.opentelemetry.io/otel v1.19.0 h1:MuS/TNf4/j4IXsZuJegVzI1cwut7Qc00344rgH7p8bs= +go.opentelemetry.io/otel v1.19.0/go.mod h1:i0QyjOq3UPoTzff0PJB2N66fb4S0+rSbSB15/oyH9fY= +go.opentelemetry.io/otel/trace v1.19.0 h1:DFVQmlVbfVeOuBRrwdtaehRrWiL1JoVs9CPIQ1Dzxpg= +go.opentelemetry.io/otel/trace v1.19.0/go.mod h1:mfaSyvGyEJEI0nyV2I4qhNQnbBOUUmYZpYojqMnX2vo= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= @@ -280,8 +280,8 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.12.0 h1:tFM/ta59kqch6LlvYnPa0yx5a83cL2nHflFhYKvv9Yk= -golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= +golang.org/x/crypto v0.14.0 h1:wBqGXzWJW6m1XrIKlAH0Hs1JJ7+9KBwnIO8v66Q9cHc= +golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= golang.org/x/exp v0.0.0-20230810033253-352e893a4cad h1:g0bG7Z4uG+OgH2QDODnjp6ggkk1bJDsINcuWmJN1iJU= golang.org/x/exp v0.0.0-20230810033253-352e893a4cad/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -304,8 +304,8 @@ golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.14.0 h1:BONx9s002vGdD9umnlX1Po8vOZmrgH34qlHcD1MfK14= -golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= +golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= +golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -329,8 +329,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.11.0 h1:eG7RXZHdqOJ1i+0lgLgCpSXAp6M3LYlAo6osgSi0xOM= -golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.14.0 h1:Vz7Qs629MkJkGyHxUlRHizWJRG2j8fbQKjELVSNhy7Q= +golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= diff --git a/output/clickhouse.go b/output/clickhouse.go index 25aad982..e85aff3f 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -16,7 +16,6 @@ limitations under the License. package output import ( - "context" "encoding/json" "expvar" "fmt" @@ -27,7 +26,6 @@ import ( "time" "github.com/ClickHouse/clickhouse-go/v2" - "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/avast/retry-go/v4" "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/model" @@ -180,7 +178,7 @@ func (c *ClickHouse) AllowWriteSeries(sid, mid int64) (allowed bool) { return } -func (c *ClickHouse) writeSeries(rows model.Rows, conn clickhouse.Conn) (err error) { +func (c *ClickHouse) writeSeries(rows model.Rows, conn *pool.Conn) (err error) { var seriesRows model.Rows for _, row := range rows { if len(*row) != c.NumDims { @@ -220,7 +218,7 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( if len(*batch.Rows) == 0 { return } - var conn clickhouse.Conn + var conn *pool.Conn if conn, *dbVer, err = sc.NextGoodReplica(*dbVer); err != nil { return } @@ -276,7 +274,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, sc *pool.ShardConn) { } } -func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { +func (c *ClickHouse) initSeriesSchema(conn *pool.Conn) (err error) { if !c.taskCfg.PrometheusSchema { c.IdxSerID = -1 return @@ -344,14 +342,27 @@ func (c *ClickHouse) initSeriesSchema(conn clickhouse.Conn) (err error) { c.Dims = append(c.Dims, seriesDims[1:]...) // Generate SQL for series INSERT - serDimsQuoted := make([]string, len(seriesDims)) - for i, serDim := range seriesDims { - serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) - } - c.promSerSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", - c.dbName, - c.seriesTbl, - strings.Join(serDimsQuoted, ",")) + if c.cfg.Clickhouse.Protocol == clickhouse.HTTP.String() { + serDimsQuoted := make([]string, len(seriesDims)) + for i, serDim := range seriesDims { + serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) + } + var params = make([]string, len(seriesDims)) + for i := range params { + params[i] = "?" + } + c.promSerSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.TableName + " (" + strings.Join(serDimsQuoted, ",") + ") " + + "VALUES (" + strings.Join(params, ",") + ")" + } else { + serDimsQuoted := make([]string, len(seriesDims)) + for i, serDim := range seriesDims { + serDimsQuoted[i] = fmt.Sprintf("`%s`", serDim.Name) + } + c.promSerSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", + c.dbName, + c.seriesTbl, + strings.Join(serDimsQuoted, ",")) + } util.Logger.Info(fmt.Sprintf("promSer sql=> %s", c.promSerSQL), zap.String("task", c.taskCfg.Name)) // Check distributed series table @@ -394,7 +405,7 @@ func (c *ClickHouse) initSchema() (err error) { c.seriesTbl = c.taskCfg.SeriesTableName sc := pool.GetShardConn(0) - var conn clickhouse.Conn + var conn *pool.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } @@ -416,19 +427,38 @@ func (c *ClickHouse) initSchema() (err error) { return } // Generate SQL for INSERT - c.NumDims = len(c.Dims) - numDims := c.NumDims - if c.taskCfg.PrometheusSchema { - numDims = c.IdxSerID + 1 - } - quotedDms := make([]string, numDims) - for i := 0; i < numDims; i++ { - quotedDms[i] = fmt.Sprintf("`%s`", c.Dims[i].Name) + if c.cfg.Clickhouse.Protocol == clickhouse.HTTP.String() { + c.NumDims = len(c.Dims) + numDims := c.NumDims + if c.taskCfg.PrometheusSchema { + numDims = c.IdxSerID + 1 + } + quotedDms := make([]string, numDims) + for i := 0; i < numDims; i++ { + quotedDms[i] = fmt.Sprintf("`%s`", c.Dims[i].Name) + } + var params = make([]string, c.NumDims) + for i := range params { + params[i] = "?" + } + c.prepareSQL = "INSERT INTO " + c.cfg.Clickhouse.DB + "." + c.TableName + " (" + strings.Join(quotedDms, ",") + ") " + + "VALUES (" + strings.Join(params, ",") + ")" + + } else { + c.NumDims = len(c.Dims) + numDims := c.NumDims + if c.taskCfg.PrometheusSchema { + numDims = c.IdxSerID + 1 + } + quotedDms := make([]string, numDims) + for i := 0; i < numDims; i++ { + quotedDms[i] = fmt.Sprintf("`%s`", c.Dims[i].Name) + } + c.prepareSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", + c.dbName, + c.TableName, + strings.Join(quotedDms, ",")) } - c.prepareSQL = fmt.Sprintf("INSERT INTO `%s`.`%s` (%s)", - c.dbName, - c.TableName, - strings.Join(quotedDms, ",")) util.Logger.Info(fmt.Sprintf("Prepare sql=> %s", c.prepareSQL), zap.String("task", c.taskCfg.Name)) // Check distributed metric table @@ -512,7 +542,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } sc := pool.GetShardConn(0) - var conn clickhouse.Conn + var conn *pool.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } @@ -520,7 +550,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { alterTable := func(tbl, col string) error { query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, tbl, onCluster, col) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) - return conn.Exec(context.Background(), query) + return conn.Exec(query) } if len(alterSeries) != 0 { @@ -554,7 +584,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { func (c *ClickHouse) getDistTbls(table string) (distTbls []DistTblInfo, err error) { taskCfg := c.taskCfg sc := pool.GetShardConn(0) - var conn clickhouse.Conn + var conn *pool.Conn if conn, _, err = sc.NextGoodReplica(0); err != nil { return } @@ -562,8 +592,8 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []DistTblInfo, err erro FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(engine_full, 'Distributed\(\'.*\', \'%s\', \'%s\'.*\)')`, c.dbName, c.dbName, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) - var rows driver.Rows - if rows, err = conn.Query(context.Background(), query); err != nil { + var rows *pool.Rows + if rows, err = conn.Query(query); err != nil { err = errors.Wrapf(err, "") return } diff --git a/output/clickhouse_util.go b/output/clickhouse_util.go index 8280257a..8b46e346 100644 --- a/output/clickhouse_util.go +++ b/output/clickhouse_util.go @@ -1,12 +1,9 @@ package output import ( - "context" "fmt" "github.com/ClickHouse/clickhouse-go/v2" - "github.com/ClickHouse/clickhouse-go/v2/lib/driver" - "github.com/RoaringBitmap/roaring" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/pool" "github.com/housepower/clickhouse_sinker/util" @@ -30,57 +27,13 @@ func shouldReconnect(err error, sc *pool.ShardConn) bool { return true } -func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn clickhouse.Conn) (numBad int, err error) { - var errExec error - var batch driver.Batch - if batch, err = conn.PrepareBatch(context.Background(), prepareSQL); err != nil { - err = errors.Wrapf(err, "clickhouse.Conn.PrepareBatch %s", prepareSQL) - return - } - var bmBad *roaring.Bitmap - for i, row := range rows { - if err = batch.Append((*row)[idxBegin:idxEnd]...); err != nil { - if bmBad == nil { - errExec = errors.Wrapf(err, "driver.Batch.Append") - bmBad = roaring.NewBitmap() - } - bmBad.AddInt(i) - } - } - if errExec != nil { - _ = batch.Abort() - numBad = int(bmBad.GetCardinality()) - util.Logger.Warn(fmt.Sprintf("writeRows skipped %d rows of %d due to invalid content", numBad, len(rows)), zap.Error(errExec)) - // write rows again, skip bad ones - if batch, err = conn.PrepareBatch(context.Background(), prepareSQL); err != nil { - err = errors.Wrapf(err, "clickhouse.Conn.PrepareBatch %s", prepareSQL) - return - } - for i, row := range rows { - if !bmBad.ContainsInt(i) { - if err = batch.Append((*row)[idxBegin:idxEnd]...); err != nil { - break - } - } - } - if err = batch.Send(); err != nil { - err = errors.Wrapf(err, "driver.Batch.Send") - _ = batch.Abort() - return - } - return - } - if err = batch.Send(); err != nil { - err = errors.Wrapf(err, "driver.Batch.Send") - _ = batch.Abort() - return - } - return +func writeRows(prepareSQL string, rows model.Rows, idxBegin, idxEnd int, conn *pool.Conn) (numBad int, err error) { + return conn.Write(prepareSQL, rows, idxBegin, idxEnd) } -func getDims(database, table string, excludedColumns []string, parser string, conn clickhouse.Conn) (dims []*model.ColumnWithType, err error) { - var rs driver.Rows - if rs, err = conn.Query(context.Background(), fmt.Sprintf(selectSQLTemplate, database, table)); err != nil { +func getDims(database, table string, excludedColumns []string, parser string, conn *pool.Conn) (dims []*model.ColumnWithType, err error) { + var rs *pool.Rows + if rs, err = conn.Query(fmt.Sprintf(selectSQLTemplate, database, table)); err != nil { err = errors.Wrapf(err, "") return } diff --git a/pool/ck_cli.go b/pool/ck_cli.go new file mode 100644 index 00000000..abde2c50 --- /dev/null +++ b/pool/ck_cli.go @@ -0,0 +1,254 @@ +package pool + +import ( + "context" + "database/sql" + "fmt" + + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + "github.com/RoaringBitmap/roaring" + "github.com/housepower/clickhouse_sinker/model" + "github.com/housepower/clickhouse_sinker/util" + "github.com/thanos-io/thanos/pkg/errors" + "go.uber.org/zap" +) + +type Row struct { + proto clickhouse.Protocol + r1 *sql.Row + r2 driver.Row +} + +func (r *Row) Scan(dest ...any) error { + if r.proto == clickhouse.HTTP { + return r.r1.Scan(dest...) + } else { + return r.r2.Scan(dest...) + } +} + +type Rows struct { + Proto clickhouse.Protocol + Rs1 *sql.Rows + Rs2 driver.Rows +} + +func (r *Rows) Close() error { + if r.Proto == clickhouse.HTTP { + return r.Rs1.Close() + } else { + return r.Rs2.Close() + } +} + +func (r *Rows) Columns() ([]string, error) { + if r.Proto == clickhouse.HTTP { + return r.Rs1.Columns() + } else { + return r.Rs2.Columns(), nil + } +} + +func (r *Rows) Next() bool { + if r.Proto == clickhouse.HTTP { + return r.Rs1.Next() + } else { + return r.Rs2.Next() + } +} + +func (r *Rows) Scan(dest ...any) error { + if r.Proto == clickhouse.HTTP { + return r.Rs1.Scan(dest...) + } else { + return r.Rs2.Scan(dest...) + } +} + +// func (r *Rows) ColumnTypes() (interface{}, error) { +// if r.Proto == clickhouse.HTTP { +// return r.Rs1.ColumnTypes() +// } else { +// return r.Rs2.ColumnTypes(), nil +// } +// } + +type Conn struct { + protocol clickhouse.Protocol + c driver.Conn + db *sql.DB + ctx context.Context +} + +func (c *Conn) Query(query string, args ...any) (*Rows, error) { + var rs Rows + rs.Proto = c.protocol + if c.protocol == clickhouse.HTTP { + rows, err := c.db.Query(query, args) + if err != nil { + return &rs, err + } else { + rs.Rs1 = rows + } + } else { + rows, err := c.c.Query(c.ctx, query, args) + if err != nil { + return &rs, err + } else { + rs.Rs2 = rows + } + } + return &rs, nil +} + +func (c *Conn) QueryRow(query string, args ...any) *Row { + var row Row + row.proto = c.protocol + if c.protocol == clickhouse.HTTP { + row.r1 = c.db.QueryRow(query, args) + } else { + row.r2 = c.c.QueryRow(c.ctx, query, args) + } + return &row +} + +func (c *Conn) Exec(query string, args ...any) error { + if c.protocol == clickhouse.HTTP { + _, err := c.db.Exec(query, args) + return err + } else { + return c.c.Exec(c.ctx, query, args) + } +} + +func (c *Conn) Ping() error { + if c.protocol == clickhouse.HTTP { + return c.db.Ping() + } else { + return c.c.Ping(c.ctx) + } +} + +func (c *Conn) write_v1(prepareSQL string, rows model.Rows, idxBegin, idxEnd int) (numBad int, err error) { + var errExec error + + var stmt *sql.Stmt + var tx *sql.Tx + tx, err = c.db.Begin() + if err != nil { + err = errors.Wrapf(err, "pool.Conn.Begin") + return + } + + if stmt, err = tx.Prepare(prepareSQL); err != nil { + err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) + return + } + defer stmt.Close() + + var bmBad *roaring.Bitmap + for i, row := range rows { + if _, err = stmt.Exec((*row)[idxBegin:idxEnd]...); err != nil { + if bmBad == nil { + errExec = errors.Wrapf(err, "driver.Batch.Append") + bmBad = roaring.NewBitmap() + } + bmBad.AddInt(i) + } + + } + if errExec != nil { + _ = tx.Rollback() + numBad = int(bmBad.GetCardinality()) + util.Logger.Warn(fmt.Sprintf("writeRows skipped %d rows of %d due to invalid content", numBad, len(rows)), zap.Error(errExec)) + // write rows again, skip bad ones + if stmt, err = tx.Prepare(prepareSQL); err != nil { + err = errors.Wrapf(err, "tx.Prepare %s", prepareSQL) + return + } + for i, row := range rows { + if !bmBad.ContainsInt(i) { + if _, err = stmt.Exec((*row)[idxBegin:idxEnd]...); err != nil { + break + } + } + } + if err = tx.Commit(); err != nil { + err = errors.Wrapf(err, "tx.Commit") + _ = tx.Rollback() + return + } + return + } + if err = tx.Commit(); err != nil { + err = errors.Wrapf(err, "tx.Commit") + _ = tx.Rollback() + return + } + return +} + +func (c *Conn) write_v2(prepareSQL string, rows model.Rows, idxBegin, idxEnd int) (numBad int, err error) { + var errExec error + var batch driver.Batch + if batch, err = c.c.PrepareBatch(c.ctx, prepareSQL); err != nil { + err = errors.Wrapf(err, "pool.Conn.PrepareBatch %s", prepareSQL) + return + } + var bmBad *roaring.Bitmap + for i, row := range rows { + if err = batch.Append((*row)[idxBegin:idxEnd]...); err != nil { + if bmBad == nil { + errExec = errors.Wrapf(err, "driver.Batch.Append") + bmBad = roaring.NewBitmap() + } + bmBad.AddInt(i) + } + } + if errExec != nil { + _ = batch.Abort() + numBad = int(bmBad.GetCardinality()) + util.Logger.Warn(fmt.Sprintf("writeRows skipped %d rows of %d due to invalid content", numBad, len(rows)), zap.Error(errExec)) + // write rows again, skip bad ones + if batch, err = c.c.PrepareBatch(c.ctx, prepareSQL); err != nil { + err = errors.Wrapf(err, "pool.Conn.PrepareBatch %s", prepareSQL) + return + } + for i, row := range rows { + if !bmBad.ContainsInt(i) { + if err = batch.Append((*row)[idxBegin:idxEnd]...); err != nil { + break + } + } + } + if err = batch.Send(); err != nil { + err = errors.Wrapf(err, "driver.Batch.Send") + _ = batch.Abort() + return + } + return + } + if err = batch.Send(); err != nil { + err = errors.Wrapf(err, "driver.Batch.Send") + _ = batch.Abort() + return + } + return +} + +func (c *Conn) Write(prepareSQL string, rows model.Rows, idxBegin, idxEnd int) (numBad int, err error) { + if c.protocol == clickhouse.HTTP { + return c.write_v1(prepareSQL, rows, idxBegin, idxEnd) + } else { + return c.write_v2(prepareSQL, rows, idxBegin, idxEnd) + } +} + +func (c *Conn) Close() error { + if c.protocol == clickhouse.HTTP { + return c.db.Close() + } else { + return c.c.Close() + } +} diff --git a/pool/conn.go b/pool/conn.go index 324499c8..d7dbc9a3 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -19,6 +19,7 @@ package pool // Clickhouse connection pool import ( + "context" "crypto/tls" "fmt" "sync" @@ -38,12 +39,13 @@ var ( // ShardConn a datastructure for storing the clickhouse connection type ShardConn struct { lock sync.Mutex - db clickhouse.Conn + conn *Conn dbVer int opts clickhouse.Options replicas []string //ip:port list of replicas nextRep int //index of next replica writingPool *util.WorkerPool //the all tasks' writing ClickHouse, cpu-net balance + protocol clickhouse.Protocol } func (sc *ShardConn) SubmitTask(fn func()) (err error) { @@ -54,7 +56,7 @@ func (sc *ShardConn) SubmitTask(fn func()) (err error) { func (sc *ShardConn) GetReplica() (replica string) { sc.lock.Lock() defer sc.lock.Unlock() - if sc.db != nil { + if sc.conn != nil { curRep := (len(sc.replicas) + sc.nextRep - 1) % len(sc.replicas) replica = sc.replicas[curRep] } @@ -65,9 +67,9 @@ func (sc *ShardConn) GetReplica() (replica string) { func (sc *ShardConn) Close() { sc.lock.Lock() defer sc.lock.Unlock() - if sc.db != nil { - sc.db.Close() - sc.db = nil + if sc.conn != nil { + sc.conn.Close() + sc.conn = nil } if sc.writingPool != nil { sc.writingPool.StopWait() @@ -75,10 +77,10 @@ func (sc *ShardConn) Close() { } // NextGoodReplica connects to next good replica -func (sc *ShardConn) NextGoodReplica(failedVer int) (db clickhouse.Conn, dbVer int, err error) { +func (sc *ShardConn) NextGoodReplica(failedVer int) (db *Conn, dbVer int, err error) { sc.lock.Lock() defer sc.lock.Unlock() - if sc.db != nil { + if sc.conn != nil { if sc.dbVer > failedVer { // Another goroutine has already done connection. // Notice: Why recording failure version instead timestamp? @@ -86,38 +88,52 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db clickhouse.Conn, dbVer i // conn1 = NextGood(0); conn2 = NexGood(0); conn1.Exec failed at ts1; // conn3 = NextGood(ts1); conn2.Exec failed at ts2; // conn4 = NextGood(ts2) will close the good connection and break users. - return sc.db, sc.dbVer, nil + return sc.conn, sc.dbVer, nil } - sc.db.Close() - sc.db = nil + sc.conn.Close() + sc.conn = nil } savedNextRep := sc.nextRep // try all replicas, including the current one + conn := Conn{ + protocol: sc.protocol, + ctx: context.Background(), + } for i := 0; i < len(sc.replicas); i++ { replica := sc.replicas[sc.nextRep] sc.opts.Addr = []string{replica} sc.nextRep = (sc.nextRep + 1) % len(sc.replicas) - sc.db, err = clickhouse.Open(&sc.opts) + if sc.protocol == clickhouse.HTTP { + conn.db = clickhouse.OpenDB(&sc.opts) + } else { + conn.c, err = clickhouse.Open(&sc.opts) + } if err != nil { util.Logger.Warn("clickhouse.Open failed", zap.String("replica", replica), zap.Error(err)) continue } sc.dbVer++ util.Logger.Info("clickhouse.Open succeeded", zap.Int("dbVer", sc.dbVer), zap.String("replica", replica)) - return sc.db, sc.dbVer, nil + sc.conn = &conn + return sc.conn, sc.dbVer, nil } err = errors.Newf("no good replica among replicas %v since %d", sc.replicas, savedNextRep) return nil, sc.dbVer, err } -// Each shard has a clickhouse.Conn which connects to one replica inside the shard. +// Each shard has a pool.Conn which connects to one replica inside the shard. // We need more control than replica single-point-failure. func InitClusterConn(hosts [][]string, port int, db, username, password string, secure, skipVerify bool, - maxOpenConns int) (err error) { + maxOpenConns int, protocol string) (err error) { lock.Lock() defer lock.Unlock() freeClusterConn() + proto := clickhouse.Native + if protocol == clickhouse.HTTP.String() { + proto = clickhouse.HTTP + } + for _, replicas := range hosts { numReplicas := len(replicas) replicaAddrs := make([]string, numReplicas) @@ -138,10 +154,11 @@ func InitClusterConn(hosts [][]string, port int, db, username, password string, Username: username, Password: password, }, - DialTimeout: time.Minute * 10, - MaxOpenConns: maxOpenConns, - MaxIdleConns: maxOpenConns, - ConnMaxLifetime: time.Minute * 10, + Protocol: proto, + DialTimeout: time.Minute * 10, + // MaxOpenConns: maxOpenConns, + // MaxIdleConns: maxOpenConns, + // ConnMaxLifetime: time.Minute * 10, }, writingPool: util.NewWorkerPool(maxOpenConns, 1), } @@ -150,6 +167,12 @@ func InitClusterConn(hosts [][]string, port int, db, username, password string, tlsConfig.InsecureSkipVerify = skipVerify sc.opts.TLS = tlsConfig } + if proto == clickhouse.Native { + sc.opts.MaxOpenConns = maxOpenConns + sc.opts.MaxIdleConns = maxOpenConns + sc.opts.ConnMaxLifetime = time.Minute * 10 + } + sc.protocol = proto if _, _, err = sc.NextGoodReplica(0); err != nil { return } diff --git a/task/sinker.go b/task/sinker.go index fa486e19..c1c800cb 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -26,8 +26,6 @@ import ( "sync" "time" - "github.com/ClickHouse/clickhouse-go/v2" - "github.com/google/uuid" "github.com/housepower/clickhouse_sinker/config" cm "github.com/housepower/clickhouse_sinker/config_manager" @@ -307,7 +305,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { // 1. Initialize clickhouse connections chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { + chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.Protocol); err != nil { return } @@ -347,7 +345,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 2. Initialize clickhouse connections. chCfg := &newCfg.Clickhouse if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns); err != nil { + chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.Protocol); err != nil { return } @@ -517,7 +515,7 @@ func (s *Sinker) initBmSeries() (err error) { return true }) - var conn clickhouse.Conn + var conn *pool.Conn if conn, _, err = pool.GetShardConn(0).NextGoodReplica(0); err != nil { return } @@ -585,7 +583,7 @@ func (s *Sinker) reloadBmSeries() (err error) { return } - var conn clickhouse.Conn + var conn *pool.Conn if conn, _, err = pool.GetShardConn(0).NextGoodReplica(0); err != nil { return } @@ -608,7 +606,7 @@ func (s *Sinker) reloadBmSeries() (err error) { return } -func loadBmSeries(conn clickhouse.Conn, sqKey string, tasks []*Service, activeSeriesRange int) (result map[int64]int64, err error) { +func loadBmSeries(conn *pool.Conn, sqKey string, tasks []*Service, activeSeriesRange int) (result map[int64]int64, err error) { // merge all metric tables to get the latest timestamp // old bmseries record won't be loaded into memory to avoid OOM var reg string @@ -622,21 +620,21 @@ func loadBmSeries(conn clickhouse.Conn, sqKey string, tasks []*Service, activeSe dbname := strings.Split(sqKey, ".")[0] query := fmt.Sprintf(createTableSQL, mergetable, dbname, tasks[0].clickhouse.GetMetricTable(), dbname, reg[:len(reg)-1]) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) - if err = conn.Exec(context.Background(), query); err != nil { + if err = conn.Exec(query); err != nil { return } var count uint64 query = fmt.Sprintf(countSeriesSQL, mergetable, sqKey, mergetable, activeSeriesRange) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) - if err = conn.QueryRow(context.Background(), query).Scan(&count); err != nil { + if err = conn.QueryRow(query).Scan(&count); err != nil { return } seriesMap := make(map[int64]int64, count) query = fmt.Sprintf(loadSeriesSQL, mergetable, sqKey, mergetable, activeSeriesRange) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) - rs, err := conn.Query(context.Background(), query) + rs, err := conn.Query(query) if err != nil { return nil, err } @@ -651,7 +649,7 @@ func loadBmSeries(conn clickhouse.Conn, sqKey string, tasks []*Service, activeSe } query = fmt.Sprintf(dropTableSQL, mergetable) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) - err = conn.Exec(context.Background(), query) + err = conn.Exec(query) return seriesMap, err } From a2c89aaa6dbf2c8e4d6ee63b1e609ebf3da25cb3 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Fri, 10 Nov 2023 18:07:35 +0800 Subject: [PATCH 363/404] feat: set connect params while protocol is http --- pool/ck_cli.go | 44 ++++++++++++++++++-------------------------- pool/conn.go | 36 +++++++++++++++++++----------------- task/sinker.go | 6 ++---- 3 files changed, 39 insertions(+), 47 deletions(-) diff --git a/pool/ck_cli.go b/pool/ck_cli.go index abde2c50..6624c480 100644 --- a/pool/ck_cli.go +++ b/pool/ck_cli.go @@ -29,51 +29,43 @@ func (r *Row) Scan(dest ...any) error { } type Rows struct { - Proto clickhouse.Protocol - Rs1 *sql.Rows - Rs2 driver.Rows + protocol clickhouse.Protocol + rs1 *sql.Rows + rs2 driver.Rows } func (r *Rows) Close() error { - if r.Proto == clickhouse.HTTP { - return r.Rs1.Close() + if r.protocol == clickhouse.HTTP { + return r.rs1.Close() } else { - return r.Rs2.Close() + return r.rs2.Close() } } func (r *Rows) Columns() ([]string, error) { - if r.Proto == clickhouse.HTTP { - return r.Rs1.Columns() + if r.protocol == clickhouse.HTTP { + return r.rs1.Columns() } else { - return r.Rs2.Columns(), nil + return r.rs2.Columns(), nil } } func (r *Rows) Next() bool { - if r.Proto == clickhouse.HTTP { - return r.Rs1.Next() + if r.protocol == clickhouse.HTTP { + return r.rs1.Next() } else { - return r.Rs2.Next() + return r.rs2.Next() } } func (r *Rows) Scan(dest ...any) error { - if r.Proto == clickhouse.HTTP { - return r.Rs1.Scan(dest...) + if r.protocol == clickhouse.HTTP { + return r.rs1.Scan(dest...) } else { - return r.Rs2.Scan(dest...) + return r.rs2.Scan(dest...) } } -// func (r *Rows) ColumnTypes() (interface{}, error) { -// if r.Proto == clickhouse.HTTP { -// return r.Rs1.ColumnTypes() -// } else { -// return r.Rs2.ColumnTypes(), nil -// } -// } - type Conn struct { protocol clickhouse.Protocol c driver.Conn @@ -83,20 +75,20 @@ type Conn struct { func (c *Conn) Query(query string, args ...any) (*Rows, error) { var rs Rows - rs.Proto = c.protocol + rs.protocol = c.protocol if c.protocol == clickhouse.HTTP { rows, err := c.db.Query(query, args) if err != nil { return &rs, err } else { - rs.Rs1 = rows + rs.rs1 = rows } } else { rows, err := c.c.Query(c.ctx, query, args) if err != nil { return &rs, err } else { - rs.Rs2 = rows + rs.rs2 = rows } } return &rs, nil diff --git a/pool/conn.go b/pool/conn.go index d7dbc9a3..cad3023e 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -26,6 +26,7 @@ import ( "time" "github.com/ClickHouse/clickhouse-go/v2" + "github.com/housepower/clickhouse_sinker/config" "github.com/housepower/clickhouse_sinker/util" "github.com/thanos-io/thanos/pkg/errors" "go.uber.org/zap" @@ -46,6 +47,7 @@ type ShardConn struct { nextRep int //index of next replica writingPool *util.WorkerPool //the all tasks' writing ClickHouse, cpu-net balance protocol clickhouse.Protocol + chCfg *config.ClickHouseConfig } func (sc *ShardConn) SubmitTask(fn func()) (err error) { @@ -105,6 +107,9 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db *Conn, dbVer int, err er sc.nextRep = (sc.nextRep + 1) % len(sc.replicas) if sc.protocol == clickhouse.HTTP { conn.db = clickhouse.OpenDB(&sc.opts) + conn.db.SetMaxOpenConns(sc.chCfg.MaxOpenConns) + conn.db.SetMaxIdleConns(sc.chCfg.MaxOpenConns) + conn.db.SetConnMaxLifetime(time.Minute * 10) } else { conn.c, err = clickhouse.Open(&sc.opts) } @@ -123,53 +128,50 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db *Conn, dbVer int, err er // Each shard has a pool.Conn which connects to one replica inside the shard. // We need more control than replica single-point-failure. -func InitClusterConn(hosts [][]string, port int, db, username, password string, secure, skipVerify bool, - maxOpenConns int, protocol string) (err error) { +func InitClusterConn(chCfg *config.ClickHouseConfig) (err error) { lock.Lock() defer lock.Unlock() freeClusterConn() proto := clickhouse.Native - if protocol == clickhouse.HTTP.String() { + if chCfg.Protocol == clickhouse.HTTP.String() { proto = clickhouse.HTTP } - for _, replicas := range hosts { + for _, replicas := range chCfg.Hosts { numReplicas := len(replicas) replicaAddrs := make([]string, numReplicas) for i, ip := range replicas { // Changing hostnames to IPs breaks TLS connections in many cases - if !secure { + if !chCfg.Secure { if ips2, err := util.GetIP4Byname(ip); err == nil { ip = ips2[0] } } - replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, port) + replicaAddrs[i] = fmt.Sprintf("%s:%d", ip, chCfg.Port) } sc := &ShardConn{ replicas: replicaAddrs, + chCfg: chCfg, opts: clickhouse.Options{ Auth: clickhouse.Auth{ - Database: db, - Username: username, - Password: password, + Database: chCfg.DB, + Username: chCfg.Username, + Password: chCfg.Password, }, Protocol: proto, DialTimeout: time.Minute * 10, - // MaxOpenConns: maxOpenConns, - // MaxIdleConns: maxOpenConns, - // ConnMaxLifetime: time.Minute * 10, }, - writingPool: util.NewWorkerPool(maxOpenConns, 1), + writingPool: util.NewWorkerPool(chCfg.MaxOpenConns, 1), } - if secure { + if chCfg.Secure { tlsConfig := &tls.Config{} - tlsConfig.InsecureSkipVerify = skipVerify + tlsConfig.InsecureSkipVerify = chCfg.InsecureSkipVerify sc.opts.TLS = tlsConfig } if proto == clickhouse.Native { - sc.opts.MaxOpenConns = maxOpenConns - sc.opts.MaxIdleConns = maxOpenConns + sc.opts.MaxOpenConns = chCfg.MaxOpenConns + sc.opts.MaxIdleConns = chCfg.MaxOpenConns sc.opts.ConnMaxLifetime = time.Minute * 10 } sc.protocol = proto diff --git a/task/sinker.go b/task/sinker.go index c1c800cb..cc24f349 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -304,8 +304,7 @@ func (s *Sinker) applyFirstConfig(newCfg *config.Config) (err error) { util.Logger.Info("going to apply the first config", zap.Any("config", newCfg)) // 1. Initialize clickhouse connections chCfg := &newCfg.Clickhouse - if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.Protocol); err != nil { + if err = pool.InitClusterConn(chCfg); err != nil { return } @@ -344,8 +343,7 @@ func (s *Sinker) applyAnotherConfig(newCfg *config.Config) (err error) { // 2. Initialize clickhouse connections. chCfg := &newCfg.Clickhouse - if err = pool.InitClusterConn(chCfg.Hosts, chCfg.Port, chCfg.DB, chCfg.Username, chCfg.Password, - chCfg.Secure, chCfg.InsecureSkipVerify, chCfg.MaxOpenConns, chCfg.Protocol); err != nil { + if err = pool.InitClusterConn(chCfg); err != nil { return } From e4dd60bab7389c92a68bb07f8e7fa10ec11437ed Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Mon, 27 Nov 2023 10:49:45 +0800 Subject: [PATCH 364/404] feat: reset kafka sasl domain_realm for some cloud service, like huawei mrs --- config/config.go | 26 +++++++++++++++++++++++--- go.mod | 2 +- go.sum | 14 ++------------ input/kafka_franz.go | 2 +- 4 files changed, 27 insertions(+), 17 deletions(-) diff --git a/config/config.go b/config/config.go index df001f76..2e33ea76 100644 --- a/config/config.go +++ b/config/config.go @@ -16,6 +16,7 @@ limitations under the License. package config import ( + "net" "os" "regexp" "strings" @@ -45,9 +46,10 @@ type Config struct { // KafkaConfig configuration parameters type KafkaConfig struct { - Brokers string - Security map[string]string - TLS struct { + Brokers string + ResetSaslRealm bool + Security map[string]string + TLS struct { Enable bool CaCertFiles string // CA cert.pem with which Kafka brokers certs be signed. Leave empty for certificates trusted by the OS ClientCertFile string // Required for client authentication. It's client cert.pem. @@ -250,6 +252,11 @@ func (cfg *Config) Normallize(constructGroup bool, httpAddr string, cred util.Cr err = errors.Newf("kafka SASL mechanism %s is unsupported", cfg.Kafka.Sasl.Mechanism) return } + + if cfg.Kafka.ResetSaslRealm { + port := getKfkPort(cfg.Kafka.Brokers) + os.Setenv("DOMAIN_REALM", net.JoinHostPort("hadoop."+strings.ToLower(cfg.Kafka.Sasl.GSSAPI.Realm), port)) + } } if cfg.Clickhouse.RetryTimes < 0 { cfg.Clickhouse.RetryTimes = 0 @@ -477,3 +484,16 @@ func readConfig(config string) map[string]string { } return configMap } + +func getKfkPort(brokers string) string { + hosts := strings.Split(brokers, ",") + var port string + for _, host := range hosts { + _, p, err := net.SplitHostPort(host) + if err != nil { + port = p + break + } + } + return port +} diff --git a/go.mod b/go.mod index 1d3e937f..9b20c4a1 100644 --- a/go.mod +++ b/go.mod @@ -5,6 +5,7 @@ go 1.20 require ( github.com/ClickHouse/clickhouse-go/v2 v2.15.0 github.com/RoaringBitmap/roaring v1.3.0 + github.com/YenchangChan/franz-go/pkg/sasl/kerberos v0.0.0-20231127011105-840a25342a2e github.com/avast/retry-go/v4 v4.5.0 github.com/bytedance/sonic v1.10.0-rc3 github.com/cespare/xxhash/v2 v2.2.0 @@ -24,7 +25,6 @@ require ( github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 github.com/twmb/franz-go v1.14.3 github.com/twmb/franz-go/pkg/kadm v1.9.0 - github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/plugin/kzap v1.1.2 github.com/valyala/fastjson v1.6.4 go.uber.org/zap v1.25.0 diff --git a/go.sum b/go.sum index e6c48de4..f9b7a32d 100644 --- a/go.sum +++ b/go.sum @@ -7,6 +7,8 @@ github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3P github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/RoaringBitmap/roaring v1.3.0 h1:aQmu9zQxDU0uhwR8SXOH/OrqEf+X8A0LQmwW3JX8Lcg= github.com/RoaringBitmap/roaring v1.3.0/go.mod h1:plvDsJQpxOC5bw8LRteu/MLWHsHez/3y6cubLI4/1yE= +github.com/YenchangChan/franz-go/pkg/sasl/kerberos v0.0.0-20231127011105-840a25342a2e h1:oK5q2SkjL7lbAXipbG9qVLz6Fm81K3z3YxdehKBp0Yo= +github.com/YenchangChan/franz-go/pkg/sasl/kerberos v0.0.0-20231127011105-840a25342a2e/go.mod h1:/lT6Rz1hqrp0IVOvyM6AUOrrvJ3iPX4rofskVigwLfw= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= @@ -95,7 +97,6 @@ github.com/jcmturner/gofork v1.7.6 h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVET github.com/jcmturner/gofork v1.7.6/go.mod h1:1622LH6i/EZqLloHfE7IeZ0uEJwMSUyQ/nDd82IeqRo= github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= -github.com/jcmturner/gokrb5/v8 v8.4.3/go.mod h1:dqRwJGXznQrzw6cWmyo6kH+E7jksEQG/CyVWsJEsJO0= github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh687T8= github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= @@ -117,7 +118,6 @@ github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7V github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.17.2 h1:RlWWUY/Dr4fL8qk9YG7DTZ7PDgME2V4csBXA8L/ixi4= github.com/klauspost/compress v1.17.2/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= @@ -155,7 +155,6 @@ github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b/go.mo github.com/paulmach/orb v0.10.0 h1:guVYVqzxHE/CQ1KpfGO077TR0ATHSNjp4s6XGLn3W9s= github.com/paulmach/orb v0.10.0/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= -github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= github.com/pierrec/lz4/v4 v4.1.18/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -226,16 +225,12 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNij github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= -github.com/twmb/franz-go v1.7.0/go.mod h1:PMze0jNfNghhih2XHbkmTFykbMF5sJqmNJB31DOOzro= github.com/twmb/franz-go v1.14.3 h1:cq8rxAnVYU1uF3SRVn8eEaUf+AaXKWlB0Cl3Ca7JSa4= github.com/twmb/franz-go v1.14.3/go.mod h1:nMAvTC2kHtK+ceaSHeHm4dlxC78389M/1DjpOswEgu4= github.com/twmb/franz-go/pkg/kadm v1.9.0 h1:UgwBu0YCd6P8HLdg6ZRA4v9W6/zoI1042fOd2CvvLBE= github.com/twmb/franz-go/pkg/kadm v1.9.0/go.mod h1:eG3f+GHUndq1CUSVvjp+WdNq5zePeJi3tEHzyTkao6g= -github.com/twmb/franz-go/pkg/kmsg v1.2.0/go.mod h1:SxG/xJKhgPu25SamAq0rrucfp7lbzCpEXOC+vH/ELrY= github.com/twmb/franz-go/pkg/kmsg v1.6.1 h1:tm6hXPv5antMHLasTfKv9R+X03AjHSkSkXhQo2c5ALM= github.com/twmb/franz-go/pkg/kmsg v1.6.1/go.mod h1:se9Mjdt0Nwzc9lnjJ0HyDtLyBnaBDAd7pCje47OhSyw= -github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 h1:alKdbddkPw3rDh+AwmUEwh6HNYgTvDSFIe/GWYRR9RM= -github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0/go.mod h1:k8BoBjyUbFj34f0rRbn+Ky12sZFAPbmShrg0karAIMo= github.com/twmb/franz-go/plugin/kzap v1.1.2 h1:0arX5xJ0soUPX1LlDay6ZZoxuWkWk1lggQ5M/IgRXAE= github.com/twmb/franz-go/plugin/kzap v1.1.2/go.mod h1:53Cl9Uz1pbdOPDvUISIxLrZIWSa2jCuY1bTMauRMBmo= github.com/uber/jaeger-client-go v2.30.0+incompatible h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o= @@ -277,8 +272,6 @@ golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.0.0-20220817201139-bc19a97f63c8/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.14.0 h1:wBqGXzWJW6m1XrIKlAH0Hs1JJ7+9KBwnIO8v66Q9cHc= golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= @@ -300,8 +293,6 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= -golang.org/x/net v0.0.0-20220812174116-3211cb980234/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= @@ -327,7 +318,6 @@ golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.14.0 h1:Vz7Qs629MkJkGyHxUlRHizWJRG2j8fbQKjELVSNhy7Q= golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 87d4c4a7..db2ef0a9 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -24,13 +24,13 @@ import ( "sync" "time" + "github.com/YenchangChan/franz-go/pkg/sasl/kerberos" krb5client "github.com/jcmturner/gokrb5/v8/client" krb5config "github.com/jcmturner/gokrb5/v8/config" "github.com/jcmturner/gokrb5/v8/keytab" "github.com/thanos-io/thanos/pkg/errors" "github.com/twmb/franz-go/pkg/kgo" "github.com/twmb/franz-go/pkg/sasl" - "github.com/twmb/franz-go/pkg/sasl/kerberos" "github.com/twmb/franz-go/pkg/sasl/plain" "github.com/twmb/franz-go/pkg/sasl/scram" "github.com/twmb/franz-go/plugin/kzap" From 9788be38703b31269c850eafba08c8002c65370b Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Wed, 29 Nov 2023 18:22:41 +0800 Subject: [PATCH 365/404] feat: if security and sasl configured both, use sasl config --- config/config.go | 44 +++++++++++++++--------------- util/common.go | 65 +++++++++++++++++++++++++++++++++++++++++++++ util/common_test.go | 33 +++++++++++++++++++++++ 3 files changed, 119 insertions(+), 23 deletions(-) diff --git a/config/config.go b/config/config.go index 2e33ea76..485ec684 100644 --- a/config/config.go +++ b/config/config.go @@ -414,45 +414,43 @@ func (cfg *Config) convertKfkSecurity() { } if strings.Contains(protocol, "SSL") { - cfg.Kafka.TLS.Enable = true - cfg.Kafka.TLS.EndpIdentAlgo = cfg.Kafka.Security["ssl.endpoint.identification.algorithm"] - cfg.Kafka.TLS.TrustStoreLocation = cfg.Kafka.Security["ssl.truststore.location"] - cfg.Kafka.TLS.TrustStorePassword = cfg.Kafka.Security["ssl.truststore.password"] - cfg.Kafka.TLS.KeystoreLocation = cfg.Kafka.Security["ssl.keystore.location"] - cfg.Kafka.TLS.KeystorePassword = cfg.Kafka.Security["ssl.keystore.password"] + util.TrySetValue(&cfg.Kafka.TLS.Enable, true) + util.TrySetValue(&cfg.Kafka.TLS.EndpIdentAlgo, cfg.Kafka.Security["ssl.endpoint.identification.algorithm"]) + util.TrySetValue(&cfg.Kafka.TLS.TrustStoreLocation, cfg.Kafka.Security["ssl.truststore.location"]) + util.TrySetValue(&cfg.Kafka.TLS.TrustStorePassword, cfg.Kafka.Security["ssl.truststore.password"]) + util.TrySetValue(&cfg.Kafka.TLS.KeystoreLocation, cfg.Kafka.Security["ssl.keystore.location"]) + util.TrySetValue(&cfg.Kafka.TLS.KeystorePassword, cfg.Kafka.Security["ssl.keystore.password"]) } if strings.Contains(protocol, "SASL") { - cfg.Kafka.Sasl.Enable = true - cfg.Kafka.Sasl.Mechanism = cfg.Kafka.Security["sasl.mechanism"] + util.TrySetValue(&cfg.Kafka.Sasl.Enable, true) + util.TrySetValue(&cfg.Kafka.Sasl.Mechanism, cfg.Kafka.Security["sasl.mechanism"]) if config, ok := cfg.Kafka.Security["sasl.jaas.config"]; ok { configMap := readConfig(config) if strings.Contains(cfg.Kafka.Sasl.Mechanism, "GSSAPI") { // GSSAPI if configMap["useKeyTab"] != "true" { - // Username and password - cfg.Kafka.Sasl.GSSAPI.AuthType = 1 - cfg.Kafka.Sasl.GSSAPI.Username = configMap["username"] - cfg.Kafka.Sasl.GSSAPI.Password = configMap["password"] + //Username and password + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.AuthType, 1) + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.Username, configMap["username"]) + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.Password, configMap["password"]) } else { - // Keytab - cfg.Kafka.Sasl.GSSAPI.AuthType = 2 - cfg.Kafka.Sasl.GSSAPI.KeyTabPath = configMap["keyTab"] + //Keytab + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.AuthType, 2) + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.KeyTabPath, configMap["keyTab"]) if principal, ok := configMap["principal"]; ok { username := strings.Split(principal, "@")[0] realm := strings.Split(principal, "@")[1] - cfg.Kafka.Sasl.GSSAPI.Username = username - cfg.Kafka.Sasl.GSSAPI.Realm = realm - } - cfg.Kafka.Sasl.GSSAPI.ServiceName = cfg.Kafka.Security["sasl.kerberos.service.name"] - if cfg.Kafka.Sasl.GSSAPI.KerberosConfigPath == "" { - cfg.Kafka.Sasl.GSSAPI.KerberosConfigPath = defaultKerberosConfigPath + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.Username, username) + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.Realm, realm) } + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.ServiceName, cfg.Kafka.Security["sasl.kerberos.service.name"]) + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.KerberosConfigPath, defaultKerberosConfigPath) } } else { // PLAIN, SCRAM-SHA-256 or SCRAM-SHA-512 - cfg.Kafka.Sasl.Username = configMap["username"] - cfg.Kafka.Sasl.Password = configMap["password"] + util.TrySetValue(&cfg.Kafka.Sasl.Username, configMap["username"]) + util.TrySetValue(&cfg.Kafka.Sasl.Password, configMap["password"]) } } } diff --git a/util/common.go b/util/common.go index 7777e0f5..616fdd04 100644 --- a/util/common.go +++ b/util/common.go @@ -252,3 +252,68 @@ func SetLogLevel(newLogLevel string) { logAtomLevel.SetLevel(lvl) } } + +// set v2 to v1, if v1 didn't bind any value +// FIXME: how about v1 bind default value? +func TrySetValue(v1, v2 interface{}) bool { + var ok bool + rt := reflect.TypeOf(v1) + rv := reflect.ValueOf(v1) + + if rt.Kind() != reflect.Ptr { + return ok + } + for rt.Kind() == reflect.Ptr { + rt = rt.Elem() + rv = rv.Elem() + } + + if rv.IsValid() && rv.IsZero() { + ok = true + switch rt.Kind() { + case reflect.Uint: + v, _ := v2.(uint) + rv.SetUint(uint64(v)) + case reflect.Uint8: + v, _ := v2.(uint8) + rv.SetUint(uint64(v)) + case reflect.Uint16: + v, _ := v2.(uint16) + rv.SetUint(uint64(v)) + case reflect.Uint32: + v, _ := v2.(uint32) + rv.SetUint(uint64(v)) + case reflect.Uint64: + v, _ := v2.(uint64) + rv.SetUint(uint64(v)) + case reflect.Int: + v, _ := v2.(int) + rv.SetInt(int64(v)) + case reflect.Int8: + v, _ := v2.(int8) + rv.SetInt(int64(v)) + case reflect.Int16: + v, _ := v2.(int16) + rv.SetInt(int64(v)) + case reflect.Int32: + v, _ := v2.(int32) + rv.SetInt(int64(v)) + case reflect.Int64: + v, _ := v2.(int64) + rv.SetInt(int64(v)) + case reflect.Float32: + v, _ := v2.(float32) + rv.SetFloat(float64(v)) + case reflect.Float64: + v, _ := v2.(float64) + rv.SetFloat(float64(v)) + case reflect.String: + rv.SetString(v2.(string)) + case reflect.Bool: + rv.SetBool(v2.(bool)) + default: + ok = false + } + } + return ok +} diff --git a/util/common_test.go b/util/common_test.go index e16b7e72..3dfb82cf 100644 --- a/util/common_test.go +++ b/util/common_test.go @@ -4,6 +4,7 @@ import ( "os" "testing" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -98,3 +99,35 @@ func TestMisc(t *testing.T) { }) } } + +func TestSetValue(t *testing.T) { + a := 5 + TrySetValue(&a, 10) + assert.Equal(t, a, 5) + var b string + TrySetValue(&b, "hello") + assert.Equal(t, b, "hello") + TrySetValue(&b, "world") + assert.Equal(t, b, "hello") + + var c uint32 + TrySetValue(&c, uint32(100)) + assert.Equal(t, c, uint32(100)) + TrySetValue(&c, uint32(500)) + assert.Equal(t, c, uint32(100)) + + var d float32 + TrySetValue(&d, float32(6.998)) + assert.Equal(t, d, float32(6.998)) + TrySetValue(&d, float32(3.14)) + assert.Equal(t, d, float32(6.998)) + + var e int + TrySetValue(&e, 25) + assert.Equal(t, e, 25) + + // f := 0 + // TrySetValue(&f, 25) + // assert.Equal(t, f, 0) + +} From f9102ad7d80d3bd3b3c2415314e4a6c8edf341a5 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Thu, 30 Nov 2023 09:17:11 +0800 Subject: [PATCH 366/404] fix: avoid panic when principal invalid --- config/config.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/config/config.go b/config/config.go index 485ec684..689a5646 100644 --- a/config/config.go +++ b/config/config.go @@ -439,10 +439,11 @@ func (cfg *Config) convertKfkSecurity() { util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.AuthType, 2) util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.KeyTabPath, configMap["keyTab"]) if principal, ok := configMap["principal"]; ok { - username := strings.Split(principal, "@")[0] - realm := strings.Split(principal, "@")[1] - util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.Username, username) - util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.Realm, realm) + prins := strings.Split(principal, "@") + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.Username, prins[0]) + if len(prins) > 1 { + util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.Realm, prins[1]) + } } util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.ServiceName, cfg.Kafka.Security["sasl.kerberos.service.name"]) util.TrySetValue(&cfg.Kafka.Sasl.GSSAPI.KerberosConfigPath, defaultKerberosConfigPath) From 9f9cd3b207cd42304a14db4c7fe33234b5587a8a Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Fri, 1 Dec 2023 10:00:42 +0800 Subject: [PATCH 367/404] feat: http port start from 10000 --- cmd/clickhouse_sinker/main.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index eb487501..a4524638 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -49,6 +49,10 @@ var ( runner *task.Sinker ) +const ( + HttpPortBase = 10000 +) + func initCmdOptions() { // 1. Set options to default value. cmdOps = util.CmdOptions{ @@ -177,7 +181,7 @@ func main() { // cmdOps.HTTPPort=0: let OS choose the listen port, and record the exact metrics URL to log. httpPort := cmdOps.HTTPPort if httpPort == 0 { - httpPort = util.GetSpareTCPPort(httpPort) + httpPort = util.GetSpareTCPPort(HttpPortBase) } httpHost := cmdOps.HTTPHost From d4f813739eb91d908bdb0904a2e9b52f4f6e6429 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Fri, 1 Dec 2023 10:28:14 +0800 Subject: [PATCH 368/404] chore: change makefile to generate executable process to bin/ --- .gitignore | 1 + Makefile | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 2a03a7d7..5963f398 100644 --- a/.gitignore +++ b/.gitignore @@ -13,4 +13,5 @@ nacos_cache *.p12 dist vendor +bin/ diff --git a/Makefile b/Makefile index d837a02c..27b7b1fb 100644 --- a/Makefile +++ b/Makefile @@ -18,11 +18,11 @@ pre: .PHONY: build build: pre - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o . ./... + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -o bin/ ./... .PHONY: debug debug: pre - $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o . ./... + $(GOBUILD) -ldflags '$(SINKER_LDFLAGS)' -gcflags "all=-N -l" -o bin/ ./... .PHONY: benchtest benchtest: pre From 07f3646eabdf15ba4a2bb9f8d19ba1263d99d21d Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Fri, 8 Dec 2023 13:17:57 +0800 Subject: [PATCH 369/404] docs: update doucument --- docs/configuration/config.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/configuration/config.md b/docs/configuration/config.md index e1f50e65..3a57e065 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -36,7 +36,9 @@ // retryTimes when error occurs in inserting datas "retryTimes": 0, // max open connections with each clickhouse node. default to 1. - "maxOpenConns": 1 + "maxOpenConns": 1, + // native or http, if configured secure and http both, means support https. default to native. + "protocol": "native" }, // Kafka config @@ -50,6 +52,8 @@ "sasl.mechanism":"GSSAPI", "sasl.jaas.config":"com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true debug=true keyTab=\"/etc/security/mmmtest.keytab\" principal=\"mmm@ALANWANG.COM\";" }, + // whether reset domain realm. if this option is true, domain realm will replaced by "hadoop.{toLower(GSSAPI.Realm)}:{port}", this feature is worked when clickhouse_sinker connect to HUAWEI MRS kerberos kafka. + "resetSaslRealm": false, // SSL "tls": { From d2683fa48b4573d593ddc1b75f7fa753bd13b0b8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 10 Dec 2023 17:22:03 +0800 Subject: [PATCH 370/404] Bump golang.org/x/net from 0.9.0 to 0.17.0 (#181) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.9.0 to 0.17.0. - [Commits](https://github.com/golang/net/compare/v0.9.0...v0.17.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Zhichang Yu From bbc8b5bacd74f64b13b8bd378ccdb7a2e33c0cc3 Mon Sep 17 00:00:00 2001 From: guangwu Date: Sun, 10 Dec 2023 17:22:35 +0800 Subject: [PATCH 371/404] fix: typo (#183) Signed-off-by: guoguangwu --- docs/dev/introduction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/introduction.md b/docs/dev/introduction.md index 1e522322..4a5b6ea9 100644 --- a/docs/dev/introduction.md +++ b/docs/dev/introduction.md @@ -327,7 +327,7 @@ Metrics are exposed at `http://ip:port/metrics`. IP is the outbound IP of this m Sinker registers with Nacos if CLI `--consul-cfg-enable` or env `CONSUL_REGISTER_ENABLE` is present. However Prometheus is [unable](https://github.com/alibaba/nacos/issues/1032) to obtain dynamic service list from nacos server. -- Push to promethues +- Push to prometheus If CLI `--metric-push-gateway-addrs` or env `METRIC_PUSH_GATEWAY_ADDRS` (a list of comma-separated urls) is present, metrics are pushed to one of given URLs regualarly. From e21fa865e4c8a6b5261263661f738bdd2853a137 Mon Sep 17 00:00:00 2001 From: exfly Date: Wed, 25 Oct 2023 18:32:23 +0800 Subject: [PATCH 372/404] feat: DynamicSchema add opt NotNullable --- config/config.go | 5 +++-- output/clickhouse.go | 15 ++++++++++----- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/config/config.go b/config/config.go index 689a5646..de79af87 100644 --- a/config/config.go +++ b/config/config.go @@ -133,8 +133,9 @@ type TaskConfig struct { } `json:"dims"` // DynamicSchema will add columns present in message to clickhouse. Requires AutoSchema be true. DynamicSchema struct { - Enable bool - MaxDims int // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack + Enable bool + NotNullable bool + MaxDims int // the upper limit of dynamic columns number, <=0 means math.MaxInt16. protecting dirty data attack // A column is added for new key K if all following conditions are true: // - K isn't in ExcludeColumns // - number of existing columns doesn't reach MaxDims-1 diff --git a/output/clickhouse.go b/output/clickhouse.go index e85aff3f..fd208bf3 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -512,21 +512,26 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { var strVal string switch intVal { case model.Bool: - strVal = "Nullable(Bool)" + strVal = "Bool" case model.Int64: - strVal = "Nullable(Int64)" + strVal = "Int64" case model.Float64: - strVal = "Nullable(Float64)" + strVal = "Float64" case model.String: - strVal = "Nullable(String)" + strVal = "String" case model.DateTime: - strVal = "Nullable(DateTime64(3))" + strVal = "DateTime64(3)" case model.Object: strVal = model.GetTypeName(intVal) default: err = errors.Newf("%s: BUG: unsupported column type %s", taskCfg.Name, model.GetTypeName(intVal)) return false } + + if !taskCfg.DynamicSchema.NotNullable { + strVal = fmt.Sprintf("Nullable(%v)", strVal) + } + if c.taskCfg.PrometheusSchema && intVal == model.String { alterSeries = append(alterSeries, fmt.Sprintf("ADD COLUMN IF NOT EXISTS `%s` %s", strKey, strVal)) } else { From 5ba6308854fe1c71f6390a982720f4e6b463fe4e Mon Sep 17 00:00:00 2001 From: YenchangChan <43897067+YenchangChan@users.noreply.github.com> Date: Tue, 2 Jan 2024 17:01:52 +0800 Subject: [PATCH 373/404] ci/cd failed issue (#187) * fix: clickhouse query issue * fix: make systest --- go.test.sh | 18 +++++++++--------- pool/ck_cli.go | 12 ++++++------ 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/go.test.sh b/go.test.sh index 3df7bb08..98a4ba98 100755 --- a/go.test.sh +++ b/go.test.sh @@ -49,9 +49,9 @@ sudo docker exec kafka kafka-topics --bootstrap-server localhost:9093 --topic to sudo docker exec kafka sh /tmp/send.sh echo "start clickhouse_sinker to consume" -timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_fixed_schema.hjson -timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_auto_schema.hjson -timeout 60 ./clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.hjson +timeout 30 ./bin/clickhouse_sinker --local-cfg-file docker/test_fixed_schema.hjson +timeout 30 ./bin/clickhouse_sinker --local-cfg-file docker/test_auto_schema.hjson +timeout 60 ./bin/clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.hjson echo "check result 1" count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` @@ -75,19 +75,19 @@ curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' curl "localhost:8123" -d 'TRUNCATE TABLE test_dynamic_schema' echo "publish clickhouse_sinker config" -./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema --local-cfg-file docker/test_fixed_schema.hjson -./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema --local-cfg-file docker/test_auto_schema.hjson -./nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.hjson +./bin/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema --local-cfg-file docker/test_fixed_schema.hjson +./bin/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema --local-cfg-file docker/test_auto_schema.hjson +./bin/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.hjson echo "start clickhouse_sinker to consume" sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_fixed_schema --all-topics --to-earliest -timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema +timeout 30 ./bin/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_auto_schema --all-topics --to-earliest -timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema +timeout 30 ./bin/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_dynamic_schema --all-topics --to-earliest -timeout 30 ./clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema +timeout 30 ./bin/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema echo "check result 2" count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` diff --git a/pool/ck_cli.go b/pool/ck_cli.go index 6624c480..d2d98670 100644 --- a/pool/ck_cli.go +++ b/pool/ck_cli.go @@ -77,14 +77,14 @@ func (c *Conn) Query(query string, args ...any) (*Rows, error) { var rs Rows rs.protocol = c.protocol if c.protocol == clickhouse.HTTP { - rows, err := c.db.Query(query, args) + rows, err := c.db.Query(query, args...) if err != nil { return &rs, err } else { rs.rs1 = rows } } else { - rows, err := c.c.Query(c.ctx, query, args) + rows, err := c.c.Query(c.ctx, query, args...) if err != nil { return &rs, err } else { @@ -98,19 +98,19 @@ func (c *Conn) QueryRow(query string, args ...any) *Row { var row Row row.proto = c.protocol if c.protocol == clickhouse.HTTP { - row.r1 = c.db.QueryRow(query, args) + row.r1 = c.db.QueryRow(query, args...) } else { - row.r2 = c.c.QueryRow(c.ctx, query, args) + row.r2 = c.c.QueryRow(c.ctx, query, args...) } return &row } func (c *Conn) Exec(query string, args ...any) error { if c.protocol == clickhouse.HTTP { - _, err := c.db.Exec(query, args) + _, err := c.db.Exec(query, args...) return err } else { - return c.c.Exec(c.ctx, query, args) + return c.c.Exec(c.ctx, query, args...) } } From a07826d4454c2d932090a5e7c5a7bd1b1130c9e6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 2 Jan 2024 17:20:52 +0800 Subject: [PATCH 374/404] Bump golang.org/x/crypto from 0.14.0 to 0.17.0 (#185) Bumps [golang.org/x/crypto](https://github.com/golang/crypto) from 0.14.0 to 0.17.0. - [Commits](https://github.com/golang/crypto/compare/v0.14.0...v0.17.0) --- updated-dependencies: - dependency-name: golang.org/x/crypto dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 4 ++-- go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 9b20c4a1..4db3ee67 100644 --- a/go.mod +++ b/go.mod @@ -77,10 +77,10 @@ require ( go.opentelemetry.io/otel/trace v1.19.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.4.0 // indirect - golang.org/x/crypto v0.14.0 // indirect + golang.org/x/crypto v0.17.0 // indirect golang.org/x/net v0.17.0 // indirect golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.14.0 // indirect + golang.org/x/sys v0.15.0 // indirect google.golang.org/protobuf v1.31.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index f9b7a32d..441d09e9 100644 --- a/go.sum +++ b/go.sum @@ -273,8 +273,8 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.14.0 h1:wBqGXzWJW6m1XrIKlAH0Hs1JJ7+9KBwnIO8v66Q9cHc= -golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= +golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20230810033253-352e893a4cad h1:g0bG7Z4uG+OgH2QDODnjp6ggkk1bJDsINcuWmJN1iJU= golang.org/x/exp v0.0.0-20230810033253-352e893a4cad/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -319,8 +319,8 @@ golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.14.0 h1:Vz7Qs629MkJkGyHxUlRHizWJRG2j8fbQKjELVSNhy7Q= -golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= +golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= From afe8bd4f455ed9692611bcd2756dc766d9e72fc4 Mon Sep 17 00:00:00 2001 From: Gorobets Denis Date: Tue, 2 Jan 2024 10:25:39 +0100 Subject: [PATCH 375/404] Updated documentation and packages (#186) * update doc and packages * removed unused dependencies * set versions of images * cleanup --------- Co-authored-by: Zhichang Yu --- .github/workflows/tests.yml | 2 +- Dockerfile | 8 +- Dockerfile_goreleaser | 6 +- docker-compose.yml | 4 +- docs/guide/run.md | 28 ++++-- go.mod | 79 ++++++++-------- go.sum | 183 +++++++++++++++++++++--------------- 7 files changed, 175 insertions(+), 135 deletions(-) diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index 285dbcfc..5a26af2d 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -16,7 +16,7 @@ jobs: - name: Set up Go 1.x uses: actions/setup-go@v4 with: - go-version: '1.20' + go-version: '1.21' id: go - name: Check out code into the Go module directory diff --git a/Dockerfile b/Dockerfile index a8d6c8cb..c68f9a73 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,13 +1,13 @@ -FROM golang:latest AS builder +FROM golang:1.21-alpine3.19 AS builder ADD . /app WORKDIR /app RUN go env -w GOPROXY=https://goproxy.cn,direct RUN make build -FROM alpine:latest -RUN apk --no-cache add ca-certificates tzdata -RUN echo "UTC" > /etc/timezone +FROM alpine:3.19 +RUN apk --no-cache add ca-certificates tzdata && \ + echo "UTC" > /etc/timezone COPY --from=builder /app/clickhouse_sinker /usr/local/bin/clickhouse_sinker COPY --from=builder /app/nacos_publish_config /usr/local/bin/nacos_publish_config COPY --from=builder /app/kafka_gen_log /usr/local/bin/kafka_gen_log diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index fbc6662e..d804a1b9 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -1,6 +1,6 @@ -FROM alpine:latest -RUN apk --no-cache add ca-certificates tzdata -RUN echo "UTC" > /etc/timezone +FROM alpine:3.19 +RUN apk --no-cache add ca-certificates tzdata && \ + echo "UTC" > /etc/timezone ADD dist/clickhouse_sinker_linux_amd64_v1/clickhouse_sinker /usr/local/bin/clickhouse_sinker ADD dist/nacos_publish_config_linux_amd64_v1/nacos_publish_config /usr/local/bin/nacos_publish_config diff --git a/docker-compose.yml b/docker-compose.yml index 98471402..6357ab4a 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -14,7 +14,7 @@ services: security_opt: - label:disable kafka: - image: confluentinc/cp-kafka:5.5.0 + image: confluentinc/cp-kafka:7.5.3 container_name: kafka restart: always hostname: kafka @@ -34,7 +34,7 @@ services: security_opt: - label:disable clickhouse: - image: clickhouse/clickhouse-server:22.8 + image: clickhouse/clickhouse-server:23.8 restart: always ports: - "8123:8123" diff --git a/docs/guide/run.md b/docs/guide/run.md index bef02631..a73c5006 100644 --- a/docs/guide/run.md +++ b/docs/guide/run.md @@ -6,7 +6,7 @@ Note: Ensure `clickhouse-server` and `kafka` work before running clickhouse_sink ## Configs -> There are two ways to get config: a local single config, or Nacos. +There are two ways to get config: a local single config, or Nacos. - For local file: @@ -16,7 +16,7 @@ Note: Ensure `clickhouse-server` and `kafka` work before running clickhouse_sink `clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema` -> Read more detail descriptions of config in [here](../configuration/config.html) +Read more detail descriptions of config in [here](../configuration/config.html) ## Example @@ -27,14 +27,19 @@ Let's follow up a piece of the systest script. - let's checkout `clickhouse_sinker` ```bash - $ git clone https://github.com/housepower/clickhouse_sinker.git - $ cd clickhouse_sinker + git clone https://github.com/housepower/clickhouse_sinker.git + cd clickhouse_sinker ``` - let's start standalone clickhouse-server and kafka in container: ```bash - $ docker-compose up -d + docker compose up -d + ``` +* Connect to the ClickHouse server + + ```bash + docker exec -it clickhouse_sinker-clickhouse-1 clickhouse client ``` * Create a simple table in Clickhouse @@ -59,7 +64,11 @@ Let's follow up a piece of the systest script. > I use [kaf](https://github.com/birdayz/kaf) tool to create topics. ```bash - $ kaf topic create topic1 -p 1 -r 1 + kaf topic create topic1 -p 1 -r 1 + ``` + + Output: + ```bash ✅ Created topic! Topic Name: topic1 Partitions: 1 @@ -71,7 +80,7 @@ Let's follow up a piece of the systest script. * Run clickhouse_sinker ```bash - $ ./clickhouse_sinker --local-cfg-file docker/test_auto_schema.hjson + ./bin/clickhouse_sinker --local-cfg-file docker/test_auto_schema.hjson ``` @@ -87,9 +96,10 @@ Let's follow up a piece of the systest script. ```sql SELECT count() FROM test_auto_schema; - + ``` + Output: + ```bash 3 rows in set. Elapsed: 0.016 sec. - ``` ## Run as a daemon diff --git a/go.mod b/go.mod index 4db3ee67..dcd71947 100644 --- a/go.mod +++ b/go.mod @@ -1,52 +1,53 @@ module github.com/housepower/clickhouse_sinker -go 1.20 +go 1.21 + +toolchain go1.21.4 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.15.0 - github.com/RoaringBitmap/roaring v1.3.0 + github.com/ClickHouse/clickhouse-go/v2 v2.17.1 + github.com/RoaringBitmap/roaring v1.7.0 github.com/YenchangChan/franz-go/pkg/sasl/kerberos v0.0.0-20231127011105-840a25342a2e - github.com/avast/retry-go/v4 v4.5.0 - github.com/bytedance/sonic v1.10.0-rc3 + github.com/avast/retry-go/v4 v4.5.1 + github.com/bytedance/sonic v1.10.2 github.com/cespare/xxhash/v2 v2.2.0 - github.com/google/gops v0.3.27 - github.com/google/uuid v1.4.0 - github.com/hjson/hjson-go/v4 v4.3.0 + github.com/google/gops v0.3.28 + github.com/google/uuid v1.5.0 + github.com/hjson/hjson-go/v4 v4.4.0 github.com/jcmturner/gokrb5/v8 v8.4.4 - github.com/jinzhu/copier v0.3.5 + github.com/jinzhu/copier v0.4.0 github.com/matoous/go-nanoid/v2 v2.0.0 github.com/nacos-group/nacos-sdk-go v1.1.4 - github.com/prometheus/client_golang v1.16.0 - github.com/prometheus/common v0.44.0 + github.com/prometheus/client_golang v1.18.0 + github.com/prometheus/common v0.45.0 github.com/shopspring/decimal v1.3.1 github.com/stretchr/testify v1.8.4 - github.com/thanos-io/thanos v0.31.0 - github.com/tidwall/gjson v1.16.0 + github.com/thanos-io/thanos v0.33.0 + github.com/tidwall/gjson v1.17.0 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 - github.com/twmb/franz-go v1.14.3 - github.com/twmb/franz-go/pkg/kadm v1.9.0 + github.com/twmb/franz-go v1.15.4 + github.com/twmb/franz-go/pkg/kadm v1.10.0 github.com/twmb/franz-go/plugin/kzap v1.1.2 github.com/valyala/fastjson v1.6.4 - go.uber.org/zap v1.25.0 - golang.org/x/exp v0.0.0-20230810033253-352e893a4cad - golang.org/x/time v0.3.0 + go.uber.org/zap v1.26.0 + golang.org/x/exp v0.0.0-20231226003508-02704c960a9b + golang.org/x/time v0.5.0 gopkg.in/natefinch/lumberjack.v2 v2.2.1 ) require ( - github.com/ClickHouse/ch-go v0.58.2 // indirect - github.com/aliyun/alibaba-cloud-sdk-go v1.62.510 // indirect + github.com/ClickHouse/ch-go v0.61.0 // indirect + github.com/aliyun/alibaba-cloud-sdk-go v1.62.648 // indirect github.com/andybalholm/brotli v1.0.6 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/bits-and-blooms/bitset v1.8.0 // indirect + github.com/bits-and-blooms/bitset v1.13.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d // indirect - github.com/chenzhuoyu/iasm v0.9.0 // indirect - github.com/davecgh/go-spew v1.1.1 // indirect - github.com/go-errors/errors v1.4.2 // indirect + github.com/chenzhuoyu/iasm v0.9.1 // indirect + github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect + github.com/go-errors/errors v1.5.1 // indirect github.com/go-faster/city v1.0.1 // indirect - github.com/go-faster/errors v0.7.0 // indirect - github.com/golang/protobuf v1.5.3 // indirect + github.com/go-faster/errors v0.7.1 // indirect github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/jcmturner/aescts/v2 v2.0.0 // indirect github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect @@ -54,34 +55,34 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.17.2 // indirect - github.com/klauspost/cpuid/v2 v2.2.5 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/klauspost/compress v1.17.4 // indirect + github.com/klauspost/cpuid/v2 v2.2.6 // indirect + github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b // indirect github.com/paulmach/orb v0.10.0 // indirect - github.com/pierrec/lz4/v4 v4.1.18 // indirect + github.com/pierrec/lz4/v4 v4.1.19 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect - github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/procfs v0.11.1 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/streadway/amqp v1.1.0 // indirect github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.1 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect - github.com/twmb/franz-go/pkg/kmsg v1.6.1 // indirect - go.opentelemetry.io/otel v1.19.0 // indirect - go.opentelemetry.io/otel/trace v1.19.0 // indirect + github.com/twmb/franz-go/pkg/kmsg v1.7.0 // indirect + go.opentelemetry.io/otel v1.21.0 // indirect + go.opentelemetry.io/otel/trace v1.21.0 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/arch v0.4.0 // indirect + golang.org/x/arch v0.6.0 // indirect golang.org/x/crypto v0.17.0 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sync v0.3.0 // indirect + golang.org/x/net v0.19.0 // indirect + golang.org/x/sync v0.5.0 // indirect golang.org/x/sys v0.15.0 // indirect - google.golang.org/protobuf v1.31.0 // indirect + google.golang.org/protobuf v1.32.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index 441d09e9..5d664e49 100644 --- a/go.sum +++ b/go.sum @@ -1,39 +1,42 @@ +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.58.2 h1:jSm2szHbT9MCAB1rJ3WuCJqmGLi5UTjlNu+f530UTS0= -github.com/ClickHouse/ch-go v0.58.2/go.mod h1:Ap/0bEmiLa14gYjCiRkYGbXvbe8vwdrfTYWhsuQ99aw= -github.com/ClickHouse/clickhouse-go/v2 v2.15.0 h1:G0hTKyO8fXXR1bGnZ0DY3vTG01xYfOGW76zgjg5tmC4= -github.com/ClickHouse/clickhouse-go/v2 v2.15.0/go.mod h1:kXt1SRq0PIRa6aKZD7TnFnY9PQKmc2b13sHtOYcK6cQ= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/ClickHouse/ch-go v0.61.0 h1:22JYeFJoFNAU/Vod4etAeUEY28cYt7Ixnwqj1+EUfro= +github.com/ClickHouse/ch-go v0.61.0/go.mod h1:POJBl0MxEMS91Zd0uTgDDt05KfXEjf5KIwW6lNhje9Y= +github.com/ClickHouse/clickhouse-go/v2 v2.17.1 h1:ZCmAYWpu75IyEi7+Yrs/uaAjiCGY5wfW5kXo64exkX4= +github.com/ClickHouse/clickhouse-go/v2 v2.17.1/go.mod h1:rkGTvFDTLqLIm0ma+13xmcCfr/08Gvs7KmFt1tgiWHQ= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/RoaringBitmap/roaring v1.3.0 h1:aQmu9zQxDU0uhwR8SXOH/OrqEf+X8A0LQmwW3JX8Lcg= -github.com/RoaringBitmap/roaring v1.3.0/go.mod h1:plvDsJQpxOC5bw8LRteu/MLWHsHez/3y6cubLI4/1yE= +github.com/HdrHistogram/hdrhistogram-go v1.1.2/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= +github.com/RoaringBitmap/roaring v1.7.0 h1:OZF303tJCER1Tj3x+aArx/S5X7hrT186ri6JjrGvG68= +github.com/RoaringBitmap/roaring v1.7.0/go.mod h1:6AXUsoIEzDTFFQCe1RbGA6uFONMhvejWj5rqITANK90= github.com/YenchangChan/franz-go/pkg/sasl/kerberos v0.0.0-20231127011105-840a25342a2e h1:oK5q2SkjL7lbAXipbG9qVLz6Fm81K3z3YxdehKBp0Yo= github.com/YenchangChan/franz-go/pkg/sasl/kerberos v0.0.0-20231127011105-840a25342a2e/go.mod h1:/lT6Rz1hqrp0IVOvyM6AUOrrvJ3iPX4rofskVigwLfw= +github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.510 h1:mvveZfYcJUOyj0jJqbYxWrM298JXt+ltj7dMbekjraI= -github.com/aliyun/alibaba-cloud-sdk-go v1.62.510/go.mod h1:Api2AkmMgGaSUAhmk76oaFObkoeCPc/bKAqcyplPODs= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.648 h1:I63hduiA3oSEAtZcHD49b0BDe55urq3LeUowhOZSLms= +github.com/aliyun/alibaba-cloud-sdk-go v1.62.648/go.mod h1:CJJYa1ZMxjlN/NbXEwmejEnBkhi0DV+Yb3B2lxf+74o= github.com/andybalholm/brotli v1.0.6 h1:Yf9fFpf49Zrxb9NlQaluyE92/+X7UVHlhMNJN2sxfOI= github.com/andybalholm/brotli v1.0.6/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= -github.com/avast/retry-go/v4 v4.5.0 h1:QoRAZZ90cj5oni2Lsgl2GW8mNTnUCnmpx/iKpwVisHg= -github.com/avast/retry-go/v4 v4.5.0/go.mod h1:7hLEXp0oku2Nir2xBAsg0PTphp9z71bN5Aq1fboC3+I= -github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= +github.com/avast/retry-go/v4 v4.5.1 h1:AxIx0HGi4VZ3I02jr78j5lZ3M6x1E0Ivxa6b0pUUh7o= +github.com/avast/retry-go/v4 v4.5.1/go.mod h1:/sipNsvNB3RRuT5iNcb6h73nw3IBmXJ/H3XrCQYSOpc= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= -github.com/bits-and-blooms/bitset v1.8.0 h1:FD+XqgOZDUxxZ8hzoBFuV9+cGWY9CslN6d5MS5JVb4c= -github.com/bits-and-blooms/bitset v1.8.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= +github.com/bits-and-blooms/bitset v1.12.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= +github.com/bits-and-blooms/bitset v1.13.0 h1:bAQ9OPNFYbGHV6Nez0tmNI0RiEu7/hxlYJRUA0wFAVE= +github.com/bits-and-blooms/bitset v1.13.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= github.com/bytedance/sonic v1.10.0-rc/go.mod h1:ElCzW+ufi8qKqNW0FY314xriJhyJhuoJ3gFZdAHF7NM= -github.com/bytedance/sonic v1.10.0-rc3 h1:uNSnscRapXTwUgTyOF0GVljYD08p9X/Lbr9MweSV3V0= -github.com/bytedance/sonic v1.10.0-rc3/go.mod h1:iZcSUejdk5aukTND/Eu/ivjQuEL0Cu9/rf50Hi0u/g4= +github.com/bytedance/sonic v1.10.2 h1:GQebETVBxYB7JGWJtLBi07OVzWwt+8dWA00gEVW2ZFE= +github.com/bytedance/sonic v1.10.2/go.mod h1:iZcSUejdk5aukTND/Eu/ivjQuEL0Cu9/rf50Hi0u/g4= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -41,19 +44,25 @@ github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d h1:77cEq6EriyTZ0g/qfRdp61a3Uu/AWrgIq2s0ClJV1g0= github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d/go.mod h1:8EPpVsBuRksnlj1mLy4AWzRNQYxauNi62uWcE3to6eA= -github.com/chenzhuoyu/iasm v0.9.0 h1:9fhXjVzq5hUy2gkhhgHl95zG2cEAhw9OSGs8toWWAwo= github.com/chenzhuoyu/iasm v0.9.0/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= +github.com/chenzhuoyu/iasm v0.9.1 h1:tUHQJXo3NhBqw6s33wkGn9SP3bvrWLdlVIJ3hQBL7P0= +github.com/chenzhuoyu/iasm v0.9.1/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/efficientgo/core v1.0.0-rc.2 h1:7j62qHLnrZqO3V3UA0AqOGd5d5aXV3AX6m/NZBHp78I= +github.com/efficientgo/core v1.0.0-rc.2/go.mod h1:FfGdkzWarkuzOlY04VY+bGfb1lWrjaL6x/GLcQ4vJps= +github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= -github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= -github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= +github.com/go-errors/errors v1.5.1 h1:ZwEMSLRCapFLflTpT7NKaAc7ukJ8ZPEjzlxt8rPN8bk= +github.com/go-errors/errors v1.5.1/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= github.com/go-faster/city v1.0.1 h1:4WAxSZ3V2Ws4QRDrscLEDcibJY8uf41H6AhXDrNDcGw= github.com/go-faster/city v1.0.1/go.mod h1:jKcUJId49qdW3L1qKHH/3wPeUstCVpVSXTM6vO3VcTw= -github.com/go-faster/errors v0.7.0 h1:UnD/xusnfUgtEYkgRZohqL2AfmPTwv13NAJwwFFaNYc= -github.com/go-faster/errors v0.7.0/go.mod h1:5ySTjWFiphBs07IKuiL69nxdfd5+fzh1u7FPGZP2quo= +github.com/go-faster/errors v0.7.1 h1:MkJTnDoEdi9pDabt1dpWf7AA8/BaSYZqibYyhZ20AYg= +github.com/go-faster/errors v0.7.1/go.mod h1:5ySTjWFiphBs07IKuiL69nxdfd5+fzh1u7FPGZP2quo= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= @@ -62,33 +71,34 @@ github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/me github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptGaCkuDUx6wNykzzlUixGxvkme+H/lnzb+A= +github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= -github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/gops v0.3.27 h1:BDdWfedShsBbeatZ820oA4DbVOC8yJ4NI8xAlDFWfgI= -github.com/google/gops v0.3.27/go.mod h1:lYqabmfnq4Q6UumWNx96Hjup5BDAVc8zmfIy0SkNCSk= +github.com/google/gops v0.3.28 h1:2Xr57tqKAmQYRAfG12E+yLcoa2Y42UJo2lOrUFL9ark= +github.com/google/gops v0.3.28/go.mod h1:6f6+Nl8LcHrzJwi8+p0ii+vmBFSlB4f8cOOkTJ7sk4c= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4= -github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.5.0 h1:1p67kYwdtXjb0gL0BPiP1Av9wiZPo5A8z2cWkTZ+eyU= +github.com/google/uuid v1.5.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hjson/hjson-go/v4 v4.3.0 h1:dyrzJdqqFGhHt+FSrs5n9s6b0fPM8oSJdWo+oS3YnJw= -github.com/hjson/hjson-go/v4 v4.3.0/go.mod h1:KaYt3bTw3zhBjYqnXkYywcYctk0A2nxeEFTse3rH13E= +github.com/hjson/hjson-go/v4 v4.4.0 h1:D/NPvqOCH6/eisTb5/ztuIS8GUvmpHaLOcNk1Bjr298= +github.com/hjson/hjson-go/v4 v4.4.0/go.mod h1:KaYt3bTw3zhBjYqnXkYywcYctk0A2nxeEFTse3rH13E= github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= @@ -101,8 +111,8 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/jinzhu/copier v0.3.5 h1:GlvfUwHk62RokgqVNvYsku0TATCF7bAHVwEXoBh3iJg= -github.com/jinzhu/copier v0.3.5/go.mod h1:DfbEm0FYsaqBcKcFuvmOZb218JkPGtvSHsKg8S8hyyg= +github.com/jinzhu/copier v0.4.0 h1:w3ciUoD19shMCRargcpm0cm91ytaBhDvuRpz1ODO/U8= +github.com/jinzhu/copier v0.4.0/go.mod h1:DfbEm0FYsaqBcKcFuvmOZb218JkPGtvSHsKg8S8hyyg= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= @@ -115,28 +125,31 @@ github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnr github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.17.2 h1:RlWWUY/Dr4fL8qk9YG7DTZ7PDgME2V4csBXA8L/ixi4= -github.com/klauspost/compress v1.17.2/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.17.4 h1:Ej5ixsIri7BrIjBkRZLTo6ghwrEtHFk7ijlczPW4fZ4= +github.com/klauspost/compress v1.17.4/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= -github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= -github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= +github.com/klauspost/cpuid/v2 v2.2.6 h1:ndNyv040zDGIDh8thGkXYjnFtiN02M1PVVF+JE/48xc= +github.com/klauspost/cpuid/v2 v2.2.6/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/knz/go-libedit v1.10.1/go.mod h1:MZTVkCWyz0oBc7JOWP3wNAzd002ZbM/5hgShxwh4x8M= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/matoous/go-nanoid v1.5.0/go.mod h1:zyD2a71IubI24efhpvkJz+ZwfwagzgSO6UNiFsZKN7U= github.com/matoous/go-nanoid/v2 v2.0.0 h1:d19kur2QuLeHmJBkvYkFdhFBzLoo1XVm2GgTpL+9Tj0= github.com/matoous/go-nanoid/v2 v2.0.0/go.mod h1:FtS4aGPVfEkxKxhdWPAspZpZSh1cOjtM7Ej/So3hR0g= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= +github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= +github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -150,13 +163,14 @@ github.com/mschoch/smat v0.2.0/go.mod h1:kc9mz7DoBKqDyiRL7VZN8KvXQMWeTaVnttLRXOl github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nacos-group/nacos-sdk-go v1.1.4 h1:qyrZ7HTWM4aeymFfqnbgNRERh7TWuER10pCB7ddRcTY= github.com/nacos-group/nacos-sdk-go v1.1.4/go.mod h1:cBv9wy5iObs7khOqov1ERFQrCuTR4ILpgaiaVMxEmGI= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b h1:FfH+VrHHk6Lxt9HdVS0PXzSXFyS2NbZKXv33FYPol0A= github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b/go.mod h1:AC62GU6hc0BrNm+9RK9VSiwa/EUe1bkIeFORAMcHvJU= github.com/paulmach/orb v0.10.0 h1:guVYVqzxHE/CQ1KpfGO077TR0ATHSNjp4s6XGLn3W9s= github.com/paulmach/orb v0.10.0/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= -github.com/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= -github.com/pierrec/lz4/v4 v4.1.18/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pierrec/lz4/v4 v4.1.19 h1:tYLzDnjDXh9qIxSTKHwXwOYmm9d887Y7Y1ZkyXYHAN4= +github.com/pierrec/lz4/v4 v4.1.19/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -167,24 +181,25 @@ github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.18.0 h1:HzFfmkOzH5Q8L8G+kSJKUx5dtG87sewO+FoDDqP5Tbk= +github.com/prometheus/client_golang v1.18.0/go.mod h1:T+GXkCk5wSJyOqMIzVgvvjFDlkOQntgjkJWKrN5txjA= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= +github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= +github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= +github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM= +github.com/prometheus/common v0.45.0/go.mod h1:YJmSTw9BoKxJplESWWxlbyttQR4uaEcGyv9MZjVOJsY= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/prometheus/procfs v0.11.1 h1:xRC8Iq1yyca5ypa9n1EZnWZkt7dwcoRPQwX/5gwaUuI= -github.com/prometheus/procfs v0.11.1/go.mod h1:eesXgaPo1q7lBpVMoMy0ZOFTth9hBn4W/y0/p/ScXhY= +github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= +github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= +github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= @@ -211,10 +226,10 @@ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= -github.com/thanos-io/thanos v0.31.0 h1:jOCmaiIXwpByWXoVtHnktLm3YDB9xDQQzmZvd1XG5oY= -github.com/thanos-io/thanos v0.31.0/go.mod h1:5ux+jb2oKr59+3XsCC0mX+JuAbPGJEMijjhcmnL/PMo= -github.com/tidwall/gjson v1.16.0 h1:SyXa+dsSPpUlcwEDuKuEBJEz5vzTvOea+9rjyYodQFg= -github.com/tidwall/gjson v1.16.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= +github.com/thanos-io/thanos v0.33.0 h1:TYDq9dS3HF7/p3aIGHeImowuQF7CMw0ZODwYoLGhyls= +github.com/thanos-io/thanos v0.33.0/go.mod h1:qeDC74QOf5hWzTlvIrLT8WlNGg67nORFON0T2VF4qgg= +github.com/tidwall/gjson v1.17.0 h1:/Jocvlh98kcTfpN2+JzGQWQcqrPQwDrVEMApx/M5ZwM= +github.com/tidwall/gjson v1.17.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= @@ -225,12 +240,12 @@ github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 h1:gXMUcxcNij github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1/go.mod h1:pP0oMOo7iBmOHY2PCqfaANItDLaYrwHbb97DpOnxhLU= github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= -github.com/twmb/franz-go v1.14.3 h1:cq8rxAnVYU1uF3SRVn8eEaUf+AaXKWlB0Cl3Ca7JSa4= -github.com/twmb/franz-go v1.14.3/go.mod h1:nMAvTC2kHtK+ceaSHeHm4dlxC78389M/1DjpOswEgu4= -github.com/twmb/franz-go/pkg/kadm v1.9.0 h1:UgwBu0YCd6P8HLdg6ZRA4v9W6/zoI1042fOd2CvvLBE= -github.com/twmb/franz-go/pkg/kadm v1.9.0/go.mod h1:eG3f+GHUndq1CUSVvjp+WdNq5zePeJi3tEHzyTkao6g= -github.com/twmb/franz-go/pkg/kmsg v1.6.1 h1:tm6hXPv5antMHLasTfKv9R+X03AjHSkSkXhQo2c5ALM= -github.com/twmb/franz-go/pkg/kmsg v1.6.1/go.mod h1:se9Mjdt0Nwzc9lnjJ0HyDtLyBnaBDAd7pCje47OhSyw= +github.com/twmb/franz-go v1.15.4 h1:qBCkHaiutetnrXjAUWA99D9FEcZVMt2AYwkH3vWEQTw= +github.com/twmb/franz-go v1.15.4/go.mod h1:rC18hqNmfo8TMc1kz7CQmHL74PLNF8KVvhflxiiJZCU= +github.com/twmb/franz-go/pkg/kadm v1.10.0 h1:3oYKNP+e3HGo4GYadrDeRxOaAIsOXmX6LBVMz9PxpCU= +github.com/twmb/franz-go/pkg/kadm v1.10.0/go.mod h1:hUMoV4SRho+2ij/S9cL39JaLsr+XINjn0ZkCdBY2DXc= +github.com/twmb/franz-go/pkg/kmsg v1.7.0 h1:a457IbvezYfA5UkiBvyV3zj0Is3y1i8EJgqjJYoij2E= +github.com/twmb/franz-go/pkg/kmsg v1.7.0/go.mod h1:se9Mjdt0Nwzc9lnjJ0HyDtLyBnaBDAd7pCje47OhSyw= github.com/twmb/franz-go/plugin/kzap v1.1.2 h1:0arX5xJ0soUPX1LlDay6ZZoxuWkWk1lggQ5M/IgRXAE= github.com/twmb/franz-go/plugin/kzap v1.1.2/go.mod h1:53Cl9Uz1pbdOPDvUISIxLrZIWSa2jCuY1bTMauRMBmo= github.com/uber/jaeger-client-go v2.30.0+incompatible h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o= @@ -247,24 +262,26 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.mongodb.org/mongo-driver v1.11.4/go.mod h1:PTSz5yu21bkT/wXpkS7WR5f0ddqw5quethTUn9WM+2g= -go.opentelemetry.io/otel v1.19.0 h1:MuS/TNf4/j4IXsZuJegVzI1cwut7Qc00344rgH7p8bs= -go.opentelemetry.io/otel v1.19.0/go.mod h1:i0QyjOq3UPoTzff0PJB2N66fb4S0+rSbSB15/oyH9fY= -go.opentelemetry.io/otel/trace v1.19.0 h1:DFVQmlVbfVeOuBRrwdtaehRrWiL1JoVs9CPIQ1Dzxpg= -go.opentelemetry.io/otel/trace v1.19.0/go.mod h1:mfaSyvGyEJEI0nyV2I4qhNQnbBOUUmYZpYojqMnX2vo= +go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= +go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= +go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= +go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= +go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= go.uber.org/goleak v1.2.1 h1:NBol2c7O1ZokfZ0LEU9K6Whx/KnwvepVetCUhtKja4A= +go.uber.org/goleak v1.2.1/go.mod h1:qlT2yGI9QafXHhZZLxlSuNsMw3FFLxBr+tBRlmO1xH4= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.25.0 h1:4Hvk6GtkucQ790dqmj7l1eEnRdKm3k3ZUrUMS2d5+5c= -go.uber.org/zap v1.25.0/go.mod h1:JIAUzQIH94IC4fOJQm7gMmBJP5k7wQfdcnYdPoEXJYk= +go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= +go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.4.0 h1:A8WCeEWhLwPBKNbFi5Wv5UTCBx5zzubnXDlMOFAzFMc= -golang.org/x/arch v0.4.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.6.0 h1:S0JTfE48HbRj80+4tbvZDYsJ3tGv6BUU3XxyZ7CirAc= +golang.org/x/arch v0.6.0/go.mod h1:FEVrYAQjsQXMVJ1nsMoVVXPZg6p2JE2mx8psSWTDQys= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -278,7 +295,9 @@ golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq golang.org/x/exp v0.0.0-20230810033253-352e893a4cad h1:g0bG7Z4uG+OgH2QDODnjp6ggkk1bJDsINcuWmJN1iJU= golang.org/x/exp v0.0.0-20230810033253-352e893a4cad/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= @@ -295,8 +314,8 @@ golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= -golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.19.0 h1:zTwKpTd2XuCqf8huc7Fo2iSy+4RHPd10s4KzeTnVr1c= +golang.org/x/net v0.19.0/go.mod h1:CfAk/cbD4CthTvqiEl8NpboMuiuOYsAr/7NOjZJtv1U= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -304,11 +323,12 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= -golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.5.0 h1:60k92dhOjHxJkrqnwsfl8KuaHbn/5dl0lUPUklKo3qE= +golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -329,13 +349,16 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= -golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= +golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -346,15 +369,20 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= +gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= +gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= +gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= -google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= +google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.66.2/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= @@ -367,8 +395,9 @@ gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= From f64cea3712bbc84a2e99f12a0f92c7ca9d9c42c6 Mon Sep 17 00:00:00 2001 From: YenchangChan <43897067+YenchangChan@users.noreply.github.com> Date: Thu, 4 Jan 2024 21:26:48 +0800 Subject: [PATCH 376/404] fix: systest issue (#188) --- Dockerfile | 8 ++++---- go.metrictest.sh | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/Dockerfile b/Dockerfile index c68f9a73..a3d7f20b 100644 --- a/Dockerfile +++ b/Dockerfile @@ -8,10 +8,10 @@ RUN make build FROM alpine:3.19 RUN apk --no-cache add ca-certificates tzdata && \ echo "UTC" > /etc/timezone -COPY --from=builder /app/clickhouse_sinker /usr/local/bin/clickhouse_sinker -COPY --from=builder /app/nacos_publish_config /usr/local/bin/nacos_publish_config -COPY --from=builder /app/kafka_gen_log /usr/local/bin/kafka_gen_log -COPY --from=builder /app/kafka_gen_metric /usr/local/bin/kafka_gen_metric +COPY --from=builder /app/bin/clickhouse_sinker /usr/local/bin/clickhouse_sinker +COPY --from=builder /app/bin/nacos_publish_config /usr/local/bin/nacos_publish_config +COPY --from=builder /app/bin/kafka_gen_log /usr/local/bin/kafka_gen_log +COPY --from=builder /app/bin/kafka_gen_metric /usr/local/bin/kafka_gen_metric # clickhouse_sinker gets config from local file "/etc/clickhouse_sinker.hjson" by default. # Customize behavior with following env variables: diff --git a/go.metrictest.sh b/go.metrictest.sh index e87045c0..558ee160 100755 --- a/go.metrictest.sh +++ b/go.metrictest.sh @@ -39,7 +39,7 @@ sudo docker exec kafka kafka-topics --bootstrap-server localhost:9093 --topic te sudo docker exec kafka sh /tmp/send.sh echo "start clickhouse_sinker to consume" -timeout 30 ./clickhouse_sinker --local-cfg-file docker/test_prom_metric.hjson +timeout 30 ./bin/clickhouse_sinker --local-cfg-file docker/test_prom_metric.hjson schema=`curl "localhost:8123" -d 'DESC test_prom_metric' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` echo "Got test_prom_metric schema => $schema" From d2a2d6cac96f21031fe748ecd1e0b2b8f3839731 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=91=A8=E7=BB=8F=E7=BA=AC?= Date: Tue, 31 Oct 2023 15:40:19 +0800 Subject: [PATCH 377/404] feat: add __ with __series_id and __mgmt_id --- cmd/kafka_gen_prom/main.go | 28 +- docker/test_prom_metric.data | 20000 ++++++++++++++++---------------- docker/test_prom_metric.hjson | 2 +- go.metrictest.sh | 14 +- output/clickhouse.go | 12 +- task/sinker.go | 8 +- task/task.go | 12 +- 7 files changed, 10038 insertions(+), 10038 deletions(-) diff --git a/cmd/kafka_gen_prom/main.go b/cmd/kafka_gen_prom/main.go index b96540aa..cca874e6 100644 --- a/cmd/kafka_gen_prom/main.go +++ b/cmd/kafka_gen_prom/main.go @@ -18,57 +18,57 @@ CREATE TABLE default.dist_prom_extend ON CLUSTER abc AS prom_extend ENGINE = Dis -- Prometheus metric solution 2 - seperated table for datapoints and series labels can join on series id CREATE TABLE default.prom_metric ON CLUSTER abc ( - __series_id Int64, + __series_id__ Int64, timestamp DateTime CODEC(DoubleDelta, LZ4), value Float32 CODEC(ZSTD(15)) ) ENGINE=ReplicatedReplacingMergeTree() PARTITION BY toYYYYMMDD(timestamp) -ORDER BY (__series_id, timestamp); +ORDER BY (__series_id__, timestamp); CREATE TABLE default.dist_prom_metric ON CLUSTER abc AS prom_metric ENGINE = Distributed(abc, default, prom_metric); CREATE TABLE default.prom_metric_series ON CLUSTER abc ( - __series_id Int64, - __mgmt_id Int64, + __series_id__ Int64, + __mgmt_id__ Int64, labels String, __name__ String ) ENGINE=ReplicatedReplacingMergeTree() -ORDER BY (__name__, __series_id); +ORDER BY (__name__, __series_id__); CREATE TABLE default.dist_prom_metric_series ON CLUSTER abc AS prom_metric_series ENGINE = Distributed(abc, default, prom_metric_series); CREATE TABLE default.prom_metric_agg ON CLUSTER abc ( - __series_id Int64, + __series_id__ Int64, timestamp DateTime CODEC(DoubleDelta, LZ4), max_value AggregateFunction(max, Float32), min_value AggregateFunction(min, Float32), avg_value AggregateFunction(avg, Float32) ) ENGINE=ReplicatedReplacingMergeTree() PARTITION BY toYYYYMMDD(timestamp) -ORDER BY (__series_id, timestamp); +ORDER BY (__series_id__, timestamp); CREATE TABLE default.dist_prom_metric_agg ON CLUSTER abc AS prom_metric_agg ENGINE = Distributed(abc, default, prom_metric_agg); -SELECT __series_id, +SELECT __series_id__, toStartOfDay(timestamp) AS timestamp, maxMerge(max_value) AS max_value, minMerge(min_value) AS min_value, avgMerge(avg_value) AS avg_value FROM default.dist_prom_metric_agg -WHERE __series_id IN (-9223014754132113609, -9223015002162651005) -GROUP BY __series_id, timestamp -ORDER BY __series_id, timestamp; +WHERE __series_id__ IN (-9223014754132113609, -9223015002162651005) +GROUP BY __series_id__, timestamp +ORDER BY __series_id__, timestamp; -- Activate aggregation for future datapoints by creating a materialized view CREATE MATERIALIZED VIEW default.prom_metric_mv ON CLUSTER abc TO prom_metric_agg -AS SELECT __series_id, +AS SELECT __series_id__, toStartOfHour(timestamp) AS timestamp, maxState(value) AS max_value, minState(value) AS min_value, avgState(value) AS avg_value FROM prom_metric -GROUP BY __series_id, timestamp; +GROUP BY __series_id__, timestamp; -- Deactivate aggregation by dropping the materialized view. You can revise and create it later as you will. DROP TABLE default.prom_metric_mv ON CLUSTER abc SYNC; @@ -144,7 +144,7 @@ func (dp Datapoint) MarshalJSON() ([]byte, error) { return nil, err } labels2 := labels[1 : len(labels)-1] - msg := fmt.Sprintf(`{"timestamp":"%s", "value":%f, "value1":%g, "value2":%d, "value3":%t, "__name__":"%s", %s, "__series_id":%d, "__mgmt_id":%d}`, + msg := fmt.Sprintf(`{"timestamp":"%s", "value":%f, "value1":%g, "value2":%d, "value3":%t, "__name__":"%s", %s, "__series_id__":%d, "__mgmt_id__":%d}`, dp.Timestamp.Format(time.RFC3339), dp.Value, dp.Value1, dp.Value2, dp.Value3, dp.Name, labels2, seriesID, mgmtID) return []byte(msg), nil } diff --git a/docker/test_prom_metric.data b/docker/test_prom_metric.data index eae3e67e..5792ca40 100644 --- a/docker/test_prom_metric.data +++ b/docker/test_prom_metric.data @@ -1,10000 +1,10000 @@ -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.860439, "value1":0.30809143156043617, "value2":8296859231789889769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697105, "value1":0.07464704753781262, "value2":3626726022956376302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499281, "value1":0.2598747988733079, "value2":1472024199460334836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389559, "value1":0.9035230182410444, "value2":5735152112214129185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259346, "value1":0.6088348511469825, "value2":2118449310673513905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_6":"k","key_7":"e","key_3":"f", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265140, "value1":0.41315907782222017, "value2":6967606456828959585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273243, "value1":0.3670231186776885, "value2":3094287292036347916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697145, "value1":0.4712183861515544, "value2":233543654937300193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.570507, "value1":0.11967792858372435, "value2":1467009477255473639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421572, "value1":0.25352243267763397, "value2":5433840270731074230, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423291, "value1":0.6724498851050211, "value2":652727500980374465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072653, "value1":0.4078722430828867, "value2":6479204450678692857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.576237, "value1":0.03506340983128008, "value2":2409142807432889102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928873, "value1":0.9749671198417696, "value2":806127466997750314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.121356, "value1":0.3710118368988569, "value2":2961209029330787894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.527082, "value1":0.427919527676362, "value2":4168209888386865559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971852, "value1":0.4558638608790214, "value2":7271190379082523197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758405, "value1":0.642984416191772, "value2":635944986021178390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.191870, "value1":0.4028374291432409, "value2":4925274338231144549, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.177741, "value1":0.112412024356175, "value2":1799812085131373089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.562889, "value1":0.9463319128677472, "value2":8604221624040619668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.145145, "value1":0.6344301595600267, "value2":5934019334637844764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512765, "value1":0.12698238608047413, "value2":3179145500529665851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.947023, "value1":0.9362933587076374, "value2":6552003615608175727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479188, "value1":0.8368835012768996, "value2":1048667356432629681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.886895, "value1":0.147063588220359, "value2":2554258082886382046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495557, "value1":0.4897837285320758, "value2":3386687451644953805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267510, "value1":0.7137029646630675, "value2":4723657657521756601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662037, "value1":0.39308757655823606, "value2":5160142240464547787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.001360, "value1":0.2816582812804884, "value2":6076665375380746353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.501969, "value1":0.4236689406205709, "value2":725918256085862567, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.956392, "value1":0.4674704766085337, "value2":4478503621053198593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442861, "value1":0.15012494378573155, "value2":6132679974455498265, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394657, "value1":0.8655032404449484, "value2":2033181385746149676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.574670, "value1":0.8578324023084722, "value2":9030512731073145161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844866, "value1":0.401882448563829, "value2":6461587433040802039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.677301, "value1":0.17574043431028194, "value2":13200916467489149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643407, "value1":0.09478762453322248, "value2":9040728601657069686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.461584, "value1":0.2313038745196538, "value2":8542291140491717734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480373, "value1":0.33926139409918454, "value2":1470854458100424231, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.387467, "value1":0.9027179517679272, "value2":2152676801390448497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.081729, "value1":0.5930629975633843, "value2":3441892662554339501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.523164, "value1":0.27807377593445504, "value2":8726571622906981746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643880, "value1":0.8856942439946653, "value2":6532657219501338712, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.233730, "value1":0.12117132337083063, "value2":5262200042073716921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113242, "value1":0.15754750121773875, "value2":3512597140842356375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.592540, "value1":0.8501957441234611, "value2":1120617963379990831, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.754300, "value1":0.2516365689787487, "value2":1009529273537079157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.738781, "value1":0.9115797570978498, "value2":654040880693718238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_5":"j","key_2":"a","key_3":"b", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730945, "value1":0.567968215036453, "value2":3597506204077270120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.940173, "value1":0.9712119929757523, "value2":5983635236065602039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326670, "value1":0.07295490390000714, "value2":6022184335907772623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.571799, "value1":0.5813154467308365, "value2":9191368645337346592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.547202, "value1":0.12691857025568046, "value2":3820781159866636536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097732, "value1":0.6481650315571341, "value2":6773475876286839908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.184127, "value1":0.5736971101036478, "value2":7284609149743260027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.925677, "value1":0.3771049289873205, "value2":1181794630742224031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.157660, "value1":0.5233716379944181, "value2":4014079834827187874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.413654, "value1":0.21255841055237895, "value2":4422822169208563976, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_9":"a","key_0":"c","key_4":"d", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282429, "value1":0.5134755380331291, "value2":2875914720894216239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510799, "value1":0.4811695837424015, "value2":2131765002626290228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855059, "value1":0.1803859944439004, "value2":4143870194342227353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841548, "value1":0.8921162909498762, "value2":2995639098842929032, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904367, "value1":0.6827455696983789, "value2":4486024070965990492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903308, "value1":0.5275068815139481, "value2":4431489892412543662, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369269, "value1":0.3088848570530923, "value2":2711948390039375929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.246425, "value1":0.22469514297321577, "value2":3311794715258074199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.627147, "value1":0.5085084020761419, "value2":7449844520407226749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757065, "value1":0.7340080407469496, "value2":1399142812283896331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404001, "value1":0.6927872548099725, "value2":4611922647039691390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.548980, "value1":0.9504303368975154, "value2":784663243740669037, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088930, "value1":0.5112126366657939, "value2":1061303860942908201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808273, "value1":0.07227725639854525, "value2":705291637800185980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.195134, "value1":0.9915791690400784, "value2":1380206536675804157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588271, "value1":0.9410252962292832, "value2":2976150472594021014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830440, "value1":0.5990964794781901, "value2":6208539757039787288, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_9":"a","key_1":"g","key_6":"b", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.988764, "value1":0.12421784563285446, "value2":600116094525731026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793643, "value1":0.013993638720735705, "value2":249013181149620349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_5":"j","key_1":"d", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537335, "value1":0.9710360883032237, "value2":3990188937402642001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.783911, "value1":0.5202417888338783, "value2":9207852713084863631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_4":"b","key_6":"f","key_2":"k", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000369, "value1":0.4757654945131251, "value2":617662898409566699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720049, "value1":0.11616063522827627, "value2":1134111314294224704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850537, "value1":0.856287241545387, "value2":3915015074142268096, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117449, "value1":0.23938708551093754, "value2":6182563875693526929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720636, "value1":0.6929840624902686, "value2":7184719611552077354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890463, "value1":0.39990668829147297, "value2":5236439635142539127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598510, "value1":0.794840854439141, "value2":7790247245768394828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_2":"e","key_7":"j","key_1":"c", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.103618, "value1":0.837788909134762, "value2":1746463551678400384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890891, "value1":0.701693361054133, "value2":750546117784279688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243694, "value1":0.04252945224095913, "value2":7022047191936829657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698720, "value1":0.8318935119459053, "value2":2403728383571794535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.362462, "value1":0.5425805070018154, "value2":7722798226519636439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447582, "value1":0.6901117678727204, "value2":1225566659535077888, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884009, "value1":0.9848633762091424, "value2":1508601246687431755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.895176, "value1":0.1370326687436836, "value2":7673037864112111033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.521633, "value1":0.14853540424497083, "value2":6875647668629772843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388849, "value1":0.856681841687182, "value2":289008565390108021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.747443, "value1":0.735786478790032, "value2":7309545501096360932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528790, "value1":0.010059663849976508, "value2":3366632872144662566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.586068, "value1":0.8316103940598538, "value2":6216090017784726937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154422, "value1":0.6449454723419513, "value2":5588370277273470937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.476987, "value1":0.688075774177057, "value2":3232344568636281729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.176248, "value1":0.8630722552735859, "value2":7054623290814757387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.553861, "value1":0.6275284095417549, "value2":8413581575795957099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482699, "value1":0.30152766631318734, "value2":2728742455813076667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228157, "value1":0.09912287337858451, "value2":1368124455579266783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.363726, "value1":0.2556344439654489, "value2":8083466247113334458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598631, "value1":0.00654705967457182, "value2":678136325524679028, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188097, "value1":0.33693220746708563, "value2":5349169986765993063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.591197, "value1":0.3158077067804786, "value2":8408038257495371649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.723537, "value1":0.8935982855510853, "value2":6781320910854884909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.714830, "value1":0.8483752317059017, "value2":4114947449469032042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510153, "value1":0.8569271663104114, "value2":2163617609666100779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.556403, "value1":0.46846499185707535, "value2":4543935722642344774, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115027, "value1":0.13242237154578534, "value2":4001391755557856082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700402, "value1":0.31919253163578853, "value2":9168039730828764814, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473591, "value1":0.914966327309789, "value2":7574079944969934140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657212, "value1":0.7123671399711553, "value2":5845893001941155972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389666, "value1":0.46892477810031413, "value2":3773448585226589360, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365529, "value1":0.10038955567825718, "value2":7677417085782135970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.181780, "value1":0.3807188874415628, "value2":213730861941923164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632990, "value1":0.6352610882615881, "value2":7695215886438259449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.801703, "value1":0.5435516239407236, "value2":1197124892718412788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943146, "value1":0.6459645738185148, "value2":257718117926443886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047331, "value1":0.2599891672558978, "value2":1859555081827581052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.329003, "value1":0.7536967883211322, "value2":8944365307464980699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750531, "value1":0.8871777984871712, "value2":4584326435451561707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820378, "value1":0.6275908409813838, "value2":2808244151477882254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.664852, "value1":0.971622999773127, "value2":3565786429109692783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502157, "value1":0.19081539446623427, "value2":1479393056820182163, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396316, "value1":0.5123967312660013, "value2":8943627767884961054, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934631, "value1":0.19466544110858217, "value2":3956006751549658946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206549, "value1":0.017226775576345523, "value2":7020366329672792405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118256, "value1":0.5188472954068934, "value2":8138153858856845378, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584343, "value1":0.14448963894972008, "value2":4790553180218846981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.877457, "value1":0.6880296047171478, "value2":1215785358113912003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409285, "value1":0.05411761237341633, "value2":4709001633389005280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364854, "value1":0.8998190824029622, "value2":2511578657661549153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.973548, "value1":0.8069354470874932, "value2":2595652771451514899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.073905, "value1":0.2517811560702055, "value2":6140023292659546232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.262163, "value1":0.5630653619265122, "value2":3018188000144695876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641275, "value1":0.6900145697913901, "value2":5216098013140123128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392564, "value1":0.018717055152633055, "value2":8077083634713789634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932220, "value1":0.8388125579510224, "value2":2685186121830778256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206042, "value1":0.49376606624615665, "value2":5850726108082863170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215184, "value1":0.356531340492791, "value2":4642776908341780996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.558758, "value1":0.575409651955595, "value2":2207683625879983306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953416, "value1":0.9073618953615663, "value2":5651749773058007750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.648335, "value1":0.46358450476088653, "value2":1290620516019440255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.487910, "value1":0.012547280333283235, "value2":4919259233823782968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.378926, "value1":0.7085830860476057, "value2":2171649428344764858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_8":"j","key_3":"b","key_4":"k", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802501, "value1":0.31728201505130527, "value2":5419079224591774860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.656795, "value1":0.24875018847328167, "value2":485829881375178402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938831, "value1":0.08769582144908596, "value2":6951540802412020838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231956, "value1":0.7863053106916462, "value2":1030265256144215264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.631246, "value1":0.46917511334330375, "value2":3774063736313175623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305077, "value1":0.834556547518894, "value2":636203159633468096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808070, "value1":0.5245784354956413, "value2":5259003801435693215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641827, "value1":0.7755759714857714, "value2":3467599768461222251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.306769, "value1":0.47958408429359173, "value2":5365936635633468542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294040, "value1":0.5158994633208911, "value2":7429034992856357163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.961943, "value1":0.7435043392340425, "value2":8500250963468273773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935561, "value1":0.5801624578827164, "value2":8754050195411098007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444846, "value1":0.030970547839451835, "value2":8387858736748608077, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358418, "value1":0.5401368375372775, "value2":7107146838213157067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.546704, "value1":0.46330699877542014, "value2":8769213962266908815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.270889, "value1":0.46793759900681503, "value2":6364231395353410490, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549392, "value1":0.3430929972372329, "value2":4966477192488000261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428697, "value1":0.7243138275027047, "value2":2836857643122176656, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993399, "value1":0.40084653235121936, "value2":6794009333725376121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.506725, "value1":0.07262914457616802, "value2":3459356114730929347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202924, "value1":0.07381206265687722, "value2":2649194281139606297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.411238, "value1":0.9684314554036981, "value2":3705584143921682379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_5":"h","key_6":"k","key_4":"c", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358603, "value1":0.534711919539449, "value2":4138168909279457429, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989681, "value1":0.19207319251633845, "value2":1593591737448226138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.052714, "value1":0.27756026777792103, "value2":6377557735209128817, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614036, "value1":0.018601161874125947, "value2":151552447676967905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305181, "value1":0.031288435638661224, "value2":3937949086874523036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715232, "value1":0.6397334976781403, "value2":7614941574913315366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705549, "value1":0.6996034412212425, "value2":6200772172174019130, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.302276, "value1":0.5239925847096263, "value2":1057710529698426724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200111, "value1":0.2567985665426193, "value2":4529602646404884783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_2":"g","key_8":"f","key_1":"a", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.727436, "value1":0.43685005741153693, "value2":3304580792456252079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.351586, "value1":0.3164739225094333, "value2":3897998541684002875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.665954, "value1":0.5906227199646251, "value2":6746215572356057637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361921, "value1":0.3364933451115498, "value2":761972198573021391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.889779, "value1":0.36794570756599093, "value2":2090465055639313788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.699184, "value1":0.5483254879513706, "value2":7720905658039555449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388162, "value1":0.5287855210410441, "value2":7195732574645022274, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154496, "value1":0.6641476496489667, "value2":9137939751947784172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327473, "value1":0.5518373975194654, "value2":3971910670147599886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063454, "value1":0.43820340286329656, "value2":5405166066848368254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.781841, "value1":0.031101766659472223, "value2":4189381224837071716, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171755, "value1":0.46498479978119844, "value2":4931395791650935239, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.559617, "value1":0.8752912652134259, "value2":8586908304160851241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.323852, "value1":0.35821728605182146, "value2":5163275989952284916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030550, "value1":0.8225486647874358, "value2":2481647473431927932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.712533, "value1":0.368964069168292, "value2":8062115249497297834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298843, "value1":0.30262512534550123, "value2":5999722310995378983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778244, "value1":0.3716152000047593, "value2":6978129792906951032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.144352, "value1":0.42812039567105226, "value2":6591666290600795514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616112, "value1":0.5501694032291233, "value2":5953070646927834342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843613, "value1":0.7957093053513518, "value2":779447045934204434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.021516, "value1":0.3651263217338076, "value2":1101615937937937836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.707183, "value1":0.9474415697823648, "value2":3163832075607398652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807066, "value1":0.328067579350897, "value2":1404186285892115543, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327059, "value1":0.4531450671817094, "value2":3186214527240993583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.629874, "value1":0.555595498018525, "value2":7512625623217629070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203064, "value1":0.24518269440439186, "value2":7974348531674940058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865398, "value1":0.7125714321236861, "value2":1185618794516530658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.098668, "value1":0.6050958429061423, "value2":7737776979675298527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659514, "value1":0.8901996322311656, "value2":7752156196236457080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_6":"h","key_9":"d","key_5":"g", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650309, "value1":0.7869508663362883, "value2":6010716207202506080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.779127, "value1":0.6366846028665012, "value2":7797452629031711950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.281652, "value1":0.9118975722819298, "value2":6103834325448620626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868834, "value1":0.18117722257021981, "value2":6098726761586315616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865255, "value1":0.5168875940701739, "value2":7616170573520868343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000667, "value1":0.8879098452429443, "value2":2716291319460579102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938952, "value1":0.27931352835868933, "value2":2351410106131072829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524371, "value1":0.5375950258397566, "value2":7234763394924546015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718980, "value1":0.7918266548241415, "value2":1156714261082582562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532951, "value1":0.4063578415664877, "value2":2401980297367889095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780599, "value1":0.15947463153475608, "value2":6812388220933080605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078621, "value1":0.6070073793267291, "value2":1012316510571499763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404621, "value1":0.9613908610430107, "value2":6888867545087821595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.130043, "value1":0.26321351304547996, "value2":4446851417687808950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602508, "value1":0.38864606344069585, "value2":2797998328851090673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244218, "value1":0.5263437247014857, "value2":884821963010111580, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415267, "value1":0.3515494200462383, "value2":3356463166369801537, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303765, "value1":0.5270577861362743, "value2":2138283533246685033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.254002, "value1":0.303619701206094, "value2":7629281847530948891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588498, "value1":0.5481446854335168, "value2":7641330013258723790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.620575, "value1":0.6420150180085478, "value2":7965173856733830205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243174, "value1":0.400523011031962, "value2":4931302022232098402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_9":"c","key_4":"d","key_8":"d", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.104291, "value1":0.4501470045022247, "value2":6419830717700946282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605242, "value1":0.5489672412234975, "value2":25272414308366080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705962, "value1":0.2696978321817019, "value2":7266661553560831399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.204594, "value1":0.050667752810758936, "value2":4939383554296719335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.694791, "value1":0.8451677349011486, "value2":1862202581082766068, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178458, "value1":0.47634419936555994, "value2":3993368962808231698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879184, "value1":0.05969390345398635, "value2":5274917008847651276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854470, "value1":0.8497662902937875, "value2":3035062077693412013, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.397422, "value1":0.08213093074253511, "value2":5967759139282299209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584032, "value1":0.08273000380656788, "value2":2735658568798834919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822339, "value1":0.581737952515166, "value2":2604615641240906232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.308039, "value1":0.34242828701171635, "value2":3092061640651599674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.913723, "value1":0.005597195694116873, "value2":2620704762797681315, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_3":"b","key_8":"c","key_1":"f", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618957, "value1":0.6413425640197833, "value2":5287503767189177120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454987, "value1":0.32512559877918956, "value2":2506424876460932044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447066, "value1":0.20850990987663237, "value2":7977478851800531911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.683345, "value1":0.838649315370365, "value2":4999282712485449007, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935535, "value1":0.4089194818619663, "value2":722917113975878160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439326, "value1":0.8812716905619982, "value2":6658838831662104981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069099, "value1":0.15024751394280958, "value2":1328305014122533446, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_7":"f","key_3":"a","key_5":"e", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473312, "value1":0.9280459649583414, "value2":3438360323166932806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215711, "value1":0.8566516621694267, "value2":1663749581830575806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312134, "value1":0.566681231011358, "value2":6911167899525793851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862090, "value1":0.5364584771183928, "value2":7729755763890728353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456397, "value1":0.7096399907515605, "value2":5531147306124199678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.587843, "value1":0.8985325214047336, "value2":4030305760543278105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710467, "value1":0.5099257391896752, "value2":1179017586099298270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188204, "value1":0.8723809422062396, "value2":5093258983701866401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.681846, "value1":0.23032248565211208, "value2":3357194399728955391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_2":"j","key_4":"c","key_1":"i", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614220, "value1":0.9249072784237294, "value2":4254302260697988757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.028000, "value1":0.2236758406407766, "value2":5499863050282572666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923027, "value1":0.4209047854675161, "value2":4430077989918325993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203873, "value1":0.8404470170534011, "value2":532376833912389989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_2":"f","key_6":"b","key_1":"d", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.541847, "value1":0.24084743744285772, "value2":2314457094095648600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932124, "value1":0.7743913731102018, "value2":7842905515694643998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.986636, "value1":0.18488491173653657, "value2":1233277417792576747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_6":"f","key_0":"j","key_2":"i", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.491189, "value1":0.5404267488261955, "value2":5193038540757554939, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.449064, "value1":0.9915502184839048, "value2":3914891481887672833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203773, "value1":0.11544040430435766, "value2":8720914144754355340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.637865, "value1":0.5397959742632489, "value2":5016935922539098494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.164476, "value1":0.40957356383804516, "value2":8101990321123394900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212208, "value1":0.31170589199654203, "value2":8172096864950686883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427578, "value1":0.9500075990560398, "value2":4448206698192420471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375888, "value1":0.08043137384857349, "value2":5428459057669693793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.253419, "value1":0.30426981771564987, "value2":3701960397375147641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.064111, "value1":0.6613366419136151, "value2":9214004723321204161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618634, "value1":0.28919063506417325, "value2":2549339895850480456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.581311, "value1":0.5011828194867549, "value2":2394702011956086199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.619872, "value1":0.3579296101205142, "value2":3114339742928723546, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_6":"f","key_0":"c", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928952, "value1":0.7843473952520837, "value2":4007100053019753408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.678699460362967, "value2":1449401258062210262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.791597, "value1":0.5225036343492986, "value2":7759663471842896645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794616, "value1":0.3395042657376233, "value2":1662502503940889595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.466754, "value1":0.9041538990976105, "value2":535273624649157699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693734, "value1":0.009440601901378784, "value2":2148867264362048197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841713, "value1":0.059732250656449104, "value2":6677903169635737819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.746032, "value1":0.16006431951166478, "value2":5022444125278904149, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.158292, "value1":0.8619758377742992, "value2":3793514584183151927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364688, "value1":0.7050682785197309, "value2":8826236482901828470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_9":"f","key_2":"e","key_7":"e", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273970, "value1":0.40265473643498934, "value2":8303926135399734280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030530, "value1":0.7286415975665108, "value2":160039363895729267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_5":"j","key_3":"g", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266449, "value1":0.033547786099323434, "value2":7620487585623102750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025977, "value1":0.17793807237193235, "value2":8744525786084307091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.806031, "value1":0.5052650450730324, "value2":4060064708931174932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035732, "value1":0.9743453198182447, "value2":4099915557434576704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.646092, "value1":0.21956311768718226, "value2":8589888788620337480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078229, "value1":0.4079449893422866, "value2":6173402600986499343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666881, "value1":0.9863097557928853, "value2":8279527785132450943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.741108, "value1":0.31651275509392585, "value2":1547895388613496809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866620, "value1":0.6883437972608208, "value2":8802986125706842330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718942, "value1":0.8847629636653906, "value2":800711854768452001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375188, "value1":0.9532398566953418, "value2":2394341312296044798, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624088, "value1":0.6934451210618476, "value2":8474880456434193538, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680602, "value1":0.2209467711169388, "value2":8621094656072865370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.522533, "value1":0.5439997834907557, "value2":8198708215843245045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439975, "value1":0.8101860664171935, "value2":8929518033480490042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602144, "value1":0.17084337908018365, "value2":6000683736434217499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594415, "value1":0.3959186882607224, "value2":1454959879954934650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917433, "value1":0.849867816493911, "value2":8615713829914363729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439894, "value1":0.7578484758847723, "value2":3887712481108512291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.965697, "value1":0.8251003697840577, "value2":836422853680285494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259251, "value1":0.4596388195214864, "value2":1597474370288095053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512513, "value1":0.783528381108141, "value2":8045517537006666296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_4":"i","key_6":"g","key_3":"c", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.452258, "value1":0.4477592871421156, "value2":1459300159451076858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735408, "value1":0.5067192602284456, "value2":7587131980242978873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.108601, "value1":0.3076343047473968, "value2":5066182910326911353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419278, "value1":0.12962316194187043, "value2":8864033357034639760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336677, "value1":0.24448588819210298, "value2":4567827434979480913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131541, "value1":0.02538083388681256, "value2":6449819714583284688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235312, "value1":0.7537710944451972, "value2":3840033782203811299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713440, "value1":0.6660557019192793, "value2":6317879942694845725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250100, "value1":0.2226451425042414, "value2":6756354166360377412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.545039, "value1":0.7203310314493744, "value2":6637075765032095898, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846538, "value1":0.5133638544522705, "value2":6146673076285104795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.099134, "value1":0.26713136173061386, "value2":5047631075889612988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.999599, "value1":0.19120338269199497, "value2":3009559225203387009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456369, "value1":0.6496125050532283, "value2":8358687825156497010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.882684, "value1":0.4767076839685737, "value2":2058131066392704343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326891, "value1":0.0072166654150041835, "value2":3877874561619455924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.753206, "value1":0.9891987223468547, "value2":2500770621881849387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.505150, "value1":0.33485751726586077, "value2":3075226382158740800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_6":"k","key_3":"h", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.337847, "value1":0.7587725777413157, "value2":4030061892629350122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.061555, "value1":0.5715183894256888, "value2":1380477982282562120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102225, "value1":0.18128048387136908, "value2":1107305884228635610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.663571, "value1":0.2082433172088735, "value2":2331488443653185612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400352, "value1":0.36344640764280667, "value2":4156784209596494480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567282, "value1":0.6853752986060114, "value2":1738184775840508143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580625, "value1":0.21556187277712435, "value2":1716362597563830669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.068750, "value1":0.5617303119904301, "value2":215978150703712746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948465, "value1":0.4015798445122638, "value2":4918723285934260509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818268, "value1":0.4923649301244579, "value2":2640505566674328479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344093, "value1":0.24900402040054898, "value2":3729697701643357874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096571, "value1":0.84402994415599, "value2":5393515911153006286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880117, "value1":0.7901487854013907, "value2":2074107282217049907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923332, "value1":0.6838866899470619, "value2":1947909482655854334, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.432183, "value1":0.35953677139878504, "value2":3191972662401496477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251030, "value1":0.9314169082560206, "value2":2792551904040732916, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.315849, "value1":0.9521659578315848, "value2":278261838735054052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.790453, "value1":0.9152891069156645, "value2":1668356628128321724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.260477, "value1":0.2126605025888137, "value2":6877799862560969541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.213669, "value1":0.023418304769384207, "value2":4056268380917249508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.647354, "value1":0.1608657659472805, "value2":7997008030753632127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364473, "value1":0.2924502770902306, "value2":2291771922128424424, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.399561, "value1":0.6211538305973102, "value2":6848308885300872697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135252, "value1":0.6565821688014496, "value2":4331814359581194434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365542, "value1":0.9118204132722055, "value2":2417655238291803790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758482, "value1":0.17112508145844177, "value2":1991118257939966294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.032543, "value1":0.08125554957598005, "value2":6803892048654362563, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.520072, "value1":0.4275632442269591, "value2":2250122493336577938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.900940, "value1":0.14035051591842881, "value2":3790466116523762354, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794030, "value1":0.6000173402567552, "value2":1696544400981072969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.760746, "value1":0.10139199442682838, "value2":2726344565529290448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687331, "value1":0.770768299751441, "value2":8765361349894188719, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035711, "value1":0.48562696854381604, "value2":95585559412343711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739784, "value1":0.5825982451536238, "value2":2923240563522303723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454300, "value1":0.4006888000990195, "value2":2468975659398879205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884626, "value1":0.8026957862944495, "value2":3605855909002945057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395577, "value1":0.28352828399222396, "value2":7602348181385211161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250646, "value1":0.2316228525268274, "value2":29961024893090406, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421024, "value1":0.7990468794285002, "value2":2264750479744979673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380413, "value1":0.677589393211687, "value2":1235396941989449609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_4":"f","key_1":"j", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680912, "value1":0.25625503996995874, "value2":2396418552817135385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624528, "value1":0.5836651399092099, "value2":8089438271359233374, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.578517, "value1":0.7858613429590018, "value2":3691936139324370664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_7":"h","key_0":"b","key_2":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.824035, "value1":0.3471915666356168, "value2":7523600740099466883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069777, "value1":0.6471590850539041, "value2":545376892289254611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390523, "value1":0.8681181997676135, "value2":5603080382596898622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.590994, "value1":0.6390330221088167, "value2":6898933669096943734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222958, "value1":0.7361676398339201, "value2":7798894502817396812, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.313614, "value1":0.9956617360294369, "value2":8114343594051219927, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.471497, "value1":0.2028990027253043, "value2":8672046519486478878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299387, "value1":0.90821231206369, "value2":7139029616253188226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_7":"c","key_9":"g","key_2":"j", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822834, "value1":0.7462012785016429, "value2":2497140784779218477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.958689, "value1":0.6493610280581492, "value2":7375518635495424751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755680, "value1":0.3173769236575059, "value2":1038885732982663879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.906259, "value1":0.7803156579997583, "value2":1671762930600330124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320586, "value1":0.047928946477745184, "value2":2364817790122909266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.237873, "value1":0.5919698389786022, "value2":2456361237893496498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_9":"c","key_2":"j","key_4":"d", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.183324, "value1":0.25157377599570596, "value2":3302857039571325572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427134, "value1":0.9570675751476113, "value2":241686111761765133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.353599, "value1":0.19102611162010216, "value2":7635824178108933655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933576, "value1":0.5497618436271264, "value2":178483151772454986, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.888923, "value1":0.6996148110998511, "value2":7261028738310365771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414681, "value1":0.45596913295291375, "value2":8486467978432271730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354962, "value1":0.6882059355313528, "value2":4219371503140919650, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149786, "value1":0.048507930319514725, "value2":3361043402215780354, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320396, "value1":0.48978604231367256, "value2":8391350623429868941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.189903, "value1":0.49045511147932325, "value2":4568889448781542889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.531173, "value1":0.015565420626107091, "value2":1532700275127700107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.911622, "value1":0.9315116645649003, "value2":845964128004551720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447558, "value1":0.6852687200103666, "value2":1997580526601582597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389703, "value1":0.7679907754096043, "value2":8650394236218808766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221750, "value1":0.8409665471906171, "value2":8068148192803213421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.679200, "value1":0.5889377996015627, "value2":6603283664639278646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.658212, "value1":0.20335938525562192, "value2":6814120282733950138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_8":"c","key_4":"d","key_7":"g", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917850, "value1":0.07209516605018491, "value2":3465054040297242627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.463048, "value1":0.8121431937413348, "value2":7711657955494481167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280683, "value1":0.9792224347770722, "value2":6637142200642308638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938341, "value1":0.6616785812007566, "value2":4430772053877820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778057, "value1":0.21301644528770988, "value2":5550954213897210258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.435560, "value1":0.25143911919093503, "value2":8602791844656832235, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980296, "value1":0.4483217784998845, "value2":8061950324304064559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.845189, "value1":0.23223688199088727, "value2":2019205569808444797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030298, "value1":0.009150371898544056, "value2":2179401507551835981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_9":"e","key_4":"j","key_7":"k", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439657, "value1":0.5569344336018628, "value2":3896413214679997959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037278, "value1":0.0752646633945245, "value2":2986036073412577705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005641, "value1":0.470809080359862, "value2":7672259019023724818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169543, "value1":0.6908284999086921, "value2":6236799599986001574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.168435, "value1":0.1656190822947285, "value2":2938449310867540367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901700, "value1":0.6243405512256929, "value2":2234990933462726180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282060, "value1":0.14169460807404005, "value2":820280722029319872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868372, "value1":0.8809198418653182, "value2":7786878634034667094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700989, "value1":0.028109982788470232, "value2":1201575202891520734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.304734, "value1":0.8385104418360362, "value2":6703304420616815596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901530, "value1":0.8011320750502264, "value2":7539936763030777880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862233, "value1":0.47963788532278256, "value2":3960941336665870754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.511307, "value1":0.1832580868434161, "value2":7188980001326450399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700199, "value1":0.6958679254162045, "value2":2878046054907323632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299563, "value1":0.31054488059495927, "value2":8075467755079023785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383092, "value1":0.290554447303189, "value2":6694033035935161393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.696053, "value1":0.16066281765668294, "value2":3961201674767505870, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392128, "value1":0.5736001777474582, "value2":168331305812241403, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_9":"h","key_3":"j","key_5":"a", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752731, "value1":0.4825725339245043, "value2":7800665573914476062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293547, "value1":0.12126767569308682, "value2":1612512565132344690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_8":"a","key_0":"b","key_3":"d", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874204, "value1":0.7014202076829791, "value2":476647849984184674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.132714, "value1":0.5122323816164429, "value2":646780496771729886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358264, "value1":0.8238092496263985, "value2":9061776775816645023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358876, "value1":0.15679289393208987, "value2":8163846432367416312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921374, "value1":0.8339787140810333, "value2":3649804286907707134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395687, "value1":0.875997280853855, "value2":9212367459114589124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.459642, "value1":0.7528040980492385, "value2":4150547104368392230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782674, "value1":0.7121212215364499, "value2":6531310437181949577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.229939, "value1":0.32814366772056375, "value2":6374161097327904245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594347, "value1":0.6163078141302316, "value2":1538685760517154018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.009874, "value1":0.36102842691855586, "value2":1131193318052688252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.606094, "value1":0.7074086026345966, "value2":7018588286889757726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.777708, "value1":0.9795566345359867, "value2":7358687237436509918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_4":"a","key_2":"i","key_3":"k", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.963990, "value1":0.9872552891680104, "value2":6171290753066239119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772158, "value1":0.4830977012506835, "value2":6211566808050254036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.952768, "value1":0.43798623227092814, "value2":3513514828808633618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566018, "value1":0.7713759201698291, "value2":5320413786417127410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.563469, "value1":0.6599121888373632, "value2":1404388254364767252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786109, "value1":0.9075953036601083, "value2":8494075902382221765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812207, "value1":0.5042296550006388, "value2":2362200937900291190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093099, "value1":0.3386496883604208, "value2":7071312823587957868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.381119, "value1":0.031714519630475106, "value2":2849328345417435639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.815827, "value1":0.14737457839891155, "value2":199255383994378266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050041, "value1":0.5503190199883978, "value2":7356958752349024534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.472083, "value1":0.12465887530301682, "value2":6392360821928490144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713056, "value1":0.12067504962982098, "value2":4673070422231359809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.034034, "value1":0.25231429703394476, "value2":5489447259456982836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502818, "value1":0.4156967719918407, "value2":4612443309724781532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427071, "value1":0.24921718574451088, "value2":7646229328406586652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643652, "value1":0.07902787804812644, "value2":34188484750195348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.488097, "value1":0.7204444807131233, "value2":4412925367441450672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.180195, "value1":0.3056776312063749, "value2":7687653658691030932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.873382, "value1":0.6827451433784463, "value2":1751876103177277887, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529999, "value1":0.39651784939201806, "value2":1747188854631655971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971674, "value1":0.8695844176281972, "value2":2878591399245613156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702173, "value1":0.9786793419693177, "value2":4388723377141224465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844886, "value1":0.690355517305138, "value2":6283981589400857877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298362, "value1":0.6666814629745373, "value2":6187669811817639042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266804, "value1":0.7091063369563013, "value2":4664946045775147969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400948, "value1":0.8630878707275965, "value2":7287257182404940937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_3":"f","key_6":"k","key_0":"b", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135133, "value1":0.6122563921627874, "value2":2139209888725866722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.370672, "value1":0.01650778323939298, "value2":3489547069770540399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.279256, "value1":0.7619200195662909, "value2":5915401631721082002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170496, "value1":0.01005373661086586, "value2":1482576688135431000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499284, "value1":0.36435815211219275, "value2":1326048436273736829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185099, "value1":0.9391170841095738, "value2":8924546898546218998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772826, "value1":0.7574432732083424, "value2":4488845007322620734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_6":"h","key_5":"f", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169744, "value1":0.5739689347042248, "value2":741420931729893242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.998044, "value1":0.8965817632976586, "value2":6239062200473429860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594662, "value1":0.6301979977894112, "value2":2054875786295626784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755433, "value1":0.06261106040596412, "value2":5455645386351916850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.682584, "value1":0.03167274357401835, "value2":310883965859994658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072455, "value1":0.9148162667555992, "value2":382836579698407350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.725171, "value1":0.3647728943926306, "value2":2421892347307790025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798022, "value1":0.22510760744817873, "value2":2442091377551743699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688933, "value1":0.07828818881955478, "value2":6735636563745227652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.649962, "value1":0.9679958182791478, "value2":2607996809419409341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_8":"d","key_9":"h","key_1":"d", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.382199, "value1":0.14064207963814382, "value2":1623159006724541520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293647, "value1":0.919909760697381, "value2":58019425907871235, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715683, "value1":0.40584027014114893, "value2":7884356555342436928, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.496280, "value1":0.6458580424511011, "value2":1385234809290785609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037411, "value1":0.5356364996267606, "value2":4485181030850732806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.092167, "value1":0.08471879259284998, "value2":1047090205307301338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203791, "value1":0.9626197309889007, "value2":458130535579789756, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_5":"b","key_0":"d","key_2":"i", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395018, "value1":0.7684680824595476, "value2":787781374740198173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408536, "value1":0.147741569501137, "value2":3967017244497727033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693578, "value1":0.011896444619318294, "value2":4942230313540034093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780418, "value1":0.8750655230631833, "value2":8170450018971591774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812004, "value1":0.011539724469496575, "value2":1314191368195668298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.090381, "value1":0.870578773268634, "value2":5701169500016143920, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512721, "value1":0.9238819426049193, "value2":5759090244464429103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938235, "value1":0.5991043580028484, "value2":7404773139120927140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.751490, "value1":0.1795090853269717, "value2":6360470713237738478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267071, "value1":0.625707646354563, "value2":6040902311152523365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.002110, "value1":0.08011219998212876, "value2":5850595301611238867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257610, "value1":0.8187562858163207, "value2":8061987864802598079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.446622, "value1":0.7315172185892279, "value2":2428635533374338014, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730403, "value1":0.7209673234694193, "value2":8416924883569369777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.814732, "value1":0.8815237283497432, "value2":1852270652578106849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.318594, "value1":0.610086630044871, "value2":4524314099691048466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.852591, "value1":0.9383715352044596, "value2":453270928675336465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989585, "value1":0.8337913033606761, "value2":6440744488245743508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666424, "value1":0.9182270518817748, "value2":6618322333494361668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.010788, "value1":0.6687390259133706, "value2":1979105606239962267, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_9":"a","key_0":"c","key_6":"j", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752852, "value1":0.4586423067764851, "value2":3289626005749005579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005307, "value1":0.4780784157090063, "value2":2142574953538675780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752795, "value1":0.7908677214518265, "value2":5326267086837385392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202377, "value1":0.001998326079086547, "value2":2359915256333417411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216786, "value1":0.03377151611828389, "value2":5536807322346137933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243069, "value1":0.48298272047333335, "value2":6984955197084383580, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199895, "value1":0.6668507451944505, "value2":1160751712622009388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605559, "value1":0.014501073348709297, "value2":2976544013604671560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046312, "value1":0.45283346882557124, "value2":1106313893608419094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841418, "value1":0.9439193659449161, "value2":1585819181629654484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.816329, "value1":0.8314432076191595, "value2":671388396305183300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.672039, "value1":0.19428152515853786, "value2":4678503251263728412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.652936, "value1":0.05999646842980615, "value2":6549432288618958292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212442, "value1":0.6061066332871502, "value2":6044859751282824016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200351, "value1":0.9858548441265623, "value2":6095603519503403992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047144, "value1":0.8823748802173365, "value2":2346527148275584633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272669, "value1":0.49903571079222264, "value2":7606961266627055929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244016, "value1":0.3505008339279857, "value2":9165454668662797150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_6":"e","key_0":"a","key_4":"f", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595436, "value1":0.1769134432972796, "value2":4183298646463696655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_7":"e","key_6":"f", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.832887, "value1":0.05781334979440461, "value2":2976605700126921242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.417486, "value1":0.03855593830020744, "value2":6499287743834404350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850592, "value1":0.8819483685824858, "value2":3365942495366951256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904339, "value1":0.4096492831256871, "value2":7870603688751891889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849740, "value1":0.5985549217817763, "value2":5972692186135936344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080962, "value1":0.6508975263449264, "value2":620195756611450814, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.172604, "value1":0.36758717062599394, "value2":3093546059237948200, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.611964, "value1":0.09212647541898711, "value2":600064894791793802, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.136731, "value1":0.3518781010346145, "value2":4172668838781409561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221150, "value1":0.7696791785774878, "value2":4882972115027784491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118097, "value1":0.3822037091545315, "value2":5547505099823892976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.209184, "value1":0.3119528694472212, "value2":8126435007719933970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.708227, "value1":0.8414289260599137, "value2":1887259462865725736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127263, "value1":0.417919506626244, "value2":3560935891426387222, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.500871, "value1":0.16110435350355187, "value2":2428171488019731436, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.848583, "value1":0.4961159478803282, "value2":5456304085389531044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.468311, "value1":0.9289954682212651, "value2":4285742227173179225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.160825, "value1":0.59740490878922, "value2":9140238758107732393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_4":"d","key_1":"f","key_2":"j", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423818, "value1":0.2325925821959117, "value2":7321787707684172397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_7":"k","key_5":"i","key_6":"g", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.689012, "value1":0.5753370946686959, "value2":5125074341961139213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941993, "value1":0.009508850201920292, "value2":2477284877979206282, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344874, "value1":0.012483207365877868, "value2":4898429158465049577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625728, "value1":0.9281594076368007, "value2":771428325581269474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701217, "value1":0.8504319614540372, "value2":3030161014659152421, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.288648, "value1":0.7225335235603462, "value2":4184194222447924377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735765, "value1":0.7338027307218647, "value2":8053319255149398119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853312, "value1":0.31226366748735784, "value2":48160047989763460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289371, "value1":0.5473984505709404, "value2":2530747852470429293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825683, "value1":0.2907108545003152, "value2":3061253498575467483, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532231, "value1":0.5859058541023265, "value2":4975638816250812874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512440, "value1":0.511793469522306, "value2":5419688896766823090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709116, "value1":0.0431590704274129, "value2":565903028384253279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709410, "value1":0.45910825737421945, "value2":425558835807211279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.970542, "value1":0.1219905172153899, "value2":24331626097709718, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.043601, "value1":0.49284843085319935, "value2":7672646678928068810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739151, "value1":0.4246916831482226, "value2":4312260845153215448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489280, "value1":0.8582555742505251, "value2":3110601136234274420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935791, "value1":0.4664631517475295, "value2":4539773222290649379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.654384, "value1":0.10389342147634097, "value2":5731036292461383708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193783, "value1":0.5459140576966053, "value2":6006070322533917783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409382, "value1":0.472091611755319, "value2":4277330658526282078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423822, "value1":0.4751601960954781, "value2":2652715507916534737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.660521, "value1":0.38225985421870495, "value2":9011573276516869930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975891, "value1":0.5238616298909866, "value2":472916590695431743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843227, "value1":0.6679925899402425, "value2":906046342124593465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829312, "value1":0.16298135421846682, "value2":1838361763539535864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480674, "value1":0.259388407981778, "value2":6440770726430652573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690879, "value1":0.0735354737983211, "value2":9181104589648553044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126782, "value1":0.4486397942700849, "value2":7462094819783188022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948901, "value1":0.9959329498625173, "value2":5479458805805092746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127088, "value1":0.9575385572151783, "value2":2275563983258531267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_7":"b","key_0":"k","key_1":"e", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259685, "value1":0.3372857504409974, "value2":6098187454917492671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336884, "value1":0.24553452230487985, "value2":8241989909296706238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_3":"a","key_9":"e","key_1":"g", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169076, "value1":0.17392408086361133, "value2":7622066852267723117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977238, "value1":0.5433155948072138, "value2":6898148363520327391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.278455, "value1":0.6005033834895888, "value2":8001662694808432224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993722, "value1":0.9597867721155696, "value2":1299805439881399559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.530999, "value1":0.2209888826520892, "value2":4066778144109849949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782378, "value1":0.7523792160016672, "value2":5556510651542494721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.219804, "value1":0.1717195689538366, "value2":7048403972015561437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.946856, "value1":0.5985084131744225, "value2":4885828087902675075, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759489, "value1":0.27172565847552677, "value2":7664668067438271399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687774, "value1":0.42250850460802286, "value2":7088707511295378411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560049, "value1":0.2616853019800064, "value2":2950184265013984605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097585, "value1":0.2684028422029654, "value2":675122184817942746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205813, "value1":0.44731824636014533, "value2":1428863070750239410, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.543640, "value1":0.544977219392789, "value2":2157919960028175094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566856, "value1":0.8008181036581771, "value2":7402387212023409312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.765221, "value1":0.8596387394697663, "value2":1232636922510789742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.386997, "value1":0.10171781350903462, "value2":2933191729226405023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.674282, "value1":0.05504930364585503, "value2":724010655267136473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820827, "value1":0.1512145830665697, "value2":6612187124412654634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537712, "value1":0.6818284933003707, "value2":833546983771380703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.020134, "value1":0.7293182995525076, "value2":2126763229656176078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.919742, "value1":0.47707314485124014, "value2":8035884626808128837, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_5":"d","key_7":"j","key_1":"k", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941486, "value1":0.13322259126479358, "value2":2182305210138540441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.089091, "value1":0.6999475934755898, "value2":5942262186593070970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.922220, "value1":0.01137462679977003, "value2":4337636877649564119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846010, "value1":0.10326336153396976, "value2":8556623018979218867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_9":"b","key_2":"j","key_7":"f", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361361, "value1":0.9603555432714893, "value2":7276258796427191775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303375, "value1":0.9198009606494528, "value2":3654960055261327174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354372, "value1":0.9154736221447883, "value2":6754725607301294498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836769, "value1":0.17500751807732254, "value2":5225830062891206724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.514481, "value1":0.712612132062798, "value2":5405795909104498480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.857662, "value1":0.4771648896771537, "value2":9197955420056459031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218504, "value1":0.43388279897052595, "value2":4876293374213376800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854318, "value1":0.29673697161398044, "value2":2557767286012732771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.621120, "value1":0.07462842813102197, "value2":5246869953360521284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482880, "value1":0.4261787117422624, "value2":7122235599178548659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880280, "value1":0.38443671400954443, "value2":5167341510277945257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849192, "value1":0.5161558417921461, "value2":4578699148249247151, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_8":"g","key_9":"d","key_1":"i", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826648, "value1":0.873997512950574, "value2":7985090519609883665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_3":"i","key_7":"j","key_2":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199619, "value1":0.8346974218021976, "value2":8292656933621980920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.959402, "value1":0.6444072126847691, "value2":4320840087483118841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117065, "value1":0.9272062952128088, "value2":4910297155014939874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.551716, "value1":0.13475804603313432, "value2":8222696695222433380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643768, "value1":0.8311693472811903, "value2":7965400634034447655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.484767, "value1":0.5064114690168263, "value2":5755735513825830154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377345, "value1":0.1504036301883094, "value2":2988203951245132390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991676, "value1":0.972772556086247, "value2":1470890803841584803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361353, "value1":0.6540839592057779, "value2":985314732060254627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585584, "value1":0.7709261757498476, "value2":2565995410162492585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.704212, "value1":0.3118041755703027, "value2":6201623885191421475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.518173, "value1":0.9814365163985729, "value2":8388580742116222397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.187366, "value1":0.6732011290892043, "value2":5408814647988876882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.296035, "value1":0.5028422803953023, "value2":7236554080452628944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.768145, "value1":0.8394583660519598, "value2":5325173567069752762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549064, "value1":0.12911593374108088, "value2":8305911548302819241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_9":"h","key_1":"e","key_2":"e", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758468, "value1":0.0009372001034959489, "value2":5525854549400493996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.525822, "value1":0.9778487221852099, "value2":2741333418274183362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396148, "value1":0.9074357631670523, "value2":8809499083604789974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414129, "value1":0.8132493023109502, "value2":4942425104780689529, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.398880, "value1":0.39814811004109957, "value2":6997600198610651439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.667851, "value1":0.7812827159518755, "value2":111949400321343617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.065047, "value1":0.5608722991855313, "value2":128957686943303349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.494481, "value1":0.17816439204883128, "value2":8141342025149658982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.515335, "value1":0.9138017166447161, "value2":4317898835838327168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713454, "value1":0.43899440111826654, "value2":143487884063527116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185777, "value1":0.5460630190657308, "value2":3180840112779032107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450264, "value1":0.9313181115462055, "value2":8328870888335802008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_5":"k","key_6":"e","key_1":"d", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567912, "value1":0.6046426149595721, "value2":3206194822374359841, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.896555, "value1":0.6026393505223315, "value2":4823045987798108744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710017, "value1":0.5880756405721221, "value2":4714155846612246340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265002, "value1":0.824988737224255, "value2":8146626468221994382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029140, "value1":0.9928569418325714, "value2":1115616498765210900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.994788, "value1":0.6191499887981317, "value2":1491651902318479820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310266, "value1":0.6540537390689438, "value2":8869916203173244502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786610, "value1":0.5759332726822003, "value2":7232889220252179533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036697, "value1":0.31309276046690226, "value2":4197456548662367361, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.684262, "value1":0.39030765212524027, "value2":8590977643860948235, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.295632, "value1":0.4747190839423866, "value2":2819114746608115923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.613481, "value1":0.02792670630263574, "value2":4669805487355644126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.462933, "value1":0.7475159252861773, "value2":2383279322222812533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179395, "value1":0.19102052131729996, "value2":1856517930791738382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991666, "value1":0.3546374487300389, "value2":6354333338863606534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.568174, "value1":0.4332276746738033, "value2":8645759887904070759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.321899, "value1":0.2992200951220634, "value2":3116962887997170979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390308, "value1":0.02747272568305014, "value2":926243844801181597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.805520, "value1":0.5096841522295752, "value2":7414378226567615667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230674, "value1":0.03960399937089453, "value2":1710851993634315848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.425150, "value1":0.4167707239792382, "value2":5244023674899540768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879335, "value1":0.4867774224489378, "value2":553896137255768087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.021415180040758975, "value2":6576886716131511876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.875807, "value1":0.22236040526017317, "value2":8247944924957719281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.596786, "value1":0.5382707900600514, "value2":6504957777564642511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853544, "value1":0.6591705273657046, "value2":6314593273328284879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.792273, "value1":0.8726534877662263, "value2":8304088035724056780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_8":"a","key_3":"i","key_5":"h", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050549, "value1":0.962790936441039, "value2":4045656520780865827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493970, "value1":0.39340114596435655, "value2":5294670971014916935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251929, "value1":0.5462996764305353, "value2":5897296033220091874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115726, "value1":0.8575474248088841, "value2":2146148594913460918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.787443, "value1":0.8185914401069716, "value2":6891346883169646879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.982163, "value1":0.8404209142239473, "value2":2046890513913645569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529940, "value1":0.15216580562912718, "value2":235763707139823565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036141, "value1":0.21797894391861414, "value2":2037043718914992589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489425, "value1":0.578085671307559, "value2":1968373430108550733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377536, "value1":0.5230650685409198, "value2":1948377520185967051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.644803, "value1":0.09118740593062584, "value2":3209698888069603256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149640, "value1":0.2149828010685946, "value2":324803347557649497, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.788006, "value1":0.35151805877228876, "value2":5736730574020425869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528326, "value1":0.12892234054630594, "value2":40722644873633005, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.517453, "value1":0.12096018419440699, "value2":8591818996293676301, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580879, "value1":0.18567989692889683, "value2":2424108579790648867, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.418474, "value1":0.10109548853357979, "value2":6805501278544557691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192558, "value1":0.5389855440079544, "value2":2468356112053849635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_6":"h","key_9":"d","key_5":"e", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700652, "value1":0.8426222872981881, "value2":5586434719045643994, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836161, "value1":0.6261463649204317, "value2":7998455739612119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.410987, "value1":0.19322483253914569, "value2":7602154106800870019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920472, "value1":0.07895320792744571, "value2":3621399116338923589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_8":"a","key_9":"g","key_1":"c", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058885, "value1":0.038506638786952846, "value2":8443519094492233492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.357953, "value1":0.49873265104684444, "value2":6689514258543734175, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444144, "value1":0.7257783571311162, "value2":2132381976912359864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.003034, "value1":0.21025100795379004, "value2":4437597220292182437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794651, "value1":0.0987214108191527, "value2":4856891238525466286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.101167, "value1":0.5530900889416458, "value2":5869924593453820746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813524, "value1":0.7032391307910569, "value2":2962680547459287448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102641, "value1":0.05280577735508823, "value2":6197989994765662479, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.042807, "value1":0.46787236774210145, "value2":6987091087352317601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023412, "value1":0.4117202092054757, "value2":5734736277249168119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.582529, "value1":0.14733901739015626, "value2":2008347713878225513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843555, "value1":0.45648522904759514, "value2":8748231721848476899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.609920, "value1":0.5332106841490003, "value2":3789122902542055079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_9":"d","key_0":"h","key_1":"b", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.004873, "value1":0.9250068088247592, "value2":4060601419917043509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598410, "value1":0.20935422794575423, "value2":3458524447228408749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327673, "value1":0.6622534351078718, "value2":1975526610989314262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701834, "value1":0.018453031281240855, "value2":7208203361477507972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.544888, "value1":0.022192838541852087, "value2":420807956823837942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_8":"e","key_2":"i", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.333657, "value1":0.6116791297905804, "value2":6780782056382077493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.224370, "value1":0.58595743970275, "value2":2273745727787058306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_5":"h","key_0":"f","key_1":"a", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874813, "value1":0.4160482798834957, "value2":5130896603060858656, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_8":"k","key_4":"i", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800557, "value1":0.6485257630686503, "value2":8645194213531066428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_6":"b","key_2":"g","key_3":"e", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798578, "value1":0.3562881109226897, "value2":1936635714706401113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222829, "value1":0.18800753190841593, "value2":3868467029118261102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394596, "value1":0.9715849939407708, "value2":2892398559190644533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088141, "value1":0.19677712178571452, "value2":1290002108232651066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046604, "value1":0.7961651460552367, "value2":6142675575156594148, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755275, "value1":0.26382736011465824, "value2":4790349544446362608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.300482, "value1":0.6458439650096045, "value2":5076464481756841749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294901, "value1":0.3937755673758644, "value2":4837753175838197566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717849, "value1":0.8465941273216573, "value2":4538488428913350745, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018027, "value1":0.7099513420503992, "value2":6589927070191288064, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686148, "value1":0.31341443389871104, "value2":4542163293900247497, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554562, "value1":0.6021518819649633, "value2":657935532838913237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549050, "value1":0.9913051319754902, "value2":9133655827436281017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.564021, "value1":0.3933144754852601, "value2":8691343872756358848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.555563, "value1":0.12216755310514293, "value2":8181517965767093469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991822, "value1":0.9046701031504584, "value2":7169764745600183537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536610, "value1":0.6245723927615995, "value2":5410550698568344254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.640481, "value1":0.7953791600529995, "value2":8896491255145975602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.983650, "value1":0.39162313489970674, "value2":5865250699451794074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.748358, "value1":0.3770029381067136, "value2":7671767291617145847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632308, "value1":0.9672641688276767, "value2":3043416827366033954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829101, "value1":0.63693380465596, "value2":3688916592463553468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.271187, "value1":0.7822523952945989, "value2":7538651950678502209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757851, "value1":0.4993335289484615, "value2":8433879559771522559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.022503, "value1":0.5685717542133112, "value2":5232283843486040657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046586, "value1":0.6991953129630171, "value2":767301512539468704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170088, "value1":0.9099698273235788, "value2":6375281583140134087, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.819909, "value1":0.9395198166293325, "value2":9208855013717569494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078675, "value1":0.35105170816316494, "value2":7441300028994205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977562, "value1":0.27747207540421476, "value2":2610474267165011454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.577027, "value1":0.4084658901530552, "value2":7348713799224820014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705040, "value1":0.13989635719875274, "value2":8770545555481284876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.821460, "value1":0.5135499070770553, "value2":1707896495831766919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369139, "value1":0.07309187404043328, "value2":5262455623289086241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536485, "value1":0.3546193597483191, "value2":5532328840062432345, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632890, "value1":0.87913605477188, "value2":4206324763882989783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.352893, "value1":0.22467668553540646, "value2":8020391351501404610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058110, "value1":0.08476564402932071, "value2":2402316898103266567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312920, "value1":0.14715981410041812, "value2":515215788552749041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335941, "value1":0.5607746607254104, "value2":4029056354863603408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493922, "value1":0.5228229815154212, "value2":5917395680452547526, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.721451, "value1":0.8180794492453682, "value2":1670905717947422538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.347574, "value1":0.7935601761833552, "value2":6084751528698562987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.722657, "value1":0.4075231573944958, "value2":7748408088453297294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.451863, "value1":0.3657057400324225, "value2":7302526748126423806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495468, "value1":0.10049524830706323, "value2":7515132691589300738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193570, "value1":0.5847911145927462, "value2":4656707981929867290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866643, "value1":0.9003805975682342, "value2":3827225598428319501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.535740, "value1":0.5365482035917958, "value2":3695801733453249426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.197799, "value1":0.7228560209917014, "value2":5697296448849523746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536756, "value1":0.2916665533141402, "value2":8616484406626615233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231668, "value1":0.5491980830875538, "value2":5401247188111922849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836905, "value1":0.6668911346325062, "value2":5956397501314084543, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.060078, "value1":0.41433534941101835, "value2":259739043603601632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933921, "value1":0.15068665469322526, "value2":4615815953640952732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.147080, "value1":0.17748824888193676, "value2":838395034976203435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662077, "value1":0.7882764714348716, "value2":4457961488907810285, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023154, "value1":0.24361462379316554, "value2":7839025824541823553, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826915, "value1":0.569504859021508, "value2":6251982817444532308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585595, "value1":0.8340020457470065, "value2":2938024413765410558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146190, "value1":0.9581727102338353, "value2":4413141265483838870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.245440, "value1":0.6417183759653234, "value2":5364021531028647444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493343, "value1":0.4938700840392069, "value2":3091997225543832896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807379, "value1":0.4690524091058998, "value2":3061093742738985330, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.208752, "value1":0.8639618600039569, "value2":3187072739982271504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.011216, "value1":0.18367276511432976, "value2":2360627840986057827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_8":"d","key_6":"i", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.440539, "value1":0.6977700817568234, "value2":3581694346818534680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489990, "value1":0.309872070478555, "value2":7457465639523954616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113130, "value1":0.6897957784827682, "value2":90019976280058795, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.604031, "value1":0.797243453747431, "value2":1234756804789352079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.881184, "value1":0.28155440991949554, "value2":4070028978929644169, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.367426, "value1":0.016167903292426008, "value2":4863636678376461059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.769132, "value1":0.967909876972054, "value2":9131005500727231455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.905490, "value1":0.6194024275206055, "value2":210353074026913884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.915138, "value1":0.6781979759382127, "value2":6372020977221093753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093709, "value1":0.6187727010105798, "value2":312866041452100946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758312, "value1":0.8273322264897901, "value2":7512565884920091651, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.284383, "value1":0.39739645167776044, "value2":8773985992873355471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216803, "value1":0.8653155039130678, "value2":8937229562905505386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.770263, "value1":0.9293790766191896, "value2":4560937013728510181, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096191, "value1":0.22323546625261562, "value2":8010469458396680860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.984434, "value1":0.9454942818827864, "value2":660593493740706018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222194, "value1":0.02420800916517378, "value2":5602553742039024765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752900, "value1":0.3672193480709247, "value2":6716506478528347816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.967703, "value1":0.2944363320939628, "value2":7783913193752729609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.600560, "value1":0.1440348243160143, "value2":9110640288768831559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400353, "value1":0.2233756240085349, "value2":2151981935371948568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.269597, "value1":0.9641297731318997, "value2":8169933506866910786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.486730, "value1":0.1006988853266946, "value2":3116477845424070312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953298, "value1":0.4756644907363518, "value2":2464865227545117596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705992, "value1":0.8173093209310923, "value2":8219739418749111053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.851844, "value1":0.28729444422491424, "value2":191979366247033806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616037, "value1":0.08828307120467993, "value2":1006916952645293364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.007265, "value1":0.43891779173017603, "value2":4642029911324023153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408095, "value1":0.9633558663387003, "value2":2764041604557827140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825923, "value1":0.12824733189500837, "value2":7952369571604255477, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853643, "value1":0.8488628783599163, "value2":3110877858489702411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686891, "value1":0.5538740126823626, "value2":7618257543247002890, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408459, "value1":0.2427343835864084, "value2":1637171870696137413, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.635245, "value1":0.42768890444095087, "value2":4013428109133869754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793205, "value1":0.23512439386454564, "value2":5799487535459689842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920141, "value1":0.9415332414772868, "value2":1520706400052294853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.930133, "value1":0.47818008921632604, "value2":4567447002026438924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.599550, "value1":0.9293362893760267, "value2":1195481394235809552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903387, "value1":0.11210859774886604, "value2":3335652489364502188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_4":"h","key_0":"f", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698033, "value1":0.3148016773642106, "value2":4939787017916634516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813706, "value1":0.3325590668831616, "value2":3252287249574729793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_2":"d","key_6":"b","key_1":"i", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.048837, "value1":0.053013772957975554, "value2":4819883401481045104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310950, "value1":0.41839244392518726, "value2":8733225969530074129, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.062225, "value1":0.858129811136947, "value2":7961256500615532109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428880, "value1":0.2502872252911392, "value2":7064927828432353118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_6":"c","key_0":"c","key_3":"b", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560002, "value1":0.11724155985130476, "value2":3115965127965032390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657940, "value1":0.5473444163031204, "value2":8727302484888278411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_9":"d","key_0":"k","key_5":"g", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554826, "value1":0.376684927630788, "value2":2585922874708871992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079791, "value1":0.9982500603963644, "value2":7391513557121794219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_8":"e","key_1":"h","key_4":"k", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.174380, "value1":0.13350980294424206, "value2":9115933928609206333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690448, "value1":0.8022085172421284, "value2":5503579136981394270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178959, "value1":0.5153762777175099, "value2":4750624720468516504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880368, "value1":0.3826318223193023, "value2":3994162284701586149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594211, "value1":0.0032667252772956594, "value2":211951125332337865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080419, "value1":0.3371270895888467, "value2":970359359029436118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354035, "value1":0.6830560620508377, "value2":351440888758257733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686006, "value1":0.15832224781392848, "value2":7973904723234304705, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688086, "value1":0.14780349920952468, "value2":2027463870413046281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.407959, "value1":0.04437078472668187, "value2":1566333717176214387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_6":"d","key_3":"i","key_5":"a", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943264, "value1":0.6495072511389037, "value2":5853361075489349645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400773, "value1":0.2676114612948212, "value2":6666522398737719346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228255, "value1":0.29124903807175784, "value2":128611338987653451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702835, "value1":0.39306652609769904, "value2":4553551121428133233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346580, "value1":0.9157209666515556, "value2":4177607716262676740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000990, "value1":0.3155726084083431, "value2":7473596942971857359, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_8":"d","key_1":"b","key_4":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380807, "value1":0.5730835803736063, "value2":6788452783424900655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018359, "value1":0.11046900836774676, "value2":61241953976306769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097897, "value1":0.6529098432421965, "value2":9123131154054179256, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118936, "value1":0.312336727346538, "value2":7294156215339952794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.256520, "value1":0.817765585396917, "value2":4276314621602341409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.239028, "value1":0.27988068073239764, "value2":8048493989378214976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.359463, "value1":0.33204054027814056, "value2":4286372625461580954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625120, "value1":0.8053374337323631, "value2":1700890355040268131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.519453, "value1":0.2764225180730322, "value2":3111696426903897415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.019685, "value1":0.5511793734194003, "value2":3991968620227079942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.040695, "value1":0.6967938051735891, "value2":8919927137208366483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750443, "value1":0.9319592750217813, "value2":7402732360820495311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.497932, "value1":0.6545667018250457, "value2":7616613351817353333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.773399, "value1":0.5580363664821956, "value2":314222158348617253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.869323, "value1":0.7418088662526952, "value2":8657709877874943583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812016, "value1":0.4480192862830272, "value2":8470890763908250393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079326, "value1":0.3872756599905638, "value2":4349661970334556580, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.186686, "value1":0.8815811840060216, "value2":938888750792010987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235117, "value1":0.8673030373216688, "value2":4951632884410168399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335964, "value1":0.6073261439232968, "value2":8008517076279782152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063573, "value1":0.12394184604832796, "value2":1362755555847887698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.685355, "value1":0.2361653248224142, "value2":2210259628223947558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029806, "value1":0.3517864573309168, "value2":6754043647766872898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.593026, "value1":0.38937941917051955, "value2":1945974862147158833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595711, "value1":0.9836966964700219, "value2":4159340747167974849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.143026, "value1":0.5945202518873367, "value2":3646754986945085167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733493, "value1":0.18903944802082667, "value2":8837255893708962102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.120646, "value1":0.4261710458329441, "value2":6946544380857137617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.163665, "value1":0.24245216879890383, "value2":555667735507028882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934782, "value1":0.31976487170355755, "value2":6301272802564976177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975795, "value1":0.2007851651150272, "value2":866319078855783895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630757, "value1":0.2788228743238552, "value2":2540783851122363232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.661317, "value1":0.5218066875354199, "value2":8869741999431102494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980967, "value1":0.7624092409640837, "value2":628203903335846730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380622, "value1":0.4837921884626896, "value2":3628530218991586661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.248953, "value1":0.02097807162306582, "value2":4598542592315882026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.384743, "value1":0.7608954854738191, "value2":1886663813765371215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.950140, "value1":0.2550158514673341, "value2":4065397223823067237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.516999, "value1":0.6292445006161491, "value2":4125156478992786356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733841, "value1":0.37206344867268365, "value2":3082112298619357468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086119, "value1":0.5738504787109479, "value2":5573707094695082413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800475, "value1":0.7767745931253813, "value2":6702226199646136395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.175876, "value1":0.8346328055447434, "value2":1066680083349750545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752359, "value1":0.7257723037237084, "value2":7568317247183343047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.165816, "value1":0.19937439179339156, "value2":2068495750767911677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.597980, "value1":0.6555890535602861, "value2":2883065227864053972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.138569, "value1":0.22532513024599726, "value2":8297154458719010882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.668715, "value1":0.11645984529741488, "value2":5555518408390232617, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063265, "value1":0.001988016117254865, "value2":8483308279719955566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086644, "value1":0.4644931461203292, "value2":6366519140385299368, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.140571, "value1":0.5416650487590775, "value2":1665979086436164947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.045805, "value1":0.9888897436719353, "value2":7697021265197143502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272912, "value1":0.2088625700639475, "value2":8643238667192348189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_9":"k","key_3":"e","key_7":"b", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289278, "value1":0.005139072821858179, "value2":1003021960883868455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.242063, "value1":0.32725526486053264, "value2":2911166424297597473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.393077, "value1":0.6314586962746932, "value2":2631691711252794732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.274319, "value1":0.9496413905130506, "value2":3462918152034782288, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659954, "value1":0.9955163986622194, "value2":2213038727032863641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657842, "value1":0.11877952023160425, "value2":2754577958762836530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146197, "value1":0.5152391159350134, "value2":4396760573611697803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759786, "value1":0.051119901574673096, "value2":965796598734594927, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.891935, "value1":0.6695409149263635, "value2":7204593893035944941, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096828, "value1":0.7785299263381299, "value2":843269019498255824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822107, "value1":0.06887356745350987, "value2":3890683236062212546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818726, "value1":0.3529788280342954, "value2":2023962428787621327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.356164, "value1":0.8626419946204176, "value2":3137373073213134707, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855692, "value1":0.3727202998903748, "value2":1868074746512330224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.899737, "value1":0.49932818019000214, "value2":5247723900283585273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205758, "value1":0.752048623202744, "value2":405978535011434243, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.492517, "value1":0.7587591048194695, "value2":570855823822973980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025167, "value1":0.9352906688766658, "value2":409025014085718364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524374, "value1":0.456131184246001, "value2":6492711389883342442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.173025, "value1":0.7216031002138218, "value2":1900883846963453532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991241, "value1":0.9435537729789849, "value2":6744125432328938384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126845, "value1":0.9483471232225058, "value2":2429204263257823531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079973, "value1":0.14998274238182085, "value2":1609358502486987778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257883, "value1":0.09750371692825593, "value2":5050315729103715843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.074081, "value1":0.21018141055409362, "value2":9169596995481234183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218738, "value1":0.9047134659722221, "value2":9140209877004134552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.745794, "value1":0.39056909591189853, "value2":4170147818303980952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179295, "value1":0.9580414905588824, "value2":4615545745316982257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442990, "value1":0.018901929544229163, "value2":2360761388936503714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.422188, "value1":0.5272513199907374, "value2":3200950480296318673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192831, "value1":0.33577000839194515, "value2":8465993285574620045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.227691, "value1":0.814891459750041, "value2":5020817184863163049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_7":"f","key_1":"d","key_2":"h", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171624, "value1":0.32281185598307915, "value2":8148752929610663940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_2":"c","key_7":"h","key_1":"b", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.490347, "value1":0.4777481785589013, "value2":2903978983753469512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217872, "value1":0.6839140142887666, "value2":3130769003103218800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.992440, "value1":0.1702388495618268, "value2":6704155277469548917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.834388, "value1":0.855009067710764, "value2":8656769438285137265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230809, "value1":0.6026285375258597, "value2":1622052925759175826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344218, "value1":0.6418025631389815, "value2":8245500230948114788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086815, "value1":0.450124296857116, "value2":4866053118697224871, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802304, "value1":0.9289196028774398, "value2":6637963067322995548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_6":"j","key_7":"d","key_1":"e", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.840991, "value1":0.6695124623580792, "value2":5437276365728147032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.833585, "value1":0.22730504686256575, "value2":1520963367111518952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.744045, "value1":0.8193501328748979, "value2":279541774649644194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_9":"g","key_0":"b","key_2":"h", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.736126, "value1":0.6177732267307673, "value2":738077807298530261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131891, "value1":0.46856485152824584, "value2":1245296884531108950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419961, "value1":0.9237029205265445, "value2":1223547652782219884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512633, "value1":0.3170684617445516, "value2":3777467556351824325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630355, "value1":0.021704073637768814, "value2":8674907845809369167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.309670, "value1":0.16245539540033113, "value2":5206956291412580457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.348146, "value1":0.7658591727792013, "value2":8113096525411729139, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346099, "value1":0.09833103805087655, "value2":3329067421410580481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086337, "value1":0.8850305283927763, "value2":8626010840258936486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479373, "value1":0.20404187483062786, "value2":2412557827504873850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921982, "value1":0.7871473016496364, "value2":1874563226125050237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415457, "value1":0.9305937676701617, "value2":4246181816757480088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035701, "value1":0.11812212256088726, "value2":7168381109733048473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.225156, "value1":0.9869870812720318, "value2":137813480312065053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836380, "value1":0.860500054517146, "value2":6804060104865284962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289062, "value1":0.385288303480441, "value2":3917001550422656910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_7":"g","key_1":"k","key_2":"a", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885780, "value1":0.1315685249986314, "value2":2596575576892432777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948363, "value1":0.36071231455630776, "value2":628986020889092104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369788, "value1":0.40617296244272144, "value2":1817522335798570734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086188, "value1":0.6439285363835136, "value2":4867721675008841698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.995659, "value1":0.2954211415738649, "value2":316663728175376955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.129393, "value1":0.050215829131005675, "value2":6231361900716566583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400907, "value1":0.39112211803964236, "value2":5808809619414218326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_7":"f","key_0":"d","key_4":"d", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717436, "value1":0.943850779238098, "value2":1447465700474277383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217700, "value1":0.1707128166287673, "value2":4689212155367810810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_9":"e","key_0":"b","key_7":"c", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.908215, "value1":0.13836337191309858, "value2":3286491733476954470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383068, "value1":0.3171639131993519, "value2":5768120780013343252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758654, "value1":0.8248294031178783, "value2":1259633164417337834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885742, "value1":0.7054910487633926, "value2":965825539457944695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450755, "value1":0.5160791051802902, "value2":980291621965651603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280407, "value1":0.20023478153485316, "value2":216774102527727538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953286, "value1":0.17708932991625895, "value2":7861520056013490110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428154, "value1":0.3840775072053381, "value2":5128389599856507984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650144, "value1":0.5475742595996844, "value2":37238921657180760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880799, "value1":0.27347132520371564, "value2":7086278310918665364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179258, "value1":0.09483479166767275, "value2":3722096865477015794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025175, "value1":0.24738190446733826, "value2":2326403401160366476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680444, "value1":0.6441894024482266, "value2":2704266700740984761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830593, "value1":0.6314118537062262, "value2":8997005770669681872, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_9":"h","key_3":"c","key_5":"b", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177647, "value1":0.3203303212434556, "value2":8965923512665629393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923489, "value1":0.29642566667944875, "value2":1203313530737226131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.811446, "value1":0.5561455146353509, "value2":570880137152142782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544802, "value1":0.9164364466036977, "value2":3723655635493793654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.542566, "value1":0.7689437007202478, "value2":3938961297360550598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819459, "value1":0.4225560622422357, "value2":2765920289473813577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971131, "value1":0.7588578513174423, "value2":554701584189491661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638701, "value1":0.9383542993382983, "value2":2675775339679034911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658985, "value1":0.8385957143790167, "value2":9118144992996965702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449591, "value1":0.6354046681746691, "value2":4796404340287643460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.851526, "value1":0.7827632730352198, "value2":4383742830241177102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929626, "value1":0.7374713580232243, "value2":6076523832966313810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.289625, "value1":0.6865021261714863, "value2":2978049869639889816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394738, "value1":0.6344060546968167, "value2":293482016535092319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.487799, "value1":0.01884442354627017, "value2":6314666407653758310, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.189751, "value1":0.5131352606856101, "value2":5370657937415537626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.532641, "value1":0.7500243297537671, "value2":5814407549612083184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413938, "value1":0.35879356972502013, "value2":7060341260629410136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464627, "value1":0.5028910273480031, "value2":4767839332233680092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.840321, "value1":0.13633703551235915, "value2":2307803453888302997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177282, "value1":0.5212414108183161, "value2":7948818597964566740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394656, "value1":0.22032364884959718, "value2":8010953658147063588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_4":"b","key_2":"j", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.193561, "value1":0.6737972615452449, "value2":7205681986933877897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919748, "value1":0.5508022412545106, "value2":1935155841798114792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.549841, "value1":0.6412542942797744, "value2":5725068566993480801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755602, "value1":0.7570856701300407, "value2":2582449631680230244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569314, "value1":0.9682571063393881, "value2":3213998604408735177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220995, "value1":0.4682404519517274, "value2":1296712240822053674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030188, "value1":0.13463362539315563, "value2":6167313691161388680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955102, "value1":0.12893740940580098, "value2":937550317375879979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912054, "value1":0.23584661351992364, "value2":8270059203922837294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248552, "value1":0.873739706342713, "value2":5483897643292875706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.668856, "value1":0.06480463060086489, "value2":3429839268398826587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628990, "value1":0.06856495811246685, "value2":2353082280387987376, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.364427, "value1":0.8257731797802594, "value2":8162649044337833619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488610, "value1":0.47476344691483213, "value2":7563742944450884141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.659600, "value1":0.22983785061318257, "value2":5566259052586123346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715473, "value1":0.5778415124343756, "value2":379000163266106555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265567, "value1":0.856852436111815, "value2":96718295321822248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647902, "value1":0.9996700403145788, "value2":6837057660421261848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.866334, "value1":0.8219801167193398, "value2":8280931000915145951, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144409, "value1":0.604589772741639, "value2":4205152374038619548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192185, "value1":0.27115827101755235, "value2":5770338217899591097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300761, "value1":0.009332804989256857, "value2":4018403627826051427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.312951, "value1":0.45431720098201106, "value2":2937253947933810175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_4":"d","key_0":"j","key_1":"c", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647625, "value1":0.3268497558683786, "value2":5914220923444612314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878656, "value1":0.9409966965511731, "value2":1289273008938800365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057134, "value1":0.3301792558402197, "value2":3318812578583185116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.439955, "value1":0.2652202727470005, "value2":6508684150448276421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827995, "value1":0.5808204202626653, "value2":2858312188918612316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863057, "value1":0.013094412054074, "value2":8217074167029879656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936866, "value1":0.17076998229838808, "value2":1405785613574281058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.188817, "value1":0.4102266284907626, "value2":9200292054736167191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919291, "value1":0.849823159364745, "value2":8783211669019586620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.684825, "value1":0.5612249810486886, "value2":2459193643509253683, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.132931, "value1":0.06165133682856612, "value2":3724912889225979746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939721, "value1":0.7022523416408233, "value2":3787551523413319921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423514, "value1":0.16397574471331047, "value2":2217544936229011402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556341, "value1":0.3363631636072514, "value2":8296724122152500212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.797045, "value1":0.9775102685569385, "value2":8717038963718857217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.037101, "value1":0.5188119627862039, "value2":8253271773680540946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.472369, "value1":0.723269226446299, "value2":1830427992517853225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350765, "value1":0.45427831592086265, "value2":2459543486390463113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.172879, "value1":0.9405309891223543, "value2":2118657389939971315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902535, "value1":0.42952097429562314, "value2":3142662533255542276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863286, "value1":0.06876792355358291, "value2":1443459731022840202, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.137921, "value1":0.6964987593861792, "value2":7256675183245953924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.477178, "value1":0.3581558179516239, "value2":6351662172251250145, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286204, "value1":0.148299090562339, "value2":1443868893374130520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286734, "value1":0.45842257280884463, "value2":1988366284391476937, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787180, "value1":0.5028191040745897, "value2":4148707399092925318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.586935, "value1":0.0518139414168187, "value2":8586911735374624592, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_9":"k","key_0":"g","key_3":"g", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021854, "value1":0.8978228143933898, "value2":1413164249310483673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_6":"b","key_4":"k", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318004, "value1":0.5329901050180244, "value2":700502281930980696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381834, "value1":0.37533803419078043, "value2":1751644850942508358, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781575, "value1":0.7741607009730129, "value2":7455242385880444942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.901091, "value1":0.9760509636456571, "value2":1834882752759920223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805005, "value1":0.18640631542045302, "value2":3539140922926641426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734757, "value1":0.14536274157280238, "value2":6893258320350162281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.340380, "value1":0.2042782161421506, "value2":2006565586318459391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.271132, "value1":0.008492488378786304, "value2":6129670920743570493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947658, "value1":0.35165969319536833, "value2":4951697844001184553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.735049, "value1":0.4770090728759761, "value2":2452955784370225782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337551, "value1":0.7050663230338188, "value2":4249578695875539573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.633134, "value1":0.6874967232601639, "value2":8449525705234658507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412613, "value1":0.8000775398308927, "value2":5399334563984433319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.520666, "value1":0.3288020539774752, "value2":1684836256027900746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063862, "value1":0.4985615503389622, "value2":5250749533704010894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.776154, "value1":0.039036691522280384, "value2":8622652029902283303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.115396, "value1":0.2649092470624877, "value2":678667215803198769, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.728254, "value1":0.3531001293726284, "value2":2004607802946409884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242732, "value1":0.9820165334533849, "value2":7506111952193927451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969092, "value1":0.45874224514029954, "value2":6402112089715656350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.528740, "value1":0.8320354986171156, "value2":8412309334445466976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.907053, "value1":0.18136550029947487, "value2":6212227004856794281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630067, "value1":0.8986694902601496, "value2":9213857752862359137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133736, "value1":0.8699930415468984, "value2":4081508478635244211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.198697, "value1":0.017191657622434766, "value2":1486415534842367537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.690493, "value1":0.3223491896226482, "value2":7372352323440542564, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.404617, "value1":0.6240663182478166, "value2":4691037631120171189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120685, "value1":0.9919062682339983, "value2":8460618157376864427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.688005, "value1":0.3953177302348419, "value2":5961626802588462897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093767, "value1":0.23061987147968935, "value2":2892986034831352810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.543984, "value1":0.16501271900248737, "value2":317259514400114949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842969, "value1":0.25446808569640833, "value2":6002379582827585880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269131, "value1":0.8842302265963506, "value2":2140205221312415384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972033, "value1":0.9857791825041854, "value2":8003941482296671818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508672, "value1":0.5492264125933453, "value2":4184449934578666153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.812228, "value1":0.3691868332062656, "value2":940690450246364953, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033340, "value1":0.4347207480857335, "value2":7925423134984867831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921934, "value1":0.7008140011057534, "value2":3950204529429726081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_8":"e","key_3":"i", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.159043, "value1":0.4015464920656329, "value2":6141367601228839763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_8":"h","key_4":"e","key_5":"i", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207886, "value1":0.923323917554864, "value2":2538142530573084913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560000, "value1":0.988479108310322, "value2":3590995919192964810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324806, "value1":0.14623600559543842, "value2":7983994507018983485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.278078, "value1":0.47875025775481017, "value2":9135606324353342316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_9":"e","key_5":"i", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978376, "value1":0.646562029258319, "value2":6969654958747462453, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.038811, "value1":0.27278148584909157, "value2":3766949452289754200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174153, "value1":0.456204859878279, "value2":5412969439184865040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152394, "value1":0.6304591834176713, "value2":7508490770671251540, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079217, "value1":0.7093512830025359, "value2":8041107853739821078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736508, "value1":0.9456361684627089, "value2":2695850503301130596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.689579, "value1":0.8717416155878271, "value2":914033909063513507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_7":"d","key_2":"d","key_3":"e", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716602, "value1":0.6181082501584888, "value2":7525886338688800823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485128, "value1":0.805498923411989, "value2":3515190992320449261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_2":"c","key_0":"k","key_1":"a", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761534, "value1":0.3987574394917293, "value2":565378419404884224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234797, "value1":0.32687033897000584, "value2":4678721697918141366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_9":"k","key_1":"g","key_8":"i", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099258, "value1":0.5165730450377122, "value2":7897877163898991964, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.855254, "value1":0.6113796894495002, "value2":5439835292581168290, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098218, "value1":0.6331475060559588, "value2":7978662530169854255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.392907, "value1":0.7806228764363937, "value2":5349308434124377933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174842, "value1":0.7994951238912167, "value2":213268455628772714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.654871, "value1":0.23130216238112586, "value2":6353589500271587044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255121, "value1":0.029192800621726373, "value2":2049991280590108935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.454836, "value1":0.6345686250008671, "value2":8305250329173845884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.196946, "value1":0.9679432223428265, "value2":4005113690213723694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.514102, "value1":0.9460847081604973, "value2":2275533825263312731, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.267365, "value1":0.5628401308203138, "value2":2718632724301463408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.702558, "value1":0.3615743138551236, "value2":5265258619209069561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.157371, "value1":0.9914192491349261, "value2":7107333633697743515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367362, "value1":0.30046228876641146, "value2":8731213027442541680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502940, "value1":0.33029188840622603, "value2":7282340644676767522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773440, "value1":0.7398342089000881, "value2":6064862964524030137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370133, "value1":0.5464083248908671, "value2":8595565634194089280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819624, "value1":0.22563478104006193, "value2":6291072404703346422, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.822444, "value1":0.6431602385348965, "value2":3571751115605471571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154889, "value1":0.661415650101091, "value2":2897716721956096259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.706177, "value1":0.9541112843149568, "value2":4382405639585131280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.892582, "value1":0.273637580698836, "value2":1569537455975017743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912404, "value1":0.02958676932622315, "value2":864163789414463895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164999, "value1":0.8749507184007825, "value2":9170076213645968437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.609248, "value1":0.6230590575325244, "value2":1182671592625390639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402824, "value1":0.8555668782169765, "value2":3430464294959504849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.905017, "value1":0.6152490194289306, "value2":3258848281050130827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_8":"j","key_9":"d","key_5":"i", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.669648, "value1":0.6859907392101928, "value2":2219730699877135652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435602, "value1":0.12441938240097333, "value2":3663118450934184743, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736951, "value1":0.9806346124707411, "value2":6495060662787801507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502755, "value1":0.6962847488848957, "value2":7479724851955507611, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.618421, "value1":0.31789340504585667, "value2":8738918799232514086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827476, "value1":0.7869780281846211, "value2":5474883946012414730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490095, "value1":0.6962736110006601, "value2":6409713241665137545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985050, "value1":0.2615773286934893, "value2":6009116516814474343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502430, "value1":0.19305751383330208, "value2":1654152222563323901, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_3":"e","key_8":"b","key_2":"g", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349955, "value1":0.7772270603571774, "value2":73068209272270733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781804, "value1":0.6130002187106758, "value2":5094562490162409190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.158107, "value1":0.12199479281505482, "value2":887695971080523742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.575029, "value1":0.9090539881797902, "value2":3726306097696429975, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.197382, "value1":0.010351058305108105, "value2":3390177787518625882, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959617, "value1":0.10545857069830916, "value2":8523151888175341811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789446, "value1":0.7606780493785521, "value2":2782849609854347711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384471, "value1":0.3297433712153455, "value2":8732496713018613730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.440642, "value1":0.05991406624275138, "value2":8221843418889420393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485145, "value1":0.4324948321911064, "value2":1223798989569651011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930239, "value1":0.797530519094042, "value2":3933281842105360464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_7":"j","key_0":"h","key_4":"c", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921694, "value1":0.3827550255479697, "value2":1163470615431943983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780691, "value1":0.8316468698353685, "value2":3969005124884722997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563279, "value1":0.5313651121371132, "value2":1275907147042029939, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099445, "value1":0.8066245132140291, "value2":1404833370385176869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.897823, "value1":0.395354453907472, "value2":4125787827950753058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562742, "value1":0.0008610845550682808, "value2":2683131454862577523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981050, "value1":0.6811899501971241, "value2":2963479936998599518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.090076, "value1":0.7872489837661779, "value2":5920042185451264661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904987, "value1":0.582385162743904, "value2":3052394546288639081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.807620, "value1":0.8276864738323398, "value2":1770112128995343558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.313153, "value1":0.7000557306246208, "value2":8804754539464742313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648178, "value1":0.6035738426070452, "value2":274869451073352526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382503, "value1":0.7570596198853545, "value2":8494223821423922555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981825, "value1":0.4954131435773338, "value2":5546139769442373925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.896817, "value1":0.3057668662697838, "value2":2146747890118623175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_5":"k","key_1":"b", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857258, "value1":0.7535991261099572, "value2":7241562330594412223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.954553, "value1":0.6272550393596944, "value2":7059824369010485707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611570, "value1":0.896770550906892, "value2":4157172203390286832, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.200060, "value1":0.6395440920752251, "value2":2313959820318356273, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_9":"k","key_2":"g","key_3":"j", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.305817, "value1":0.7944717065357103, "value2":9079049510197634026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771177, "value1":0.5686457722223018, "value2":5926649416008895009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.194460, "value1":0.2030568091950128, "value2":5211606615226569619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_2":"c","key_6":"b","key_1":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541777, "value1":0.38229879472765593, "value2":8330651996424197311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566620, "value1":0.24307591832233774, "value2":2839998819511281148, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376323, "value1":0.5739727137071835, "value2":5399793491142182010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480111, "value1":0.7895749118902282, "value2":7450043826116159674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453645, "value1":0.4875344798695633, "value2":6926040231429456296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734559, "value1":0.3865838466523982, "value2":9001676782517223243, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.171659, "value1":0.33364493236506704, "value2":5739131266488499620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.966196, "value1":0.1807801172943221, "value2":7703455497589255747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087000, "value1":0.14745151659831435, "value2":3780273793092000978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537557, "value1":0.6679609579098379, "value2":7524235382942600940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624433, "value1":0.43524838478159544, "value2":8707151351953730491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.764878, "value1":0.9857854697655962, "value2":2628899540411422018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087516, "value1":0.9235134986127471, "value2":1244667248757759893, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.092707, "value1":0.8123903328089516, "value2":1510741329638496293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212617, "value1":0.7803051371802152, "value2":1782941121051923216, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110139, "value1":0.9520250869731262, "value2":8515109734542519287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217411, "value1":0.569745410096317, "value2":7158729472403289306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578681, "value1":0.4421036889589163, "value2":5172660091174561754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.917747, "value1":0.9646587109518411, "value2":4932202474624997807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490742, "value1":0.18375289991112742, "value2":5134730937919546380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698691, "value1":0.8462874020691947, "value2":6789099466073417633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987429, "value1":0.47456618667904216, "value2":7333730409059985654, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.552031, "value1":0.8731675557704086, "value2":2383267067188016407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.024510, "value1":0.7398931410894473, "value2":4688375563960023614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.885940, "value1":0.5267822661948509, "value2":3106942003445818164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.128634, "value1":0.17304999217293912, "value2":4202058405826758012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919245, "value1":0.5246293149604686, "value2":7962001396490479168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.784582, "value1":0.9051007718281217, "value2":6345441367106704431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.631365, "value1":0.18200897376922306, "value2":2743187358723124435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632586, "value1":0.49203338164778787, "value2":310459975834593918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.986910, "value1":0.21502125960121787, "value2":9213029499233048926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.903530, "value1":0.20307359325640195, "value2":3425839773997981556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495175, "value1":0.42906594953086846, "value2":1910613263092063181, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560265, "value1":0.27498766656590357, "value2":8893579102721900369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.298348, "value1":0.05982261910728163, "value2":8036644582013512610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624831, "value1":0.8394666399726175, "value2":1345154507982673038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.123368, "value1":0.13834539586421774, "value2":6342191610731262762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.086336, "value1":0.17439021756766837, "value2":6128400320706208802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421248, "value1":0.39843282468447905, "value2":8680280397492385017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139454, "value1":0.456434331043375, "value2":4836743646420773621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351219, "value1":0.777135306661456, "value2":3903934605632263710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.414090, "value1":0.8143501039400677, "value2":4413351639615300728, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_7":"f","key_0":"d","key_5":"h", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231721, "value1":0.3138412091160347, "value2":9168517044165068794, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_1":"h","key_0":"d", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.230702, "value1":0.8804508000643604, "value2":512236854100511456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.708957, "value1":0.47311614120900003, "value2":7136526398161878544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.475928, "value1":0.32108039730398563, "value2":7292530251105851041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.643383, "value1":0.9689860631255371, "value2":1734391035548707204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.792203, "value1":0.8844015947765959, "value2":6318884852252048224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.323993, "value1":0.18316994233066625, "value2":3785806626345479252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.055153, "value1":0.9975090536646855, "value2":9101914663697333394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762055, "value1":0.14538112505308043, "value2":6589674161435979608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457017, "value1":0.9313588309515796, "value2":2386410717081263989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118975, "value1":0.2538090940895061, "value2":2359952847484695171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887012, "value1":0.26127955373376677, "value2":7323797183712585078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465777, "value1":0.39956363244564946, "value2":4290799816610955241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919817, "value1":0.24654070770474018, "value2":6141631506827241212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878721, "value1":0.7298060681766352, "value2":7628091662411871299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308905, "value1":0.8062336262819177, "value2":3004783841504994012, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630086, "value1":0.9740479758667072, "value2":7021198218943014146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661157, "value1":0.0030467411917742306, "value2":8705979164600447258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865574, "value1":0.36919272421680055, "value2":4523649159933689100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.185605, "value1":0.5443213859941823, "value2":3228781454351122262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865038, "value1":0.09385639771871399, "value2":7227272424843868171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798023, "value1":0.8684611657481685, "value2":2233953571760268466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730804, "value1":0.08710486562835452, "value2":6236391362613657084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.202824, "value1":0.4792114852099944, "value2":445497349923776863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173553, "value1":0.039235946095809125, "value2":7869971288440786845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908688, "value1":0.5052600498486294, "value2":4519674082380595002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341267, "value1":0.007052586435709472, "value2":3150628489852334883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881706, "value1":0.2885539517449333, "value2":1797762712648874995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216914, "value1":0.4200170713130975, "value2":8711469515389655710, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.219622, "value1":0.8598302806140433, "value2":3863311400727592540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.656021, "value1":0.6463537908056233, "value2":1050074674903654190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641122, "value1":0.40560841010225995, "value2":742946648471490108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_9":"d","key_6":"j","key_8":"c", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755478, "value1":0.42687719134929036, "value2":8589479839598188558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_8":"d","key_0":"a","key_1":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.848432, "value1":0.9962729980353733, "value2":780000159187900550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.195275, "value1":0.6773988604333073, "value2":5522000829662160867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787345, "value1":0.6486468864516003, "value2":4138825123686123358, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.553619, "value1":0.7566207604613129, "value2":5341523136140526945, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982202, "value1":0.8174248836774913, "value2":4875324336295746206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317829, "value1":0.25574317763537074, "value2":3850513628767917465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366612, "value1":0.634692324496315, "value2":8508176811169559100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.931082, "value1":0.9070818575294899, "value2":2941126198839449331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419466, "value1":0.6035953996525765, "value2":5226492465996637298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207446, "value1":0.7148273386681443, "value2":6830161919863899962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339115, "value1":0.9680080839353958, "value2":3346579129519946511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329518, "value1":0.5052033589082856, "value2":8326166807060477796, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_2":"c","key_0":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314104, "value1":0.23744990369573346, "value2":6679180757317175227, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177954, "value1":0.6916907655144761, "value2":6598624424043999988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220478, "value1":0.9419151786449748, "value2":6868454728302182507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.962386, "value1":0.29200153388880246, "value2":3531740125944089227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.522912, "value1":0.4384617611060238, "value2":7012028395173793023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.701530, "value1":0.008494038169472137, "value2":4096870035163582532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019574, "value1":0.540551382306373, "value2":7575443997806932703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558742, "value1":0.30242558718747614, "value2":1312291174476606343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.597198, "value1":0.9990604033963557, "value2":5377437181323103863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321750, "value1":0.8571357709325697, "value2":3223326577646383979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976585, "value1":0.794736424631542, "value2":7297990135456560387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053373, "value1":0.9010563829072253, "value2":5702408670366972568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.500145, "value1":0.16016689363963474, "value2":7655127244946192653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.347755, "value1":0.19254449629837572, "value2":1438411533080776332, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.047080, "value1":0.9596252631833848, "value2":4943940353426671568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.737278, "value1":0.6304724450380977, "value2":5075506433971212474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842472, "value1":0.9682019821931608, "value2":7537248780629599850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210066, "value1":0.016073835132125416, "value2":1148941512571603438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019003, "value1":0.15376503679496498, "value2":6744619422647456093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972351, "value1":0.6522824510722263, "value2":3635978187235320184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464404, "value1":0.1987173098042302, "value2":2325620189736714816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505822, "value1":0.8969967220271704, "value2":2772428523245878847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.858017, "value1":0.42532104055952885, "value2":7057664582771584406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827767, "value1":0.11843519151662998, "value2":4642257980991996610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457807, "value1":0.6680540521232995, "value2":1413093760337990232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307237, "value1":0.2483558233491924, "value2":1657535478313961463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187495, "value1":0.8787274497214357, "value2":1012440122187350073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_5":"h","key_6":"k","key_2":"b", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.935106, "value1":0.6622345557285527, "value2":873018521033719372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212483, "value1":0.03664769373990424, "value2":8662944034974337497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.279577, "value1":0.06940570148672788, "value2":3397490112016095138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.679928, "value1":0.94467398885254, "value2":8140288354763339321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348611, "value1":0.09177297933527545, "value2":2243053697955925595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033795, "value1":0.004266699548950342, "value2":8869596053230724066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716957, "value1":0.952311087977132, "value2":3488323066281405120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275244, "value1":0.9153132837467353, "value2":6017024334903831134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523199, "value1":0.8894820330529598, "value2":2495751532034294665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.548536, "value1":0.36452752239967207, "value2":4087473006792877554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_9":"g","key_2":"b", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452175, "value1":0.03794808741352215, "value2":8378297363369384496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156999, "value1":0.3422567916396813, "value2":3950482866588863620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508543, "value1":0.5441836551476523, "value2":5324891351187729379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397072, "value1":0.8991563074648984, "value2":484650303993845367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074791, "value1":0.08635154390096551, "value2":7636185572736407362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214033, "value1":0.529318647161814, "value2":3428935700531030330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023178, "value1":0.9863231971076605, "value2":952801547411635885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.655562, "value1":0.1342499141610202, "value2":6341506872735720793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834066, "value1":0.5801423696428913, "value2":2989261336292506730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.107225, "value1":0.08595942695174727, "value2":7162816956772874074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.303930, "value1":0.5732092807009482, "value2":6606617148160680700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.915970, "value1":0.10263255284529503, "value2":8480457956761770794, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356355, "value1":0.6141228451134201, "value2":7156945200312274543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645757, "value1":0.7597072830851564, "value2":2605675246955821391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_9":"e","key_1":"f","key_4":"b", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760044, "value1":0.048059905194606296, "value2":3102725962018871003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949450, "value1":0.803383211503761, "value2":5157179277981661443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124187, "value1":0.16836795463840862, "value2":5627024937490827799, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621202, "value1":0.11873453496853606, "value2":971627245321019634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762026, "value1":0.5957950719970895, "value2":1706809321238862597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391493, "value1":0.7101377073031195, "value2":6966030075201405586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.168903, "value1":0.22863352792292077, "value2":1427332489767053365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698944, "value1":0.5031422927319381, "value2":2536670790706398066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985917, "value1":0.7010791398723792, "value2":2424373004486759234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611987, "value1":0.023052053167488012, "value2":1572028785346213886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.084732, "value1":0.8225285464369793, "value2":7247308476502074358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476584, "value1":0.6320619568566006, "value2":3013845267574452926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.358865, "value1":0.9597368730404013, "value2":1895755548879639735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559280, "value1":0.7939551495955407, "value2":9185811549617457738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338153, "value1":0.6433609372521104, "value2":5456961662990895069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120489, "value1":0.02598857402797232, "value2":2690067127374441413, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.026856, "value1":0.3509430162667547, "value2":5912250260618117609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.911457, "value1":0.5658091707632997, "value2":3970272443516058585, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563432, "value1":0.24659033075994388, "value2":728046743750740729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.596212, "value1":0.5615056517168183, "value2":7033619460114819908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051460, "value1":0.5771358250664421, "value2":8203484790496543671, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.824484, "value1":0.05697931696566816, "value2":6635808805223685263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627182, "value1":0.9130566878289693, "value2":8110244852760276756, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_3":"e","key_9":"e","key_2":"d", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207192, "value1":0.7274854390257628, "value2":5772335372245409001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799146, "value1":0.5204858317687999, "value2":128757778781441708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014826, "value1":0.7097519460607616, "value2":9006806179184131719, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648719, "value1":0.01042589454449313, "value2":1630317652722526040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_6":"j","key_9":"k","key_3":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.665998, "value1":0.6184281613297867, "value2":3187261205020385384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.501321, "value1":0.23807711629991066, "value2":1285207877035639838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184165, "value1":0.7331865941823509, "value2":8439731025981805138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_9":"f","key_0":"a","key_5":"j", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801569, "value1":0.9860558447882487, "value2":6774600722253198808, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384223, "value1":0.22369926816026842, "value2":5826656674460169842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.980495, "value1":0.6229185444454209, "value2":4872634527752777210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.380826, "value1":0.7806130098804899, "value2":384791734560367497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163193, "value1":0.5398716860823138, "value2":4387950486971806786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.823934, "value1":0.5810656874809516, "value2":450490831163611562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.608002, "value1":0.7930931659235, "value2":7050949938710758790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363629, "value1":0.5705690869328316, "value2":4003836970877260330, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_6":"e","key_7":"a","key_5":"i", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.997480, "value1":0.2946419071937674, "value2":5707538095602523121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.835470, "value1":0.023853067484955352, "value2":5038338303194879356, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.720798, "value1":0.48474572095528634, "value2":682322783035637435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.573983, "value1":0.3514813766672496, "value2":2903766540993385647, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730487, "value1":0.8217554814463921, "value2":2948047805979067123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016747, "value1":0.12213673109770266, "value2":3888549493731213502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.081414, "value1":0.18665321280658956, "value2":6792269259144278363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300754, "value1":0.9731729557371976, "value2":7090692680090167231, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337262, "value1":0.4023504772804693, "value2":4873228147801762854, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990413, "value1":0.6497908067795414, "value2":3504328384430870621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.276664, "value1":0.785362017819063, "value2":1481672280882279109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.357896, "value1":0.03050056454630596, "value2":6649840382760115378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.095552, "value1":0.31901448906359003, "value2":7956338378631849376, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.518227, "value1":0.922032137621878, "value2":785067472984268171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094243, "value1":0.6170957228718421, "value2":6792110304537802959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782822, "value1":0.20221744914264872, "value2":3178187263071942184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.539464, "value1":0.9703158465709303, "value2":1060145839109092280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773211, "value1":0.9517836987483416, "value2":7463838442968425219, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.695794, "value1":0.34514076166527163, "value2":1111746250317888633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074230, "value1":0.19136080585905593, "value2":4573587278465589025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_6":"c","key_7":"b","key_0":"j", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982043, "value1":0.2232136951539441, "value2":7688263110504645334, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939724, "value1":0.3301646825680086, "value2":5317348954333303134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275048, "value1":0.1567976387714283, "value2":8628385701502049400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.251237, "value1":0.601867452525591, "value2":4258975777323734306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051509, "value1":0.87447596502846, "value2":390569829027012116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453014, "value1":0.7517900109611292, "value2":5319112069730300764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.859148, "value1":0.03968223769223721, "value2":6092044766196787249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564895, "value1":0.8282334480284984, "value2":8177460795118738605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.213064, "value1":0.37245338326241034, "value2":5313011500830153125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234759, "value1":0.6391009807318464, "value2":3598159575817066989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922540, "value1":0.8881740732484977, "value2":4573970560902118758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247842, "value1":0.6867454879676517, "value2":2359279288069431907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350225, "value1":0.6867879732162172, "value2":37306184729818872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881388, "value1":0.4406197726220826, "value2":724124821973247241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.950564, "value1":0.1322036312488787, "value2":4198819767892784951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.666395, "value1":0.2251980228042216, "value2":606006971675717615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.344497, "value1":0.029652052432054064, "value2":5846707393638380751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.703773, "value1":0.6376539079560296, "value2":6681127841717987695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295458, "value1":0.5526352586149329, "value2":7043209016069060231, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435502, "value1":0.09713433968048614, "value2":1019638678991819627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598132, "value1":0.963189125504533, "value2":6458016500274290636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627640, "value1":0.4641061727729553, "value2":8689246494106761338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254760, "value1":0.8326486356175677, "value2":4407797931839765907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805732, "value1":0.6423132619728287, "value2":6833967417402080897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.326823, "value1":0.9067956612338547, "value2":2841002873709302821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.088104, "value1":0.06965395874499884, "value2":1789284717431249823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.948712, "value1":0.9884448336440479, "value2":1290858986155893057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511709, "value1":0.8105556502461225, "value2":3470144572708895882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163730, "value1":0.8658645130208062, "value2":6466367002189635673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260153, "value1":0.5010858345134921, "value2":555883752624156654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.274796, "value1":0.8267418905611275, "value2":7590347778036203754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338150, "value1":0.7906313079639156, "value2":7837780129396014797, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938845, "value1":0.03371523590847688, "value2":2642591228031199330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254822, "value1":0.42059549793106427, "value2":4614016824012787995, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003989, "value1":0.05975709544316542, "value2":1382695454770978636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.577151, "value1":0.8937534168973357, "value2":6521815205949311706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.253675, "value1":0.40184282199800475, "value2":8286880530272716169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205638, "value1":0.7633247932732301, "value2":6913337973948968095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939093, "value1":0.03322965819693288, "value2":2797917596305577577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248192, "value1":0.5307465178536894, "value2":6134584955455781673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_9":"j","key_1":"j","key_4":"c", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.593487, "value1":0.05419262497736568, "value2":9190929989995856492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.182515, "value1":0.035610068777040586, "value2":6469776877211677470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160124, "value1":0.662164487805106, "value2":8768604308735587138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317998, "value1":0.9757873511779303, "value2":1199801679672428467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922339, "value1":0.9554985330074205, "value2":8286943158249961765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023973, "value1":0.01669695080382125, "value2":4408497776354893523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.399088, "value1":0.37811424022789003, "value2":499643435878747492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.974576, "value1":0.6392617923906977, "value2":1832499551397243524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423426, "value1":0.1431493881970496, "value2":4016805275891833987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220477, "value1":0.006541995474356055, "value2":5608461794122610089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014455, "value1":0.20051860544075079, "value2":4298831902740539404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771702, "value1":0.9711913949733121, "value2":4548136488165596312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837484, "value1":0.3199900890395432, "value2":7619719494433917806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632963, "value1":0.8179930116909135, "value2":42611344634328983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739321, "value1":0.8571068088303897, "value2":1929086780074260377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169110, "value1":0.2241172770388991, "value2":3015343857994873164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558214, "value1":0.5245963976733786, "value2":4636974583954671385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551826, "value1":0.8529460688457188, "value2":7676686801578586657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317389, "value1":0.34597315939279244, "value2":8526544610592186971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131556, "value1":0.9023875402465826, "value2":3147425851236763117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330907, "value1":0.7187683220932978, "value2":6304935909987923644, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419527, "value1":0.3464275048663091, "value2":800017209533647436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325070, "value1":0.32409707654758807, "value2":3093719017131899877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094166, "value1":0.6576755239099957, "value2":4375841020494708801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184361, "value1":0.454784333846635, "value2":8511384704032602153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424067, "value1":0.7746556148202395, "value2":8409657959538616305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.282726, "value1":0.19274194173178774, "value2":4484284290811930974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.044679, "value1":0.17881938606509734, "value2":2991918820431220404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622917, "value1":0.5058188298556251, "value2":2680090310035550156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.686600, "value1":0.019991692028205214, "value2":8990563930001611676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.977493, "value1":0.8045660581528546, "value2":3965120458727174879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.910280, "value1":0.15532301386230277, "value2":2751777764504449883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981678, "value1":0.6634934084628059, "value2":3338856320416894701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_5":"a","key_0":"g","key_3":"e", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099077, "value1":0.4991967563162609, "value2":7696501412266603819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_3":"c","key_0":"a","key_1":"j", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169242, "value1":0.7573800949559079, "value2":4682651688122452104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.854077, "value1":0.38315712266426233, "value2":7277684317066282125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017721, "value1":0.43030519598600153, "value2":8176728734487897933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449823, "value1":0.0905672633924478, "value2":8038615141368276704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505199, "value1":0.29347749204963386, "value2":5939807103722962422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.429412, "value1":0.3821527042310373, "value2":7616377560194630586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645364, "value1":0.31800894756504583, "value2":5324276910842211792, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789951, "value1":0.12346800778619027, "value2":7282935021388766519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.005934, "value1":0.4652775738745803, "value2":5150985627139247681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770208, "value1":0.8318069314007921, "value2":8693296121103057557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.036795, "value1":0.31504722243353955, "value2":5332439897432100886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.557676, "value1":0.04547498827953966, "value2":1747115516814035342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295720, "value1":0.25765308010583876, "value2":7172964053161542723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.862280, "value1":0.6690305275979546, "value2":8530225967231964009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464755, "value1":0.015096554191777058, "value2":2158727717387461667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902724, "value1":0.3729818170388733, "value2":2937474665692773587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.150211, "value1":0.9010548939518067, "value2":8689173712211659171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502794, "value1":0.637718327451074, "value2":1058416603563820737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756334, "value1":0.9093486580620052, "value2":3914139843013624351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109763, "value1":0.3482544181233465, "value2":3984218999735374107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_7":"g","key_2":"f","key_3":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.007357, "value1":0.530193826214922, "value2":7784257234281286192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133042, "value1":0.14994561696634548, "value2":6913972686763727632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.499117, "value1":0.9693770819840257, "value2":292923979595115495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.256986, "value1":0.8560085418753359, "value2":1633179891830212156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348017, "value1":0.4525195870224779, "value2":1498113059165586202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.389164, "value1":0.4747861877903318, "value2":5663253941505962378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.712219, "value1":0.23770447169372633, "value2":7127281555547067699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799547, "value1":0.4771589732318728, "value2":2601851094636806560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761432, "value1":0.9902875537117704, "value2":2441735233721216793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.594915, "value1":0.9327820854039164, "value2":7435962671657247487, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.018250, "value1":0.35410376862709275, "value2":6088069648662464618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.640810, "value1":0.6282310495408681, "value2":4206520774588356560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929377, "value1":0.30194565379846455, "value2":371174649294622523, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.817275, "value1":0.4745483085572834, "value2":2810603774900787473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.046061, "value1":0.9540450542751275, "value2":6567731530813672613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976385, "value1":0.2747180574854192, "value2":7052984344810070260, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833030, "value1":0.8111270330849011, "value2":2406611793520084033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146154, "value1":0.019440584474240373, "value2":1431804086688415352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541319, "value1":0.42910271299796143, "value2":5008131544913964869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.478974, "value1":0.3992297014608565, "value2":6638251650936294767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.591553, "value1":0.9631516171967569, "value2":6328755963396316225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.836675, "value1":0.8925642028256048, "value2":2169296529295363291, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_6":"g","key_3":"e","key_5":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240233, "value1":0.5601423681829663, "value2":2372802153601657174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926558, "value1":0.8669054217402387, "value2":8750941328640682000, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.785340, "value1":0.9294643708923529, "value2":8967392703697598518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321372, "value1":0.6020812110006013, "value2":1942102877471692500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578919, "value1":0.17388396608421544, "value2":5054354296681821565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830121, "value1":0.06504953435269434, "value2":7812868363877570134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110821, "value1":0.12907416408116543, "value2":1398896999021006979, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318471, "value1":0.6665917244504806, "value2":5816507337427212221, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881196, "value1":0.35509697330525436, "value2":3799008703902351704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503152, "value1":0.686811705668727, "value2":1281918055642507847, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.393488, "value1":0.8980510009425112, "value2":3157487117727981952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993113, "value1":0.8339917765115039, "value2":1528182230124691520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441452, "value1":0.5723022066521427, "value2":8611586563060872543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902474, "value1":0.21553093452664723, "value2":1591141288904852990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.937761, "value1":0.5975557556565823, "value2":1778079885483684894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.509879, "value1":0.8107703383515009, "value2":7338854303608988296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321562, "value1":0.7839743525651934, "value2":407341572079351661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_5":"k","key_0":"g","key_3":"g", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.083126, "value1":0.8340311472033722, "value2":414058498508156341, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.266082, "value1":0.7848792912531005, "value2":1487975780946804498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.001354, "value1":0.042658133657431034, "value2":2686477010080712855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984348, "value1":0.12772415816911367, "value2":1744454398245616628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.545130, "value1":0.012749865379431171, "value2":3614758343709591135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.211075, "value1":0.26931470974880833, "value2":5335597373851226907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517993, "value1":0.8277823010534222, "value2":441076351209513172, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.783905, "value1":0.6446774753163863, "value2":2444284131966329588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140755, "value1":0.7307559658816861, "value2":1750453360454837174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.243464, "value1":0.5611645445748624, "value2":5031122288522829688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247597, "value1":0.26059857605419207, "value2":6822156384989813472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053905, "value1":0.6298269719196294, "value2":6661136217666523640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.058995, "value1":0.7306532383370615, "value2":5311315770055711299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030527, "value1":0.08351380687475202, "value2":7155821886727737087, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.498371, "value1":0.2731396400770269, "value2":89197923125259866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397662, "value1":0.0870595280214176, "value2":4020978501682465140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173834, "value1":0.7945264755888277, "value2":3882559205765407193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497055, "value1":0.09364593584875654, "value2":891682366412177732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065765, "value1":0.835147994300049, "value2":2505979555724543511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.765901, "value1":0.992206355507561, "value2":5788507149763623428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_6":"a","key_2":"k","key_3":"h", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622912, "value1":0.44608293916585673, "value2":946502417712586156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366924, "value1":0.9938167708003688, "value2":7200121887993962206, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.072114, "value1":0.001073896039030629, "value2":8685581835197772410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265774, "value1":0.30130760063683054, "value2":8411046196525683207, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.749589, "value1":0.9239863104020335, "value2":4219975890726145199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190061, "value1":0.9734172533312344, "value2":8137079207203857103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.519945, "value1":0.07187194342466732, "value2":7825744617647803155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.649495, "value1":0.6408236940193898, "value2":3048798030528052662, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320456, "value1":0.5081060875649162, "value2":719352320345764599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476698, "value1":0.052307893890373255, "value2":2742427547083463898, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_8":"d","key_2":"i", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571531, "value1":0.4132003718642955, "value2":6484766831533191693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682702, "value1":0.44751287991255284, "value2":4055042383084807325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_4":"e","key_6":"b","key_3":"i", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.890718, "value1":0.33741266801359987, "value2":6934783766842267527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452747, "value1":0.3663537726308516, "value2":3312655545498739922, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199996, "value1":0.15068464159094366, "value2":5665646131255744951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.983795, "value1":0.8775221552243728, "value2":9069927522091992178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.583294, "value1":0.838152847228304, "value2":1035319401212201491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212587, "value1":0.8012210929096193, "value2":5585326063344558523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641779, "value1":0.9247211843697717, "value2":2117637859665911236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.406293, "value1":0.6198872109290668, "value2":5911640262763562610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.277897, "value1":0.027423430536495416, "value2":7628618445567261117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.832579, "value1":0.5331161695907243, "value2":8305462638082561375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205268, "value1":0.018199222166683245, "value2":475925606235710366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.615075, "value1":0.9915416739226851, "value2":2204646069144435635, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_6":"d","key_7":"e","key_1":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424612, "value1":0.9757938651409276, "value2":6266511548869428739, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286078, "value1":0.0846398426026624, "value2":3081995387268873440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_9":"g","key_4":"f", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787171, "value1":0.8365446036859473, "value2":5979460348923075463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381115, "value1":0.9123835513620147, "value2":3398482109219075054, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.114157, "value1":0.804291931301114, "value2":6012424958858906101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134911, "value1":0.00736667824808292, "value2":2400000632738793301, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164780, "value1":0.21287762980212552, "value2":8645521771535465924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710538, "value1":0.7411138160092244, "value2":8685307998868002992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585644, "value1":0.08786909591629084, "value2":7343395510792995060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125543, "value1":0.7332366768127239, "value2":8370868046930361421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.932834, "value1":0.9015644945676353, "value2":5236314268804910193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093732, "value1":0.497896904599941, "value2":6397325408101186539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486123, "value1":0.5097658314843312, "value2":1337466807316653828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904889, "value1":0.6556792189077274, "value2":4516335409033295305, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.387498, "value1":0.9910872699779568, "value2":1557997601588004148, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710931, "value1":0.5132845499506883, "value2":3319795795293053140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.515215, "value1":0.049432502275940204, "value2":2300709145422274852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821813, "value1":0.6825139883510115, "value2":8194491426131939536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865373, "value1":0.7044541486613806, "value2":3703555177588303304, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837580, "value1":0.8572176121238669, "value2":2344100903459752932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314910, "value1":0.4347186053772807, "value2":7676829676819086499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.946948, "value1":0.7950628854157453, "value2":2267010392465955325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015107, "value1":0.8624602552899475, "value2":9148330689311945183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488041, "value1":0.4618418780870866, "value2":8123711349904092581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710726, "value1":0.4226915062812881, "value2":5491521860827839646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841455, "value1":0.9825903973183027, "value2":3077499782633025057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.592689, "value1":0.9193972918242918, "value2":929238328644383650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993922, "value1":0.14695440679147787, "value2":8882303717901413025, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_4":"b","key_8":"b","key_0":"a", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731320, "value1":0.9934902241842611, "value2":1321453009633438522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604377, "value1":0.24174531205655925, "value2":5870091064000157680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628164, "value1":0.8064136026821553, "value2":8786888366338651141, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.484350, "value1":0.5079394885547678, "value2":6920245211342331930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978330, "value1":0.9354707910737625, "value2":7581764658078201917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632366, "value1":0.23595321295755245, "value2":1511878176205605753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142194, "value1":0.6306719216610874, "value2":2431345756658073693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397195, "value1":0.7323027634538346, "value2":8521159495965572956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.996298, "value1":0.8484427529278864, "value2":7246857551445151219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.328211, "value1":0.8531255802404514, "value2":4506921974707672140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308884, "value1":0.5324739320542478, "value2":6505796324920546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_4":"g","key_9":"i","key_1":"e", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.721813, "value1":0.2030510736245931, "value2":493933611483820748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325138, "value1":0.6318807488600625, "value2":3427677815511198978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.717950, "value1":0.13850468444662895, "value2":3714125763364851426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984959, "value1":0.2313158857459622, "value2":6179178857252455230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767352, "value1":0.04737365094297706, "value2":7312736110130720225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517326, "value1":0.38308887661557994, "value2":4397262624851606256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537458, "value1":0.8630190277228629, "value2":1405251953788249605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.589024, "value1":0.6853812314292282, "value2":6568591835038822877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013121, "value1":0.9022971605517441, "value2":5391410131187089227, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585652, "value1":0.027029007730938633, "value2":6270487841856618503, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.069148, "value1":0.5452016240009335, "value2":7230673973301658030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870318, "value1":0.8893099847439015, "value2":2072311642337421510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.525589, "value1":0.9489296395713415, "value2":1934395693866121179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.411127, "value1":0.5543622037481519, "value2":869973100557162310, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870905, "value1":0.1429227422154167, "value2":1580405259981052384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214057, "value1":0.007127408341584659, "value2":5021161294907689526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324636, "value1":0.2886203159980936, "value2":3536603792281429135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566190, "value1":0.2435195285039059, "value2":7314231811000308948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.302446, "value1":0.3969373764014023, "value2":861549335043188440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265270, "value1":0.8392596283575425, "value2":229012885919848214, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306041, "value1":0.6755326135952276, "value2":687305007623331029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.920842, "value1":0.14122582181388352, "value2":8912506495776009399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970414, "value1":0.32375225588910955, "value2":3874667148173594242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098047, "value1":0.5080753238940934, "value2":259459341225408666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814708, "value1":0.5851579994151463, "value2":4279032101744191170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857725, "value1":0.8750883468056723, "value2":8160658677842295703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288649, "value1":0.3318827172603761, "value2":8934632128032959148, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.651119, "value1":0.31525481325301247, "value2":8244510685212193822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065672, "value1":0.08637516713839134, "value2":707728667641140855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620357, "value1":0.8983386747310453, "value2":3756826533006983142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707292, "value1":0.4495030238963459, "value2":5996025393915497140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.443961, "value1":0.930457120039426, "value2":6519425791992110608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109167, "value1":0.09168467762421384, "value2":2057706949013098262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.481603, "value1":0.3529978146205129, "value2":6544227882618708766, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770746, "value1":0.6378677748113686, "value2":1182963631296444077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967875, "value1":0.8756904550723122, "value2":516292034493240162, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324624, "value1":0.03689035176673176, "value2":6723940798795470811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556389, "value1":0.7611742354824225, "value2":6343475929372871725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538576, "value1":0.1188977043392838, "value2":1679649620772949796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_6":"k","key_7":"a","key_1":"j", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.531730, "value1":0.2793539457328025, "value2":8309265375086957584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488778, "value1":0.8806255322454197, "value2":2637346612964090286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174493, "value1":0.4058829790367517, "value2":4908420813809791761, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.875676, "value1":0.8084081237917962, "value2":1370185119560523418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214353, "value1":0.22148397653369978, "value2":3829075843579764833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402832, "value1":0.9983967808941802, "value2":373850521427271723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.574399, "value1":0.1403913922108672, "value2":1025325851242540472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_4":"j","key_2":"i", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538957, "value1":0.5637672549308663, "value2":180146514999582015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043198, "value1":0.6621042931314224, "value2":4672271958823003044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.455391, "value1":0.37463255563131875, "value2":7050585404363795282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118353, "value1":0.7019235890065948, "value2":6561897677595087821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562842, "value1":0.4216296582302376, "value2":4671864964861835857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.613979, "value1":0.9101669225413574, "value2":5476069179621211969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079079, "value1":0.6285830126630634, "value2":1800090753316916399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990168, "value1":0.4228502254109105, "value2":4769618752656230984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630399, "value1":0.0593879368852638, "value2":8425926359679957872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381514, "value1":0.3170447182462336, "value2":134775839032388004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306533, "value1":0.09290124329204048, "value2":3878589204829724343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363988, "value1":0.7494592976903763, "value2":588119535129085693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.726401, "value1":0.2843848594033847, "value2":797824336547840313, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.425192, "value1":0.6747059449207641, "value2":6618945540166900762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486703, "value1":0.9234732601693532, "value2":7284867992059891857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.162651, "value1":0.4998767239554866, "value2":3758664262196916666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638502, "value1":0.14222618431131662, "value2":7183266992692325102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814797, "value1":0.04744514500578937, "value2":8807075724873574935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617563, "value1":0.7113712918981732, "value2":7369990382580999103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391047, "value1":0.4667345592294561, "value2":2593217151341486604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423210, "value1":0.2993388901152061, "value2":7736063295280636221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.233156, "value1":0.2669621811468438, "value2":9163725732269873210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780343, "value1":0.4836321272043857, "value2":6239122865745239252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270794, "value1":0.8124904828437979, "value2":5664361933553425625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_2":"k","key_5":"k","key_0":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.567390, "value1":0.1509148834766825, "value2":769811157679024909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502511, "value1":0.3496329905819851, "value2":8064435486484275503, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.995318, "value1":0.07944294849213016, "value2":7290799161307742730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160027, "value1":0.9323372972664894, "value2":8694402707307208350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265068, "value1":0.7804375599856387, "value2":2823069570137165166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852790, "value1":0.04269382129150226, "value2":4247785862022220763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.813337, "value1":0.3041270110008107, "value2":1162341534698170174, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_6":"a","key_9":"c","key_1":"k", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821104, "value1":0.05863251524747595, "value2":5517633377182805397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801029, "value1":0.7039209651517294, "value2":5777192245942090986, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.292328, "value1":0.0789099713548181, "value2":4341141001609491414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.650140, "value1":0.6547556555322551, "value2":488100188330068974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881952, "value1":0.6384170556890851, "value2":7393134503131526080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019410, "value1":0.4521619097561246, "value2":2730473646049580719, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_7":"b","key_8":"c","key_2":"b", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413658, "value1":0.35836104075629194, "value2":5913006016396086189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.388542, "value1":0.347328329631466, "value2":3415671885617145536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908514, "value1":0.682932258250745, "value2":1535326911373174401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453424, "value1":0.011430118976046925, "value2":6615057602694952188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661892, "value1":0.04490375141966574, "value2":3586691570128338490, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598897, "value1":0.9840071764206562, "value2":6756637664735447989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.614741, "value1":0.45336506256132186, "value2":7160462685010833845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959437, "value1":0.33491802794663117, "value2":4263644093884006866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394710, "value1":0.09508520954191016, "value2":8086669194950309707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.400389, "value1":0.748914928293926, "value2":7561318401530983897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_2":"a","key_7":"c","key_0":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551520, "value1":0.29033239272123923, "value2":219973352444233301, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.433012, "value1":0.8026759332563937, "value2":1035833541375117050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.097876, "value1":0.3123782080487283, "value2":97482352559342166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971923, "value1":0.09350074090215915, "value2":6603209689656348295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385293, "value1":0.025071498526727982, "value2":7821611042801827653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.582177, "value1":0.17895122462677535, "value2":4851272893829158392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715577, "value1":0.12145793831580125, "value2":6773868244129033160, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_9":"d","key_5":"e","key_6":"h", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.245667, "value1":0.5083106874300092, "value2":5342790051223313569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.191947, "value1":0.7213188287503117, "value2":8184500849109232134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.527410, "value1":0.2848785581313827, "value2":8059297953437098619, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023841, "value1":0.08963330011439892, "value2":8744048608508050406, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.861975, "value1":0.9569428255590289, "value2":5863440324678791040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_4":"h","key_6":"i","key_0":"j", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.020139, "value1":0.5328792585389814, "value2":6851298079925452135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383316, "value1":0.6736870667139099, "value2":1034982274545531879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490723, "value1":0.7069141316227696, "value2":1238860616186948989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774619, "value1":0.15017350186839917, "value2":4145739713631541991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.724625, "value1":0.41136382513830944, "value2":2470375246542099625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014004, "value1":0.45322459474776666, "value2":3424033124770728498, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370213, "value1":0.7936480778398273, "value2":7651653355869440262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930663, "value1":0.28375382304907193, "value2":4573094024293120951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.108477, "value1":0.2514498975622512, "value2":6565807048098733104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318641, "value1":0.09067602148053568, "value2":2118726105929309608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341849, "value1":0.5146408700742566, "value2":5544935195040202056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016279, "value1":0.2887329883061575, "value2":4016937139552222061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696532, "value1":0.35345697419681527, "value2":7198819764217760607, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356409, "value1":0.6101596793669094, "value2":2338966817998610022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882128, "value1":0.8757983585066079, "value2":1264113150578508254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214730, "value1":0.04033669277556926, "value2":2311999714610864651, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125040, "value1":0.8510776359004306, "value2":9203972476999185975, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617466, "value1":0.4880211576700948, "value2":7876488006268555933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.580824, "value1":0.738663404243217, "value2":6159559973328325759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.894051, "value1":0.0394639970488642, "value2":477392450960926806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112539, "value1":0.11952574252776728, "value2":2511563277039044978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.918404, "value1":0.30833608301592924, "value2":8563798643952041377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385588, "value1":0.17478857683593713, "value2":6552271330519870932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139243, "value1":0.42492669927426513, "value2":7691897138631144930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715207, "value1":0.8647651137464607, "value2":8497862595581854195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760625, "value1":0.24627316716260067, "value2":1409311771773591601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.880305, "value1":0.7511958117841795, "value2":8663780143091433962, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.121986, "value1":0.503001087851081, "value2":86404765376501910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.378220, "value1":0.7154879418930685, "value2":4699613906263091859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367776, "value1":0.3357733171654874, "value2":2109430818036060968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051853, "value1":0.46132527877007656, "value2":1864815246368296251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503984, "value1":0.5450705451757059, "value2":2199582755716092199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270031, "value1":0.15857947720543178, "value2":5111440735129620852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376563, "value1":0.08077432861716789, "value2":7964637453463198864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017382, "value1":0.4697079883099492, "value2":6862028548685305626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938781, "value1":0.3698227408392578, "value2":1642534448554432249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.869838, "value1":0.4366942978360045, "value2":4007380928114996020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.309900, "value1":0.05509289000926218, "value2":8881026026944444737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321699, "value1":0.5508642725465765, "value2":1065357963175430930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242134, "value1":0.05194281440569991, "value2":2955861771024227742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394641, "value1":0.6802778418350491, "value2":7829903676952590571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.011752, "value1":0.2679251298930048, "value2":46390372055260426, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156821, "value1":0.8759623901682069, "value2":6663991521677358276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621515, "value1":0.8461818248462066, "value2":4793482631115954367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.906690, "value1":0.4454231810933155, "value2":1968123317376605778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833508, "value1":0.5580322939662696, "value2":542437654648170086, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.864441, "value1":0.13709821675852493, "value2":3775094265545831122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349796, "value1":0.5463888647509524, "value2":3843833923457334958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210241, "value1":0.13053751777334896, "value2":2547919115148938351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.534237, "value1":0.13104275846768723, "value2":2757438080188257397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.479346, "value1":0.6936882347927468, "value2":3593937018182399578, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013374, "value1":0.647079383488511, "value2":6027310083597491507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353768, "value1":0.5713686637635252, "value2":3396180021694291926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704388, "value1":0.028813882790100154, "value2":6631675215659864441, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350375, "value1":0.09828506398906811, "value2":262432636815220332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130984, "value1":0.05874581283090899, "value2":5455068607213223389, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269711, "value1":0.16897643510231944, "value2":2188619802248362391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798909, "value1":0.49628972013165096, "value2":8871333460189078193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314472, "value1":0.003776179814725882, "value2":5465069878443166467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841977, "value1":0.016019153138836187, "value2":5090055543864076033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.431741, "value1":0.4503254906693805, "value2":3437452976536184232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021144, "value1":0.7157770425720729, "value2":1627465028436054242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.492677, "value1":0.170722769795088, "value2":5333195898301269961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.572483, "value1":0.07518223050679318, "value2":749837753058622914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.070547, "value1":0.7819934575507779, "value2":1257841836293236196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834097, "value1":0.7836504674507244, "value2":7822820024554629611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.725415, "value1":0.6205675290840149, "value2":7546425291277331498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756846, "value1":0.4577877981753711, "value2":521428696388054794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551591, "value1":0.4656287234062602, "value2":3666151433771478256, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367967, "value1":0.3303667849726779, "value2":1336037347437447384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124888, "value1":0.9362913257587775, "value2":8565731764409689287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356964, "value1":0.763103278688178, "value2":7384734667573077593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.336557, "value1":0.18917586865284508, "value2":3164250330171533065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_4":"e","key_1":"c", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947920, "value1":0.7279862820897369, "value2":5585150661666129475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320214, "value1":0.02069659588289354, "value2":4096509130796828102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339884, "value1":0.3585015534256289, "value2":1745908303614221947, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736733, "value1":0.6300320540961489, "value2":4976479107859231308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843896, "value1":0.8618710521424175, "value2":409303658417169135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351779, "value1":0.31536023315738654, "value2":7469097912657923197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146297, "value1":0.47961745828082863, "value2":120290133475149079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.299654, "value1":0.953430875530432, "value2":1804061300635059106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.653880, "value1":0.6130300298494126, "value2":3600820249700194803, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.241886, "value1":0.4261506936601424, "value2":6410123065530378279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.719862, "value1":0.5557094068088962, "value2":6619604443134241152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192985, "value1":0.4372446351220488, "value2":5419318851099904514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_5":"i","key_8":"b","key_0":"j", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.062254, "value1":0.35078106654250335, "value2":6337545142680530695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_8":"i","key_9":"d","key_1":"f", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.826832, "value1":0.03911586835937395, "value2":8510739605077299393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.466577, "value1":0.8386295972027331, "value2":3712956826274262169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774816, "value1":0.2874652343932132, "value2":1661160295946815788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497396, "value1":0.37671291600810797, "value2":7809177916397546184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.061112, "value1":0.6756138122485794, "value2":3821150629117787840, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.711530, "value1":0.9726549946550717, "value2":4361793701491206585, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419723, "value1":0.8422780123510716, "value2":8387473970870580835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740430, "value1":0.3687691277865257, "value2":49347396348935758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544648, "value1":0.34026181496926355, "value2":3168616117094841931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.031693, "value1":0.35157464321590376, "value2":6956034409157435335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682310, "value1":0.4968537230062955, "value2":1715909753935447521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571570, "value1":0.7444971868018181, "value2":773947913317871760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412817, "value1":0.17790194335965473, "value2":453971816672635527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154029, "value1":0.695415820029077, "value2":4412491024371312379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187573, "value1":0.025946190955721947, "value2":8268616070900349780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297634, "value1":0.39036160577230816, "value2":9117936615987168462, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882305, "value1":0.6498141826508745, "value2":1021392582780518091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288985, "value1":0.6094023317572287, "value2":5522861781820654768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.516532, "value1":0.4913308899910666, "value2":2877526282810678571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.663856, "value1":0.6052287079408516, "value2":5244892044841001586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177812, "value1":0.8518899507803316, "value2":4611087097036165557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_5":"f","key_7":"d","key_3":"c", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789804, "value1":0.9253854154062459, "value2":4746855558380871130, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163177, "value1":0.054868844952025875, "value2":4671327502125519320, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753554, "value1":0.0480631824565462, "value2":7078677034582099346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537888, "value1":0.3970428718849328, "value2":8317481519599523582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887867, "value1":0.9625491057630078, "value2":851265403943296561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739174, "value1":0.3876147534715181, "value2":5378120884849587893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263149, "value1":0.6578316627430515, "value2":199930804061246062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926006, "value1":0.9018811245039401, "value2":8736925807617681839, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559532, "value1":0.017637548071686292, "value2":2677656350646305112, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.940793, "value1":0.36532273109204866, "value2":7505680103024683972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416010, "value1":0.7581313246147069, "value2":7561931289424116286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130651, "value1":0.31761722951334315, "value2":7760092958155535983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330390, "value1":0.3514672157667692, "value2":1963737467345840913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731094, "value1":0.5668209441148129, "value2":6834927574629705768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003464, "value1":0.44994944703983886, "value2":1268238147301227399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488880, "value1":0.15568001025824393, "value2":3802105015027414434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099752, "value1":0.49578700566450296, "value2":6660233803101117286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131762, "value1":0.8279103415624635, "value2":8189053738835694283, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.818080, "value1":0.7683869146409037, "value2":4901733131698699140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502240, "value1":0.2102196895388282, "value2":8419378719359168551, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130382, "value1":0.00017527290277822845, "value2":7209832064279367432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697720, "value1":0.977742512542472, "value2":5886772442159349223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152164, "value1":0.5452125862262094, "value2":2301031936204454118, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_9":"i","key_0":"f", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307306, "value1":0.27827706723478657, "value2":8114605959354081308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.687815, "value1":0.407696242982513, "value2":8251434258695514324, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.957403, "value1":0.9248822561074296, "value2":5435740408050442827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297405, "value1":0.735268728902142, "value2":6544542317901773513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505182, "value1":0.6434041992526011, "value2":6723613820733288827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953853, "value1":0.08239495901880306, "value2":7797622216144272838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.816223, "value1":0.026034555587018983, "value2":8247070392223609361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781228, "value1":0.009350455430296063, "value2":6344110503212444401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993318, "value1":0.7246275363827985, "value2":1117977296074531052, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.346481, "value1":0.04745532854484892, "value2":8841440982349072666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240566, "value1":0.6244142217945766, "value2":4416869640928141403, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627039, "value1":0.8121528941309705, "value2":2364077225142023978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.605014, "value1":0.041614775390040444, "value2":5587679092002855017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_2":"f","key_4":"g","key_1":"d", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.675250, "value1":0.22265983684713628, "value2":852855496341453985, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.626117, "value1":0.13978101794521924, "value2":4898929345737475246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.272053, "value1":0.8981952631418685, "value2":4155867998384044766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216853, "value1":0.7930982218994412, "value2":1990843980684870249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231757, "value1":0.11381414833270587, "value2":5629821232198202573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967616, "value1":0.9325867021628451, "value2":6458012454076604601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.105465, "value1":0.20026280910311015, "value2":1519145667363005058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959715, "value1":0.8692424311015335, "value2":3441946116265084781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.916752, "value1":0.09650654250745302, "value2":7435925512873628832, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314127, "value1":0.5339606052113064, "value2":8918601923652673265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955659, "value1":0.21237559939985842, "value2":6700000271323408810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.462571, "value1":0.8031758318689627, "value2":4200089724641662061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704628, "value1":0.42388353536260726, "value2":1676230088334959718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781871, "value1":0.7853951443498608, "value2":4677884824803639265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093020, "value1":0.44592927039430036, "value2":5106772041668223293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.066421, "value1":0.7167334468150509, "value2":916960599462889530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753333, "value1":0.8855172765987326, "value2":3498076904045993942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263558, "value1":0.039242126444255614, "value2":1994687158267196250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.529319, "value1":0.11390721956642542, "value2":6770255083967052135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.642302, "value1":0.25917237536080345, "value2":3561054032858646761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.800222, "value1":0.5648139239977721, "value2":7859467413386922358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.714908, "value1":0.3172565417584279, "value2":4018189432904936827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495891, "value1":0.1699695672081134, "value2":2951696383925672800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.927247, "value1":0.5193998219352067, "value2":7461947996252867047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465786, "value1":0.04561481590524083, "value2":8852624735734959732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970044, "value1":0.9502884615051707, "value2":5544298343674474830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.224365, "value1":0.7835642667167426, "value2":7989661444310811565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970171, "value1":0.044357973684653176, "value2":6240172232063272107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.022705, "value1":0.2909464675793184, "value2":6791113566169749215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.791002, "value1":0.30758766636926194, "value2":4087253744537420622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.526835, "value1":0.26218100233396685, "value2":761773481307699191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_9":"c","key_3":"d","key_6":"a", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057757, "value1":0.5941262990661297, "value2":5203993881422592468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621587, "value1":0.9726557103153585, "value2":4065263619832011048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936496, "value1":0.4220278511438201, "value2":537814119030374569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476922, "value1":0.7453981222005723, "value2":3086149467770264289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544208, "value1":0.1648752329298873, "value2":1528219915951406823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480134, "value1":0.5864292944345278, "value2":1915038657623169872, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.634200, "value1":0.06085546420380519, "value2":1586772996290355012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.290157, "value1":0.4479966729250899, "value2":603667588545059551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523397, "value1":0.8019975659364824, "value2":2263229113306312715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260677, "value1":0.49446211956319075, "value2":5384294748496824064, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.040570, "value1":0.2634688640512974, "value2":7081899407358309984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.670159, "value1":0.4242108328120063, "value2":1003697559007887859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.181699, "value1":0.7357308047023233, "value2":7674299813279760629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.437960, "value1":0.34081697732260335, "value2":1366760825001014571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_5":"f","key_0":"b","key_3":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585340, "value1":0.9880202469237767, "value2":5993013748473116382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758920, "value1":0.528974137063132, "value2":5387464413435497334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_4":"c","key_7":"j","key_0":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190405, "value1":0.24309504115260316, "value2":8699961182370479292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.045104, "value1":0.42560682798506067, "value2":3749517767528292571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.239270, "value1":0.11837048427737668, "value2":4852849985587057450, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647950, "value1":0.24581023177518593, "value2":7946632180521405772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969355, "value1":0.36754556888022905, "value2":7897267252653597746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416692, "value1":0.9421287451192997, "value2":7881806374967586751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.052040, "value1":0.27660174901670026, "value2":5294738319793885165, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112174, "value1":0.8996109424352536, "value2":5125031699148488781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696416, "value1":0.42330986040349045, "value2":7061433961041368821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127933, "value1":0.6743423459526591, "value2":5703788929519053213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_9":"a","key_3":"k","key_4":"h", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421658, "value1":0.6878542321395752, "value2":3553548366690488868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697493, "value1":0.169783750302659, "value2":7055001682108205827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.550811, "value1":0.35231569219578235, "value2":1455572534223159241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.493018, "value1":0.08084254027780423, "value2":2900266808751124367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949248, "value1":0.8847972628692381, "value2":4051840894997158475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142802, "value1":0.3270345064861823, "value2":2685380632911712697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.004387, "value1":0.822755239334881, "value2":7165522026622587078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620895, "value1":0.40285181555826477, "value2":7279952302942542040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.988553, "value1":0.42488091295486163, "value2":7195769969726296289, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314916, "value1":0.24528560160904253, "value2":3412797507763244235, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_7":"b","key_3":"e","key_4":"e", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740841, "value1":0.7678978195926474, "value2":7353627394146413053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767489, "value1":0.9849856324971001, "value2":4180092134684532856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.681082, "value1":0.08407929345646925, "value2":2247545351887675892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987608, "value1":0.3158564571415852, "value2":3372134575580371247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953355, "value1":0.6808029749520165, "value2":6416401290847025754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.934118, "value1":0.3815229371385279, "value2":5398231187188543220, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349522, "value1":0.008036202059570214, "value2":6632834525194225583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852636, "value1":0.6082938917169154, "value2":2209452475312580755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.491711, "value1":0.7070391677298287, "value2":5867779644024249420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329284, "value1":0.23697436114335285, "value2":2747492895656100802, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217160, "value1":0.013558152161070156, "value2":543215008088311315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376202, "value1":0.3967940240541657, "value2":4348396634707940652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134419, "value1":0.2570940106942101, "value2":5495331795379375428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.693581, "value1":0.6383501980446401, "value2":7332225459965791621, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_8":"i","key_7":"h", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843591, "value1":0.374550163160159, "value2":1957979143426115517, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127314, "value1":0.16562410193780275, "value2":5514782476287105847, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564210, "value1":0.47834402715548674, "value2":7864140566183633471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.673777, "value1":0.656108656648855, "value2":6919590747679310147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.071957, "value1":0.3923477172589352, "value2":7954657800672945455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497858, "value1":0.21899200976683347, "value2":5922902740218283725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063776, "value1":0.819166087262725, "value2":497846112853863498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.039234, "value1":0.06262728093114268, "value2":976242972769343091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658955, "value1":0.48196452599993433, "value2":2417102300700963835, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255288, "value1":0.13639741045424703, "value2":7715812700116132347, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.496112, "value1":0.1915457705996175, "value2":7008066197109595718, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144334, "value1":0.011357361010319224, "value2":7600237475494790795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.129843, "value1":0.45829887966275845, "value2":1540110795133843915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.828068, "value1":0.8516996075016723, "value2":8405984298577097957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870240, "value1":0.16467566190225996, "value2":1556056753506941522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199050, "value1":0.7512917767548035, "value2":3201410658564101147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830765, "value1":0.6115828253610872, "value2":1648965358200873623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385351, "value1":0.44894635523974136, "value2":2127367263922899732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.555613, "value1":0.566989554108829, "value2":4275387093192470829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758445, "value1":0.45045350568682, "value2":6540058212119502734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051410, "value1":0.07872483972360628, "value2":7661444664176600116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.027756, "value1":0.1866583235803357, "value2":4402492796625360355, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_1":"i","key_5":"g","key_0":"c", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043704, "value1":0.3399629891033194, "value2":2843620218251838442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.218394, "value1":0.6792705695737844, "value2":3717810518285734277, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.489979, "value1":0.2797515798899019, "value2":5603667040320572095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394844, "value1":0.47076120967369356, "value2":2551470543073926967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.176040, "value1":0.28649026862589944, "value2":8089352485313652968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015062, "value1":0.8503661846365486, "value2":9115866206649167217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.867138, "value1":0.18222497145716, "value2":748036164126661849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560015, "value1":0.16282129139411827, "value2":7758713833273415824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782776, "value1":0.3488004874020465, "value2":715341283187680350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.547186, "value1":0.07214699982761937, "value2":8071660169425611932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383243, "value1":0.8304505195235525, "value2":4079430116382080627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551280, "value1":0.6676351538773236, "value2":4287835647581016602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.853493, "value1":0.3478301116617329, "value2":7815908195604525371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511951, "value1":0.7049888722888338, "value2":1948135132689353800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173192, "value1":0.5458720430431239, "value2":5871330428575335057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413126, "value1":0.46553694703913845, "value2":7586060029018039815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781439, "value1":0.705616760521343, "value2":1127274508185282406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.732371, "value1":0.37205742400318453, "value2":5213749986182819618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569115, "value1":0.03375202087271923, "value2":118632103331380976, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.102614, "value1":0.2822023902913289, "value2":1391807523431966730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140423, "value1":0.44611401204912965, "value2":828680952801675744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.085991, "value1":0.7893667946565419, "value2":8411361994319110857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.975471, "value1":0.7162325854468116, "value2":79462509030833666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.244787, "value1":0.40407729683075916, "value2":2829176965864438571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.304099, "value1":0.2959757161094293, "value2":5764137406549726855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837106, "value1":0.4258452743236353, "value2":5341156407592196309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707606, "value1":0.031525280019894485, "value2":868621351492758022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_9":"g","key_0":"k","key_3":"f", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441177, "value1":0.3225860911008584, "value2":3070966375799941978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.077150, "value1":0.044429243112689185, "value2":8848387799149411370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604969, "value1":0.8353491972869558, "value2":5277008629054064552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537157, "value1":0.4103644112452139, "value2":9148697464094805661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353178, "value1":0.8020014291715372, "value2":611786907029670311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505264, "value1":0.10898025782239415, "value2":5937127177824419205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_2":"h","key_7":"f","key_0":"c", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.228410, "value1":0.43346447533111176, "value2":7102784791187505051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321463, "value1":0.9635846235850585, "value2":7091075060854093514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.999019, "value1":0.15889354502430938, "value2":7074771352556045066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_5":"d","key_6":"a","key_4":"k", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.766619, "value1":0.4943151198279696, "value2":6293058242069710093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382183, "value1":0.47579561094609324, "value2":2482575383850980092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.054382, "value1":0.7191240626399533, "value2":7914822293313169877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255873, "value1":0.055171339528865125, "value2":1928364223437432733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.535958, "value1":0.40417404005201557, "value2":8512533852699686666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394085, "value1":0.8317795195864898, "value2":6061937875766891692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.752931, "value1":0.1811338646540346, "value2":3103290849364510613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.049507, "value1":0.11124247410560267, "value2":8253232345223333946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.067068, "value1":0.02792863936331565, "value2":1942738411033963266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746589, "value1":0.43603502537000616, "value2":5153565802559198697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903064, "value1":0.3556713649253779, "value2":3489353708315351817, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.856458, "value1":0.28933348763855665, "value2":5247367677027973150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397179, "value1":0.7130148119357395, "value2":5728198037062323847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.927222, "value1":0.7639936733901646, "value2":101990787192301191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005808, "value1":0.03609120161396754, "value2":9216428818715898202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812165, "value1":0.4866254594918793, "value2":3026112033227979237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420851, "value1":0.7402262167603453, "value2":6177484091405440598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836705, "value1":0.0336701243488239, "value2":4778937227074309153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097260, "value1":0.6626429057561898, "value2":7257306969130399155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496088, "value1":0.36910009322506554, "value2":3657880390271983286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870639, "value1":0.7067162570632016, "value2":1102103625009499222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.020747, "value1":0.46587247219742656, "value2":5967339025562916268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.724139, "value1":0.8163768770313428, "value2":2067500216739340556, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.369554, "value1":0.7671983408631192, "value2":4937837819229078201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382589, "value1":0.94426076786917, "value2":575895696952869704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115327, "value1":0.47575759482061636, "value2":2541949700573197783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260734, "value1":0.8252666210219215, "value2":5316874961818228049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.872358, "value1":0.8794099168533623, "value2":5800876034859982246, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.054141, "value1":0.05686166977091221, "value2":5301631175885078161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759774, "value1":0.8076817854451119, "value2":2675452361460215935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921855, "value1":0.0467410376485931, "value2":7851926061831999904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578515, "value1":0.9029496688391901, "value2":3737429676030030627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812804, "value1":0.47887479306222563, "value2":1687173944192158201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472625, "value1":0.8441160499425835, "value2":1515286134892734672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760814, "value1":0.33274037254007843, "value2":1188458814270546919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788501, "value1":0.15942957594654944, "value2":2916736695606434573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098854, "value1":0.8030098284257553, "value2":2336601872331852302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894386, "value1":0.4139981184160349, "value2":7897864995270774175, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_9":"k","key_5":"b","key_8":"g", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117046, "value1":0.5674622805765219, "value2":4791162537839256650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.542843, "value1":0.19807908776500033, "value2":6807107659814067311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_1":"h","key_9":"f","key_0":"i", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703834, "value1":0.5788042510612239, "value2":5007564675868943302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945291, "value1":0.3701740719620502, "value2":2459541142030887319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070410, "value1":0.2627776276203505, "value2":9110732671786561088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.305776, "value1":0.24403108521016115, "value2":3139654072672113070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.417910, "value1":0.549110839930364, "value2":2406040915101429610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410216, "value1":0.8784609507591513, "value2":3467112924963980779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186300, "value1":0.13500098796508653, "value2":4869858265003017538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934357, "value1":0.7572145499314314, "value2":3619701318200032332, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871110, "value1":0.4402673304510715, "value2":5018886154821992035, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.493200, "value1":0.1348835909121323, "value2":3845890133642201057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875968, "value1":0.1364875892800979, "value2":570804136752641105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.469938, "value1":0.5776925075890766, "value2":8313002338216576665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.667608, "value1":0.9950302066492306, "value2":1913698187097493343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497144, "value1":0.7591363060077458, "value2":4304322015184203986, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731147, "value1":0.743025674257032, "value2":3480752580915944185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878760, "value1":0.5947269209561762, "value2":7998247484475194678, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.606309, "value1":0.663213778377581, "value2":3067341192985682212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.207494, "value1":0.8601914108563131, "value2":256790036338793553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.144417, "value1":0.30286956006124033, "value2":5377965606661311252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.134377, "value1":0.609971735912583, "value2":3774215546443285103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.434483, "value1":0.986170039019965, "value2":5064754870308124270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.509040, "value1":0.5522321797777098, "value2":5012812814115566068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903746, "value1":0.20330422291674133, "value2":6986910292987554191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282547, "value1":0.04694512830804489, "value2":6329857494201458927, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354275, "value1":0.16833606088021735, "value2":3136536256813456866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774256, "value1":0.5410085627082251, "value2":3587475793865363544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.762751, "value1":0.806841924275979, "value2":7358998298622408616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584082, "value1":0.6563492986452016, "value2":387593409104388354, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.080055, "value1":0.5509982288474451, "value2":6251069533288673484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695506, "value1":0.8632479367125148, "value2":2061552818267451584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489099, "value1":0.6265315870043634, "value2":4304737020249244240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098523, "value1":0.5339406052485941, "value2":4381041775949231103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945882, "value1":0.23785010902188283, "value2":6729562900414289120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.167154, "value1":0.3701120650676118, "value2":622381534969203518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775199, "value1":0.6184507954864329, "value2":5210419683928766855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_9":"j","key_3":"f","key_4":"g", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.730074, "value1":0.38121631002699524, "value2":419350096079979395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.013737, "value1":0.8747949109587233, "value2":1018960453244065142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.091852, "value1":0.629803885859078, "value2":859062102274314528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166974, "value1":0.47395530253799945, "value2":6411055007256508887, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.524472, "value1":0.9186585978945081, "value2":3492305084628232737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241236, "value1":0.3787867675534582, "value2":4760531059679736084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918712, "value1":0.05290704362542312, "value2":1966436853098488271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636045, "value1":0.9510956232200477, "value2":8704302710827007347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187946, "value1":0.6635616634234764, "value2":1433605612315108919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180096, "value1":0.173261531213523, "value2":3306945339384428282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.453459, "value1":0.11094008182846338, "value2":5724966973695631624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562127, "value1":0.3576118834730829, "value2":3463667757785807622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137341, "value1":0.6255329096036213, "value2":1381955744525597553, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913181, "value1":0.29162415237583095, "value2":2202030046648119531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.443734, "value1":0.3209051100037936, "value2":3274712051745788108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.302887, "value1":0.38589486316701666, "value2":1489273904023316120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205658, "value1":0.07820373463872678, "value2":7665901880569799873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.211623, "value1":0.32477633025580727, "value2":6808357456829544888, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.697819, "value1":0.26244647649977987, "value2":6232754250974380613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853360, "value1":0.3310597786987246, "value2":7565924568483154342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.135295, "value1":0.049613381396899345, "value2":6155724101399152110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701749, "value1":0.9716574010135762, "value2":3704214151049295904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527554, "value1":0.6532141955951636, "value2":2786469958733724605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866043, "value1":0.03422399382505321, "value2":5496039574170303988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.415307, "value1":0.682109162147481, "value2":4924263240012975638, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631078, "value1":0.7274897566111301, "value2":8520095366826456731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.549725, "value1":0.7077865773466269, "value2":2977543758339035237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.154150, "value1":0.14439730391589514, "value2":7810498955756317569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007570, "value1":0.8087681893470107, "value2":4017899853890170979, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.055629, "value1":0.44977291321386215, "value2":3453485470431150033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189056, "value1":0.012064619774392913, "value2":3564594888238224552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.545459, "value1":0.412297537176583, "value2":7021178975362987850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.116215, "value1":0.3276649448479836, "value2":3143364922694929733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529456, "value1":0.4708787441244093, "value2":2999813825732785635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_8":"d","key_1":"i", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.966668, "value1":0.04918146120370962, "value2":870818154766064140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601524, "value1":0.4723442759712491, "value2":5773596887056998694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712431, "value1":0.04260252713403969, "value2":4497318416983710614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452968, "value1":0.03699415334331326, "value2":3190104546355404213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.394341, "value1":0.05586282401938057, "value2":2019354346274899062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.084367, "value1":0.35383668982583555, "value2":14041413767842483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227017, "value1":0.7278732120270098, "value2":8901905120302064125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494493, "value1":0.0910027308098097, "value2":3614458258424175747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059547, "value1":0.2958045744232146, "value2":5813283620430694858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.423389, "value1":0.9221509815116762, "value2":6742495498273189642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501855, "value1":0.44198471706869047, "value2":7662744388645110665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293650, "value1":0.512834114916101, "value2":5984393328938374467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614743, "value1":0.8714098760733937, "value2":937734757736172502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406459, "value1":0.17711840072487878, "value2":749048739837121472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711112, "value1":0.5465711310503617, "value2":2943773951314932011, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335697, "value1":0.8515828942170133, "value2":7615557480020888685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548405, "value1":0.13026413723454033, "value2":4513263208882824306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.351448, "value1":0.28264554746804893, "value2":1701229548455066314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501558, "value1":0.7474271542447126, "value2":4300562906938627060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107125, "value1":0.5220893826927367, "value2":3433346642469038441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968552, "value1":0.32921550916944375, "value2":4204134061287881453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.438315, "value1":0.5786006279022659, "value2":7016921001360584415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.834753, "value1":0.15159677478724748, "value2":3404421917920731295, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454269, "value1":0.5292362536342466, "value2":6241633082661695873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137463, "value1":0.4976726181585776, "value2":7634359338923847776, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744484, "value1":0.7709542443985684, "value2":2984932547540133289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.653649, "value1":0.40168424113567774, "value2":9208485552143870835, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420303, "value1":0.8834331384252292, "value2":6602388659890449882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.995276, "value1":0.35489802779155194, "value2":3627299399432728520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.609717, "value1":0.32106453173709887, "value2":6997589541390821763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.764384, "value1":0.448976213692369, "value2":2970007228856945436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548393, "value1":0.15204462152678552, "value2":7646037909450468928, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379101, "value1":0.7807002939639928, "value2":783999596388900221, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.278309, "value1":0.014487780492841778, "value2":1057643982300790379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.016491, "value1":0.40903973612790867, "value2":453932377401006766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347001, "value1":0.5622168932310977, "value2":5705011066230310463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.086911, "value1":0.6494793319729653, "value2":861844073039964568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.997904, "value1":0.8338228010696771, "value2":1349114457177401043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266213, "value1":0.43023507437229846, "value2":2397780341037792730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068462, "value1":0.6058806411980994, "value2":78381759723174420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358282, "value1":0.2854157607268251, "value2":9047800747480831869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845468, "value1":0.009988708005244632, "value2":2440593977353687165, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780415, "value1":0.6187822246822943, "value2":4482602300599359335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_1":"g","key_4":"k","key_0":"c", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700223, "value1":0.018684194191095932, "value2":6529685586040726217, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.792345, "value1":0.9111940168010332, "value2":1575039147844860051, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275696, "value1":0.01410734034670126, "value2":8535583193313743852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458977, "value1":0.23636701604224677, "value2":314843807550918715, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_9":"d","key_3":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711597, "value1":0.1315269488618045, "value2":914748291669479577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003734, "value1":0.48566417416368673, "value2":5809881749485776988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.587116, "value1":0.411572893699433, "value2":582776826400065747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000207, "value1":0.25509895950578154, "value2":5802214523620869062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082969, "value1":0.48435989736978513, "value2":7352735247711683766, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.042350, "value1":0.16157141460949973, "value2":1607874054697622469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.765374, "value1":0.15664293959132114, "value2":3675493379037113802, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.359224, "value1":0.887036256695556, "value2":7786059543942584229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293533, "value1":0.267785564160632, "value2":2723125994900979360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964294, "value1":0.716239726237387, "value2":8651041612345523274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076213, "value1":0.9749455558859894, "value2":5297461003834502762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.896241, "value1":0.7102104646771853, "value2":2795778846362770396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903120, "value1":0.14414851964547873, "value2":7334142904098197, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.843431, "value1":0.8361555205773405, "value2":7976205336110881786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731615, "value1":0.4179919061811569, "value2":1554283199870929425, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.953374, "value1":0.7537780463182495, "value2":2591672750977463865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400999, "value1":0.7471726171159019, "value2":7300350647929857982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232296, "value1":0.7720730243870363, "value2":1060384392061020703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219798, "value1":0.16805543570986609, "value2":1462620146278820176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364330, "value1":0.7255148218772242, "value2":713753994079127610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152830, "value1":0.13410318940561705, "value2":9140011796352402602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585566, "value1":0.4130103851752984, "value2":2970454230901035575, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.718094, "value1":0.5500539331076559, "value2":8305617631847453693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.624058, "value1":0.9500416769324029, "value2":2935568409193402069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539303, "value1":0.05177136725388866, "value2":7442175860869173746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.172986, "value1":0.4456449462659514, "value2":137655062794989901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631420, "value1":0.9356892224700029, "value2":8946945917645913041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948627, "value1":0.9988907819529511, "value2":7978995836658846886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946323, "value1":0.5613474012283862, "value2":648588536937617868, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361054, "value1":0.4585620050578102, "value2":5573215420419545492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.817729, "value1":0.3241298418342054, "value2":2657202509964663855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_6":"i","key_7":"h","key_2":"b", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713670, "value1":0.7338301625220077, "value2":7217765288314019300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.521076, "value1":0.5030076209647331, "value2":3777966789579493364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.855317, "value1":0.7172152098421418, "value2":8060266815999977878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849239, "value1":0.9561486509896251, "value2":266727525021228946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008199, "value1":0.11740640397171373, "value2":3689690733597943394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936773, "value1":0.795334335887194, "value2":4733404788345141419, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_5":"d","key_7":"k","key_1":"g", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888974, "value1":0.9298030625919304, "value2":6088282595701162644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247054, "value1":0.1558941325474189, "value2":460928608228918469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853182, "value1":0.8511278083029166, "value2":5598797292292719359, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.252753, "value1":0.16156992511009938, "value2":4368635811765896864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309449, "value1":0.5849973411963314, "value2":683197153873995001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.222187, "value1":0.6637363404506509, "value2":201431956551451886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805299, "value1":0.03082292423828661, "value2":3135350466088347105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489193, "value1":0.47023015827549575, "value2":301183049195776618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_6":"b","key_1":"b","key_2":"c", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.058774, "value1":0.7620086190415307, "value2":2178333587571299612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098933, "value1":0.696442402358351, "value2":427216344556980093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497026, "value1":0.18415993654905655, "value2":8786493859199944811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257640, "value1":0.6674105512961352, "value2":395753979079709412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_9":"g","key_1":"d","key_4":"d", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490944, "value1":0.5114071451614219, "value2":2042645288775439509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455445, "value1":0.7431063214848364, "value2":8165863124378358117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.796800, "value1":0.6414858212054014, "value2":2442066548061613486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616766, "value1":0.17517079750483502, "value2":2134452812266479738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060207, "value1":0.5103041206672632, "value2":6515531846051600666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677324, "value1":0.2542456767596465, "value2":3968114223247313347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.487523, "value1":0.6903207412936282, "value2":1424416428597884416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601220, "value1":0.7953800597000228, "value2":3247496941215159385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.642959, "value1":0.2541441152051556, "value2":7121652419723619875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268832, "value1":0.025435318199453064, "value2":8277309445690585191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591576, "value1":0.98502525758626, "value2":3261712292707439368, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696131, "value1":0.602844278741062, "value2":9150721424292798196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.799050, "value1":0.6397730903214256, "value2":322841440821332477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703698, "value1":0.7179811740811757, "value2":6911091771320522730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.276968, "value1":0.16623158682521033, "value2":5763688015485978321, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.932237, "value1":0.9953773456479392, "value2":1648531648235955285, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562810, "value1":0.4786357292651346, "value2":3349388012721867031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496221, "value1":0.10819101220823944, "value2":5211051877655840528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720431, "value1":0.552698162040604, "value2":7021843927329324470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.087213, "value1":0.8472571863880398, "value2":7745061885728922306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.039167, "value1":0.21533794157440472, "value2":3121229544727240655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309356, "value1":0.1590321315931928, "value2":710479249184112134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.191661, "value1":0.040375310871212516, "value2":1055365248220450266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189905, "value1":0.8222460270476257, "value2":9109111690041797580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517425, "value1":0.7168387126170568, "value2":6945048447223853682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916881, "value1":0.37193142727878464, "value2":2747126651524473231, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.395094, "value1":0.6414572510838905, "value2":2905014786439561176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.163189, "value1":0.7311236407483467, "value2":5070208982215207577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827333, "value1":0.19235777061627696, "value2":1042609530408390018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107479, "value1":0.11539837467269841, "value2":7629188841800355160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186100, "value1":0.23799692868617986, "value2":3114291156883869525, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790442, "value1":0.4974549500784167, "value2":3142531162087829163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.797183, "value1":0.252808246395566, "value2":7867371410034903023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269019, "value1":0.4459897405815389, "value2":5769972800266784623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299688, "value1":0.7900084672305991, "value2":4400086899085085411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.295251, "value1":0.4992500267841033, "value2":8698415822608825668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.992903, "value1":0.9703688948654993, "value2":699484569689785049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694010, "value1":0.24590491974037657, "value2":6378096972799959498, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916779, "value1":0.03667685970441408, "value2":8236016681410946158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.018930, "value1":0.7762719211488223, "value2":5653992369569699642, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673243, "value1":0.9005498092358467, "value2":2541259280078576557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.403564, "value1":0.7066442393226117, "value2":7468478788293853391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912162, "value1":0.4479784585493673, "value2":2451129253364350697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622846, "value1":0.6985884362308626, "value2":1822728881940552123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707469, "value1":0.052057120254889885, "value2":640515677830640949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433576, "value1":0.4097754058261397, "value2":2475392911974322330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374459, "value1":0.1353067477342801, "value2":7368967313707141232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_7":"k","key_3":"g","key_5":"g", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993651, "value1":0.06711618906287087, "value2":1750993237587336816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219084, "value1":0.5316119999916927, "value2":5790597415731545344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527236, "value1":0.6470232539834205, "value2":5706809590317379695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566174, "value1":0.9745838789323467, "value2":774234656544468494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803752, "value1":0.8708197206158679, "value2":3775873873449515760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.388879, "value1":0.4862649086023325, "value2":8962983929778901050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471944, "value1":0.25578979603582636, "value2":7732635688969517720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017485, "value1":0.4968624551232297, "value2":202940911271524303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_8":"c","key_1":"f","key_3":"b", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.333780, "value1":0.8299780273551013, "value2":895214629529935810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.164082, "value1":0.9321061863901389, "value2":7407307236544459993, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225068, "value1":0.9163062543062077, "value2":8739271132887488724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304547, "value1":0.45077586944816295, "value2":1933302507685789986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.538110, "value1":0.4128481092985977, "value2":3178182543294139274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541173, "value1":0.7483924955860858, "value2":212944257013368531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255386, "value1":0.14438260405328707, "value2":7645966424158688784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.073000, "value1":0.7628193459385832, "value2":4973597494190231390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408853, "value1":0.5564603800067697, "value2":4401411283807625564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447512, "value1":0.8939878640344981, "value2":2693396599053914337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048017, "value1":0.34230394362828026, "value2":3182689568942606164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057151, "value1":0.8042880864594656, "value2":2051969816473375198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746476, "value1":0.7332341667442952, "value2":461488002348760082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_6":"d","key_1":"i","key_4":"h", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180890, "value1":0.31049348146155, "value2":1020023460971269956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550108, "value1":0.4362790210713837, "value2":7901529415931806303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345598, "value1":0.11823802615751057, "value2":8356166230167153323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804625, "value1":0.5643553567149021, "value2":2154734444228835687, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_8":"c","key_9":"d","key_6":"j", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247313, "value1":0.6975496564473892, "value2":3820052334665018388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805735, "value1":0.31084346436090265, "value2":4462974662487547374, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364439, "value1":0.1744566217814337, "value2":607910964577919864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589313, "value1":0.37010824428777245, "value2":5796079422541877022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931293, "value1":0.5772458152930102, "value2":1027336122468786461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527679, "value1":0.2977670251954975, "value2":3769527680254605539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.404475, "value1":0.38496439646172187, "value2":2284351654690866371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772590, "value1":0.07775425303960136, "value2":8477700234367515830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722168, "value1":0.17927054370632736, "value2":7192360571641392544, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712775, "value1":0.576069645932529, "value2":4594581975868935835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.126652, "value1":0.5061094013712955, "value2":3728550795113093627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321145, "value1":0.13357161718693558, "value2":588806792143157793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.101550, "value1":0.6394371793311785, "value2":3045802217092318405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.547834, "value1":0.5303880277608923, "value2":244812322497139670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_5":"f","key_6":"f","key_2":"k", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.022447, "value1":0.813105541632356, "value2":4932627202785941294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_8":"b","key_9":"k","key_5":"b", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382289, "value1":0.3508103161551219, "value2":2376325264202595095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.611464, "value1":0.9618639190404236, "value2":2197184281244482985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.911985, "value1":0.3625352195059055, "value2":211893969967676184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268188, "value1":0.6899041268799436, "value2":3263903700628732234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229932, "value1":0.7830409765107325, "value2":3581295224187821255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921675, "value1":0.15132831757259205, "value2":8941349375775958121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.738768, "value1":0.28818505448700765, "value2":5311365998633458584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.670459, "value1":0.7891122563117983, "value2":5569764792017894880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.893664, "value1":0.863874518524225, "value2":4812640494718764785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484101, "value1":0.6065100518640419, "value2":3350972220626617309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048572, "value1":0.15248106917852358, "value2":6593923637797377418, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.043823, "value1":0.6456350127530551, "value2":3888964250592362865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109495, "value1":0.6845812758644145, "value2":167613517010834758, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.328450, "value1":0.42643027428907515, "value2":8129659398848977292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.429304, "value1":0.14015794848590404, "value2":4681962038728020905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.776267, "value1":0.3289665984746857, "value2":3318083720052608693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496512, "value1":0.3624675397742157, "value2":6289410424653796398, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441324, "value1":0.9094233421689666, "value2":2912289985138568601, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.650459, "value1":0.3752516427539381, "value2":4398293649161961566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.350383, "value1":0.1741603700667637, "value2":3124160867383954120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.051974, "value1":0.5242492694985263, "value2":6760802540060503552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_9":"i","key_5":"d","key_8":"f", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.291840, "value1":0.5576830705012547, "value2":4155187805029620559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853505, "value1":0.13344813988799406, "value2":7220245991757698014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_4":"b","key_1":"f","key_2":"f", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.480822, "value1":0.34390971406763976, "value2":4047856794594370906, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192205, "value1":0.46496172552445103, "value2":2904944897204510913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668616, "value1":0.9800566015891223, "value2":3768818727218430516, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.940469, "value1":0.8502278446477077, "value2":5643864232780485221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_4":"c","key_9":"i","key_0":"a", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775796, "value1":0.26560945264856717, "value2":4386556505603475589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_4":"c","key_9":"g","key_2":"f", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484224, "value1":0.10363743205437767, "value2":7594526116903338826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.889136, "value1":0.4627114619785784, "value2":4680730415934145352, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739229, "value1":0.1719548250883568, "value2":6803717844294234894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_8":"i","key_4":"f","key_6":"k", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057765, "value1":0.3081555912049451, "value2":8499651141401941047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.450693, "value1":0.8168675860307004, "value2":7949382588313472478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770167, "value1":0.272960033553786, "value2":1543554906153054869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411881, "value1":0.5252975573521564, "value2":7223448923269287286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.892105, "value1":0.15152104472471403, "value2":3531358618763240055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.882618, "value1":0.45516649656010977, "value2":2288125701245215412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913392, "value1":0.6620512228658078, "value2":4363674990969315932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.835393, "value1":0.7513525395048825, "value2":5883431192279251653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.563057, "value1":0.9008981984383352, "value2":6465724162584986904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007001, "value1":0.9502622268086611, "value2":6989543064471041687, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688255, "value1":0.07451084769674933, "value2":2957263551734111550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751629, "value1":0.2617947338456188, "value2":684618542420066244, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930787, "value1":0.6308597904360509, "value2":6601900249320434792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.363243, "value1":0.8139409689083822, "value2":1497160325123262680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676085, "value1":0.9711732869605988, "value2":1463546357374490230, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.161858, "value1":0.6374277110955211, "value2":6435065381978315913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757941, "value1":0.910443361797238, "value2":2854035357843167779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.331899, "value1":0.6895825352637633, "value2":621471764809456095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473746, "value1":0.2844187612168192, "value2":5704986401969616545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048060, "value1":0.9984260267119236, "value2":4771344600707329825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.030112, "value1":0.9637634957448155, "value2":5568130719531472451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257160, "value1":0.5940271387669745, "value2":142865511754737819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760326, "value1":0.4334581523321684, "value2":6413870941968139402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827687, "value1":0.8136794802681319, "value2":746965665674196204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447680, "value1":0.3168283761550826, "value2":7144161377877398366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690596, "value1":0.3996428785716089, "value2":3598801694226963836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.009204, "value1":0.21753660582669873, "value2":1790271338088923331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.432520, "value1":0.8760470338201594, "value2":2329654518443264597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558450, "value1":0.5241764449812006, "value2":8062760419700144076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.999165, "value1":0.4211253053940594, "value2":6241537222130424287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.492382, "value1":0.19419908429465416, "value2":8132562796409382183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968488, "value1":0.8740668460370395, "value2":5462263580744941212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566020, "value1":0.3116629548456918, "value2":1660846180499855539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899292, "value1":0.4455952655758791, "value2":5641720897227183391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213999, "value1":0.7107285930376039, "value2":320006539559036783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177859, "value1":0.6189254938745993, "value2":3916861895859871884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.942059, "value1":0.3464916414346185, "value2":1739217875853687994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637964, "value1":0.4027137396066144, "value2":498622351647583844, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994610, "value1":0.7530828834500288, "value2":5890892402021233384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.727487, "value1":0.28182955033112955, "value2":4630833361364112333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.915590, "value1":0.5470194104324091, "value2":1666728635470642191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757932, "value1":0.5371051329548212, "value2":1493513093655584392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.254302, "value1":0.06977656180754654, "value2":5979425073849108121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.475877, "value1":0.8550799756217883, "value2":624522672976326814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247089, "value1":0.39743079305559637, "value2":4830779617580090488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015595, "value1":0.2577347123153436, "value2":2406835300488043720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945719, "value1":0.3129335496013211, "value2":820286347352483926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081969, "value1":0.6051672450326654, "value2":8261558360059795606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948207, "value1":0.7815897187663656, "value2":3429909037501105433, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.935696, "value1":0.909809980891043, "value2":6385383912360129741, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.248852, "value1":0.9247408795385116, "value2":9175721939467544545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934738, "value1":0.6334884200328613, "value2":327826698570505025, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102837, "value1":0.44571438142580455, "value2":4747306485524936018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.885921, "value1":0.35994074928816305, "value2":1395907621788196782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.949368, "value1":0.7895021421389461, "value2":7949753331449968910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.633040, "value1":0.5458972679689471, "value2":4245189254820710416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.150358, "value1":0.7548359500337994, "value2":8481176564399105584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.557167, "value1":0.37427466994480085, "value2":1133639012241894316, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.093134, "value1":0.29043113295756745, "value2":6947797053182077820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_7":"d","key_1":"j","key_6":"a", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.136231, "value1":0.7872857259243544, "value2":2466956534940008420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.329501, "value1":0.01079935258997052, "value2":4968522926518293278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.825131, "value1":0.6767719760068325, "value2":8529334985450219291, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807971, "value1":0.7351117134307579, "value2":3019346910002173461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_9":"k","key_3":"f","key_7":"a", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017959, "value1":0.6253725541571399, "value2":8835344453005145545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_1":"f","key_4":"j","key_0":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.280599, "value1":0.16481336771423033, "value2":1417072867431315310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673036, "value1":0.8178932673930933, "value2":6530240293167023011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192297, "value1":0.7505738676572953, "value2":6861957449422419505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517236, "value1":0.639000099787322, "value2":5815570450456127187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739883, "value1":0.17493563192408712, "value2":7179940437785343272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.531912, "value1":0.08360001327419847, "value2":2755543678440921421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452307, "value1":0.8726532834145122, "value2":187164282595967676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517272, "value1":0.9444444024596198, "value2":5607680802025895328, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289453, "value1":0.15619031798053584, "value2":810705789880173134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397508, "value1":0.9544477853731341, "value2":7641027031682219689, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812107, "value1":0.9448725135844366, "value2":5145031592575451663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.063548, "value1":0.922662738511914, "value2":2569888172146766439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.959005, "value1":0.7947728545231653, "value2":4794325670292386969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.570203, "value1":0.9426031560649749, "value2":8759901536059040166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.514218, "value1":0.47519036709767065, "value2":1247765910819397634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541370, "value1":0.3774216285225394, "value2":7667033726860743169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.938282, "value1":0.020047520852319107, "value2":7849085976338159942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874020, "value1":0.47048043771282444, "value2":2196635070664843443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_8":"c","key_3":"a","key_6":"e", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513588, "value1":0.012232986779526358, "value2":8478348320905895865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109700, "value1":0.7338145643706604, "value2":6797416603289211588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.488779, "value1":0.6031093288566471, "value2":5300319596464382751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931594, "value1":0.42339447548233033, "value2":969762807855801088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.553860, "value1":0.676076748018672, "value2":1192139536623982501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.132006, "value1":0.6728558661354047, "value2":1380355336108501777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.669226, "value1":0.1577792131508693, "value2":4512376080159929307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921683, "value1":0.24615483388690507, "value2":5797393083888487885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.800085, "value1":0.768303750055719, "value2":6971653838071612447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468961, "value1":0.45803974634578515, "value2":1199483949895079721, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356489, "value1":0.49471640885346424, "value2":4138046086256944727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225992, "value1":0.6741558407603926, "value2":5216648174057731100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.779458, "value1":0.11692394331277675, "value2":9011198363894466670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.683290, "value1":0.48772002686762633, "value2":7212512979389072906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053472, "value1":0.5939009672642634, "value2":1146669122452133702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790089, "value1":0.8261492714018603, "value2":6290707817110359790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145153, "value1":0.6394833078139336, "value2":7849898590496421466, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.367040, "value1":0.6490663681468899, "value2":3967326627567255091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_7":"k","key_9":"e","key_4":"j", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930388, "value1":0.6376629127038135, "value2":3197193413798224716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471623, "value1":0.06021884177520976, "value2":4773928340410962251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_7":"e","key_1":"c","key_6":"i", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201531, "value1":0.7136591237382468, "value2":6260011441045155926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064553, "value1":0.2818642070197672, "value2":2495939851413003901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282911, "value1":0.1452427125541155, "value2":8365230104234658997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279591, "value1":0.8354266947002554, "value2":1254307733558365753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849657, "value1":0.6912085697845711, "value2":1824933885382646361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_2":"d","key_8":"b","key_1":"i", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.092000, "value1":0.053974617818740134, "value2":8465803905922172931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408792, "value1":0.3821176162488874, "value2":4135301501032886839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.728999, "value1":0.9460512775720927, "value2":5232062325949200093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.706765, "value1":0.05808795299595995, "value2":3187473873599955497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689611, "value1":0.07108495627051868, "value2":8720638688812490559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.686555, "value1":0.11535487696002504, "value2":3943663806047625686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815986, "value1":0.6951101939977824, "value2":2103945278295322543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991212, "value1":0.9405521412873721, "value2":3787480236551923641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665833, "value1":0.21165663807431181, "value2":3904059491287927557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183315, "value1":0.5795855388985459, "value2":1854253684581147206, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344965, "value1":0.8721578775869344, "value2":2246100236955108961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671953, "value1":0.31034394563958956, "value2":193813594216774336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.127315, "value1":0.4080665667532778, "value2":360930115537783747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.947191, "value1":0.901998732832304, "value2":1708333379014319670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894585, "value1":0.9195635726221545, "value2":8084121448593915190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004792, "value1":0.9591461919610487, "value2":8675745838539187803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900181, "value1":0.4274569433819623, "value2":6055452196459435249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948861, "value1":0.954396067309185, "value2":2700832246849384276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_8":"k","key_9":"g","key_3":"h", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656208, "value1":0.4983661330975837, "value2":3291693517576289438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_4":"h","key_5":"c","key_1":"a", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.937253, "value1":0.2620189984734357, "value2":8181737649910995938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867100, "value1":0.06224158128949376, "value2":4580692356241748765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827749, "value1":0.03871136162211054, "value2":7089705182458313228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241402, "value1":0.4482242754297532, "value2":9114765255689214165, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.602059, "value1":0.29735202117340814, "value2":3162172555777612351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.249371, "value1":0.23918886296603434, "value2":9067747281684729145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725148, "value1":0.8102090769702104, "value2":4626069739522513533, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.928403, "value1":0.3838571333237863, "value2":8647736244601572769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358211, "value1":0.32922133279873583, "value2":8174411590575614648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347825, "value1":0.7167150468238062, "value2":8311537141574084420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875897, "value1":0.26919747077534717, "value2":9053609159881163713, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392772, "value1":0.2800052196883224, "value2":7203447031464069407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849662, "value1":0.5009846634201038, "value2":939832331440858045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.024201, "value1":0.9391704477843268, "value2":2885798586864811264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952269, "value1":0.2664477356829927, "value2":847075590517072325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.307777, "value1":0.6456685214669929, "value2":3218983200036055999, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_5":"a","key_9":"h","key_3":"j", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.832715, "value1":0.2591746701265793, "value2":3426357136923132270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.510375, "value1":0.5688430382533638, "value2":7025749889591390600, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.850762, "value1":0.6438552947023466, "value2":1197489157433338286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_6":"f","key_1":"j", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.905930, "value1":0.54095928495375, "value2":67603635153932551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636437, "value1":0.5854506016456503, "value2":5128251159570110333, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836009, "value1":0.4067978187295729, "value2":6225162557978433985, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.781439, "value1":0.8809907639802731, "value2":1945481468549859562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722225, "value1":0.6928638550983661, "value2":7984089816823103368, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631076, "value1":0.0305606186093776, "value2":3704333676945193488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.515877, "value1":0.03574457632343526, "value2":8116748669267665996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.062029, "value1":0.8880344844442181, "value2":7205869492721817899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.753934, "value1":0.5309770343574366, "value2":3276732533413239990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598973, "value1":0.9458250274999828, "value2":4588112807086674988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284154, "value1":0.6864088859089572, "value2":8516198907569397198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.337595, "value1":0.45387405891882765, "value2":2878690183576728505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743636, "value1":0.5411320007572493, "value2":7926291410401458702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751569, "value1":0.29015226479219164, "value2":550259652101408680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_6":"b","key_9":"f","key_4":"g", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696737, "value1":0.19318283293628083, "value2":8059270019889405703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558974, "value1":0.5527141281446059, "value2":4981805365285955545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.499157, "value1":0.3593055667554787, "value2":8763899128009608550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725437, "value1":0.38132949820890993, "value2":2593622500148824140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575337, "value1":0.27925006017882537, "value2":4225105729105614711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.168187, "value1":0.8455269167046058, "value2":4706079022430056900, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.223652, "value1":0.14768410638621007, "value2":2848574409182395374, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.655719, "value1":0.26992469602574687, "value2":4754721503564673004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972901, "value1":0.22308996900209793, "value2":4121883961065445917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513268, "value1":0.763713042109811, "value2":1389619580667003266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335708, "value1":0.9988121809461258, "value2":8201212630370607829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874837, "value1":0.37662051730821194, "value2":1239286216512679885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.761409, "value1":0.08352300660036191, "value2":788197175065166874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993753, "value1":0.08212627471454952, "value2":2431495613386540436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513306, "value1":0.09225555422185021, "value2":6434312790658522957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899387, "value1":0.2747071960330611, "value2":3838090571407685488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498947, "value1":0.19881833202736185, "value2":645491056031774982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851971, "value1":0.9432399401550566, "value2":3792284814302154775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912337, "value1":0.6898029391029513, "value2":8695381256661600831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060802, "value1":0.4672690888050405, "value2":509110642459148760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.988275, "value1":0.6949519126514042, "value2":2125445866842945317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.537780, "value1":0.9644299177479598, "value2":2844641921809856363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.129171, "value1":0.2870831250353682, "value2":5356758362083554180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.699225, "value1":0.7476204022496519, "value2":4681919372415529338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.798856, "value1":0.5178345021294356, "value2":8950909328927419020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356529, "value1":0.9689016091606751, "value2":778967583500360424, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.740297, "value1":0.48229504021118547, "value2":319778397891873168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082023, "value1":0.8672498335471771, "value2":2761738695533665878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.737582, "value1":0.21539065762206674, "value2":1173702480391317018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.203204, "value1":0.04428967569336166, "value2":6796467582468653168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197343, "value1":0.5038157716364858, "value2":5419554006840711744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_5":"g","key_6":"g","key_3":"e", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166454, "value1":0.4520211285816449, "value2":4140960458683685597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.810425, "value1":0.9556399667218088, "value2":2345821074968960700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708611, "value1":0.36562484484170277, "value2":7482183983302020104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614909, "value1":0.010690625631515951, "value2":4802163912353727023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529200, "value1":0.23500509922901588, "value2":1518336865591090789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957588, "value1":0.8999494296908893, "value2":5272580661109552970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900198, "value1":0.3844554073160095, "value2":4170365040112200144, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.579116, "value1":0.16747277518786968, "value2":6804977291625507700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250708, "value1":0.8577877748125339, "value2":590691746141274872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721810, "value1":0.7147800394287138, "value2":4035652902850373019, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.922402, "value1":0.0627394141655773, "value2":4185528067707010977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.198128, "value1":0.08114002676829289, "value2":2239252166892691326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803033, "value1":0.0019520065249596544, "value2":1312973954738137976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.174637, "value1":0.09909477348410087, "value2":93134409084713078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250984, "value1":0.49581172361364056, "value2":9059996107352612477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381289, "value1":0.5781636662002811, "value2":3051612121774242926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.046509, "value1":0.758530400177533, "value2":67989907532933127, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.933345, "value1":0.9339175035199608, "value2":377586629316226209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963510, "value1":0.7004527478348754, "value2":1390034556068597934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918093, "value1":0.17000680115237268, "value2":4215018843369459603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.569630, "value1":0.20175634343868196, "value2":8381338885007503558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.883668, "value1":0.06695323081792075, "value2":1589024156310802370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.178981, "value1":0.37314887727591756, "value2":518499450705948775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187208, "value1":0.4599180344115974, "value2":4887550148987301900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952934, "value1":0.8319538072999415, "value2":3671167315474130299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.588275, "value1":0.5426790448893685, "value2":6523903627951338788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.128001, "value1":0.37530546288075156, "value2":5616976404881722053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004862, "value1":0.8346907316991129, "value2":4019033006145719569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826250, "value1":0.5254457071340125, "value2":972903594117286740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344623, "value1":0.6999472357226462, "value2":1701492823670778543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639041, "value1":0.2248828857383105, "value2":4810585903388633809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.571165, "value1":0.22946219998225406, "value2":8055176566662404630, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117315, "value1":0.7792786347828345, "value2":8503346841194050818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.646178, "value1":0.29610828190559946, "value2":6707117461343778078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804384, "value1":0.8994527553958773, "value2":3055603759085197982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.503237, "value1":0.038643663095115535, "value2":97929435615191250, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.917565, "value1":0.06570063912325382, "value2":7659557506835508250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.194133, "value1":0.7280519586189449, "value2":607993872873285299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197005, "value1":0.34702978428971004, "value2":2119181822487481024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578715, "value1":0.4531709992663944, "value2":1719249508626109183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.221581, "value1":0.5476991652235295, "value2":8006956843834072386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177885, "value1":0.5137634978637854, "value2":5009892345362966157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596214, "value1":0.2368591303155911, "value2":8792401284340375040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182004, "value1":0.5444756866358181, "value2":1210706677095755681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972249, "value1":0.9909726775988351, "value2":1108183309913410572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225911, "value1":0.47194616841751835, "value2":3490783413049237610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321843, "value1":0.5905642407899484, "value2":4893835600551421123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.552992, "value1":0.8654702264058397, "value2":5106927081449112108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782381, "value1":0.8285431175840194, "value2":9168936255854371872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410849, "value1":0.31656794437122665, "value2":2071986113321022167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780581, "value1":0.051832044232056354, "value2":6484981440389875558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.072700, "value1":0.698857333675772, "value2":3443549406704488268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689951, "value1":0.5808073564555776, "value2":5048983612386176894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.733768, "value1":0.8029317643963917, "value2":7359595780765172048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364358, "value1":0.34011958562739014, "value2":999331274737013831, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.512816, "value1":0.34358105240557685, "value2":6714721903573983722, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711860, "value1":0.9821515464895118, "value2":5477696623940889135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.857930, "value1":0.697732523770638, "value2":6661494860267552531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_8":"k","key_4":"d","key_5":"d", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114607, "value1":0.8236051117026364, "value2":4748182480100358673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.628228, "value1":0.168082038026373, "value2":7102000252406916356, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.687012, "value1":0.3638576608952103, "value2":8540460395700338240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_9":"c","key_2":"k","key_5":"b", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169601, "value1":0.3111748596945155, "value2":8571165720141489367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498080, "value1":0.6187426468242411, "value2":2451438877433776073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866907, "value1":0.5131483959448067, "value2":8869138925582259535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.396171, "value1":0.28292656018705137, "value2":5391262893377663378, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114785, "value1":0.0649996585433473, "value2":3978180949228199287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671381, "value1":0.13833508369091968, "value2":5430930553797657293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.923816, "value1":0.8519197270200854, "value2":9127083562371495095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786218, "value1":0.02969493094352536, "value2":7671441472993176714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774561, "value1":0.2102387127527105, "value2":4039185760762390304, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192344, "value1":0.144819186927003, "value2":378491549245728191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076506, "value1":0.3187033255519536, "value2":8885020385104236250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213717, "value1":0.8128476179407741, "value2":2181064442818506560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528442, "value1":0.5936064804578587, "value2":5648031374498714859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.767476, "value1":0.1348520099403021, "value2":4074505568425329493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.449677, "value1":0.09144593933086395, "value2":8719212717808997934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411794, "value1":0.8990540517914553, "value2":8624156888058647365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.457637, "value1":0.3197628975782331, "value2":16763285997559821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.330356, "value1":0.8095806602517515, "value2":4384365921475656332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584148, "value1":0.16829289997155406, "value2":380241693738836834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455567, "value1":0.7529538853574959, "value2":1681989614930760275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.844936, "value1":0.6074576588778431, "value2":7722066461223215119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926267, "value1":0.5797545736876416, "value2":7883616781252732331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238789, "value1":0.014736618507623488, "value2":4669225171297242340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_9":"g","key_0":"k", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361896, "value1":0.17876601404118195, "value2":5115665659478830691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756049, "value1":0.04403510631259223, "value2":7660735889331080164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.031957, "value1":0.33256068212920453, "value2":4453878565561794031, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.828157, "value1":0.39237914319665085, "value2":7249969180646160990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952636, "value1":0.372690613847562, "value2":6192025452112268327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991675, "value1":0.5494769946982794, "value2":90791903671198766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240830, "value1":0.9786829569747533, "value2":5978162907079513697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279525, "value1":0.1494973578442592, "value2":6869066553359800738, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_8":"h","key_5":"d", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.939830, "value1":0.37432428722372013, "value2":4226472512491590319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.564559, "value1":0.7159468168647539, "value2":1512961341899043326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_7":"e","key_6":"k", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048658, "value1":0.5401236641884074, "value2":8347872809477134111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.340095, "value1":0.4533223235157631, "value2":1670286383442367211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293295, "value1":0.4258630060523724, "value2":5674260405699571928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.239916, "value1":0.9150796129317534, "value2":8474870548791768473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744309, "value1":0.6470478918019383, "value2":7704476841600519211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458393, "value1":0.05603920506738755, "value2":1942285693271412968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720119, "value1":0.6222144605927992, "value2":5009900948732974588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435194, "value1":0.6528463342414516, "value2":2372394551953155351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.847658, "value1":0.12821161771950726, "value2":1481519210091358924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.083324, "value1":0.3648356544882518, "value2":15076769986125901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_5":"h","key_0":"h","key_1":"i", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696866, "value1":0.5460078372418026, "value2":8482317071463194706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878357, "value1":0.9158893232321995, "value2":4112830082593015136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815751, "value1":0.7832798664333082, "value2":6621546193633794291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.036509, "value1":0.7258315650178256, "value2":2697123926599878172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786542, "value1":0.050969235189444476, "value2":960522431282362262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293909, "value1":0.1637483242537312, "value2":4063809715912746117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_6":"d","key_1":"g","key_2":"e", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960513, "value1":0.15412859037505927, "value2":3609332046279378552, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845902, "value1":0.9308703210994502, "value2":6609460372849170625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750261, "value1":0.12234055832157559, "value2":8912704681070762337, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458834, "value1":0.9318510467298435, "value2":8026158416345347023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859979, "value1":0.9301579137721765, "value2":7925354941740095610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.783146, "value1":0.4471773356574157, "value2":8481603802647763313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.505375, "value1":0.5948200403445978, "value2":52874652951483269, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005495, "value1":0.829951436170396, "value2":3721373420722024974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867411, "value1":0.6049905245498157, "value2":8221468316724652544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435857, "value1":0.8205378674820365, "value2":4321340481613778781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.618588, "value1":0.6552813364407445, "value2":3804139748589484208, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.027076, "value1":0.23841518035800105, "value2":7013010516589186218, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285111, "value1":0.5613630899205873, "value2":1517169132984045486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117978, "value1":0.3226349150452925, "value2":5930250973192218816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238673, "value1":0.398077529845209, "value2":8495752825324479706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_7":"d","key_8":"j","key_3":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258260, "value1":0.3301633034128501, "value2":3240643823549476632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.373493, "value1":0.3529131970207001, "value2":3035971070657682432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665486, "value1":0.1772430533425256, "value2":5999482327853244380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358796, "value1":0.18816869038874143, "value2":5134838279453664617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140177, "value1":0.8387362327402264, "value2":2310199526324609410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_9":"j","key_0":"a","key_1":"c", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059616, "value1":0.8521328351977854, "value2":8719119695607623072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121795, "value1":0.3605246283964201, "value2":1540950724302287517, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760661, "value1":0.21327358886251288, "value2":5483717677008529333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.019253, "value1":0.7234842919041711, "value2":3995891303198614565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468371, "value1":0.026572272284047523, "value2":2431068728438057213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397658, "value1":0.22598294609730654, "value2":2698499054966120972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755700, "value1":0.09282245685151808, "value2":1822149750384470026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304716, "value1":0.6358956357038871, "value2":1038515895167880998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673410, "value1":0.01619219159654624, "value2":999540262235584439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406798, "value1":0.2801610335950404, "value2":5737646822014765045, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472994, "value1":0.14109660013219333, "value2":5384594772062199779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601871, "value1":0.291815873607527, "value2":1847519318868439297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994725, "value1":0.6730562186913949, "value2":5489090928652625049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205639, "value1":0.42334867153762545, "value2":1823044221508369143, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.904945, "value1":0.598173556849254, "value2":1029832085092338611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507464, "value1":0.8462370553826191, "value2":8938945324991684299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105783, "value1":0.4362851240411875, "value2":8230814679005715011, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.130819, "value1":0.5311370726009594, "value2":2799369947502242194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.436621, "value1":0.5406911040823768, "value2":5927551572729154830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_6":"e","key_0":"b", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081043, "value1":0.9897107711937903, "value2":1409810039307886257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.360124, "value1":0.8898589942584547, "value2":1182666162202869439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473406, "value1":0.9785032169444227, "value2":7933797459692204307, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711755, "value1":0.8440901040738314, "value2":1266347640739805258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836845, "value1":0.13202825619743236, "value2":6494324863003452768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.814171, "value1":0.22371325434459086, "value2":3679612694652027947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707264, "value1":0.43001516107021553, "value2":4022260893191171434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585576, "value1":0.9028218878505339, "value2":3312996140868409539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_8":"h","key_2":"j","key_3":"d", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.253947, "value1":0.5596795145791367, "value2":9053991858276197521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.793355, "value1":0.655432892066219, "value2":2171965578246650022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003349, "value1":0.32642930281521576, "value2":4724329284506985861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532693, "value1":0.47384645211929527, "value2":166053326506373649, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258148, "value1":0.8755464478705562, "value2":1367687629182661630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284672, "value1":0.969983045672162, "value2":7709886208589610791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656648, "value1":0.7178009692517222, "value2":378528392759859903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.138190, "value1":0.17961568288211016, "value2":6086554307721297156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755588, "value1":0.7476161070292083, "value2":3339991721000583202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.160231, "value1":0.4667023272863336, "value2":8520356596039544954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888583, "value1":0.20049555103413333, "value2":2444783554472337568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008015, "value1":0.08008213078983399, "value2":3073434633965527342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094964, "value1":0.7132542439294741, "value2":5312205610465131789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.234374, "value1":0.3178507521158574, "value2":1341935169276159022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772701, "value1":0.9799507521983674, "value2":4679043722393447880, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.742787, "value1":0.8436883610291797, "value2":8206108321797356735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285496, "value1":0.02119049506813719, "value2":5643416354242611531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041734, "value1":0.8593746773495375, "value2":7177072845753511018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102405, "value1":0.258959468834628, "value2":5506364318868451234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.617097, "value1":0.5213134879577571, "value2":7820417886370593685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_9":"c","key_1":"k","key_6":"a", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614285, "value1":0.8586887753126803, "value2":146426955523604107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094475, "value1":0.3100760743003405, "value2":3216906708607385583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400149, "value1":0.40305327199936813, "value2":4063666163388559666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_4":"a","key_8":"g","key_1":"d", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.741707, "value1":0.07472258132971145, "value2":4583642984114208044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_8":"c","key_2":"b","key_7":"b", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289818, "value1":0.21965034622340104, "value2":548484284878385021, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788770, "value1":0.8645491106416102, "value2":6428112142936498186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589911, "value1":0.8352505870700639, "value2":8807067943589232363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980194, "value1":0.4222199616754522, "value2":1431823521353243628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347713, "value1":0.3792858568507692, "value2":5699682291213510899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790574, "value1":0.48263080283098414, "value2":3899441712388650234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.586360, "value1":0.39598813053011617, "value2":8951581838322949271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147821, "value1":0.696249186695191, "value2":5997811343683052027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_9":"g","key_6":"a","key_8":"d", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375267, "value1":0.30594268061015806, "value2":7589869429394133120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.662739, "value1":0.8061361815679017, "value2":5904903629197529499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_4":"h","key_1":"d","key_2":"d", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447051, "value1":0.37750215516426633, "value2":8371518067848222420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121391, "value1":0.6696130432092745, "value2":3528483712948148367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_7":"a","key_2":"h","key_5":"i", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114395, "value1":0.9303974176092045, "value2":539479413974722614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.440024, "value1":0.33141470193064243, "value2":492711817812030768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830103, "value1":0.7998316524208386, "value2":4274003622712511224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.343594, "value1":0.948502518427523, "value2":1092062054441297015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.820370, "value1":0.6468806610248711, "value2":5700638394649707417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169752, "value1":0.427387565910367, "value2":7968698722232869415, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.352398, "value1":0.008797452976395297, "value2":5389560588922529986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.574184, "value1":0.6925729377012323, "value2":7921127567234590177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708589, "value1":0.6874360024316648, "value2":7727336634389454587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.647250, "value1":0.2053256936728798, "value2":8730436625808972720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522110, "value1":0.7058378270168358, "value2":5838333049897839477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894605, "value1":0.3141493596388785, "value2":2076790219296988613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008782, "value1":0.590566630449788, "value2":571655682516576680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.424761, "value1":0.6098530285996386, "value2":7846500926381318806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591196, "value1":0.33824124551827933, "value2":646303391431941929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964204, "value1":0.7366285135213106, "value2":4137944794465811468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643081, "value1":0.3218928504433246, "value2":9121706025453589663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_9":"e","key_0":"b","key_4":"f", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804724, "value1":0.014940637092911, "value2":7451060690570027498, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694428, "value1":0.21460306796333273, "value2":2982813118003123474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_6":"g","key_9":"e","key_5":"b", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.122258, "value1":0.49140759630280006, "value2":8832276444519664682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.996796, "value1":0.2474526024511197, "value2":7613005431630356063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238269, "value1":0.8267663978098108, "value2":2724285849945025892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.644529, "value1":0.4649514742650587, "value2":7589769671176102226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.376035, "value1":0.7135202605627624, "value2":4131507979199286159, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958098, "value1":0.5742451996367589, "value2":7655662522626761261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478186, "value1":0.30259713893200063, "value2":7366343481003162441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_6":"g","key_9":"d","key_3":"a", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299728, "value1":0.5700777203840441, "value2":8077274544410191414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.685643, "value1":0.9403232361639245, "value2":809458262071959965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478225, "value1":0.7234775762369925, "value2":1256131637116314402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.445423, "value1":0.17591193935592764, "value2":6211570042356723652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.362681, "value1":0.09007902579380467, "value2":5590217114587646147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140908, "value1":0.3872422494802931, "value2":3824505998771737796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.312510, "value1":0.26492926145602147, "value2":2580605589364820036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639094, "value1":0.8695708814560201, "value2":828997494663746747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532778, "value1":0.8659642362585516, "value2":489767661854306328, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.332818, "value1":0.5085353492466338, "value2":7131627003266463948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616148, "value1":0.6665482572291218, "value2":9183191141357087143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807708, "value1":0.03703617074385954, "value2":1634911105603876624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.990782, "value1":0.8721197542810023, "value2":829062251335888311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.678575, "value1":0.8518891534609417, "value2":5490128933307846252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676580, "value1":0.032403043603468545, "value2":1860775688299500894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.011776, "value1":0.3310374930985375, "value2":6454291476355278297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.902226, "value1":0.03335506845841715, "value2":4281002708262490682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.884426, "value1":0.26587448707952976, "value2":2888077545939138944, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721322, "value1":0.9352434968753679, "value2":7587503328880398082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674491, "value1":0.13470131213667097, "value2":733064028083204302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.422073, "value1":0.34800250717845166, "value2":5657610491232907815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232687, "value1":0.8974445402000435, "value2":5670871752075567567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.603175, "value1":0.8044440847017621, "value2":8279540241317103114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_5":"b","key_6":"j","key_3":"e", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354995, "value1":0.9370809546532062, "value2":2794379758286031831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598521, "value1":0.03162604483850602, "value2":8898609192002715752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489635, "value1":0.09343989179985779, "value2":6354680197733735029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_5":"k","key_2":"d","key_4":"i", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.766631, "value1":0.6557065025532866, "value2":8867906113412686551, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.110067, "value1":0.37336348255028134, "value2":8359961637534300323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.734771, "value1":0.6467506386274888, "value2":1558727129876488311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751073, "value1":0.5011707406625697, "value2":165276254030404304, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903806, "value1":0.883542811993649, "value2":5713497784481830170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_9":"b","key_3":"a","key_5":"h", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068830, "value1":0.41287620615552584, "value2":110550725395415731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.481998, "value1":0.9176610757654294, "value2":3543653242038441064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.206639, "value1":0.6881985031716263, "value2":3991865736711226608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682333, "value1":0.7354938188464508, "value2":2336942143484925177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.909363, "value1":0.6550859218093187, "value2":7848025848676081478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.439129, "value1":0.4735897904277586, "value2":1730101214885033885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788447, "value1":0.7029400400744759, "value2":7348647096857266019, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656519, "value1":0.8226813316731755, "value2":8565628989265102630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.286018, "value1":0.1715470955708772, "value2":281139728200192436, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.316461, "value1":0.970829440713552, "value2":8266115606666039974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.131938, "value1":0.06029126490427053, "value2":6088969321338942970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936137, "value1":0.16630484422294922, "value2":368627481682565524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.632956, "value1":0.4597748954914025, "value2":614883467360042989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441534, "value1":0.3169134259464135, "value2":4894244187289767341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.536363, "value1":0.21988172452701402, "value2":3848642916965059480, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722069, "value1":0.0035036640993316324, "value2":1784910019753576066, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189179, "value1":0.5999084267877618, "value2":589928855378135734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070147, "value1":0.2816071736469606, "value2":1978576359562742327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.474328, "value1":0.12233813345274026, "value2":7757125810591739471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756802, "value1":0.8734910453077231, "value2":2858305999840238155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830462, "value1":0.7809071944843656, "value2":6070440638222443699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.158081, "value1":0.8007440801809812, "value2":3893225795949780633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_7":"g","key_1":"b","key_3":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266044, "value1":0.5291100849459248, "value2":8570983485516004790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375100, "value1":0.43060898088087474, "value2":5072556087103000232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433713, "value1":0.4065332038272028, "value2":7498601821703546789, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293147, "value1":0.2008523459507916, "value2":3469755278652299519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.464083, "value1":0.7193413705182999, "value2":8360590737420028572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182168, "value1":0.12869784027227613, "value2":2436732966596869973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152849, "value1":0.915235247509805, "value2":6575468662860533653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.106299, "value1":0.6574795299867987, "value2":5839146977664216835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.842382, "value1":0.6684180756051546, "value2":6255233029731307668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.976893, "value1":0.9921834371308854, "value2":286112572755130788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.747197, "value1":0.6687884895336262, "value2":5026368687057544137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227443, "value1":0.8444750407098351, "value2":3238045843783198862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575432, "value1":0.04959245001196866, "value2":4763760968874648742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484574, "value1":0.8794886478305205, "value2":2929187105321944273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.613332, "value1":0.4649488124035846, "value2":8038984692264433406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.156389, "value1":0.013056913554735015, "value2":1073629030912865258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614657, "value1":0.21062410476220902, "value2":6415897327379032774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238729, "value1":0.10820884210252545, "value2":9083864450943421396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241678, "value1":0.22605677165183255, "value2":191747579342742576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958794, "value1":0.08847610578305673, "value2":7897538472323716766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335228, "value1":0.7181578715519717, "value2":7450823110233453413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408629, "value1":0.007049664704797943, "value2":1505104843936344580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558681, "value1":0.17466226992103584, "value2":4472498115626926601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682989, "value1":0.9645272985758268, "value2":1982644188431990793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.179200, "value1":0.6439691808346755, "value2":8186363896835482044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.823991, "value1":0.6478071089364331, "value2":8599197823923500800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109730, "value1":0.43770117688691335, "value2":8534783631376673343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.659077, "value1":0.2388079100822681, "value2":5323293096195049116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.120405, "value1":0.7373368673794832, "value2":8559881035375574344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.355361, "value1":0.6166673498020887, "value2":5456145421934358189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.824086, "value1":0.4534358999465994, "value2":6384310233648456918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871013, "value1":0.5576949556795737, "value2":1428733241052481930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507168, "value1":0.14733425604177453, "value2":4932780474472793744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.399272, "value1":0.7455596536595573, "value2":7959578558660693956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015542, "value1":0.3661445195957949, "value2":5873286557095287645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.103786, "value1":0.04914833432556202, "value2":3473465672536909160, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201038, "value1":0.09243610446589845, "value2":317387159009776683, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637510, "value1":0.05764711949860248, "value2":683074398669967457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000515, "value1":0.27555419865710506, "value2":1427932259617448392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_6":"d","key_4":"e", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455871, "value1":0.4421289415542984, "value2":2179707427691179241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.816179, "value1":0.0315328103269199, "value2":8955771787657622788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.012467, "value1":0.2583652156574764, "value2":3152397034013397022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.323088, "value1":0.6137213421267609, "value2":6194154265710187463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361691, "value1":0.2728847707358428, "value2":1410878768807423021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507208, "value1":0.40681095070477, "value2":2031127740762892841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.943788, "value1":0.4688930142953297, "value2":4135242725922960343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528649, "value1":0.6786255340569864, "value2":5287486612492903502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_3":"f","key_5":"a","key_0":"f", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225718, "value1":0.4601298705932887, "value2":9052049426787719860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.308880, "value1":0.4295083103049244, "value2":5651329885903965921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759618, "value1":0.16482488772851353, "value2":7857083585674436753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416052, "value1":0.9111867810311565, "value2":6723934981662609251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.962464, "value1":0.4935260151955853, "value2":3632302155175562257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.771117, "value1":0.5157133736913164, "value2":2578240794064018102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690480, "value1":0.6806227775589547, "value2":478506980366332127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_3":"h","key_0":"e", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.819739, "value1":0.8780417506767497, "value2":5796860147973283852, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.393173, "value1":0.42102685254727695, "value2":655129410007603946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.648983, "value1":0.7792161980083644, "value2":3901696553081135725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746096, "value1":0.4415965960336573, "value2":5128962497088709971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550611, "value1":0.9164668411003939, "value2":8252426555782396131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.026169, "value1":0.7150672668373745, "value2":4764749266652855333, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.684226, "value1":0.7136962442214909, "value2":5607090679190953591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382560, "value1":0.4327657862928645, "value2":2199828856833521565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_7":"c","key_2":"b","key_3":"i", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.467808, "value1":0.22363679307659168, "value2":2768232122997302049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575206, "value1":0.7602213610975361, "value2":2779299954374668017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591751, "value1":0.6270567668817301, "value2":8638942281069975034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374899, "value1":0.3628294711980554, "value2":2256958770162190672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490274, "value1":0.7562635891941201, "value2":828105904736409259, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.170118, "value1":0.5429488039839387, "value2":2772778506127755922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240769, "value1":0.2820088361870865, "value2":5639438631779439142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.052386, "value1":0.43149632078191064, "value2":1748438881846630926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507233, "value1":0.961520455982136, "value2":538372936759435184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_4":"e","key_3":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705937, "value1":0.5267821110187133, "value2":2615203611933488638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750925, "value1":0.12125960803758137, "value2":9007375284837027095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631618, "value1":0.07157188871273036, "value2":8155465841105503917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695843, "value1":0.7797612580730544, "value2":174508687346238960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269295, "value1":0.7420818036952436, "value2":4380281606468315484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566418, "value1":0.31419999154507045, "value2":1448626920753535043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.125439, "value1":0.23644172282231143, "value2":6454497651916669000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260012, "value1":0.7675706994384546, "value2":5131372429329822614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674484, "value1":0.024443192261290648, "value2":1060802392645575205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870912, "value1":0.40232027987742836, "value2":7398514903536659789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.193840, "value1":0.5573502520764032, "value2":2891807756570316806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496711, "value1":0.0561679292808207, "value2":7275940738251409744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007546, "value1":0.4548616529539407, "value2":3391843142730199327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532516, "value1":0.5229547783776293, "value2":306719517858726688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.879419, "value1":0.738955824150393, "value2":4551796041817468625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.037090, "value1":0.16862399179431758, "value2":4308060124699455866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.748879, "value1":0.537219693168335, "value2":464744057316162498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.610036, "value1":0.35423811263377813, "value2":8058264218672212470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.212290, "value1":0.8726353918052155, "value2":7894596078524999317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.876606, "value1":0.6546517022010405, "value2":6758959777685985912, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140504, "value1":0.347953522010423, "value2":6223000072814932347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.829533, "value1":0.7297577355885363, "value2":7098834979468843426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770147, "value1":0.5158958137849098, "value2":7721786354129545110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960566, "value1":0.08984803039947419, "value2":7183642332674708296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.002511, "value1":0.7991212300299283, "value2":1907144646133978540, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.969474, "value1":0.8490958586264525, "value2":2475634499457331111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757044, "value1":0.08525717165380897, "value2":6126821447855017519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826913, "value1":0.3832744407117778, "value2":7549472912902563562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713539, "value1":0.040929240199395034, "value2":8725477692408636856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.235038, "value1":0.21655074702892513, "value2":6340750264489982261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713507, "value1":0.09698561541865447, "value2":6976532341223849395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926365, "value1":0.6869464705551463, "value2":1770406309916093131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_8":"h","key_0":"a", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.159778, "value1":0.37025745483297273, "value2":1667176166944781722, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964175, "value1":0.3460815492793407, "value2":8584884445165219279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.789212, "value1":0.5752530695794612, "value2":7518543931942002634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946193, "value1":0.9389444654094391, "value2":4798866777286425255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183973, "value1":0.42689788996190486, "value2":2707564478214913458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381159, "value1":0.19968138225970908, "value2":8498929039632010906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.535645, "value1":0.9768502318817727, "value2":6273017261118239502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957340, "value1":0.1584492922790321, "value2":1071530234423723880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622320, "value1":0.6014449518439046, "value2":5905866866815287771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.839825, "value1":0.6412078194818629, "value2":8581664149275929445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.925054, "value1":0.6499072539736884, "value2":3213784800648371267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.654382, "value1":0.9107398509050536, "value2":6290871463312648713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.795697, "value1":0.8789558932554133, "value2":9118966352973366626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299533, "value1":0.18449853142128472, "value2":6924501956061845944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993498, "value1":0.10358257618685637, "value2":949727120096577179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192327, "value1":0.14944601875044164, "value2":3650903859653754777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115925, "value1":0.7654222794387283, "value2":1807303188151583637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290372, "value1":0.022817260035797225, "value2":3306384333491756709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.402670, "value1":0.5073298240798392, "value2":4756233885952638662, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682572, "value1":0.44581853587828163, "value2":8977717914108281095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.176704, "value1":0.0686762673056191, "value2":418784156784186861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676421, "value1":0.9540024144812602, "value2":5040838132393700704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851525, "value1":0.4501599756607321, "value2":7291682933774791859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.593343, "value1":0.506206698395899, "value2":622013338659516904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454600, "value1":0.4498330838292854, "value2":2669650474822317581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527805, "value1":0.10877922841749062, "value2":2131131607855096268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.196472, "value1":0.04264644660597854, "value2":2656288085872651609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743799, "value1":0.2248320170707705, "value2":6330060840606217610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137446, "value1":0.5799142292246671, "value2":4848799016740596142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.405919, "value1":0.3378516880177889, "value2":3532844197361780130, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.085405, "value1":0.11768848717895854, "value2":3011043909687794254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494421, "value1":0.45496539093555294, "value2":1968423041425508639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_5":"f","key_0":"b","key_4":"h", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.846925, "value1":0.23911281903346318, "value2":1229857216789047456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.320172, "value1":0.7911586278138405, "value2":145982991700455098, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688187, "value1":0.9272721413003945, "value2":5752958588041193324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.069720, "value1":0.006222818414235851, "value2":7251719396221186260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.974434, "value1":0.7327419748667513, "value2":8892941322690231954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.891519, "value1":0.2559114110516683, "value2":8453661615833220464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596204, "value1":0.8329616164402396, "value2":6548226786173702319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539908, "value1":0.7401840588881007, "value2":2823865774842747097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865375, "value1":0.23950780446408748, "value2":6387790804623201730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121268, "value1":0.4818898421020435, "value2":5213834205313537165, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865802, "value1":0.3275415536098801, "value2":2802490407634150279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105641, "value1":0.6210077776605651, "value2":7224328772436945002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592291, "value1":0.821033330430236, "value2":3130486219263774432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497809, "value1":0.5925199380020667, "value2":341686057277963037, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097734, "value1":0.7071847698151175, "value2":3681912566029517435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.763574, "value1":0.5357729458141934, "value2":4576393986863014129, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626276, "value1":0.4467400550839313, "value2":8832239215373133572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626952, "value1":0.8566744061055088, "value2":2681684884383601627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980666, "value1":0.620719669726131, "value2":3866315867973394400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.970686, "value1":0.492660470887635, "value2":8584579737939983838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205981, "value1":0.41980273151462594, "value2":1272721076115788334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643071, "value1":0.9850402079127909, "value2":6866475251892359276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.520241, "value1":0.021531538514302357, "value2":3680419636166133733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689781, "value1":0.7246711061095096, "value2":9156928465977729149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936886, "value1":0.30593890582238353, "value2":8650088989944818585, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064882, "value1":0.9419400037260646, "value2":5139525288533697167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255204, "value1":0.6562362511494719, "value2":5377088753944330401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147096, "value1":0.12576537131594923, "value2":696189011986700613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.199366, "value1":0.8703136109731814, "value2":8758678284656449473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782148, "value1":0.21588917591543605, "value2":7157001910263594492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053985, "value1":0.9864976225072658, "value2":3999466034196061068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.409718, "value1":0.4052988292496616, "value2":7239010836693528474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918878, "value1":0.9460279119002769, "value2":8629146616157881411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744254, "value1":0.12263478198823631, "value2":8698623269262602240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_9":"i","key_4":"i","key_8":"h", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701868, "value1":0.3178667417931335, "value2":8069435536591676411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_6":"k","key_7":"h","key_5":"b", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.794445, "value1":0.7546856348043545, "value2":1056257035145843208, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.318803, "value1":0.8976234734615749, "value2":7848445975050376420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.146350, "value1":0.31748244669278847, "value2":2926268572806049868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392349, "value1":0.655725592207486, "value2":4190046781739044777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.749690, "value1":0.6239130127907853, "value2":5206683397328774242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744300, "value1":0.6764999956977692, "value2":1477381647586507515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.758303, "value1":0.5258481908865111, "value2":5235026862755190183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379762, "value1":0.890935177217856, "value2":6322796509408703364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592912, "value1":0.6886535422027805, "value2":3968885283796366911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859203, "value1":0.3806206868492738, "value2":6588306706789304138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.565175, "value1":0.823280011831958, "value2":1840222942885696142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.095858, "value1":0.18658533157232132, "value2":3040729957448861027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.567863, "value1":0.1544724870114228, "value2":5739061219420103119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739840, "value1":0.5915071280708923, "value2":6986464734573649185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.200755, "value1":0.4500618730121175, "value2":4400580498849630870, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358028, "value1":0.8986628799372285, "value2":3320447208954210899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017181, "value1":0.4972731292168526, "value2":5687205385943047330, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229539, "value1":0.5257635254456374, "value2":1562089048942850119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.171044, "value1":0.8736127789799626, "value2":7151271249368522054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.384626, "value1":0.2370246172687121, "value2":2826036906774872004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522883, "value1":0.6013577976454938, "value2":890649018657102134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.877768, "value1":0.6077458920362099, "value2":8966895480129060070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.224033, "value1":0.0076863111785604905, "value2":6569047840639595510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.311682, "value1":0.07593318166504835, "value2":3253540135578122766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490459, "value1":0.4609451528285016, "value2":2994036544656680091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.378898, "value1":0.9545401129487334, "value2":3668542248171100045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416785, "value1":0.35541532907488144, "value2":94039845626229270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275554, "value1":0.421200916650073, "value2":5818521501894570082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619269, "value1":0.30772799684273167, "value2":4876402407267868155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.702725, "value1":0.15927682148224467, "value2":1903456341847904299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232604, "value1":0.7711772747260455, "value2":3866638254479737513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.217772, "value1":0.8540182257688077, "value2":7309608218029816756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705505, "value1":0.48163362162995105, "value2":1550283929814058088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900193, "value1":0.23760307179291948, "value2":8646186763820142769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.208626, "value1":0.38931880434870053, "value2":5483530341831301025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.568347, "value1":0.2801336860166125, "value2":9164397534745927432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117458, "value1":0.981405746517363, "value2":6802896340427379587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963035, "value1":0.7214166769862517, "value2":8440456704015434339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700137, "value1":0.5934611351717767, "value2":6333818613628602547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145303, "value1":0.6922047329264945, "value2":3009832451157760657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345925, "value1":0.7232346453151443, "value2":4791014890475627712, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619327, "value1":0.9932237622196707, "value2":3624811523135333449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269730, "value1":0.7866373764087783, "value2":3746435711914598124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.265782, "value1":0.05991121042342668, "value2":1951849832097700576, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.175543, "value1":0.5444637318639066, "value2":5220283889625062347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_8":"h","key_5":"i", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677825, "value1":0.904620506580413, "value2":4669605336529676512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668600, "value1":0.17366696171797397, "value2":8249554071220606769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041222, "value1":0.3254693615668756, "value2":5262353711290630554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.243762, "value1":0.8759302538831941, "value2":7311945236881805486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958625, "value1":0.5492731854343449, "value2":5435276136514133338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290213, "value1":0.7949670770789082, "value2":7111766404096808279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025988, "value1":0.1026128579407924, "value2":1755616401770874869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595099, "value1":0.18775955592545962, "value2":1656134813479826833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.671098, "value1":0.8079739199189518, "value2":3665225572712988915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.542084, "value1":0.5181312401815146, "value2":1481804189008155276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.784956, "value1":0.40357178109169595, "value2":9035114082415618141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.276517, "value1":0.24301724893563487, "value2":4602891919019880467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064974, "value1":0.37706581069603073, "value2":7053383221578853097, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698510, "value1":0.45184858839742487, "value2":2258520036203869092, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.218133, "value1":0.3113727295782559, "value2":6841422843138500205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668881, "value1":0.25909755219748914, "value2":1185172632185740020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.049479, "value1":0.9359900704241361, "value2":8443878057371326705, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.674394, "value1":0.5330986303950987, "value2":4597086230952392696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195073, "value1":0.23095513043718424, "value2":886484372856556987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_6":"a","key_1":"e","key_5":"c", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.380352, "value1":0.9222632147295795, "value2":2011984636052581606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.302156, "value1":0.7967664965647748, "value2":1936857391665696956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673174, "value1":0.8085420465867998, "value2":6974688906284239573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.013224, "value1":0.01982064999844403, "value2":3955016096628456095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090855, "value1":0.01304461484505885, "value2":8778318340262765164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.795784, "value1":0.12217080131703652, "value2":2014947645475742270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284442, "value1":0.42738762978686595, "value2":2584606560267930457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.741760, "value1":0.5561052082820417, "value2":7263332550390986826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035524, "value1":0.908173032025188, "value2":5685807943763240264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025764, "value1":0.20925710637592332, "value2":6562668387404626293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.530188, "value1":0.6447955243050295, "value2":3888614173619999623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424157, "value1":0.17487330381635957, "value2":5301061833157028268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026934, "value1":0.0422857387218024, "value2":8303721108125129087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841847, "value1":0.5206472086096076, "value2":74348635869177909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.414469, "value1":0.4278094238360654, "value2":7010327929309184782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491788, "value1":0.747156270778759, "value2":4790193170750716406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399141, "value1":0.47388249427808166, "value2":8836306836420904183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.160250, "value1":0.29496035919361546, "value2":1937679940636334558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787511, "value1":0.40198955034703965, "value2":5313833463463440565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684988, "value1":0.6177512623201419, "value2":1449869359073755283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803253, "value1":0.8970896940692594, "value2":6178067064755804966, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964550, "value1":0.8199555931693112, "value2":140777809986023618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456407, "value1":0.2401625723920763, "value2":4030355648738753679, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849585, "value1":0.2629587147468046, "value2":7076050191155576404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.824331, "value1":0.6892589926802947, "value2":6399755277819070915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_4":"e","key_6":"h","key_2":"k", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388915, "value1":0.8127879446503501, "value2":5588278675440700300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782971, "value1":0.22771808116681516, "value2":6261899644042013732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712419, "value1":0.2905286238822736, "value2":5614218841386062674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_3":"h","key_9":"j","key_1":"d", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249696, "value1":0.9445558673553629, "value2":1404486883141712140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601814, "value1":0.6990549633969345, "value2":7198014296771075924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543813, "value1":0.07857387425059083, "value2":1217712980435541695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123893, "value1":0.07983470162468347, "value2":7171526612504871775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_4":"a","key_7":"k","key_1":"f", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703648, "value1":0.36447696895349185, "value2":5146632909527815894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166353, "value1":0.7098874569397392, "value2":5324193865568739754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399594, "value1":0.7745072356158436, "value2":7831862313374173936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.439724, "value1":0.2037239485962775, "value2":3755069786706852308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839998, "value1":0.6732355725193216, "value2":2639271267504971364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228435, "value1":0.897840654678523, "value2":5116662644663764475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973287, "value1":0.9461886626837847, "value2":5279391211817478931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.441360, "value1":0.9419535299333034, "value2":2098864241711696584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.780978, "value1":0.7212145622817746, "value2":8825784157879516761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.342306, "value1":0.40763808526506856, "value2":615440271373894957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964992, "value1":0.8428995412660205, "value2":5458378648341607708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386901, "value1":0.3242872455426699, "value2":1659876068547712992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.960102, "value1":0.2464238639205716, "value2":699750419933475491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585099, "value1":0.11986390379255613, "value2":3678288424535052375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.154037, "value1":0.7785920099800233, "value2":8275166075671164173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211892, "value1":0.7853941226491261, "value2":6369532531235861115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.746677, "value1":0.6664817644749024, "value2":4658328841108884689, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.941862, "value1":0.846994201539624, "value2":1274645788690532949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.733827, "value1":0.13169457879746155, "value2":3221287182313964764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870127, "value1":0.37466507316122794, "value2":8856106508914542017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798298, "value1":0.22462880312469063, "value2":6798911070080579970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993366, "value1":0.9961738964349245, "value2":1193549068611970903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.157412, "value1":0.4452570786392375, "value2":3487651503828456956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.529021, "value1":0.26905395113611574, "value2":2025155869621977272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.701922, "value1":0.8730175680632801, "value2":997347931310956157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_1":"a","key_9":"h","key_0":"b", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234577, "value1":0.7588051800901922, "value2":5319874977525060911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943591, "value1":0.47603601658101, "value2":5946410545801626105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.292747, "value1":0.03173464098642328, "value2":7226691638812613138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_3":"b","key_5":"j","key_2":"a", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608463, "value1":0.2957253043607161, "value2":122362680185330457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.643245, "value1":0.7167827619023723, "value2":5186237060875400709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456051, "value1":0.2188755629521717, "value2":1845588654842798990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231176, "value1":0.6418253580281563, "value2":3690123298454825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_4":"d","key_5":"d","key_3":"f", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158253, "value1":0.7945876103242618, "value2":9004322525035166688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234079, "value1":0.5202371436378874, "value2":1591535835129382659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.875339, "value1":0.37493709867362873, "value2":8328159676060260451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907034, "value1":0.20583691616211497, "value2":3631192064418800289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.518876, "value1":0.16335464977880865, "value2":4643059165781979124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185787, "value1":0.018176867671764584, "value2":3270163687436586406, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308787, "value1":0.4773555846410255, "value2":3253306657765008963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297154, "value1":0.246942999840671, "value2":6872861714776356785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_3":"k","key_9":"j","key_1":"c", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.817582, "value1":0.8165235911801363, "value2":8310469403306376604, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493041, "value1":0.38500411997916434, "value2":6747567659837060435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224106, "value1":0.9994238326374754, "value2":7099587552820760691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031476, "value1":0.09066884358892856, "value2":5945790442760569800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311356, "value1":0.5806991883130863, "value2":6895176067401972683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653725, "value1":0.15503072276739358, "value2":7449412216901454949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.598231, "value1":0.20373471056711764, "value2":1437658094331870355, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798108, "value1":0.6097451467316525, "value2":7133130757829716058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709054, "value1":0.7371923755130838, "value2":7543405219351917468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104790, "value1":0.21083306139991156, "value2":7929604552171608387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.187877, "value1":0.5608708053678212, "value2":3368623446076987813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749130, "value1":0.657437311445075, "value2":8949805346712651308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582481, "value1":0.42362389624895797, "value2":8717288037606444575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.558438, "value1":0.370755803762819, "value2":1107999166234163628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802675, "value1":0.8677299277452752, "value2":5915000168574046056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_3":"j","key_0":"j","key_1":"c", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.208039, "value1":0.38781051864576976, "value2":3375198001539598542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680278, "value1":0.886048145499295, "value2":4081527324669320081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162287, "value1":0.7118801419064047, "value2":2390093012206667849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.092015, "value1":0.9942758150465311, "value2":4638994738698984564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039345, "value1":0.641165676244722, "value2":2976540841974698815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.408130, "value1":0.5042712547968512, "value2":8947846924433036861, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259088, "value1":0.514500988249709, "value2":2770449998520564629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_4":"b","key_0":"j","key_3":"i", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293773, "value1":0.36521689212647834, "value2":67527608081891502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646136, "value1":0.04048325982754749, "value2":4743191660759249600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.047497, "value1":0.5502497475964416, "value2":7169106217640232030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876800, "value1":0.3799501707452096, "value2":6757458314368411993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968057, "value1":0.7499783045753542, "value2":1200059614173097893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.871681, "value1":0.6987940576652765, "value2":3787158661733344974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.998419, "value1":0.7974184353878208, "value2":3011281681090708365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663978, "value1":0.1875336848417326, "value2":7682956450262415743, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799777, "value1":0.13431952299825461, "value2":8472008336913527353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224565, "value1":0.7432835827699946, "value2":5363247813194187485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896272, "value1":0.1238866666650729, "value2":1249324477450532329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630720, "value1":0.15283582580094462, "value2":1819923308050662735, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405798, "value1":0.9413697480070442, "value2":6170928651103063980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249174, "value1":0.3594131764729996, "value2":6196624819261733193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628627, "value1":0.5009434579166352, "value2":4913727766417338788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_6":"d","key_9":"g","key_2":"e", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949656, "value1":0.677579274449204, "value2":4321897610912596942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.452302, "value1":0.8465391778403211, "value2":8767695810058898547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224741, "value1":0.9419255467202373, "value2":3942608565088456699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832311, "value1":0.1844332411242929, "value2":7897621420940762914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364450, "value1":0.6062623207548122, "value2":3048338263269528616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458762, "value1":0.8964243000562225, "value2":6980106557138925502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090342, "value1":0.9353992150482529, "value2":3024974544306240484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680282, "value1":0.32391766175077186, "value2":855225884463637074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949004, "value1":0.8992840018801868, "value2":5343754317476027418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.564072, "value1":0.04591172153502947, "value2":15119713343690272, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.015169, "value1":0.2775250663576938, "value2":8329492228577328961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035102, "value1":0.8955268703368635, "value2":1413978015063203049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.922310, "value1":0.5335365949387284, "value2":7752415045685811238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.862402, "value1":0.328052481246616, "value2":1049471938369225232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.632372, "value1":0.702701110691373, "value2":736041481181376478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886888, "value1":0.8185435772395079, "value2":6423113939133946645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087072, "value1":0.4700081672384728, "value2":4642202730229975792, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.936650, "value1":0.18762496890040795, "value2":3355401686890613388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032044, "value1":0.12307064568471555, "value2":5991756874915773532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093295, "value1":0.22401921911686248, "value2":6120641321792430058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.777098, "value1":0.29744828015347885, "value2":3084065102603938152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.437450, "value1":0.2884592585353877, "value2":4044254904136084327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.017135, "value1":0.7608822818277434, "value2":1205572766308158879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_7":"a","key_4":"g","key_6":"d", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907629, "value1":0.7081298253215954, "value2":7688268403734929948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924934, "value1":0.7192898599410195, "value2":1421354730652485196, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.736050, "value1":0.7502596301905261, "value2":2582210329738662673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515235, "value1":0.09310016531489594, "value2":5123514648664672005, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.117213, "value1":0.07556633746315061, "value2":3501827326756011617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.485929, "value1":0.9272991096008794, "value2":5159915940333201308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296263, "value1":0.11108812397005534, "value2":8753717861080819638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990089, "value1":0.527982415726353, "value2":5638704340740792965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835455, "value1":0.062168150269998286, "value2":1944114908616022523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297201, "value1":0.30562887567201646, "value2":2937626694886920620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974272, "value1":0.7221303752638616, "value2":4841762343070531864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.477606, "value1":0.011147194457821816, "value2":7938710785429203576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_9":"h","key_0":"d","key_2":"k", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129918, "value1":0.5408385148641213, "value2":9195358942783956586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364995, "value1":0.0773648729840119, "value2":2987992944358913043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.776833, "value1":0.8273262149027085, "value2":3110126394788152042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.610517, "value1":0.6781728258681893, "value2":4793737503812454856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896976, "value1":0.37188254064396126, "value2":3144974162911084078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202562, "value1":0.5271477787382647, "value2":6411589275983706377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906770, "value1":0.4476129119528191, "value2":424159247916997254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_9":"j","key_0":"e","key_3":"c", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289456, "value1":0.541155755467242, "value2":2999265325007368522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.860878, "value1":0.4758705080993404, "value2":6690006483968082589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.365448, "value1":0.027420735129761006, "value2":825754437394215422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212551, "value1":0.16839070098231337, "value2":1942763578810483030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870900, "value1":0.4994077584006896, "value2":5993483889801645858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.455124, "value1":0.26138016691309135, "value2":4427691071851307843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.569012, "value1":0.4778238812129043, "value2":8556858598621845387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.362641, "value1":0.2203054974232932, "value2":3477406531290022646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493850, "value1":0.4787384781915333, "value2":2198408185614770688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.513028, "value1":0.6429388402116987, "value2":7593443154020639176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386729, "value1":0.17475131034566962, "value2":3647348637356138676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053464, "value1":0.6091636844532319, "value2":5763485176228727883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628348, "value1":0.023668653461306262, "value2":6178270391876561387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913114, "value1":0.637992847841559, "value2":8248068010165238913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.413158, "value1":0.5132505144116425, "value2":3488568643829613122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517474, "value1":0.21449952384417484, "value2":4829637720497890669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_8":"d","key_4":"i","key_7":"c", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.659125, "value1":0.35348852774479717, "value2":929980299829000703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071642, "value1":0.5598789974197187, "value2":7463399784012985552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993416, "value1":0.6475276350489354, "value2":3252439000616730752, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003658, "value1":0.897666564581801, "value2":508174253835823210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.629566, "value1":0.49532352920736916, "value2":3384274756820723223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.084906, "value1":0.40471552937588173, "value2":2358566109120320700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.108543, "value1":0.8814759719911462, "value2":940297244819349653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138889, "value1":0.48057446422385713, "value2":5651682080133821762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.468024, "value1":0.6607643026338559, "value2":3567273041505939519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.278010, "value1":0.05185404011612328, "value2":9174087231001182435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.095328, "value1":0.4998002617147629, "value2":4028395944074542796, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.962889, "value1":0.7966578010574328, "value2":907724426763786137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.434695, "value1":0.596944451662117, "value2":6599947217349296930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.695515, "value1":0.23670726317490795, "value2":2923186045419354751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830693, "value1":0.43683284863611044, "value2":9110976313628947704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311809, "value1":0.18924644468054935, "value2":2283072307874475430, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405030, "value1":0.5026514010742291, "value2":7499615277819756210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.985393, "value1":0.2214787811403981, "value2":1167173892132804942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123017, "value1":0.24050302556695696, "value2":3512829503479749438, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.275825, "value1":0.8989518053152799, "value2":7332940306078599750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_3":"g","key_0":"k","key_1":"e", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.935513, "value1":0.422470893258827, "value2":1965139573445622612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_4":"e","key_2":"a", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697239, "value1":0.9715985007012725, "value2":2441260821113474270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297829, "value1":0.4121340516781959, "value2":7327020552945137669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.488215, "value1":0.22392461646214185, "value2":5911938621135820572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.206243, "value1":0.020325317058324376, "value2":9207086370751028426, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183572, "value1":0.23206347801714602, "value2":6792315880866236659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524533, "value1":0.9124779133103079, "value2":3221463708308920869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.893528, "value1":0.4848003839426891, "value2":3814492558819679029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.752371, "value1":0.8170507918067189, "value2":6527155317426998401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916551, "value1":0.342137272729241, "value2":1602041662641372111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.037592, "value1":0.6500639685761466, "value2":3567065955735150085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902253, "value1":0.7531115895009749, "value2":2543364566980292046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683579, "value1":0.30275430657154007, "value2":7640650810082868472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956614, "value1":0.5403322951696091, "value2":332198111972672347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202908, "value1":0.3832878669702297, "value2":880756788127038920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.820586, "value1":0.34286605495032285, "value2":6526500882733787563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930022, "value1":0.3539159969493719, "value2":370073818816792799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023457, "value1":0.047546161290410185, "value2":5398553854250083599, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303770, "value1":0.7186586727923895, "value2":9169471131017267259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.756889, "value1":0.31103630013784395, "value2":3594098461981248414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_8":"a","key_2":"k","key_7":"j", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212371, "value1":0.3882294279634762, "value2":2428299028227660876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.472113, "value1":0.19389271171622363, "value2":7737936344522555466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051213, "value1":0.12920469147183916, "value2":7837057207029745863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907093, "value1":0.017956122904157993, "value2":4878755396476119431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_4":"e","key_7":"a","key_2":"g", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.525258, "value1":0.49782305829726053, "value2":2466466392623808778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913492, "value1":0.4704351769152427, "value2":7908561685667216260, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290702, "value1":0.4198914182070395, "value2":6819583912532498504, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125217, "value1":0.04410603241691789, "value2":6832161683827996201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004937, "value1":0.13145801093930243, "value2":5314457688060891725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.656896, "value1":0.41192120722520353, "value2":5457893604153821743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_5":"k","key_7":"e","key_4":"a", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.700497, "value1":0.8047394500892703, "value2":8180560267702238361, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553716, "value1":0.7484558334038893, "value2":1003966921986785454, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966183, "value1":0.8167158291471999, "value2":2250434045285686743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663750, "value1":0.7127353578019723, "value2":3741760060639503306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686363, "value1":0.9712701187599743, "value2":3622116724705792870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.103723, "value1":0.4724493721726449, "value2":8181583911299132150, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.484649, "value1":0.18448759636607337, "value2":4053994543260017135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.925440, "value1":0.06382834538328273, "value2":6916231632621539461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939805, "value1":0.22424399111258897, "value2":4351472309231431136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792665, "value1":0.33436392964606154, "value2":2696421529224538356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054274, "value1":0.9272819232934056, "value2":7264556561094806515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.983823, "value1":0.7989127551970725, "value2":5239695063979297186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714766, "value1":0.6892792624795272, "value2":270252571976728726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.263106, "value1":0.64122047400478, "value2":3421795383581628735, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720260, "value1":0.8087795736115097, "value2":8426217720617766298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839070, "value1":0.5917446619321536, "value2":390571547986932413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966990, "value1":0.2432955344040808, "value2":8461111581798764026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.808072, "value1":0.3216500857620197, "value2":4440941489040196963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091958, "value1":0.3995806295500445, "value2":1180768478663237716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139366, "value1":0.11507703668400583, "value2":4516311843825616388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.251512, "value1":0.7632852678548858, "value2":6418768969046377858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963533, "value1":0.8213013619231073, "value2":6101164365690068241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.466433, "value1":0.28350424710105476, "value2":5563469458031181794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831271, "value1":0.8821293582204699, "value2":3430349356503249989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003553, "value1":0.7323647142880576, "value2":5739925461894436779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.241144, "value1":0.3850622637508932, "value2":6137203197960360313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.872100, "value1":0.3184658084196574, "value2":2061373587207508318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.900532, "value1":0.2888524955130242, "value2":6600516250776964704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_9":"c","key_3":"g","key_4":"b", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142058, "value1":0.7572218923237818, "value2":5437617110907280837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.167933, "value1":0.20087813542991115, "value2":8497531403193171323, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977657, "value1":0.49027491644969856, "value2":6846007575513730281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.337497, "value1":0.7535889949814326, "value2":4191915280375173080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091305, "value1":0.41827531366800946, "value2":3553824659208645760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.260475, "value1":0.39071909544867817, "value2":3828966522631037754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421117, "value1":0.6070501451115596, "value2":637677502764251310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_6":"d","key_3":"a","key_5":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949074, "value1":0.8719785824911194, "value2":739855289497042435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005847, "value1":0.564214486086032, "value2":3879428464931256295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608985, "value1":0.8120552257475274, "value2":6222621751128276018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.505971, "value1":0.44491083173920337, "value2":2966233843840872053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_6":"b","key_1":"d","key_2":"f", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686524, "value1":0.910573536000882, "value2":7259657495340557823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386181, "value1":0.5690887314353372, "value2":7696797346941049468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004437, "value1":0.2837421940276855, "value2":3630048754825451133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197835, "value1":0.3478392366591849, "value2":8924518323717745322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404754, "value1":0.14818758453959469, "value2":3841840031846521039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.847397, "value1":0.10611042816524652, "value2":3749767974486199531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773493, "value1":0.16710420925739453, "value2":2156267597938791257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253265, "value1":0.39770730374803015, "value2":8078016986748379936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720298, "value1":0.3023956927195079, "value2":2183933305568131133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.306860, "value1":0.13870609943567333, "value2":5783110454957795198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189190, "value1":0.3825441585131095, "value2":2493897460883724129, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842956, "value1":0.712749117246484, "value2":1337306979661530320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045437, "value1":0.9278250040101162, "value2":6116936767056337825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.606083, "value1":0.4732072236705937, "value2":6967738612762397759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094844, "value1":0.9269374913746184, "value2":2591132307871233919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369784, "value1":0.09045009486516034, "value2":7834710052863465672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032637, "value1":0.16606039490952637, "value2":2737666598235173614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.394163, "value1":0.47507719343340277, "value2":313537922371752138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.987576, "value1":0.5780445815848603, "value2":4108970207058212125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906463, "value1":0.03465970899762972, "value2":6841259554497483309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.266096, "value1":0.6742131298888875, "value2":7025803245311237704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963062, "value1":0.5904971275955457, "value2":2254362569087327315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.130304, "value1":0.868734941900672, "value2":1538504759997860509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.955104, "value1":0.5994289988135004, "value2":3881947199005484386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.480748, "value1":0.49853130606809243, "value2":727938692981473193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.682757, "value1":0.38003013064353247, "value2":8479678523613035623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792753, "value1":0.16576560662385773, "value2":7797109252869584262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.510440, "value1":0.6049726150607012, "value2":8669482832353600257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.638136, "value1":0.15014964585075946, "value2":6225706505785608080, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.522674, "value1":0.3512111717300567, "value2":3641814037368457348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785473, "value1":0.4544503987969124, "value2":4635364210597620767, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_2":"f","key_4":"b","key_1":"f", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110745, "value1":0.5221211963164464, "value2":4903849187407455848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195405, "value1":0.08594736781118724, "value2":2859597321511204836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051298, "value1":0.8256101721258909, "value2":290470632378621656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284220, "value1":0.27211501738111415, "value2":2134420797562982003, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792536, "value1":0.9372077558354763, "value2":7927244557057049645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.681776, "value1":0.4477409740262762, "value2":3995977364357841936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231810, "value1":0.6078487538143919, "value2":3261959160546744959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.131613, "value1":0.7340176818759, "value2":3565658759712151137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.908393, "value1":0.28199967714832963, "value2":3350748733246898915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698846, "value1":0.043919603046489966, "value2":4515802998753688491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803882, "value1":0.9014217952934828, "value2":8345388974528376469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_2":"a","key_3":"i","key_0":"d", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501652, "value1":0.45069419646062914, "value2":4400331011414194610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.240902, "value1":0.34849466754791153, "value2":3738192721082184330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.950263, "value1":0.5119937423488377, "value2":8919897235633479114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.443897, "value1":0.08318284858972097, "value2":8878946431124018702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653076, "value1":0.2711114129019499, "value2":4904832384185588956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199562, "value1":0.5179430768249581, "value2":2338622690587545147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902254, "value1":0.8456128690164353, "value2":3512723913452353513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424568, "value1":0.358235963026908, "value2":1951463002559117594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.891020, "value1":0.899730727943035, "value2":384925737643795292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242327, "value1":0.8194521049409685, "value2":1651591731883214718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064957, "value1":0.5094969888256025, "value2":1137790773894543912, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032406, "value1":0.3455331481317415, "value2":2958859262258749395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553817, "value1":0.484360827667417, "value2":3796071195831724862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197274, "value1":0.4874943149488065, "value2":1044578434864087492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.384646, "value1":0.3107814255944672, "value2":1519235564474675279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286271, "value1":0.42853641527210906, "value2":4722523490137371754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175308, "value1":0.3946055220580569, "value2":6954032044064595773, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.793904, "value1":0.06297728493572644, "value2":5259172994496601557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.062223, "value1":0.013288825435356888, "value2":6140160587654305146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.768622, "value1":0.39922765836242, "value2":6629678017270316706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_5":"b","key_9":"g","key_4":"k", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.203380, "value1":0.9727752066385791, "value2":960040697676306380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_8":"i","key_0":"i","key_3":"d", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.845192, "value1":0.9218669222513459, "value2":8161692681657646744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625176, "value1":0.7818690860470178, "value2":6286486510404426651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943535, "value1":0.004848635402133272, "value2":8498014405632225004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.204456, "value1":0.5833134681081519, "value2":7410659829376107064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.576114, "value1":0.9192372118815992, "value2":6657353407785586037, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118284, "value1":0.8313517670516266, "value2":7764880758391828110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.535569, "value1":0.24827312684994104, "value2":2040106521001592824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.549323, "value1":0.8166897779780931, "value2":1564430022376434258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881818, "value1":0.09926733907609549, "value2":3471220368243877954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374110, "value1":0.5149369120619428, "value2":2786214942305343351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770127, "value1":0.7580522569953325, "value2":4637323015723129783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178499, "value1":0.146709310096475, "value2":7936042836162333987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274162, "value1":0.4372447015799375, "value2":9083170531856859501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491205, "value1":0.022796016225098064, "value2":8123705023600354614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016451, "value1":0.9256503393490508, "value2":6970208660239633828, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087308, "value1":0.4462899441715343, "value2":1749521530182375024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094183, "value1":0.3475409881711034, "value2":5390535090747181247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.735505, "value1":0.53372762504523, "value2":8372678000097135865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.114823, "value1":0.029693120979849812, "value2":6638472287910475525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242928, "value1":0.3050035672228757, "value2":1792917800424529821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703291, "value1":0.31694612370345837, "value2":2183523770285388049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.331904, "value1":0.47208239473287594, "value2":6204971001548635721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.345432, "value1":0.4951178136444847, "value2":3855507055566939240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136145, "value1":0.6933172444157865, "value2":1571591012651895136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303587, "value1":0.05226903625691088, "value2":6994464805121874342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.604568, "value1":0.14657045529476723, "value2":2698838307937260948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376282, "value1":0.8133509413304958, "value2":8456674940436282992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843511, "value1":0.6514108937273068, "value2":7171882174493021649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888039, "value1":0.34434154548426427, "value2":5546478000801091154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663092, "value1":0.2145932369532521, "value2":1924108819168508153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178424, "value1":0.3811642005135661, "value2":5277232346465674780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.982420, "value1":0.509903018966164, "value2":1366973490493465213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.789914, "value1":0.5590817594064089, "value2":2545283978114147454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.858245, "value1":0.2990011327668893, "value2":7402472987921684090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045358, "value1":0.4729812652892598, "value2":6922132388747645101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212994, "value1":0.6311902604147308, "value2":4414936740257073237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357341, "value1":0.06207502931784455, "value2":4475293686563973956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.975194, "value1":0.41868388111482774, "value2":1854059337783469990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841014, "value1":0.21676180216590327, "value2":4591196137073503812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.978140, "value1":0.3174552368999545, "value2":1901352595947591853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.267125, "value1":0.18049579402909974, "value2":4625546630316797154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916965, "value1":0.5612711107070505, "value2":6029229351915879062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.927166, "value1":0.40285609194050886, "value2":5993132840270480501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791545, "value1":0.4639522865587751, "value2":7463548107279858536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770063, "value1":0.26886367149111556, "value2":3097459774572125541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077189, "value1":0.7572977836842385, "value2":8298753827870878088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.287375, "value1":0.621000003066286, "value2":5005851683916784910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517180, "value1":0.6660153705066499, "value2":8387087977559252312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.675034, "value1":0.9813881401454424, "value2":3741676304938251291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.554896, "value1":0.7715073370679849, "value2":6589060977722716090, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319288, "value1":0.12279901479898898, "value2":3422770912327659239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703895, "value1":0.8358860148624778, "value2":8257794540400854550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493064, "value1":0.048118109532117544, "value2":4134373887037968422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799328, "value1":0.11593192650373021, "value2":1597084067805135642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.848966, "value1":0.3986720724086452, "value2":2405658508790196207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792447, "value1":0.5248994141540482, "value2":6715402324504357457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376734, "value1":0.7992683845538152, "value2":636776368616526730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_8":"b","key_2":"j","key_6":"i", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646082, "value1":0.7040359351145827, "value2":8792341972386350386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.358717, "value1":0.15149972457126396, "value2":8525880284771508381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038854, "value1":0.6775307407388963, "value2":4497884453622086771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313173, "value1":0.6408664944912432, "value2":9170385529548968990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.209148, "value1":0.8824832965439914, "value2":6994782182036294339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.349829, "value1":0.6332729838723153, "value2":8895300460695986955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877356, "value1":0.9629884067987685, "value2":5874587318078108417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175639, "value1":0.2215504394755138, "value2":1645626442028267495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.867334, "value1":0.17245948139280137, "value2":7635160136876279780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.304089, "value1":0.4417775050588339, "value2":4663940359268319500, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.377260, "value1":0.3699273314519082, "value2":3098095013643812538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.687645, "value1":0.49899496293822504, "value2":3483082050221607102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.764258, "value1":0.5854336662763441, "value2":9096847150126021331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.395360, "value1":0.4621587456489886, "value2":4751829496417501448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702903, "value1":0.41598100403233135, "value2":3929201336867975918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107090, "value1":0.7914140452826858, "value2":3589076543012868088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.487563, "value1":0.4474861481593704, "value2":2779456470910748695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_6":"j","key_7":"f","key_2":"i", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512119, "value1":0.5021386207405886, "value2":5678715906223819621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_4":"k","key_7":"j","key_2":"h", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346070, "value1":0.55706872148054, "value2":2039287696528568559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.574280, "value1":0.3713448367655447, "value2":3929829805998447613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.892079, "value1":0.6942759872823322, "value2":7729227663415283404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.344002, "value1":0.42092378887559295, "value2":4794883782773019081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078411, "value1":0.6154878270099419, "value2":6707716797669517904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.571429, "value1":0.1378751349573672, "value2":8074751840319910755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.040055, "value1":0.6408943986817053, "value2":1079211030051415138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162869, "value1":0.5000621894976172, "value2":5126515351324361210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.390768, "value1":0.5289053805740587, "value2":7495442762687396988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259180, "value1":0.5649142133543891, "value2":4328071162032108601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842704, "value1":0.21495927580985347, "value2":2932451180465678632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650755, "value1":0.6984634581926075, "value2":5766024562422825714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612080, "value1":0.22851599558447847, "value2":5344041318095855549, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990865, "value1":0.7354893379622883, "value2":9025665650100041943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787926, "value1":0.33568512834020503, "value2":6804173344109850974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172355, "value1":0.9139245088414795, "value2":6155807217840657379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.979541, "value1":0.24199196392953898, "value2":5042790445163279739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.314666, "value1":0.3110154918771059, "value2":8906652806658780575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357408, "value1":0.8063999303874789, "value2":1651751776595868392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.288676, "value1":0.9293974960471005, "value2":7670397636585827003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293262, "value1":0.7818217756805212, "value2":8606918715162923765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.083534, "value1":0.9670843857632683, "value2":5887729702260327460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149981, "value1":0.6552447379991068, "value2":6725961755903338686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.492842, "value1":0.7092364133108918, "value2":7796995215212529618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944648, "value1":0.6256326288941183, "value2":4831063196252879183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.895337, "value1":0.4995752655149396, "value2":1408597026195247818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944023, "value1":0.634711000263352, "value2":5584697240807054271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809484, "value1":0.7426498850085607, "value2":8020687455958940501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289743, "value1":0.008820277060811819, "value2":6337782472142563494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.537111, "value1":0.14436676295527084, "value2":1783901321543975972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749042, "value1":0.9467021656046419, "value2":279529980240550876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.261740, "value1":0.20334209964882055, "value2":3677408684840846866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.272200, "value1":0.5140769192808836, "value2":3965788713393972569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391452, "value1":0.5620808926889774, "value2":3432925103242153786, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.797201, "value1":0.24306630830309864, "value2":2212488327265931040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296686, "value1":0.4958413853936345, "value2":2745010941408731624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.508150, "value1":0.1764099610149601, "value2":6087888705783067229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.150030, "value1":0.6475061379725164, "value2":4152410772475568903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406997, "value1":0.8552543065924282, "value2":6943121315248371505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798595, "value1":0.4482032174944134, "value2":4591949889203793788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427353, "value1":0.23427032816286456, "value2":8455901174434850105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.426947, "value1":0.2157937977395769, "value2":4812396978108020858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553964, "value1":0.6569888020417926, "value2":4304633456065882312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830280, "value1":0.6753558122351391, "value2":5709360383833750840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335737, "value1":0.6229413264866215, "value2":774141312393774432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934503, "value1":0.3184401780744072, "value2":992485213419150513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286280, "value1":0.19701195292124982, "value2":5131579316119627914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.027527, "value1":0.5595644624118316, "value2":7324008691917344768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.445835, "value1":0.3878002605330551, "value2":5283397242149349986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.317903, "value1":0.34865399024368093, "value2":919918660600814082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.988293, "value1":0.520877683344731, "value2":1362287203583282026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451918, "value1":0.17797518451009597, "value2":8069259017842130934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.416614, "value1":0.49438798293161773, "value2":8221530176741871409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930143, "value1":0.11046223267799947, "value2":1569220098038190448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841002, "value1":0.2351074608491269, "value2":6987153620614536862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172031, "value1":0.30378326240838155, "value2":3438948688273621480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427034, "value1":0.3726321157197697, "value2":3892410333661915590, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.996688, "value1":0.06148106153681236, "value2":7012776674540552021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178139, "value1":0.24652480255103007, "value2":7264782641186044646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.641890, "value1":0.18896088271258277, "value2":5281363384350384982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020470, "value1":0.5427453076719968, "value2":4211018551963613162, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548627, "value1":0.9467235670877983, "value2":1356346215173760766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923887, "value1":0.16868132666008845, "value2":2419587552380208808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388273, "value1":0.34607395217688514, "value2":6360018437765039879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199475, "value1":0.48700761251474795, "value2":4207130869528000977, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212632, "value1":0.7298887109862953, "value2":3855225371708173636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869081, "value1":0.3040832001677732, "value2":1636252957386132843, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458358, "value1":0.25499479265263875, "value2":7951534915845928364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673427, "value1":0.3560914688953808, "value2":4486505974707646981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934304, "value1":0.9086314287420252, "value2":6454407382033994812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077472, "value1":0.272412312785447, "value2":4055844210948982160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391470, "value1":0.4094306269697527, "value2":8737172085844354362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.507196, "value1":0.4293932426928758, "value2":7752579176339832618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.354922, "value1":0.8782836749723913, "value2":3092074071663841238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290283, "value1":0.6466038651665884, "value2":6445966108572410248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930993, "value1":0.2534798517547614, "value2":2212538268968967382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078571, "value1":0.766186870171555, "value2":2276988139922811436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512143, "value1":0.26492881415396713, "value2":7960469334919381452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.664663, "value1":0.9293891578195701, "value2":8363525489715122179, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.859809, "value1":0.5317709711783597, "value2":5646146960428890136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755810, "value1":0.5033479266486215, "value2":1442438201922477703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.014944, "value1":0.2736949295034319, "value2":4543402002213224798, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.019842, "value1":0.06301734338783647, "value2":7848421926410748778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.006460, "value1":0.7283128915704968, "value2":8696821133231766119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.495665, "value1":0.22357458621454834, "value2":5013935622275517771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_5":"j","key_9":"c","key_0":"k", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.829438, "value1":0.08516153321035196, "value2":6346747105122069788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120479, "value1":0.6330301971897442, "value2":3074605270601257967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406377, "value1":0.9646902889346889, "value2":9116844958473667659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258382, "value1":0.9290313183449712, "value2":7925410787408077827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340705, "value1":0.7460138699765932, "value2":1834174806073680705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.660263, "value1":0.9250659950303504, "value2":6999248688568055694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_6":"g","key_0":"h","key_5":"f", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252290, "value1":0.718659365662016, "value2":414864145410727491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404912, "value1":0.5143974391105166, "value2":6178244170296726344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.355231, "value1":0.7285417682392774, "value2":4099314188219635809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232613, "value1":0.9628251247534773, "value2":2753347640662156751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042238, "value1":0.40260777202369225, "value2":6405687740970757268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.059614, "value1":0.6706802975294311, "value2":155105754975301806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335427, "value1":0.03875943420721546, "value2":2621496673126808846, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791994, "value1":0.4069226188854093, "value2":8911123850930154192, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.152311, "value1":0.001001700535447483, "value2":8160805255392424399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.806126, "value1":0.09279090865200551, "value2":5165528109888127485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181230, "value1":0.34297422042642206, "value2":5952935302808125089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308758, "value1":0.39399952685268963, "value2":2291587630268260397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438229, "value1":0.008760206538330227, "value2":4899262456651209905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253853, "value1":0.7708129134752543, "value2":5591299897276344295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388133, "value1":0.4749440177332357, "value2":1978415349188199488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612871, "value1":0.07573141562457633, "value2":5953047090784791116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.481277, "value1":0.12674805322449534, "value2":8950535310944284362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026417, "value1":0.13620073115890582, "value2":5838422267573225081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491782, "value1":0.6697421435610075, "value2":5470323240276494634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599889, "value1":0.2218252683274155, "value2":3863404473966363047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028452, "value1":0.8531657224160458, "value2":1800467248475944001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234267, "value1":0.2783481987951185, "value2":6997165241525177240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369705, "value1":0.5636516367093837, "value2":4958650393346745090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201031, "value1":0.628281679414999, "value2":3983210515283198963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.519582, "value1":0.950842451530494, "value2":8333725115642196357, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.800667, "value1":0.47362573201727787, "value2":5531339313881866007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.022715, "value1":0.7456986400451252, "value2":6375649752818307396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026963, "value1":0.013441045212029653, "value2":8879122156886303116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849167, "value1":0.7209297073484169, "value2":5571977114960029707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916214, "value1":0.07478238462060836, "value2":4957974530052652715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.801842, "value1":0.5497676040427942, "value2":7068572572815929980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.180884, "value1":0.16934771421243305, "value2":4129529446885753081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742335, "value1":0.17996128783258833, "value2":1612889506435491518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.124131, "value1":0.6211285761421723, "value2":6936437753872174981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_4":"j","key_2":"c","key_3":"h", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.626664, "value1":0.1412142921060844, "value2":7340286601023012024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_9":"k","key_7":"e", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534599, "value1":0.1824543867650024, "value2":2710596260731528329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.198428, "value1":0.563515863993397, "value2":8658671925349980639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939981, "value1":0.5602221119588965, "value2":1332664463057731952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749870, "value1":0.27530155912076365, "value2":3294589175109845443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501991, "value1":0.28276078133217086, "value2":6074437039967216201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886807, "value1":0.3878328960046618, "value2":6751632027412168403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_8":"e","key_3":"d","key_7":"f", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.911289, "value1":0.6411363324230984, "value2":8252067497790790379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.696807, "value1":0.43753398310350267, "value2":3790908907624313334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319319, "value1":0.8437649419939369, "value2":4536112071098596928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.734427, "value1":0.7768336856428384, "value2":577814262800235815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232482, "value1":0.4639181971383516, "value2":915233793101361418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033045, "value1":0.6840737140076493, "value2":6077817095892263388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427451, "value1":0.7412294644798527, "value2":5067651533453787714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.086307, "value1":0.884916959850894, "value2":6213157740517107791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032588, "value1":0.7692661642362982, "value2":3255905758159811447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129133, "value1":0.9868818324607139, "value2":2840159926392322636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005933, "value1":0.4856659261691267, "value2":5401836471710156191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078678, "value1":0.43872499297363293, "value2":7610348477992750069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.101709, "value1":0.6553052079447295, "value2":5590613299241898040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.691765, "value1":0.5500719022328135, "value2":2311787078857040546, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.609841, "value1":0.20437362444327167, "value2":5976709159513367946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.262383, "value1":0.988890785552926, "value2":5073782211732867167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_5":"g","key_9":"g","key_1":"e", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382099, "value1":0.7056400640121613, "value2":1362786206799762754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.910924, "value1":0.740118438943497, "value2":5204241364776707180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526567, "value1":0.33926045568377805, "value2":8282435047009117311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259549, "value1":0.6968073477407744, "value2":5508984811117632066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_6":"a","key_2":"i","key_4":"b", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.575379, "value1":0.9204583134801098, "value2":7328623658034172463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.021181, "value1":0.8037794671210114, "value2":8763012613490416638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666802, "value1":0.735290222214112, "value2":5898346647756703163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836023, "value1":0.9774841401477888, "value2":262883488175479807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085239, "value1":0.2006523822964799, "value2":6927806981819194391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722778, "value1":0.29750991877070054, "value2":7269146382542606407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585823, "value1":0.7790122698056057, "value2":5454870737254023995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966399, "value1":0.9342783027092735, "value2":1736810747318512793, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.363378, "value1":0.5421736888212692, "value2":8680345426115136675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548518, "value1":0.6180502927475818, "value2":576215651567739722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169252, "value1":0.8808696517971845, "value2":3266464067570846970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032461, "value1":0.3854322648479189, "value2":5351736110848205547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418352, "value1":0.9282836374954182, "value2":4950655664039540565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.567245, "value1":0.06221685577996024, "value2":6376294794655595743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.607825, "value1":0.8284877360321535, "value2":8836357162970050736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.002720, "value1":0.9429757239354525, "value2":3712886085226721402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772087, "value1":0.8045038867848936, "value2":4759138460922621996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515031, "value1":0.2984015395169622, "value2":2069584478283962675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.141150, "value1":0.6666900797494949, "value2":268184099180001198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.102173, "value1":0.08736073078451814, "value2":3669960405293392951, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.825677, "value1":0.883585003117732, "value2":4900464079888733154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.613326, "value1":0.2003226722632331, "value2":3796686759340146598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.965080, "value1":0.24041472099907274, "value2":7454089011589464567, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139395, "value1":0.8846967167208682, "value2":3451086131264105278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709618, "value1":0.40080113749800483, "value2":7347906568423400039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.490979, "value1":0.10232414632757307, "value2":6898384399737921493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.419810, "value1":0.9521194603097142, "value2":4165710553126975748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142251, "value1":0.3656369505236563, "value2":2656042717790241170, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578775, "value1":0.42161623774225904, "value2":7959202830983535548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340081, "value1":0.47208224231080925, "value2":1119972655203312039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_3":"e","key_6":"j","key_2":"k", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201301, "value1":0.686282703890728, "value2":1347056528311053195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.905450, "value1":0.1504198445233754, "value2":4153183079049561671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.774660, "value1":0.04197441149840889, "value2":2892741854923361469, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.223828, "value1":0.9913278205978917, "value2":2634921019433133187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145135, "value1":0.9764589636119458, "value2":1863333686324511619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051457, "value1":0.9664880803510422, "value2":5468675095205300977, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.471202, "value1":0.9438594012185837, "value2":8711551131933841419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438432, "value1":0.7332734301616686, "value2":6736601196370014167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.018444, "value1":0.6166410041335356, "value2":7430067770903432071, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042785, "value1":0.3083845760668554, "value2":7529797289952981250, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.500528, "value1":0.41360737530731173, "value2":527946418421906650, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.639029, "value1":0.28120003518234893, "value2":7333246850823447562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118602, "value1":0.22780592472876143, "value2":5562449967641903101, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773447, "value1":0.34556648930141964, "value2":3454919255414457289, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.751050, "value1":0.5691145183911931, "value2":3420985612149611112, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.967122, "value1":0.010774686710818307, "value2":1857363403020110299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211203, "value1":0.5071144540997159, "value2":3201415543625346596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966034, "value1":0.10396047428400347, "value2":9014157472738440520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253319, "value1":0.43609058248995825, "value2":1183645881443278471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104187, "value1":0.4324653404306875, "value2":2865322183450789108, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185518, "value1":0.8897028996083935, "value2":7566651396044601727, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930391, "value1":0.8041371194633276, "value2":7871648174569158725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163568, "value1":0.9301616644769536, "value2":7748526202149251971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869278, "value1":0.88349271636991, "value2":554007324522749523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.213408, "value1":0.664832495384472, "value2":2758153864408048603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451136, "value1":0.3702265627491039, "value2":494580882184796857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915009, "value1":0.6519575592422996, "value2":6479471856073874185, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.581364, "value1":0.0703141245253348, "value2":899491222964044388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142057, "value1":0.9957531092014932, "value2":5996532724034214303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618561, "value1":0.31275866348570913, "value2":4261560321529985020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526426, "value1":0.8632813331612903, "value2":2280841193196363442, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.640741, "value1":0.4671010753968709, "value2":2523475617435456826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686774, "value1":0.37684937337218677, "value2":6568317226871778750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.547284, "value1":0.9412532971618462, "value2":6396494980432088364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630465, "value1":0.10697292615695254, "value2":6790210135628014818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156319, "value1":0.10115374361141051, "value2":2774605952238778168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742080, "value1":0.4345761934130475, "value2":6896687242300333410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662119, "value1":0.3886014719849511, "value2":2171378209788759856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158805, "value1":0.4473032074837117, "value2":586039993535786710, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025398, "value1":0.0651048756141861, "value2":3136658443010387115, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189314, "value1":0.11574796728414168, "value2":5756732023607570312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_9":"d","key_1":"i","key_8":"g", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469000, "value1":0.3187212262703272, "value2":4880377874531438837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698910, "value1":0.9021085577061128, "value2":8314934978791640839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228153, "value1":0.7939593629687762, "value2":5721748474729655997, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.174833, "value1":0.47180057752688764, "value2":74850886504396865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.373682, "value1":0.9894401032981868, "value2":2443814427210362559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.230070, "value1":0.840516361148932, "value2":5295744989016708862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595645, "value1":0.2788762608865926, "value2":898774770804749270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583440, "value1":0.38764558337295657, "value2":4879715356875855821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.818979, "value1":0.08550208893092236, "value2":7063465379400979205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990629, "value1":0.9735905702213057, "value2":9121417987396463994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.904697, "value1":0.06007475161136332, "value2":1399260524832392153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832693, "value1":0.9560818456694564, "value2":6313771830503199817, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.273330, "value1":0.5911156951363433, "value2":6957059227202260296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.069691, "value1":0.43486640701864104, "value2":5573463606213032969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107074, "value1":0.28194511495703845, "value2":5008401247473276713, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.133554, "value1":0.6572805795366599, "value2":1430282776047161882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.994425, "value1":0.7317293344785567, "value2":2161864319170825977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703844, "value1":0.05169084281485511, "value2":7389845001390437940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667112, "value1":0.912614805848696, "value2":7494408396730454879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175192, "value1":0.5654088358678295, "value2":7414573375849683263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.205436, "value1":0.5764549187518897, "value2":4315302504250390302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.816073, "value1":0.3021595298462732, "value2":7835612651483818643, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465800, "value1":0.7224511737802353, "value2":1475815608612889931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836860, "value1":0.13043025990273924, "value2":8725370796816365579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110112, "value1":0.2182570483232002, "value2":4212625314776126266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.401613, "value1":0.3660560702023913, "value2":7136594683517469774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772305, "value1":0.6420871129325768, "value2":346199260296482441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142556, "value1":0.18492252698973582, "value2":2410884666534888495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.710999, "value1":0.03498389655141747, "value2":3857815905441742992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588844, "value1":0.6843616917435196, "value2":925508006135613377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902010, "value1":0.7892717951683124, "value2":7453201615304707478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071475, "value1":0.8597541524726022, "value2":5247483360729978449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886399, "value1":0.896477399685476, "value2":3245685435537096080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683519, "value1":0.7125828488615462, "value2":9221328263547905112, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650600, "value1":0.8976371405804583, "value2":6562020796772331341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.316077, "value1":0.8989654150875539, "value2":2315139769444859937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.844010, "value1":0.027932169490326376, "value2":8266191397214577403, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_3":"b","key_1":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.732165, "value1":0.9768852287807686, "value2":7871357409427659276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.644252, "value1":0.3652027068824268, "value2":4777422302644847840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016877, "value1":0.4774488788727276, "value2":1359637185519527638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.121368, "value1":0.07637107971325026, "value2":225307864650781063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.661741, "value1":0.9526993459994305, "value2":1751890497684728484, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534867, "value1":0.5672837246804853, "value2":3948370290656301760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.400506, "value1":0.312527577003677, "value2":3200609992091774001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023204, "value1":0.9498281171738407, "value2":1775257641053296989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_6":"c","key_7":"f","key_1":"a", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650859, "value1":0.7200243703698949, "value2":6459489446120996262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125434, "value1":0.7513756216852647, "value2":6502722962310909561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_5":"k","key_0":"k","key_2":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.861634, "value1":0.22132024628333183, "value2":4213082281103637151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.616718, "value1":0.7492815151768418, "value2":8785198245169443750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618495, "value1":0.5367662090666573, "value2":311068731789452160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090805, "value1":0.6461806005354355, "value2":1567126231539851190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.704866, "value1":0.6742157058257261, "value2":8258394004417852586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178045, "value1":0.9347807954648532, "value2":5391643132437615590, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.903981, "value1":0.04205578457439769, "value2":4267367541005494603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038592, "value1":0.9632414356901877, "value2":7847330231249131972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753078, "value1":0.025682952887214512, "value2":4407969986032631485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.551387, "value1":0.9127538521307936, "value2":5532887277300530226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340315, "value1":0.357568695323674, "value2":5839322022227051810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253088, "value1":0.37455084749823464, "value2":919408856650004667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.061612, "value1":0.08807754228035809, "value2":7674978240888726818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350886, "value1":0.9187800273401259, "value2":3985979173089262193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.997832, "value1":0.3332675485121975, "value2":6884650671561667205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376107, "value1":0.882485292013383, "value2":980580007402095853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_5":"h","key_6":"h","key_0":"b", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666200, "value1":0.543727834482791, "value2":7683502472635473948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.690345, "value1":0.9227402770528923, "value2":5963421474121503099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831896, "value1":0.8899731417411955, "value2":3119002755449635021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583901, "value1":0.46723052632915907, "value2":4738569017001620646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254234, "value1":0.07205706866452416, "value2":1464209096722519553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.940932, "value1":0.4880590550999508, "value2":3249029228406137695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.819723, "value1":0.9180382594935601, "value2":5531777988899534137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.899553, "value1":0.5017555686141989, "value2":3526732457056334830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364242, "value1":0.726219019872585, "value2":7463421321863859910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591997, "value1":0.18380773538675038, "value2":6489374408484526555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268627, "value1":0.43767329135389277, "value2":7591255891965320507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668426, "value1":0.8905993514345967, "value2":8025287728753470033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_4":"c","key_2":"a","key_3":"b", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980089, "value1":0.9629455057472474, "value2":8613343366791814003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_6":"g","key_3":"i","key_4":"d", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.561494, "value1":0.11038616417110533, "value2":5718549668479051731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031316, "value1":0.5572446441362652, "value2":9073355869031665981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913579, "value1":0.40513173097972344, "value2":1392267543875933322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469802, "value1":0.7197223556739805, "value2":8312522501295372114, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216239, "value1":0.38424010939993836, "value2":3429512093250859609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053128, "value1":0.12167032010014549, "value2":2500133776481013473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_9":"g","key_1":"d","key_4":"b", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755093, "value1":0.1432079670350677, "value2":4460076765766388831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939117, "value1":0.022396858430100586, "value2":1677736517466813137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730771, "value1":0.5975163721595746, "value2":8564236690483063617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.763913, "value1":0.11111651544548351, "value2":2128389252611471929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225483, "value1":0.1857169909861624, "value2":1459664257900170929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.557521, "value1":0.10580492023702627, "value2":9211924519949290338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.398250, "value1":0.8985634717833737, "value2":2690743666091944740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.523236, "value1":0.8737716900990798, "value2":3717036232567418933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465788, "value1":0.8763409298655005, "value2":1486778905892801100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185627, "value1":0.4834833602007871, "value2":8283437701946785636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129438, "value1":0.4506377660858689, "value2":8577339860951120030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.937075, "value1":0.6005010987020736, "value2":4798896188528165681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.387681, "value1":0.8777928078176396, "value2":4677891530420984908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.074680, "value1":0.015131030176877407, "value2":2652924707092709319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032439, "value1":0.7762590461709376, "value2":5555968285860120909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107619, "value1":0.0013683674938652786, "value2":8724767287173452904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199777, "value1":0.2705573241568207, "value2":5731172518242608315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.883455, "value1":0.440980354957089, "value2":717450023283511313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.065153, "value1":0.7805040031236675, "value2":8249819238828017763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028771, "value1":0.4266508385350679, "value2":3525220539697375602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417030, "value1":0.9344530676800743, "value2":2840008251192450241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169783, "value1":0.4944137362673325, "value2":8143727104076341090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309365, "value1":0.8233798931001785, "value2":4143752746550875358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405563, "value1":0.36473772798628423, "value2":8368477347330201864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.080493, "value1":0.45312759136632275, "value2":5757367910796479605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591589, "value1":0.5574497820130245, "value2":4692942231335997930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770417, "value1":0.20880923062348417, "value2":4939040557737590500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923414, "value1":0.8006055497749137, "value2":6979326850811422524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249012, "value1":0.022621459630026125, "value2":6570512292324514521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787566, "value1":0.5692664113038692, "value2":3307063324843670478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.236668, "value1":0.706981031305401, "value2":6240895241125781235, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590979, "value1":0.9325091299580149, "value2":5714058795401825599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782982, "value1":0.8159590900452445, "value2":3981533983292904484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449106, "value1":0.2421958898621009, "value2":8819094433559346153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_3":"d","key_8":"c","key_1":"i", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.721732, "value1":0.7320938316381037, "value2":1840371677400026121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417920, "value1":0.3777544081714696, "value2":2517494055558796825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.467240, "value1":0.07736271112081412, "value2":3355581288729910614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802307, "value1":0.06164218213969723, "value2":6962215396306940249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599871, "value1":0.2948698648725146, "value2":1065059716477496547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.109220, "value1":0.11737528756979723, "value2":4468678220083400352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156222, "value1":0.6294465098457002, "value2":2835781199255992964, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697592, "value1":0.9317953948461911, "value2":2221981181398322475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785752, "value1":0.4235402956347778, "value2":5956435747949559681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943686, "value1":0.28355998213096456, "value2":3638916038813769869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.010104, "value1":0.22773696044954786, "value2":2360395284842638669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.239178, "value1":0.14419564768805598, "value2":5315118297850048019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136093, "value1":0.3244550022291072, "value2":164222067931855660, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_6":"i","key_7":"b","key_2":"j", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968733, "value1":0.7486844786821671, "value2":6287914493893452990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252999, "value1":0.24537563867085085, "value2":1626574969302079091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.885743, "value1":0.9307243682752692, "value2":2858030878483913913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.631011, "value1":0.7084136481624964, "value2":2044467985498488215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.853026, "value1":0.04425979450750048, "value2":1653452661642691666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.991236, "value1":0.40886253057738015, "value2":7871621921656029433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.207156, "value1":0.37687587966259445, "value2":1943347287070225574, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.708313, "value1":0.5785225454675275, "value2":1380941757138302107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770913, "value1":0.9188180749639426, "value2":3884885787383779114, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418857, "value1":0.9998412287287592, "value2":1886115929568662728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.850257, "value1":0.9796804114853653, "value2":4397109710918249601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339971, "value1":0.5089698391144044, "value2":8237713822108701891, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913742, "value1":0.9646995309003717, "value2":1173067889283022862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067798, "value1":0.6899692743360695, "value2":1520636933221389194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533696, "value1":0.8606713340189989, "value2":5737213954118900238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212415, "value1":0.39075235526474594, "value2":6592432468398327445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054486, "value1":0.34784858262092483, "value2":7843113252021213719, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479147, "value1":0.5614107726843738, "value2":8690368331747046125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_6":"d","key_9":"k","key_5":"j", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.622976, "value1":0.06775340365098863, "value2":5564636394288500497, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.315983, "value1":0.33546786831791003, "value2":3930900696554115736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_6":"j","key_7":"g","key_3":"h", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.370627, "value1":0.779233269421853, "value2":3716543858406653649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.969610, "value1":0.006014494345107452, "value2":8710808689518483016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.724048, "value1":0.07768942947746518, "value2":496409043717037404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590602, "value1":0.8288610653786054, "value2":1102610595846648990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297543, "value1":0.37578423768641916, "value2":5536977376920161050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.155394, "value1":0.7083524353328224, "value2":4002932567237644088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.459881, "value1":0.4873646265203241, "value2":5916271319076231574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791505, "value1":0.5808298895011346, "value2":7841892156688279555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067541, "value1":0.09121699649729094, "value2":8699612203169033522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138436, "value1":0.8850890106644654, "value2":9069485496265005186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.812668, "value1":0.7098616610527902, "value2":8132915524494658053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.190668, "value1":0.2761598493860578, "value2":651770389362313830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.220527, "value1":0.17936744276680264, "value2":310375881133353905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428315, "value1":0.49075022964169346, "value2":6493345006460964913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595999, "value1":0.2755442227044263, "value2":4862778030716507189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.719052, "value1":0.21758648924177262, "value2":9168976451471290619, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_4":"c","key_8":"i","key_0":"j", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166119, "value1":0.5187037640605351, "value2":7936720027148899998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.361344, "value1":0.7201073761748152, "value2":8725593566170852352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254415, "value1":0.07641284618924576, "value2":7978515171853518726, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.981898, "value1":0.027541030165662613, "value2":6121611839007131360, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625954, "value1":0.22759978693770738, "value2":2989185053134172992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350547, "value1":0.566849398731457, "value2":4327099372915065254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799208, "value1":0.11746266545752859, "value2":7451515689230242023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.322064, "value1":0.26907348347198895, "value2":262434464387240754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.919974, "value1":0.6405108138252255, "value2":7284316699831173435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_9":"f","key_3":"k","key_7":"c", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534467, "value1":0.9814271230191226, "value2":9214896283898892364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.291823, "value1":0.7119821636486502, "value2":4137668561036791084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773725, "value1":0.05006867968524374, "value2":4350082823034910080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.642706, "value1":0.8326077028765216, "value2":6301127653055186823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.624652, "value1":0.6013140948710498, "value2":7150151669380473161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663653, "value1":0.6471420893487286, "value2":2842026748176794650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533665, "value1":0.40332997931033865, "value2":8939087210798770840, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915959, "value1":0.20834521315367086, "value2":7895351758501362895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.238383, "value1":0.5265667505785955, "value2":2676293106518263900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582878, "value1":0.3420886897063641, "value2":1233598437288047014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.489464, "value1":0.6718160824398975, "value2":4468628860088387255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.649028, "value1":0.7293826474439736, "value2":3127060444172041547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254010, "value1":0.8457444172560414, "value2":6200016452741633767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.536370, "value1":0.16874770298054162, "value2":7575698105239291032, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809584, "value1":0.343753090691686, "value2":4854239248106071203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.334375, "value1":0.5540567040963329, "value2":2646076573445947055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826524, "value1":0.7273180695893363, "value2":6564419565708307637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071563, "value1":0.26385268835278003, "value2":4070457585628644142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225965, "value1":0.9321283899737617, "value2":7702159669474089190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458779, "value1":0.10195409940683517, "value2":5987602857042259052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501592, "value1":0.07688705594071264, "value2":5259725214710983339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.385240, "value1":0.8254376912417642, "value2":6760238975249610189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_6":"f","key_3":"c","key_4":"g", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.760329, "value1":0.7530171422146449, "value2":1097173296466796516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.597269, "value1":0.2403481802375547, "value2":5050041855629937310, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149349, "value1":0.43132198586517284, "value2":3287400838900732872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.499626, "value1":0.737892470781718, "value2":4461721794139931373, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449953, "value1":0.8639558735483506, "value2":2903703899322529602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.912253, "value1":0.01605090964788225, "value2":5781647102967996198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.4054065908363409, "value2":6440329633133258472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038844, "value1":0.3763702930516151, "value2":1302491802347098686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.453739, "value1":0.6265318797324549, "value2":819075302260626614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590829, "value1":0.5479451127212884, "value2":8557856578535483978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714164, "value1":0.8513319596110408, "value2":3981392010073089983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.767546, "value1":0.9500048124431112, "value2":2193055476612398128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.685203, "value1":0.9241278767545394, "value2":5146494836784446946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171752, "value1":0.05193751579817584, "value2":2407542541423061968, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620585, "value1":0.9449477088124408, "value2":293003481714135625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197927, "value1":0.9081143587822024, "value2":336470947794228646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_6":"b","key_1":"i","key_2":"d", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730165, "value1":0.4106378302625472, "value2":2305406403284031522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993455, "value1":0.5015628664485756, "value2":3942941750281172936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376150, "value1":0.08846010494510266, "value2":451409939393211294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578966, "value1":0.5184884584460681, "value2":5013311361860229047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980632, "value1":0.6414794039381226, "value2":4661775189685128413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543153, "value1":0.7779597819383781, "value2":1678453405944867509, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.728157, "value1":0.4026694195455883, "value2":1247937331490084335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039033, "value1":0.6078130901686731, "value2":6572519819620597387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020089, "value1":0.16501689332992148, "value2":2135299756770180375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159279, "value1":0.06812322597368, "value2":1853312335979487824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225355, "value1":0.31761717143878, "value2":1547635249117783016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.579591, "value1":0.45080215259792744, "value2":830280592135153306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.989155, "value1":0.3246650009893744, "value2":3832412454221483471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.898794, "value1":0.9353227689941251, "value2":5019805485542488656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_4":"g","key_1":"d","key_2":"f", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.325402, "value1":0.40452074217950085, "value2":3779976710102146862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.846682, "value1":0.8078556003817017, "value2":4563926776477066326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.538107, "value1":0.6916244463696708, "value2":2616499684497809047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601707, "value1":0.0005961819411869942, "value2":8421789457526552579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.474772, "value1":0.40247038234287125, "value2":691809847413019723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835874, "value1":0.24276779175523852, "value2":6899854861109897252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.366826, "value1":0.8778001737651254, "value2":4321676554804962434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826448, "value1":0.8031889549998205, "value2":5554188972077808449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.432681, "value1":0.15017093541000798, "value2":7513115457016757405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.353975, "value1":0.23702527674889587, "value2":1810989557365945672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303934, "value1":0.2677306312444, "value2":430799962318472061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163809, "value1":0.8138323281483592, "value2":8246241283973870498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662355, "value1":0.9144882781553422, "value2":4359470640610896382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120568, "value1":0.8056479342778862, "value2":6170593949410616781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.541822, "value1":0.248047143451303, "value2":8344606419798048221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.147354, "value1":0.35385149944297156, "value2":6006537614889971434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_8":"j","key_3":"i","key_4":"i", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.116235, "value1":0.6774547184266492, "value2":5503266725598369859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.563395, "value1":0.978792023344712, "value2":8139355376515169099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202304, "value1":0.42453838538082794, "value2":8767087562530261434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553168, "value1":0.813931872578598, "value2":838817774250793063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.476339, "value1":0.2705647850316454, "value2":6004383053661805919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067348, "value1":0.23127791420697405, "value2":614040873255409897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.063340, "value1":0.04494057741571453, "value2":7945808906854676018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877369, "value1":0.188685408716376, "value2":5653852277013510880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145318, "value1":0.6394711610065485, "value2":6391486639411558835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171038, "value1":0.9902721723985418, "value2":3163235560331954355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773084, "value1":0.40123815528773027, "value2":6772197516975478771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.976169, "value1":0.608513336182585, "value2":3898075135558136236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.460540, "value1":0.027884469593113505, "value2":1806632468596114826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.411909, "value1":0.6437101253853217, "value2":4628507915366383990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.055892, "value1":0.4209111843259915, "value2":8498738002897413709, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684315, "value1":0.3608627471653678, "value2":8911984339628672502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181887, "value1":0.04106030144724243, "value2":912973087706429869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.127221, "value1":0.8723975492127802, "value2":4688994221703849298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881735, "value1":0.8013556912079172, "value2":643258613153646569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093171, "value1":0.9983142632275949, "value2":2242609892773836818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702516, "value1":0.9951260414368723, "value2":9110402286568364937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712584, "value1":0.10494403348655837, "value2":4745554958401980793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.699467, "value1":0.9010413033988799, "value2":9007036540377523860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.132646, "value1":0.06467587338211019, "value2":1060827532336676254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.552157, "value1":0.4330362437559917, "value2":7496787358324320393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888799, "value1":0.2576583993305694, "value2":4018360458617386281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.498269, "value1":0.2224059001256479, "value2":4065063745853139952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588721, "value1":0.14516718870277823, "value2":3512331805580372297, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583233, "value1":0.7489522543167118, "value2":5610699138607696953, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290155, "value1":0.44273304014865483, "value2":427744180869204041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_9":"e","key_5":"a","key_8":"c", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183829, "value1":0.6345267428968718, "value2":1846381248733979645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.323098, "value1":0.012439339538142696, "value2":8869073391299009493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634145, "value1":0.9267919956221178, "value2":2118172993893609219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794582, "value1":0.8353549954749553, "value2":1351674467301929869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.008550, "value1":0.6040059322178531, "value2":753170312236461842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268043, "value1":0.31922185771526074, "value2":2365194637514291305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.857933, "value1":0.8196275302693926, "value2":4478227043901542114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.645585, "value1":0.17157442528490047, "value2":3055405974571687053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.277791, "value1":0.8508212157393535, "value2":475170132890875463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123450, "value1":0.35015180460716977, "value2":7931493690032322348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.918435, "value1":0.30789613557477924, "value2":4634204529169845529, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.747396, "value1":0.7119174217020244, "value2":5113967535733338451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346547, "value1":0.4105703222986521, "value2":5634326652610778742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.873833, "value1":0.5726619572582797, "value2":4834039219162125836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839553, "value1":0.3474160702051541, "value2":5912402851303338785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924943, "value1":0.25193375971882537, "value2":3157228766646586798, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309520, "value1":0.4856065585160012, "value2":12512784025771269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612385, "value1":0.06742962293903079, "value2":5561454087904294314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634744, "value1":0.8158802826640159, "value2":4407063105915178233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524442, "value1":0.05056002141365119, "value2":1069310785755082516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.122461, "value1":0.8648592352685478, "value2":2607760955410972754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308553, "value1":0.8340608354185821, "value2":5091217112123016682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.073014, "value1":0.412293314616913, "value2":1754188674597018639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382506, "value1":0.7914084871078703, "value2":3113230598922027169, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.494019, "value1":0.22683994673921629, "value2":345509562584529791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770833, "value1":0.36946692588064045, "value2":3634543347045698698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374724, "value1":0.7674203492653544, "value2":409087874753033650, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.33760436450813625, "value2":6378216485000854161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753898, "value1":0.08146706117124274, "value2":4619087622758440080, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.058417, "value1":0.0895913841094616, "value2":7775148933653129917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.889748, "value1":0.9025957275835974, "value2":9060698938664882878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.429099, "value1":0.45935248830386766, "value2":5813431686543708617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.415280, "value1":0.8845089283715811, "value2":6384109853170568567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956220, "value1":0.6163283411653032, "value2":7939487848850431764, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.813125, "value1":0.3306736514104797, "value2":717112968732186535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.464126, "value1":0.23519387446018714, "value2":2061991374504688132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.336194, "value1":0.6334359034532542, "value2":5464410230276800258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722970, "value1":0.6104432497384834, "value2":3298076355265173187, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.246842, "value1":0.8069861206050694, "value2":7793368021912415700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465645, "value1":0.6879012241509205, "value2":4196934016416565115, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876004, "value1":0.06724546167889978, "value2":7203308436755461322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667298, "value1":0.6826712318664662, "value2":6402839614176163878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.329694, "value1":0.7040366253423608, "value2":1262392626609016984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591566, "value1":0.9312106031712911, "value2":6293442498282971529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.559072, "value1":0.4448009704050545, "value2":2669159126008899565, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431852, "value1":0.8238849009316198, "value2":1021818236630467563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722762, "value1":0.5948459056041813, "value2":6253806025921861085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339501, "value1":0.14816011629555237, "value2":2437202342857552183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313849, "value1":0.14517234363613402, "value2":3032490535727906731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974064, "value1":0.5282428591902778, "value2":7306937940345849875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930949, "value1":0.6595519497499615, "value2":2818227237339798252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.995836, "value1":0.41346641045143234, "value2":457793590293209131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.527926, "value1":0.17089664849686367, "value2":5825724904068162921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_7":"d","key_1":"e","key_6":"j", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258906, "value1":0.5587901020638738, "value2":2247971244413895345, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232248, "value1":0.9182710496214209, "value2":2114818465791862400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.075755, "value1":0.36489084685089246, "value2":294204895726685309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.726685, "value1":0.6513663450138257, "value2":741257611118540876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794966, "value1":0.07728058455296861, "value2":6115920270410886698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.954148, "value1":0.6576477115837028, "value2":8442517348621889353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.305446, "value1":0.8417215600157095, "value2":3960819244598684350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749210, "value1":0.905397326364665, "value2":7392448658664215427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479499, "value1":0.8192850944227448, "value2":1109448037735485747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_8":"b","key_4":"h","key_5":"a", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228850, "value1":0.7549298352802428, "value2":2475887390541907125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.256344, "value1":0.6191375565525401, "value2":6629221972448932333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843715, "value1":0.4048974244394111, "value2":2052496354441233400, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.257771, "value1":0.11131625744600283, "value2":5527432282921695963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033616, "value1":0.18479886541727453, "value2":9028612437715086928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832552, "value1":0.6959404377768793, "value2":2173084661524542898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431933, "value1":0.7816557853866045, "value2":7424533234329790131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.264080, "value1":0.7382347817860379, "value2":5213561271316161382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085647, "value1":0.4523244522101, "value2":7889921400021958159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004893, "value1":0.6556889205138186, "value2":3025561804921198898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339616, "value1":0.2999169933424259, "value2":8184676717410157627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216679, "value1":0.42313229610434316, "value2":4008780382662754625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418495, "value1":0.49555776117242517, "value2":1018253790078366875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_7":"j","key_0":"a", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033786, "value1":0.9209475895817386, "value2":5569575246133894540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620702, "value1":0.6494468637932249, "value2":4953667604093058532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_5":"b","key_7":"e","key_2":"a", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.706504, "value1":0.12760690080118256, "value2":6290715519152270329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.483588, "value1":0.5095141564714446, "value2":6862972202733799902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159429, "value1":0.90417660237305, "value2":8461823351624136781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.868532, "value1":0.20009311827615117, "value2":2269303027016825167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.030853, "value1":0.07427016371278365, "value2":6524107618104112365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_7":"c","key_9":"e","key_0":"b", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625191, "value1":0.3705374970999065, "value2":8940487375802886850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274579, "value1":0.43745692673872216, "value2":7664930411577220439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428630, "value1":0.527310522539046, "value2":6084859156132058763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.594595, "value1":0.21322470607998698, "value2":3416879710307779950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.920870, "value1":0.47127845699263055, "value2":4564621285620268941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930512, "value1":0.41802427118606883, "value2":8549501771062307593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977996, "value1":0.1610907563584878, "value2":6578022034712623821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973727, "value1":0.9224145581231933, "value2":8668945457313971984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421891, "value1":0.40172827078543016, "value2":2817865116264178423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683482, "value1":0.8428158825760677, "value2":1087743773064912864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.001802, "value1":0.5075672608227982, "value2":3748937813625482336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924538, "value1":0.6184538471340483, "value2":8049426804122873881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427400, "value1":0.23484522887445405, "value2":360795956748828928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.371453, "value1":0.14713938237360089, "value2":2782450449945692027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105401, "value1":0.7787234859135739, "value2":7103390400724679707, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608837, "value1":0.9083492888699478, "value2":591947190434298439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839636, "value1":0.20678896707817307, "value2":1816652983048222981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.798496, "value1":0.27959153848419394, "value2":6453301245071446957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_8":"f","key_1":"a","key_2":"g", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079294, "value1":0.9921346825877959, "value2":2610009567825131673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.932174, "value1":0.616531393243407, "value2":5793240958146632836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352378, "value1":0.5817027431697587, "value2":1001435181122015935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.892678, "value1":0.09115452403074104, "value2":5100882205034466347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.976748, "value1":0.223689480612581, "value2":4832346905388106952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348707, "value1":0.4785116775888147, "value2":1474405574460091730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285197, "value1":0.2545105724094511, "value2":2069757700743153534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_8":"a","key_3":"e","key_4":"e", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886496, "value1":0.6267242677514145, "value2":5390965713876988864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_3":"k","key_8":"b","key_0":"h", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.106653, "value1":0.29720904047717794, "value2":3776030286792185916, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.360157, "value1":0.5546347320924953, "value2":330359320098225031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229710, "value1":0.36539460584181066, "value2":2852256551896569615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.107272, "value1":0.7519447135299673, "value2":479042341091737730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.653206, "value1":0.055664103121505173, "value2":4715105139689145584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339724, "value1":0.65279778916842, "value2":5051290581444035933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605595, "value1":0.7185823471699857, "value2":6840084614633305448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923752, "value1":0.4375825544173804, "value2":5512879974255226786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_5":"d","key_7":"e","key_0":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259124, "value1":0.1976251344530473, "value2":3865204942280496136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833243, "value1":0.1327195116975274, "value2":25036382112222312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181954, "value1":0.03893696644874014, "value2":8620048848181171214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241452, "value1":0.08765547856551259, "value2":478115772614972254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.346035, "value1":0.8594527006005106, "value2":7112082294884150846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_3":"f","key_9":"c","key_2":"i", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171651, "value1":0.03206744023244803, "value2":1423544456047900511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863325, "value1":0.8536761474388656, "value2":6060590565202496674, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.441838, "value1":0.172156773217081, "value2":5954759661339252215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_4":"j","key_5":"b","key_0":"a", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960980, "value1":0.32980876505437323, "value2":8725964433492010874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_7":"d","key_1":"f","key_2":"c", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000707, "value1":0.24709283071336913, "value2":6676514692655151346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.618373, "value1":0.9009517094483285, "value2":3595072304978000605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.170508, "value1":0.9280407408957537, "value2":5488749324303924425, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863252, "value1":0.12661256202486373, "value2":8664732043269436820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.849403, "value1":0.16431470929214623, "value2":7019138744327165116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464027, "value1":0.09323138351556215, "value2":7725624855401413056, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_7":"e","key_9":"c","key_6":"b", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.236399, "value1":0.8723974444748736, "value2":4852776678069519279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748433, "value1":0.955013940926474, "value2":1647976132077310759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519277, "value1":0.9878333472759755, "value2":1234045371574302701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098817, "value1":0.01452665827731407, "value2":1567075824080468956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.086812, "value1":0.507976874584688, "value2":7088565570837880582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.442740, "value1":0.05234017389082447, "value2":6744325328771713826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200886, "value1":0.26486555470244233, "value2":7063921451775904700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542647, "value1":0.44652572463811874, "value2":881040498652574470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403848, "value1":0.15638403887386948, "value2":881480808583221270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.456051, "value1":0.8059172501323808, "value2":7272729849107639168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529307, "value1":0.34486654778342174, "value2":392684728493023069, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171436, "value1":0.3729515434430422, "value2":3944928681545148043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308638, "value1":0.7632957285940241, "value2":92442464729948625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515220, "value1":0.6925288324736131, "value2":8180048442965609063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584191, "value1":0.021892941959738798, "value2":8483219855752350520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.062243, "value1":0.4180089249967854, "value2":6354312606503764778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_9":"j","key_1":"d","key_3":"h", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341779, "value1":0.7379607607767219, "value2":5150239222005969615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843413, "value1":0.6687750316343205, "value2":2316529591090571268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.402005, "value1":0.9141363896275393, "value2":6270246938486716288, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128379, "value1":0.5150067708119237, "value2":678394980878451314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108482, "value1":0.3129644434816646, "value2":8384633845422523721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_4":"a","key_9":"g","key_3":"c", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.594874, "value1":0.2519824660631957, "value2":6886759477377939773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445205, "value1":0.903159708784011, "value2":7034171128847343546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824198, "value1":0.6204070552794421, "value2":1059263092990498226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.159620, "value1":0.8686561618779289, "value2":5404253444446120581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.365387, "value1":0.4572890031222697, "value2":7934471101097198056, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521775, "value1":0.36902024174358433, "value2":2258863231460147617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.553157, "value1":0.44513850096108715, "value2":6217475414573430294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141433, "value1":0.21220874698804243, "value2":93944351046750038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.811456, "value1":0.8670934692477834, "value2":7403345830074134000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003892, "value1":0.4239005643048954, "value2":7762289626422779835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.067882, "value1":0.18864986614817061, "value2":2907310668904743062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.750570, "value1":0.6373780024843834, "value2":3686123513663951346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.386866, "value1":0.2003273945539795, "value2":5992425398301380670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515245, "value1":0.516857446811694, "value2":4876668434287590384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389071, "value1":0.9340489387136409, "value2":7504250901331893981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098316, "value1":0.29683731723021556, "value2":7480572590262205533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906295, "value1":0.7596148064915852, "value2":2005574389285214762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.469199, "value1":0.3474919541045514, "value2":4513611972684272623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735221, "value1":0.44321964186413737, "value2":284056865289241089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382330, "value1":0.4698238258381697, "value2":6544780705530113911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644228, "value1":0.34547957427274706, "value2":5643558758908257061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025533, "value1":0.9831418878764304, "value2":34382825795091331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564530, "value1":0.1974264877985391, "value2":7202346148984902707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_8":"h","key_4":"e", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.582038, "value1":0.6658499962262118, "value2":4779411307587076137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.048223, "value1":0.28118182352648263, "value2":2406664371776445226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.277281, "value1":0.3293088133074837, "value2":3125287577587960916, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711593, "value1":0.6513318644160435, "value2":6304109109655854952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.136044, "value1":0.15479605681668362, "value2":4561542839689997911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.143594, "value1":0.03881037667597162, "value2":5145183720039449728, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766270, "value1":0.5913740900789849, "value2":2880750585927778789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.330272, "value1":0.03345193869225257, "value2":5177206809108614692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893588, "value1":0.9545617077887952, "value2":1994751413642673810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200065, "value1":0.7615854624227617, "value2":2555245387882162412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.637143, "value1":0.8697167932948927, "value2":3163242108044176287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.640389, "value1":0.5659009743641423, "value2":7954937488899706120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.558497, "value1":0.928513605246244, "value2":1207841403052993714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.244339, "value1":0.9485629290996225, "value2":8066024682557174519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.368292, "value1":0.1771040772082859, "value2":5158146682102067527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883369, "value1":0.26796315495485606, "value2":500188854907829642, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.603831, "value1":0.695040513930677, "value2":7670486818087410343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635402, "value1":0.17089841071375036, "value2":2188177332357604180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246584, "value1":0.638326848995027, "value2":2467157573588247428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297753, "value1":0.4028871413838985, "value2":549392368948968713, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061298, "value1":0.28566196711825703, "value2":4345219419785285366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468603, "value1":0.15137756224435941, "value2":5857705640176057374, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705772, "value1":0.9900652870935787, "value2":4160023456685330050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628560, "value1":0.47755802914007767, "value2":2886069569872931687, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.201527, "value1":0.3765200270511815, "value2":8192740497964394547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405600, "value1":0.16568713498682006, "value2":807681981221494025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.070083, "value1":0.5411063354146068, "value2":6815859051389366015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339848, "value1":0.09610559077312085, "value2":5872072911320364504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571126, "value1":0.2497057720663749, "value2":924893218806837423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659128, "value1":0.026188484488162033, "value2":8931856805131420281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.127455, "value1":0.3964141576554178, "value2":2662798392903843365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780076, "value1":0.15040609462861912, "value2":7522359721454017998, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108734, "value1":0.6578186778027336, "value2":2079921522814480295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.476594, "value1":0.5388243902425934, "value2":1295082543190183614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377611, "value1":0.4899107617022405, "value2":1810844261495017790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.980760, "value1":0.052036560099483146, "value2":4808591658339710229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.662657, "value1":0.739342249293886, "value2":4457016924188070830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573154, "value1":0.043565004929414355, "value2":2934360743315418976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.792186, "value1":0.7194337918198027, "value2":8482147182498656703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687869, "value1":0.5159874448621534, "value2":5870856463348491189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.522440, "value1":0.8824876555801262, "value2":2534178016362914654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132515, "value1":0.8599678891940701, "value2":8395948226155967317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.096926, "value1":0.9245913183435254, "value2":2358549994381165596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_3":"e","key_7":"d","key_2":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486752, "value1":0.2842029485021584, "value2":2801356184403378281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.475238, "value1":0.22293224326621178, "value2":443811864922674372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132657, "value1":0.9371037777036672, "value2":4385576391999515299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_5":"a","key_7":"h","key_3":"e", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.549121, "value1":0.03204732941506213, "value2":8714811939867543319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796275, "value1":0.21070900388130828, "value2":8667819799137478248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927343, "value1":0.9159887689007843, "value2":7689292400204431331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685507, "value1":0.7348923960578333, "value2":330857683592078866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.865459, "value1":0.24066397037006473, "value2":203169051837200909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.827976, "value1":0.4488303067264579, "value2":3646968045887419113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900145, "value1":0.31889183176683844, "value2":5999122195912397909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.251489, "value1":0.5593865051855272, "value2":5772018587784459447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191087, "value1":0.2891985736646055, "value2":4399230413942175266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087488, "value1":0.5425699812510858, "value2":3965285909604337765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680134, "value1":0.18266599397918218, "value2":5571070206339456089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551156, "value1":0.8876027936303559, "value2":6010745863587972423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.803161, "value1":0.9394184911877158, "value2":571658979107886488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.054336, "value1":0.03957759543141472, "value2":3918244459790702522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_3":"b","key_9":"e","key_0":"g", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685390, "value1":0.5264564714839759, "value2":2232108588919897592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.161082, "value1":0.32998817001502934, "value2":8460874534662447333, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778535, "value1":0.06811024060980113, "value2":7835773491677016452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832955, "value1":0.028669939437680664, "value2":4299578964430488535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004645, "value1":0.8188081000238006, "value2":9221293293613853860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205474, "value1":0.40545870686171354, "value2":1575377844766226698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639268, "value1":0.8908404617416188, "value2":1540158992274106467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.139661, "value1":0.3668772992670227, "value2":6483649363648949947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924909, "value1":0.16190741337415684, "value2":5257368157508278570, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297500, "value1":0.4330469601614637, "value2":4632520395480129342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101765, "value1":0.4685291307083107, "value2":1614399458247460286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291465, "value1":0.6213831794241761, "value2":7754123020970248013, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369394, "value1":0.5153486584517465, "value2":2263160774029528717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719680, "value1":0.5383036464533613, "value2":3589028105702605678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262982, "value1":0.02336562179973987, "value2":2500396082328773649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.790458, "value1":0.7125005920534319, "value2":8453423485925643300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.937011, "value1":0.9029577396659265, "value2":3493444148478497784, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815751, "value1":0.34331005780857, "value2":4896770691763395578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.732239, "value1":0.5781653462685661, "value2":2815554048397289034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073195, "value1":0.6318398759739543, "value2":1443128294631046938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.335280, "value1":0.2900799960272814, "value2":319439076295666505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375292, "value1":0.8416100389726322, "value2":1104228901216289966, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543223, "value1":0.30539264418826334, "value2":2529051273510780065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850347, "value1":0.9475422757893678, "value2":3815335775316587120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010486, "value1":0.9233328326166781, "value2":1132448560381334143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509503, "value1":0.43078826324145314, "value2":2585538196535057958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440516, "value1":0.36575868540627, "value2":764298189191108299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.576334, "value1":0.9992971397552057, "value2":8607203945347812848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.970781, "value1":0.43725394836722203, "value2":8039899278253467744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746791, "value1":0.5653040341344706, "value2":6676447946324724349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636245, "value1":0.8420862976065959, "value2":1562613302672270132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285262, "value1":0.32142916834577895, "value2":5289001385309911194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.417392, "value1":0.049995803596490404, "value2":6607894685909801752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584296, "value1":0.2409216093884783, "value2":1637965971530230928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325720, "value1":0.028777043283946674, "value2":963675377332588860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566004, "value1":0.9002123352889874, "value2":4604125644132750665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.882236, "value1":0.33330368402055666, "value2":156824449049228711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.767642, "value1":0.13763101488639068, "value2":8064121309553799557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_2":"k","key_9":"h","key_0":"d", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719598, "value1":0.8986180062766446, "value2":1742955026017794842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886873, "value1":0.010775899740746808, "value2":1351720915728762737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484279, "value1":0.5158913513515657, "value2":6695543448765155807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.120818, "value1":0.38000311042969326, "value2":8387696033998291, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.471169, "value1":0.5449952216814127, "value2":7977583153767662789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.692704, "value1":0.644651424950153, "value2":1344814886763463146, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434917, "value1":0.5336620566427158, "value2":7974250102837601477, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.231472, "value1":0.18429722211173638, "value2":239717814769298968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351874, "value1":0.3302574440246314, "value2":116725965202686758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.652256, "value1":0.8832602328157859, "value2":2443207413955073666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392058, "value1":0.3666546013150111, "value2":5949156227430002451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940471, "value1":0.038777989285728266, "value2":2871780422161853348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412070, "value1":0.5197207843322247, "value2":406042026849178989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571008, "value1":0.7027610078055025, "value2":1383746034138340292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900415, "value1":0.17910816553895145, "value2":401749706711859248, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.718726, "value1":0.2276914412180115, "value2":9221447498523708978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.381086, "value1":0.9677704240195594, "value2":1303001585301279474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893466, "value1":0.2579622668275943, "value2":3971563895105696581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_9":"e","key_2":"h","key_4":"a", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.400237, "value1":0.23652480428212666, "value2":6009422694154041952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481908, "value1":0.625595272995919, "value2":2204722405376257638, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.936595, "value1":0.2680275935509345, "value2":324738255121972023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.983646, "value1":0.3650479381459544, "value2":6655499432411034428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.489964, "value1":0.6666717547984726, "value2":2225549269590513897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641209, "value1":0.578327662161616, "value2":6547750503066251015, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134486, "value1":0.67209109592655, "value2":2885152331577461710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073911, "value1":0.08294753363850706, "value2":7590236594974272032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181170, "value1":0.5500056163737558, "value2":6771546940482209584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.029045, "value1":0.6184324246909119, "value2":8362437134419045707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162843, "value1":0.4243438443026443, "value2":8729664805668824103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.808018, "value1":0.9740191657641797, "value2":811057613633969138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509186, "value1":0.5860682567407373, "value2":4715370569695436166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664283, "value1":0.9354682602818816, "value2":4771667149292350278, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220707, "value1":0.9895245632099, "value2":7384968594835669289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389342, "value1":0.5213334242916959, "value2":9064052527115170337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977547, "value1":0.9968413793490465, "value2":1018668683107683896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122093, "value1":0.22725801593699438, "value2":471678573457243344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474960, "value1":0.6106407332478643, "value2":6142821675445007328, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770612, "value1":0.38366813133853406, "value2":4771686788392964587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166046, "value1":0.6731562334089229, "value2":6765933549203517189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000435, "value1":0.7725469520964668, "value2":1526254098614108268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.596567, "value1":0.33143109459424, "value2":6988963074743099921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_2":"f","key_0":"f","key_1":"h", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832348, "value1":0.5082718126919952, "value2":7366669828299681751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_5":"j","key_7":"b","key_1":"j", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.933461, "value1":0.2177858524105436, "value2":1927680313692641282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_4":"a","key_9":"h","key_1":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800386, "value1":0.37752422688739345, "value2":3623415676274096956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602632, "value1":0.9591290646171173, "value2":149144865605034806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805853, "value1":0.46005104238615857, "value2":5399494644643471050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023064, "value1":0.2589954384060476, "value2":2161234059148173478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.663052, "value1":0.4340276216577473, "value2":1651362675477804080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425648, "value1":0.5284587146267693, "value2":2204338428823542625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297837, "value1":0.9930278173662268, "value2":1153109222914733395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.661230, "value1":0.3189372627835483, "value2":8950530915947476877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997529, "value1":0.9931406569786665, "value2":3171376634461957806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785991, "value1":0.6115707472871859, "value2":4480386318218501447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909727, "value1":0.29199552201380746, "value2":3686357880544622730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155119, "value1":0.5470008889969509, "value2":7858820522318450111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_7":"e","key_4":"a","key_5":"k", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.530629, "value1":0.9873625937302516, "value2":5471095001682678938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512132, "value1":0.14465321479404997, "value2":8852396394387214520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.643903, "value1":0.81607168234715, "value2":7750000008183770029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088371, "value1":0.9834997161657205, "value2":8265123019714389041, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.840291, "value1":0.31720704974294733, "value2":8838522133270072113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607394, "value1":0.6158114593013463, "value2":3920524769932432062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.891225, "value1":0.7260707014711243, "value2":8545845285837175547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155524, "value1":0.5324837252402642, "value2":6050562650984829349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463368, "value1":0.3636429297340659, "value2":7366387480034568615, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.051267, "value1":0.21240586224005645, "value2":3520112151921939439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.713935, "value1":0.6148054965944443, "value2":4435199374933360163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229994, "value1":0.877849746015665, "value2":4328772383519778289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.394088, "value1":0.7419214909083843, "value2":1533570891823401045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311200, "value1":0.6547222273874411, "value2":603076909255570468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593460, "value1":0.33723617074445394, "value2":4429464301654712087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016002, "value1":0.7850512802013139, "value2":9218293660844009579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293893, "value1":0.9024192531760731, "value2":874814059564514510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197015, "value1":0.6346018344137921, "value2":2275287342249871781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444015, "value1":0.07476763645648904, "value2":6294420894284163263, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.137036, "value1":0.7149572150590823, "value2":5890716035999924313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382203, "value1":0.2373007200794848, "value2":4237441552449229366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927338, "value1":0.5067232894842505, "value2":7166709656823836078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.856975, "value1":0.7551172442205076, "value2":3114426699624764241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290618, "value1":0.461652630545808, "value2":6870191814019559455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194719, "value1":0.28972301156720964, "value2":2266210405139781750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.930146, "value1":0.8041428167626514, "value2":3758081748375193976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.876259, "value1":0.8601408243336739, "value2":8153013110856241749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544542, "value1":0.4588432483941638, "value2":1078220017088765885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861984, "value1":0.33210767773987954, "value2":6423637803003548004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725768, "value1":0.13821724766383348, "value2":2685436575194046233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422624, "value1":0.6174306852475553, "value2":622178413482526248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.749470, "value1":0.7804428540523304, "value2":6875918761098582661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.903503, "value1":0.7328633216838554, "value2":3622697451467218798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.116528, "value1":0.9193757905595477, "value2":6857103271674313745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318972, "value1":0.35822522818261543, "value2":8829601973990551809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433538, "value1":0.5317306440222773, "value2":2556858263334639046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497594, "value1":0.1404686557588262, "value2":5182788796116063454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.731708, "value1":0.08352722566425953, "value2":57349948699707298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052659, "value1":0.5013772188087474, "value2":4075548779807119976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680970, "value1":0.7475466041318248, "value2":804362071276609030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644235, "value1":0.13991370513704515, "value2":8012476351028838411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.430709, "value1":0.07096170614998987, "value2":810033186472659399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.283029, "value1":0.49501003784947184, "value2":5335533898113400696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.213964, "value1":0.06985861165056967, "value2":3236599269070483646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899939, "value1":0.8432631992057813, "value2":8331656940556283499, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447604, "value1":0.4633606486187383, "value2":3282862092550905414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986819, "value1":0.31082999792855004, "value2":5075360107904774840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375656, "value1":0.6617351656082016, "value2":8930874302935418560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_5":"j","key_1":"c","key_4":"h", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787774, "value1":0.5242892599116226, "value2":4186364548758456678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918288, "value1":0.3268848136936015, "value2":2614642728372801254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535191, "value1":0.176268908097137, "value2":653040557920632015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529661, "value1":0.35212458808041835, "value2":1525600014751099598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.361165, "value1":0.4714568379001177, "value2":4598533494631122989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898032, "value1":0.22769956750018774, "value2":5631589457641828317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_7":"c","key_1":"f","key_5":"f", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483853, "value1":0.4317121876757055, "value2":564210499810148536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512418, "value1":0.14841146100443744, "value2":6375007633580482412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172575, "value1":0.4457725144006962, "value2":940411418055250112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109679, "value1":0.17465206890579033, "value2":619954528132439983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016784, "value1":0.6172432789077167, "value2":8694611599799563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405055, "value1":0.9156938137116516, "value2":1197968795715040120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.153068, "value1":0.6193599211433017, "value2":3101256038412073082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041261, "value1":0.0029965595633104653, "value2":1337872063966920132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.852796, "value1":0.021368920853982197, "value2":7987040863826958067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921132, "value1":0.47595655072486737, "value2":1700855145658274194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.383399, "value1":0.26371142871266584, "value2":6832011082222328508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.332123, "value1":0.8824714441278855, "value2":4889159017232292596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_5":"a","key_7":"j","key_0":"f", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847690, "value1":0.5825485094693434, "value2":8301349950043802905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004022, "value1":0.963420472883698, "value2":4714062367056435117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178319, "value1":0.21282157659328568, "value2":5741804038235426956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226035, "value1":0.614117409288991, "value2":7916317937107113091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188200, "value1":0.6715893678504037, "value2":7136230473959043902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709476, "value1":0.7492754619759127, "value2":6914951663040962797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879839, "value1":0.9373684799176515, "value2":3333434004719572493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174084, "value1":0.14484188554360505, "value2":2035242236177706450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.505125, "value1":0.57329431354178, "value2":5119683575480371621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742853, "value1":0.7629472821449486, "value2":5737967107040353117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226226, "value1":0.17730887465740228, "value2":4673386690110010077, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560859, "value1":0.9766459300795906, "value2":1996912721458005085, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262060, "value1":0.05094335959612934, "value2":1245357682330480297, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432269, "value1":0.8390455137158405, "value2":1878222408225434076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113677, "value1":0.7363256343069756, "value2":2885521712019062646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444967, "value1":0.7955117384805845, "value2":264780152192144506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162729, "value1":0.6712123871834628, "value2":1119778679298263778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.911266, "value1":0.9891407670464004, "value2":9103652755878870157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379007, "value1":0.2889256375011702, "value2":5327006092087239478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.092408, "value1":0.21632368500444782, "value2":3442219206298318473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032852, "value1":0.524214031146113, "value2":6530887848447144232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382209, "value1":0.7113041138812657, "value2":933816269978698137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.334638, "value1":0.7695287307455443, "value2":7489040093668157103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_4":"d","key_7":"f","key_1":"e", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372671, "value1":0.7439130206835773, "value2":4959199765009131376, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900273, "value1":0.9052840812209041, "value2":965461062104746729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.654196, "value1":0.8121385984212723, "value2":5665296699467259244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178571, "value1":0.19948342559708582, "value2":5100240681564871548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770922, "value1":0.3886846963277322, "value2":619589477307797843, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.623448, "value1":0.010709272715713756, "value2":2405417022197491275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.501430, "value1":0.49322743442420075, "value2":4840878662359361238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268501, "value1":0.9623480970104531, "value2":7490234459966550271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_7":"b","key_9":"c","key_4":"g", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403202, "value1":0.7953121912476196, "value2":6585060526257826515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078706, "value1":0.6781344722473363, "value2":3204877843782827661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.343846, "value1":0.7227864686817432, "value2":3271852738757870545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.026648, "value1":0.47137422416076813, "value2":3734918729615183416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.691254, "value1":0.7660887564670621, "value2":790090719852690753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262583, "value1":0.666841123381339, "value2":2785906959504169261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.234934, "value1":0.28467055157973764, "value2":5751420159217569904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110004, "value1":0.6775088488593821, "value2":3553287660015547262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766422, "value1":0.23227174275211976, "value2":623571748203397381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800936, "value1":0.31353934764817454, "value2":4270432397290565810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_7":"e","key_0":"k","key_3":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709087, "value1":0.45637800363407555, "value2":6299983262198737196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756471, "value1":0.4807492151985086, "value2":7279712801686741955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188086, "value1":0.36885137660554584, "value2":7300782007111153833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.948483, "value1":0.37900022039799186, "value2":659834049221243616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.589486, "value1":0.790378840630986, "value2":5696488550645422430, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783617, "value1":0.7580291430060272, "value2":3320953914407054988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321932, "value1":0.6227238552522466, "value2":8556762730182657496, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585825, "value1":0.8340787864264869, "value2":2874883662014238947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425246, "value1":0.4329750194122766, "value2":6011155334107441536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023097, "value1":0.474467449495024, "value2":9096564500742879948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625423, "value1":0.1682038891195739, "value2":4431847776626169211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289832, "value1":0.9685809921059807, "value2":1593648503211538110, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613185, "value1":0.10083138549838662, "value2":6012379324790188749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720225, "value1":0.22688014499451828, "value2":533494308206781076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.925929, "value1":0.3145810371686551, "value2":2521880979655219825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_5":"j","key_9":"f","key_0":"a", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073559, "value1":0.8510652378781342, "value2":2771699030834759801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540880, "value1":0.19703714704465763, "value2":2229904355406996428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_1":"d","key_0":"c", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358372, "value1":0.47865214913286414, "value2":991745116280869642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.985891, "value1":0.5712880806403865, "value2":7108232781064624938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787182, "value1":0.10623777547287752, "value2":5757177518524775613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725343, "value1":0.4881539523572101, "value2":9008775172737657154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.276129, "value1":0.7963976864976252, "value2":4233311333311619132, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.296009, "value1":0.9362536518865696, "value2":6472012225611436212, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572284, "value1":0.5256827892320936, "value2":6967364447645723612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490559, "value1":0.5321215426269078, "value2":6272951345279554009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.897929, "value1":0.9148681755770618, "value2":8519086271296505773, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438881, "value1":0.9657537469163081, "value2":1825676918798292475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.878233, "value1":0.44284605438890395, "value2":948269923000784878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.240505, "value1":0.2373413838092857, "value2":510374534141290052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041417, "value1":0.17592672383423122, "value2":3473808065501665173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_8":"k","key_9":"a","key_1":"j", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191879, "value1":0.3751219146996357, "value2":7106698191372720163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694969, "value1":0.8445486295845246, "value2":672504742133677359, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974994, "value1":0.6654201856510823, "value2":4512443800515268537, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.265020, "value1":0.3977309051692021, "value2":987280456712575618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_9":"d","key_0":"f","key_7":"k", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.837282, "value1":0.6667916772962499, "value2":2649317741597325862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584752, "value1":0.8184998685164273, "value2":6853243233010504233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956207, "value1":0.20740750871245983, "value2":3701599145151889680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785977, "value1":0.42059150301577825, "value2":4300206568693523992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206794, "value1":0.4065971742289675, "value2":5055750687883280777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.466356, "value1":0.24692397834834437, "value2":6503690772355582431, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374485, "value1":0.27439926954774063, "value2":2257827689674118963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859317, "value1":0.15734285025101918, "value2":2188339349886556422, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.888728, "value1":0.2275635074824085, "value2":3959472993254545104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552286, "value1":0.8140316248314068, "value2":28562363718590634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101732, "value1":0.9986972418263012, "value2":6625998065774179710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.064398, "value1":0.4719546471748007, "value2":5016107261432878232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540095, "value1":0.05564657854486056, "value2":2892021965316178106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052034, "value1":0.6085312696710864, "value2":267981962928859635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479153, "value1":0.1864971302120239, "value2":4274890765687650257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285637, "value1":0.8618179927379825, "value2":98273395449385741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160949, "value1":0.1665709809492651, "value2":3549197933610586412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479807, "value1":0.9792753435060556, "value2":544874846395596701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.715679, "value1":0.955575677088658, "value2":7351930305259561505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655220, "value1":0.7716326625888155, "value2":490280663710319116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799754, "value1":0.26438748028137354, "value2":2889376751051793411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625103, "value1":0.1721904108442133, "value2":7991543743244408523, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.690861, "value1":0.007633155868195746, "value2":3073219921658712367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308852, "value1":0.12391196742809665, "value2":1115580635235450437, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.013227, "value1":0.7267200692886595, "value2":1429240573369570759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268822, "value1":0.6684636760714637, "value2":4339881778709198457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_5":"f","key_2":"i","key_3":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160477, "value1":0.5396173638949372, "value2":4906545989468275563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599858, "value1":0.07893489799204395, "value2":410345370022207222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584892, "value1":0.8068626341311561, "value2":4979004915398024459, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.987465, "value1":0.8935603600443448, "value2":8149710572361609309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227032, "value1":0.3687576237691273, "value2":1391051851919204878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833201, "value1":0.32203470987301264, "value2":2081318068679055279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291063, "value1":0.034071004249405484, "value2":5342977357585868177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290178, "value1":0.3044083195032883, "value2":4841141697891251443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109841, "value1":0.9228991323104464, "value2":1691775942527270832, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587504, "value1":0.5332728718746681, "value2":1334529969932810461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.920462, "value1":0.18350284649923848, "value2":2590094361323084553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.966666, "value1":0.25923537198192, "value2":6116285159120956507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480100, "value1":0.8741323699909098, "value2":1633286453079731557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.175764, "value1":0.46174408884851265, "value2":5588956365079776400, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_7":"j","key_1":"i","key_5":"h", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357949, "value1":0.9471421832392611, "value2":2020538223652392065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087613, "value1":0.8391096943297279, "value2":1883071449916289062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883487, "value1":0.5961374967262479, "value2":6748555704132728387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.514904, "value1":0.43082646153613147, "value2":3827154946167603504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899576, "value1":0.926252451999718, "value2":6912098010507428750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.708901, "value1":0.5214095428253637, "value2":8239976493295716264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.784267, "value1":0.7119074977387614, "value2":5875315435697946397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318238, "value1":0.2857683698624022, "value2":2168440855638261641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434779, "value1":0.728469975577827, "value2":4027120624532411802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432104, "value1":0.9159655292186285, "value2":1266477315347168519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.453551, "value1":0.9820731220635184, "value2":472902679452044251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.239100, "value1":0.05657570742681277, "value2":1082500030056759801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918697, "value1":0.5552870083342735, "value2":3402695849932536811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689901, "value1":0.372073062318352, "value2":4891331242898882829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924258, "value1":0.5283619937827609, "value2":559122472611611400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172686, "value1":0.4344219735503553, "value2":1248792109975059381, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894193, "value1":0.43426860175202114, "value2":3497813026288966485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.327909, "value1":0.3878605121485491, "value2":5047756483827083362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220825, "value1":0.07655058465784402, "value2":5252985818354538541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742213, "value1":0.38430981024473343, "value2":5620787295533246773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.733232, "value1":0.5555223164703845, "value2":631270712021422829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940662, "value1":0.3098977203238577, "value2":5954879855194614083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.649184, "value1":0.3354932964670884, "value2":6051779552546236650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832762, "value1":0.9063274954717605, "value2":3724265762255192701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.385652, "value1":0.5036946685314467, "value2":5109085475601380385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.722579, "value1":0.772640832685559, "value2":3780594965713240464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655329, "value1":0.29356054638145535, "value2":4596604760791157242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.459104, "value1":0.8777447945394818, "value2":1186981345641271839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.399975, "value1":0.025674080171056102, "value2":8964167090414725476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.123918, "value1":0.9272811881045043, "value2":7218529778562540903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747920, "value1":0.9264629051459597, "value2":8464913590880556582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914960, "value1":0.32245869851475545, "value2":9109063864698425563, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.328190, "value1":0.7652891376156531, "value2":3024374861601192187, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593397, "value1":0.17904346621073747, "value2":3899031855173978925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229837, "value1":0.5384685110923013, "value2":5235158233416481121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.698328, "value1":0.5982638259890453, "value2":6740778632490166175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812614, "value1":0.1666081159883058, "value2":2800734738787135789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352669, "value1":0.33792886940688555, "value2":7496164581714310216, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128711, "value1":0.2991783007626336, "value2":7338677993190210677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552539, "value1":0.06968528327878533, "value2":8300480680068139329, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.581708, "value1":0.35231544880285015, "value2":1384864101595986439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778952, "value1":0.8415964882413857, "value2":1745115595251159346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.695991, "value1":0.5753223257206643, "value2":1886601101964605359, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664273, "value1":0.6725491194351899, "value2":5574699884022596470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711367, "value1":0.2595939895449639, "value2":7814193120722379636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.600545, "value1":0.944128951867823, "value2":6711650903238481221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452146, "value1":0.9554010846729131, "value2":3169390990279464268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.140642, "value1":0.16345235341026784, "value2":2148518750069655597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388132, "value1":0.5979152819074895, "value2":7002872128153528233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078323, "value1":0.5697531979956287, "value2":980455479278929153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499938, "value1":0.388179798040964, "value2":2192712143484318261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761109, "value1":0.2600517751619912, "value2":647733851046806648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_9":"d","key_6":"d","key_8":"g", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088032, "value1":0.2162359479396891, "value2":7960761017907576840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886480, "value1":0.21950291960892157, "value2":7881441150954657573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.538255, "value1":0.36627249562354336, "value2":2697185681710870928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956144, "value1":0.09482810661100612, "value2":3142048329987868522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.610158, "value1":0.7330550786147262, "value2":4897289423318138878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_5":"g","key_7":"h","key_0":"b", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641699, "value1":0.27867439634792107, "value2":621712637914424018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.218950, "value1":0.1329184934440547, "value2":3565116595465876544, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.247888, "value1":0.7166338804544903, "value2":5367509956565676215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.214416, "value1":0.3332699601933243, "value2":4699030477990361670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023652, "value1":0.5060732821933588, "value2":1384885559250896583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907328, "value1":0.1694015036022409, "value2":3592921690010847117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.642558, "value1":0.737568796988017, "value2":2406767446522063712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273826, "value1":0.8447613680154935, "value2":6080438111270024688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.984506, "value1":0.28262866688772725, "value2":6480278416681893463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.868136, "value1":0.8910201029103536, "value2":6867531047341490726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687877, "value1":0.8829884505686191, "value2":2249445178190997362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850677, "value1":0.6276341510695712, "value2":2252184121487700107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737726, "value1":0.12597210262772834, "value2":1348511802277935140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.810752, "value1":0.2102822839010783, "value2":4831043683200695850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317105, "value1":0.817061302570146, "value2":8324179065679406640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.426069, "value1":0.09982797254913842, "value2":6519984536301213031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.121073, "value1":0.5197148618479782, "value2":3010915938164137229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.264976, "value1":0.4225812260629186, "value2":5303229136813956536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572439, "value1":0.07380111555778088, "value2":4428887607089365223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526101, "value1":0.2368198468823957, "value2":5676247054773494672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.794040, "value1":0.8488571887152928, "value2":7926265788784400661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682501, "value1":0.6419011526811798, "value2":6127958830288275598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.508833, "value1":0.08505936393863323, "value2":6956791791775335476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.485450, "value1":0.24440134625086388, "value2":8675077803244715182, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341184, "value1":0.7204080689080778, "value2":7288860462608592695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300497, "value1":0.587674524187528, "value2":2136914643668219995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646074, "value1":0.5687487761791935, "value2":5117858885820847437, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405215, "value1":0.63703146651586, "value2":8601605955587466200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040361, "value1":0.9056350649746174, "value2":4395139702119619272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042190, "value1":0.25258468888663055, "value2":6275160558122438530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436823, "value1":0.3052140189316623, "value2":1176588437754725430, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_9":"c","key_2":"d","key_4":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.885585, "value1":0.996098364991821, "value2":7425087259003526051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.667291, "value1":0.01567259898271256, "value2":688757342279745926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560861, "value1":0.8815928724937075, "value2":5127701164424889645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921488, "value1":0.5280525030917018, "value2":8702967906065448733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906624, "value1":0.204252621066632, "value2":7869988567138398448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.855702, "value1":0.956230561465019, "value2":157946491688041108, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367342, "value1":0.6684623495570281, "value2":329650868812150871, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445102, "value1":0.6894360816538613, "value2":4920916512388188612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.609045, "value1":0.0027756305160863397, "value2":5637928110041308291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.632075, "value1":0.04166500110036427, "value2":6157318199778992604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972254, "value1":0.31480609981503055, "value2":4364758259634809973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.776493, "value1":0.13104837145124149, "value2":4732079918184171170, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238222, "value1":0.6559225899889957, "value2":3576156993215909053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434447, "value1":0.24817502461178453, "value2":7546679218765777183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040709, "value1":0.1860292914746309, "value2":1580899927050287482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273335, "value1":0.9526744406123745, "value2":5086267667987270481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052913, "value1":0.33164061518472737, "value2":6494564179935068135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093729, "value1":0.8324601543372498, "value2":4840824205416424699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.167756, "value1":0.8292588795543363, "value2":926691352131787790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602891, "value1":0.7468923005026717, "value2":1499084894678948825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939687, "value1":0.9093420550399373, "value2":7624268916058145495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859436, "value1":0.018264976223293757, "value2":8966202800040990455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540853, "value1":0.15398968210956118, "value2":1169217513145417422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927943, "value1":0.6497527028232101, "value2":8964902118693660858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032864, "value1":0.5440150103669782, "value2":3398396102381979841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440858, "value1":0.5805592700724092, "value2":598543749015746918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220522, "value1":0.7634462305829404, "value2":4744586200886838973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350814, "value1":0.8753641286301881, "value2":3594959380973533940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689230, "value1":0.11698455183819528, "value2":3362979353229964321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369880, "value1":0.9959022990686378, "value2":913069268809019275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499935, "value1":0.42435230326696954, "value2":5642082683494178997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.126493, "value1":0.24855180288733794, "value2":2553926153696879833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241812, "value1":0.19715345400514303, "value2":7966035951898297683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.282709, "value1":0.26568871987724985, "value2":5614603759761174315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634974, "value1":0.5611667195638738, "value2":719923428562838874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_9":"f","key_2":"i","key_3":"b", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.437429, "value1":0.044098595487335565, "value2":7598428545433157434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986084, "value1":0.15643080062821366, "value2":876340157580973000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350215, "value1":0.5350647081004255, "value2":2039458830523373741, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.263790, "value1":0.13663992546983536, "value2":4887595679013193650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761252, "value1":0.31778054800300737, "value2":4963237153707981360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_5":"k","key_0":"k","key_1":"c", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358509, "value1":0.4977744678036884, "value2":2417156657544845857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.825502, "value1":0.7927543298826962, "value2":7803607945365925463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463820, "value1":0.25407613728207273, "value2":7298874514736284603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_7":"d","key_5":"f", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815985, "value1":0.03419039527127868, "value2":2983524479323907938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648209, "value1":0.5111390924658735, "value2":4469943396931617311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560198, "value1":0.3027175063306067, "value2":8582749896805213520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293707, "value1":0.7381247559612524, "value2":267296433337218156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436607, "value1":0.1678795867934854, "value2":2882160820519571353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516040, "value1":0.02439990616294445, "value2":3459503829097802670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861717, "value1":0.8549050337904404, "value2":5330118776081944380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181801, "value1":0.014687212605387332, "value2":1741497345355098408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398203, "value1":0.3791649455492745, "value2":7827883443566154722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004152, "value1":0.4627140852343998, "value2":5127693472090561791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289305, "value1":0.9473077760244797, "value2":1972308969161346102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.407619, "value1":0.22478804543381714, "value2":4393947727507143193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_8":"g","key_5":"c","key_7":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500355, "value1":0.9148638467317476, "value2":2930801640270658847, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390931, "value1":0.03756752782865913, "value2":6450238118865267877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436518, "value1":0.7302171451231968, "value2":8892033149379515881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352126, "value1":0.6345536180749114, "value2":8076417892766647115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211476, "value1":0.7869548859510007, "value2":2137142575900025284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.620465, "value1":0.04777740137317064, "value2":2493346722675519040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166994, "value1":0.5908047667032886, "value2":2133480026426783424, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799110, "value1":0.7447928303219921, "value2":6367274514210109366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.258756, "value1":0.5720021685376423, "value2":7819250772844902436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178485, "value1":0.22728962622611987, "value2":5810329748220466185, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756847, "value1":0.09118290676810949, "value2":2438579961381457911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447596, "value1":0.2813020748043202, "value2":4446085339672311465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_6":"b","key_7":"i","key_1":"d", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.507148, "value1":0.8635348228135215, "value2":7235169187543778395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412107, "value1":0.6168652299086078, "value2":8460560637760416654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.916980, "value1":0.20506020965429841, "value2":1044309486752695204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358540, "value1":0.5251096014647666, "value2":6638181287796109637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_5":"b","key_9":"c","key_2":"k", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.199047, "value1":0.6942630163772678, "value2":3410541935645339151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_5":"h","key_6":"j","key_1":"g", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.754827, "value1":0.9837922147041048, "value2":3690131697027385202, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_4":"b","key_5":"h","key_2":"a", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.329509, "value1":0.8271404181330777, "value2":4227974754451880800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.513229, "value1":0.672862073275747, "value2":3810363550665232995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910574, "value1":0.9390557529752215, "value2":5133678849432201178, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181846, "value1":0.49307852075676845, "value2":6137025771035845818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572559, "value1":0.2408765387072259, "value2":8298872358883983458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078241, "value1":0.08393417297668128, "value2":8837983950648920911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_9":"e","key_4":"k", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.056551, "value1":0.9755466271988745, "value2":2207490383904730952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.814175, "value1":0.5043585864165596, "value2":5350947527006645760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.066736, "value1":0.5177229679064868, "value2":2703347438865819514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560402, "value1":0.4478041012053205, "value2":117552149576626326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079515, "value1":0.133747701259907, "value2":5643720599026556636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_9":"h","key_2":"f","key_3":"e", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246265, "value1":0.8787847052412423, "value2":8596628070583162060, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.011510, "value1":0.3733460731999948, "value2":6601446014598669145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735379, "value1":0.9116679871732262, "value2":5605502556155572766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.574257, "value1":0.3506079241295861, "value2":1800090074854288106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900580, "value1":0.8930823791186644, "value2":1843480248185265031, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.744935, "value1":0.754485340555253, "value2":1527411985430067052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369195, "value1":0.7284643223167715, "value2":2551804296404453319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500109, "value1":0.1715238288854149, "value2":3468590424262559119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196638, "value1":0.4131391791311713, "value2":2036494112995161182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660034, "value1":0.5162081878070082, "value2":683174465375000181, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.324311, "value1":0.6392052032119463, "value2":6854067358310903979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477405, "value1":0.9223995048777075, "value2":8678575474954356403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_5":"k","key_6":"b","key_3":"e", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894983, "value1":0.2879379950878977, "value2":1760029872301364905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_8":"b","key_0":"a","key_4":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404612, "value1":0.5964910019926303, "value2":5352075776414002891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_6":"e","key_1":"h","key_2":"j", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.406123, "value1":0.9911934759109439, "value2":2505572631856818583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_5":"c","key_0":"a","key_4":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098160, "value1":0.29265324156352274, "value2":673331618670269188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317156, "value1":0.5284207406945622, "value2":5801235018377104530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.278954, "value1":0.8339844494736254, "value2":142783045099103297, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144316, "value1":0.5981739630713123, "value2":4376705426457644175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.533900, "value1":0.7048057967645066, "value2":7668900249344587055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655047, "value1":0.9572743468227054, "value2":8615116979377517835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.164063, "value1":0.12955196466904195, "value2":8720744967395487030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211413, "value1":0.4645239019949383, "value2":1341011821671689315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.254073, "value1":0.9282656526373411, "value2":5675449932048560089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939659, "value1":0.3774328021259814, "value2":7756643154892601396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.789551, "value1":0.7757594275737888, "value2":8309777310097157940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_6":"a","key_4":"c","key_5":"j", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.256014, "value1":0.7599489616826112, "value2":210625827724423434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883947, "value1":0.13842755442158153, "value2":5303830768512577958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516070, "value1":0.17914029301657106, "value2":1056091608874344104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.446988, "value1":0.27173267141582086, "value2":6180352463990379834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.622518, "value1":0.8672352642207358, "value2":6931279306288268314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.002662, "value1":0.25623553544743577, "value2":4114415105877902794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.250822, "value1":0.1691133676662619, "value2":8315894568290440082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148935, "value1":0.5989068031663785, "value2":916798823552491100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404397, "value1":0.3854563629426573, "value2":1729425108588138938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155114, "value1":0.42561573956387977, "value2":3412690494545228649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_6":"f","key_4":"e","key_5":"k", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.185911, "value1":0.7064647650423553, "value2":6268941778110729170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_4":"i","key_7":"b","key_1":"g", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.349779, "value1":0.9255472682273418, "value2":1166345991907977547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.982678, "value1":0.6891602632892445, "value2":1740831383850752805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914669, "value1":0.4165113125409203, "value2":1826795354292330747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871167, "value1":0.7086334187759732, "value2":5543325343112219047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.762690, "value1":0.8505896053437381, "value2":5927978325036218166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.292162, "value1":0.6672535143315814, "value2":1327915299266347941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311132, "value1":0.47941008598606394, "value2":7437258572827323548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523170, "value1":0.7404188223668129, "value2":5505665204112874175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357326, "value1":0.9538155003965737, "value2":2702198679896121556, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.364597, "value1":0.17680019955559403, "value2":1753707621503560793, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452318, "value1":0.2992929781674707, "value2":8362411304781786581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518625, "value1":0.8647905294441987, "value2":5431066155038766988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.355524, "value1":0.6116936780714586, "value2":1234552975837284329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542514, "value1":0.8351923165774915, "value2":1733954151005989610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979827, "value1":0.2892109226113143, "value2":6049795733820800226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883185, "value1":0.011744889551642781, "value2":141159892430595908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.223860, "value1":0.4073953516179897, "value2":6458792499529133548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049058, "value1":0.12482723086065665, "value2":9009567363074084595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.298389, "value1":0.4827417824302383, "value2":4841281853843733128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.629693, "value1":0.6591071306029623, "value2":3355181768485869219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.147730, "value1":0.34536474665893674, "value2":3191505597112674507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.179092, "value1":0.7637896583466016, "value2":1179889369576374895, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.658439, "value1":0.9209377727035774, "value2":3422449294756383390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.071622, "value1":0.28143127213081115, "value2":7766511653375780058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537938, "value1":0.6024534595610256, "value2":8547099742960318907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646788, "value1":0.20378541750257417, "value2":7978976103929383445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747803, "value1":0.10859041477345463, "value2":1109588376642858177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564874, "value1":0.23647658766118684, "value2":6863917686945126643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042244, "value1":0.3078158837684314, "value2":4419488498667340703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.284250, "value1":0.9387293705744738, "value2":6769306683790898588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479123, "value1":0.4851789805412465, "value2":6507016270927900733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025704, "value1":0.8538216321323174, "value2":4020894753483999383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833612, "value1":0.2626123684082926, "value2":5033990782030407302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.881634, "value1":0.935752736774716, "value2":6384512508877447191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474427, "value1":0.8815963618627093, "value2":8929077315195001710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321721, "value1":0.8079053030903706, "value2":7509708901019477626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.854474, "value1":0.6950357389111533, "value2":6524710922297840763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229889, "value1":0.6795085741944863, "value2":5609613021607849758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.988904, "value1":0.5223586661895601, "value2":89977508053442787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374340, "value1":0.755589597933608, "value2":7955537057313043569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639189, "value1":0.7408346971613415, "value2":1902500838340597252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_9":"d","key_4":"c","key_5":"e", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481547, "value1":0.5004511492728786, "value2":164083172923357290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518019, "value1":0.729605886464792, "value2":9068451454110926149, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_7":"b","key_1":"c","key_5":"d", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.363805, "value1":0.7382344993077278, "value2":7045722716338732866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.630886, "value1":0.688064557117447, "value2":296768996976239880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_8":"k","key_4":"g","key_7":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.845466, "value1":0.13213135497669065, "value2":4879659946939633689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660200, "value1":0.2229370900047786, "value2":292379935200020145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073732, "value1":0.8115899146610889, "value2":8577867893469971833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.019908, "value1":0.49801059455223595, "value2":7683222481005280397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.347447, "value1":0.9276573673104045, "value2":2091335224644651701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321409, "value1":0.6126771681057066, "value2":1893407571657392417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.738125, "value1":0.3569881382718092, "value2":4029761814756595416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.665842, "value1":0.05170172426704011, "value2":8174581603042814007, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.449347, "value1":0.7466823594518665, "value2":6257307241351449210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.588421, "value1":0.4753472682300776, "value2":4728406440983577982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.257494, "value1":0.1386821629125743, "value2":332192645116862048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.728108, "value1":0.43976058351020564, "value2":1985435892430278941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.209628, "value1":0.8966889425339122, "value2":5787236380684241489, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.638851, "value1":0.9617304712840572, "value2":4707670589409962460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484422, "value1":0.020940662405038862, "value2":1545134502951937463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440131, "value1":0.17343525627253645, "value2":6009630316902812876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229422, "value1":0.02146978522387302, "value2":2111892615484907893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780591, "value1":0.6377996056471382, "value2":8896668703689799570, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.124061, "value1":0.7970190759525996, "value2":7452754965853045737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_2":"i","key_4":"g","key_0":"j", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.450958, "value1":0.8563337518192949, "value2":1606820680288550386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194998, "value1":0.8092586961600053, "value2":1104653518579575826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746061, "value1":0.888221791516023, "value2":6881584298104407367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.550870, "value1":0.45584434378222843, "value2":7394447363524184561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.684022, "value1":0.3067771551792505, "value2":1245627384806850593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289452, "value1":0.4582593754084005, "value2":5465272523370779338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235677, "value1":0.8840876989101745, "value2":551093905717196985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.890313, "value1":0.09198263253620918, "value2":6028451346704401787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.168914, "value1":0.40840563117271195, "value2":3931349792536134233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.717876, "value1":0.7410792258916561, "value2":5169350845698434472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.202605, "value1":0.5290546996961457, "value2":3475693437398263961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636633, "value1":0.824130818871441, "value2":5903284123492948390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.615008, "value1":0.04083331973299552, "value2":7577674506757874978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036559, "value1":0.5696805792367668, "value2":4633771544842561674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367723, "value1":0.355783015228386, "value2":5675601540429514938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907306, "value1":0.5698956093759964, "value2":9122457118633339454, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685953, "value1":0.4276033626449204, "value2":7616472412744453525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587193, "value1":0.9520532910468696, "value2":2039460897796769985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.322100, "value1":0.9372598841781506, "value2":8025865217929956411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155211, "value1":0.2725517811852112, "value2":7680103519195376114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964045, "value1":0.49419337991273576, "value2":8605447057400658561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.423455, "value1":0.28782941494899417, "value2":7631463815755268967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433388, "value1":0.6722143467332055, "value2":4743404486443339981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073044, "value1":0.9919783220125814, "value2":3546009072923452494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060903, "value1":0.09931972482798261, "value2":3396822536822006988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.952732, "value1":0.3775686488599951, "value2":7121618792612707640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521690, "value1":0.5851273414170087, "value2":4662473906180780029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227468, "value1":0.11958534947502218, "value2":3788112644660727919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196785, "value1":0.8966564462732959, "value2":4357533587354590088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901366, "value1":0.5571789697475874, "value2":7759726578542538969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.633498, "value1":0.9521589976099354, "value2":3412636563124759184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477196, "value1":0.6345321514502629, "value2":2968047284799701894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.245970, "value1":0.8602524837567829, "value2":239564384927138993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923521, "value1":0.1090517952569125, "value2":8200798429042044530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648242, "value1":0.5170091370557403, "value2":2725553204805480592, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812069, "value1":0.8884437199979248, "value2":9154279472010215199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962857, "value1":0.2140414709499265, "value2":5080056778998450413, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.561764, "value1":0.6433379926959468, "value2":1246414196804721612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288689, "value1":0.8910605919928337, "value2":8094720078703937078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.419833, "value1":0.6705423885099231, "value2":6631047350002617679, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.085812, "value1":0.03831227212231745, "value2":2735730201464302258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.046896, "value1":0.09940708362873886, "value2":600170391537990700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.884000, "value1":0.5332485748572247, "value2":5278799395808920103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.942254, "value1":0.6825570254740958, "value2":3505708958239891309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010039, "value1":0.007884797092278711, "value2":3745225427592325971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.712360, "value1":0.2908960224193339, "value2":9147580185923316377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109158, "value1":0.3741670076640341, "value2":1005531285446043532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680722, "value1":0.010722209112304005, "value2":826737562607372966, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.439540, "value1":0.4713372686055574, "value2":6754252699406092830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392281, "value1":0.9898544758145399, "value2":2353492636354645276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205843, "value1":0.6788358030320953, "value2":712272724855746801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003145, "value1":0.7114317643988356, "value2":3716874444298014530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.133514, "value1":0.37586422274898074, "value2":2416361460381490648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.904105, "value1":0.18756095304630882, "value2":6928964256284052382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171387, "value1":0.6820596249231228, "value2":172259903859496202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_6":"e","key_0":"i","key_4":"k", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.510204, "value1":0.06734756409519246, "value2":7819819964334132673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422546, "value1":0.1465987153240839, "value2":3085770726305992826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.039220, "value1":0.39594354666033404, "value2":6309816101603497959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.739730, "value1":0.8530985778840952, "value2":7379770578925142492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.672747, "value1":0.7865961192571739, "value2":7774331371169359686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.769896, "value1":0.802526565041674, "value2":3838614021162591095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.304485, "value1":0.6949310789221889, "value2":7501685449967872481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.678592, "value1":0.7504248678928968, "value2":7218481023042482207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764761, "value1":0.20348098581370722, "value2":5988835601145003290, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.968921, "value1":0.21895244723583734, "value2":6407608012483731307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480313, "value1":0.3910222274917921, "value2":935230396616102910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680717, "value1":0.7679413590875351, "value2":6737105603635406237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843792, "value1":0.928935788622385, "value2":8032690240756723112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544961, "value1":0.3803375288284628, "value2":808064213573953017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974943, "value1":0.2873027943668844, "value2":420303650121753052, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351352, "value1":0.9011332762961933, "value2":5787708950335762277, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049110, "value1":0.920403506389589, "value2":3542056558848585785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.473874, "value1":0.20499658001386006, "value2":6473623955229286943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_9":"e","key_5":"f","key_6":"c", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705527, "value1":0.3369786416226785, "value2":5565389573084278676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977283, "value1":0.5753145958771464, "value2":5509805379215659395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.969055, "value1":0.7944991097395847, "value2":5227490476089768493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.260822, "value1":0.37247993171967786, "value2":9100748121410350084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392300, "value1":0.9170516775216153, "value2":3337181235028553814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.917458, "value1":0.7005088931842404, "value2":4249344436835260528, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398079, "value1":0.4412594689419285, "value2":355338602844953023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605493, "value1":0.14953313696992326, "value2":7522955956227719413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.947612, "value1":0.24504972733419753, "value2":19900356272350185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.657164, "value1":0.7554151847082919, "value2":2083009730678034907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.303410, "value1":0.9047440140278514, "value2":2665949305315254597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720136, "value1":0.02233304294884983, "value2":2148351116720732280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605342, "value1":0.06576250199061187, "value2":3149195196917185744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.249421, "value1":0.10981969871005968, "value2":1489417694228791945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490002, "value1":0.2365349300258502, "value2":5413143921150744151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537535, "value1":0.056469545325025704, "value2":2942905392018720189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748755, "value1":0.5086783291906242, "value2":4857376881146282553, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235928, "value1":0.6784255332464089, "value2":3029395751611067716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.401233, "value1":0.753375539931517, "value2":8169340970187425814, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960595, "value1":0.5301594317579601, "value2":3908827718931366422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113359, "value1":0.7603991477763598, "value2":2119709520548445193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.875780, "value1":0.49587314465392923, "value2":8685987546300538378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.938401, "value1":0.9152835192021896, "value2":1076417540165823423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.869630, "value1":0.7973627027092416, "value2":2126356962780308110, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388566, "value1":0.6382739532774706, "value2":159979324198275701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.275145, "value1":0.5927046376681012, "value2":7080867181269495541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523278, "value1":0.5267214411412359, "value2":1728237597853111534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584632, "value1":0.5083502145633392, "value2":7227399514745308579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964267, "value1":0.6658213068549924, "value2":5519764789585080716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824564, "value1":0.9264931547852524, "value2":3998899671907293853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986213, "value1":0.027645492661625626, "value2":3185626691636394685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634127, "value1":0.858418006506526, "value2":5813619502423233874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.099856, "value1":0.5750664046005645, "value2":7125055104530490351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685619, "value1":0.3275765619786257, "value2":3507230098472264785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174236, "value1":0.36236865121536765, "value2":2097215901740320742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093835, "value1":0.8733207807127527, "value2":5480486627610559582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687722, "value1":0.42815747753178873, "value2":2463297564216975626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015930, "value1":0.035417712050281464, "value2":6849011969803492014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181519, "value1":0.9822679258679019, "value2":408242883289676228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370465, "value1":0.9916450645571185, "value2":3540916962390370495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.557960, "value1":0.814086825170977, "value2":975702025360269586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785667, "value1":0.18117401736664177, "value2":8964384630261548826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014126, "value1":0.6836156690970117, "value2":5150749937802379773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310715, "value1":0.8554125170001826, "value2":6356794838014021178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000193, "value1":0.43591866134299384, "value2":3781675008214883017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_5":"k","key_7":"e","key_3":"d", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310472, "value1":0.20656655062551638, "value2":2155883927623577685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348781, "value1":0.1874457704512197, "value2":3156445090401189095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737752, "value1":0.4107965308340836, "value2":6831391234474352496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.864771, "value1":0.7087377270632053, "value2":3852759604751603741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377175, "value1":0.302211293966416, "value2":3671141882815450126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035362, "value1":0.11868658008033231, "value2":4402958858442974997, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206005, "value1":0.9050640840246325, "value2":1571018380191846388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607719, "value1":0.5276900307886362, "value2":2938372021759409514, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847851, "value1":0.23589306520821895, "value2":8130082209207560079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181853, "value1":0.3318615047159984, "value2":2814234275379593797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093707, "value1":0.8735019416998069, "value2":7839357347839415099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.050872, "value1":0.045154096442812874, "value2":1387216811650422069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635715, "value1":0.7372871363614325, "value2":2653034997948603272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.807390, "value1":0.06360776205204009, "value2":8705617981394656683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893750, "value1":0.033458142425902254, "value2":2550969485909611240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599717, "value1":0.9713785482184008, "value2":1528762568291094955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544670, "value1":0.0942081326101987, "value2":5208646740238503092, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.460630, "value1":0.30899325368963804, "value2":6955190987114158639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348159, "value1":0.8338319484328468, "value2":382561011167415899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_5":"f","key_4":"d", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.866844, "value1":0.4559148181796855, "value2":4921190547959215054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.779864, "value1":0.8408779744303566, "value2":5151627244176235100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087888, "value1":0.9444160436820308, "value2":2636681313922893724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422949, "value1":0.45533312838966733, "value2":8719456700476032056, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909073, "value1":0.4871022495216044, "value2":7470894518121500308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526308, "value1":0.6556527723106481, "value2":2061496006850665904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483995, "value1":0.01917969989791312, "value2":6594173174887504625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_9":"g","key_1":"k","key_3":"h", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.704096, "value1":0.7335839167893562, "value2":7683462035683402688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796245, "value1":0.775721209508694, "value2":1762994312464074505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635089, "value1":0.08634955165534107, "value2":5435729701416464796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.146656, "value1":0.904887030211023, "value2":3845389016759808576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.975641, "value1":0.06011625753112877, "value2":2167561119971885977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.027722, "value1":0.7606346737865884, "value2":7806658536996851894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.255754, "value1":0.8709289771166605, "value2":2946981794368069885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682006, "value1":0.164502402573321, "value2":5375762287724896423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901286, "value1":0.32721693073724145, "value2":1600743249293213477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.844121, "value1":0.4592683330441191, "value2":5355971233932247551, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756958, "value1":0.06556927134836166, "value2":2324918632583888734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.686437, "value1":0.5613972309334896, "value2":1964277954793319022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.280286, "value1":0.210108301927216, "value2":4762745887015662113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300106, "value1":0.3451768742896802, "value2":5756080702879459521, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008833, "value1":0.8860643993351358, "value2":5562038867537669443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964560, "value1":0.5679670350248304, "value2":7270655908716143592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259598, "value1":0.1779142648395649, "value2":1989791279542261185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289316, "value1":0.27910110845131275, "value2":9108304266352627509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.380786, "value1":0.03821154554853608, "value2":5351444755572082025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_3":"d","key_8":"b","key_0":"c", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.619088, "value1":0.014955635682113628, "value2":1506343674187146948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.991863, "value1":0.4507410741879421, "value2":700372855016807730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.819751, "value1":0.47361005562943986, "value2":397573677309043949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962980, "value1":0.5040052190830692, "value2":7266525751259633187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.451541, "value1":0.1442134636404948, "value2":561640599502466121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.387998, "value1":0.5301085061271325, "value2":9134048967019410288, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300013, "value1":0.7373601287356207, "value2":342878676774549894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008200, "value1":0.7728904842469639, "value2":6539748237741452295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.697595, "value1":0.02404028274324282, "value2":5913828782152003967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.726484, "value1":0.45889774925126936, "value2":8474247967864647236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.696138, "value1":0.8390712125923527, "value2":8640587498190477408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.053954, "value1":0.3443648011087006, "value2":6899937907932201153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.702864, "value1":0.09419193983638496, "value2":1509375003037818758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379925, "value1":0.10612710322810476, "value2":7235965724589430822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.065471, "value1":0.03922963726603652, "value2":5920886218601602154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087291, "value1":0.4838942531586474, "value2":535199259720805783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977724, "value1":0.6375345212159459, "value2":4619853255519698247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566854, "value1":0.21378747510647453, "value2":3467239940916600883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110862, "value1":0.6797718237586716, "value2":7914262457466633074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440246, "value1":0.30120840846619557, "value2":3776365109572133674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.797803, "value1":0.7797287504617298, "value2":5037915665887516284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.586577, "value1":0.1709193163609849, "value2":5198104224414552394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879594, "value1":0.7450124911318364, "value2":8773848846938448870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886367, "value1":0.551654132389125, "value2":358026416808351667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464288, "value1":0.7461865171369122, "value2":8085418698319569937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197248, "value1":0.8352566024408136, "value2":5751810192674740519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.217337, "value1":0.19284687675681011, "value2":7864252915572676984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448294, "value1":0.42640546323275524, "value2":168095719577724557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.591254, "value1":0.037748263913037684, "value2":2863811536714848498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372170, "value1":0.6660431538229836, "value2":8567505344595141514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_6":"g","key_9":"f","key_3":"k", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317243, "value1":0.18445456220952994, "value2":4074456277125729011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.267738, "value1":0.2691217808870719, "value2":159261331127552033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585076, "value1":0.21078242008152992, "value2":1239129613801043878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979016, "value1":0.5956340156534701, "value2":9183614150234962352, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447286, "value1":0.10494392100402684, "value2":3770086671966420800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105069, "value1":0.2794799410983503, "value2":3469689160956336636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.173597, "value1":0.6860882854882017, "value2":6360441874685617414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.835787, "value1":0.7543174676127191, "value2":6129127352506150313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.772599, "value1":0.7516717167947417, "value2":8027123646004983290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.115846, "value1":0.3569203207491387, "value2":8576278949637442153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551211, "value1":0.9246333110829817, "value2":898669648143321253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.230459, "value1":0.04953444106043317, "value2":2309701856093198744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060038, "value1":0.7972052364471702, "value2":2490346122315313464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512371, "value1":0.09068245544499785, "value2":3847395208192505439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783506, "value1":0.7070750057247079, "value2":451775490327949567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000319, "value1":0.9107891782905447, "value2":4820343479119697886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628626, "value1":0.016138857727039164, "value2":6452512787294668321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.063279, "value1":0.39758013318800806, "value2":2380900021298276106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480402, "value1":0.014403992075819017, "value2":6901656754464212004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.462954, "value1":0.20570595420817753, "value2":4308695238369802814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308777, "value1":0.46154273334740287, "value2":533630259579000143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.006531, "value1":0.41769996010310556, "value2":1003539425076610908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.606413, "value1":0.8392935388898198, "value2":7138000920468723420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390008, "value1":0.9457032273317953, "value2":8816745864308273066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.545540, "value1":0.9980077865004171, "value2":3431998048103898458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.771291, "value1":0.9269154791448218, "value2":2868685560372963447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035244, "value1":0.15635792222626285, "value2":745780555759481875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.802078, "value1":0.49326419263144117, "value2":7061825021138296948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141863, "value1":0.6394029445292002, "value2":4052750041261099856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210519, "value1":0.4447889112985626, "value2":4491375706281220187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014644, "value1":0.5845639893426985, "value2":8504751940020912407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.457311, "value1":0.39232937571361776, "value2":8948083137978870620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497330, "value1":0.0554441906288732, "value2":8552872946126060990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778960, "value1":0.3021332369906045, "value2":5664412036675630494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151648, "value1":0.7180248405530221, "value2":4322565027390602745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.286112, "value1":0.7877440826847747, "value2":6988799327817384951, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.131604, "value1":0.8048208447034889, "value2":7951591877124931974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122845, "value1":0.26609349854031233, "value2":1608820578917270500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.059135, "value1":0.8932421413801345, "value2":6863045131855298597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.119289, "value1":0.6625474883897694, "value2":3938725492057076257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.114546, "value1":0.1285766450638473, "value2":3866539491041344116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_7":"e","key_6":"i", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134976, "value1":0.9092879262740383, "value2":3783610158682087147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.676253, "value1":0.24518334285373874, "value2":5037863892800296388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613311, "value1":0.5221731263503148, "value2":6112150162991672535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448627, "value1":0.09620166167463585, "value2":7262951598026367151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370708, "value1":0.9034926573578455, "value2":6264024668154911646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694827, "value1":0.4562036337293121, "value2":5132858458452108988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.409213, "value1":0.2400040814231921, "value2":5692228309415552332, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.272515, "value1":0.512618446397202, "value2":4245362498789762636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398497, "value1":0.9796136787594169, "value2":2854381927687220104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_3":"b","key_5":"g","key_1":"k", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573421, "value1":0.25717373724181153, "value2":6071818930618476006, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499699, "value1":0.5015171773350958, "value2":6590026308842608922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390872, "value1":0.15108026004608982, "value2":4054685864574240684, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.187721, "value1":0.11524534470163803, "value2":4129310514528385220, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486124, "value1":0.9297806553036565, "value2":2076740459139610542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.506484, "value1":0.48473773192721936, "value2":5279382479053069968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095317, "value1":0.5302104024468535, "value2":1006511502199967697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659995, "value1":0.7173304922658666, "value2":7272260713937588084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206009, "value1":0.21353139637435428, "value2":4614697991046896838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049549, "value1":0.9736103016824452, "value2":713014430698092731, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764793, "value1":0.5877828915841818, "value2":2743371147788843703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095972, "value1":0.7802447999330678, "value2":1247537151507568078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210056, "value1":0.13206812499303777, "value2":5119237508780620544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477450, "value1":0.6302618943375251, "value2":3505999540579430287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.182076, "value1":0.5504514321159135, "value2":2799023815129557532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262688, "value1":0.9279334258185475, "value2":3960329791115549039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601942, "value1":0.9333313963421735, "value2":8488152958276516414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095627, "value1":0.2423663973685341, "value2":3147375434225291522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325183, "value1":0.9698662748883076, "value2":8348906475599130816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535337, "value1":0.9108408536175355, "value2":548221601502028995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.465394, "value1":0.20049512890521093, "value2":6918344754347341650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805179, "value1":0.04912411768594746, "value2":7498908729880858823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635014, "value1":0.5227012296024662, "value2":6384611575231594987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481932, "value1":0.2492598145582209, "value2":7063605156324549396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.458503, "value1":0.3120534424622104, "value2":219352200445992919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144186, "value1":0.6043714345133836, "value2":5343022569080095368, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148408, "value1":0.7539243961545181, "value2":8912511444056366708, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357993, "value1":0.9968001216010738, "value2":3427619790755264753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519356, "value1":0.2148487452898843, "value2":387702087539510555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631948, "value1":0.4990937312036263, "value2":936214783188826913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631191, "value1":0.03578918325727108, "value2":2415177266799716415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.915725, "value1":0.6672040350791707, "value2":7911702372383761591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898111, "value1":0.2610115750745007, "value2":6604319492118960080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871853, "value1":0.700295668840288, "value2":4280817186446527854, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601683, "value1":0.48046941918756864, "value2":4803180873024223807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036853, "value1":0.059813074330959816, "value2":4421336201693669164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.870282, "value1":0.09570589043315565, "value2":2716621407072678469, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543069, "value1":0.31121787546995405, "value2":1276512731094194139, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367830, "value1":0.09434652170549433, "value2":2256603050688647078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468827, "value1":0.8402294714187805, "value2":299914447978500025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910520, "value1":0.6890435117531037, "value2":2224012689460052929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291500, "value1":0.2581942233541333, "value2":94642526795868830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972763, "value1":0.39798358703466546, "value2":711133600064208053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398873, "value1":0.007238280134283497, "value2":414930643350925093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003583, "value1":0.3709596459020853, "value2":7158538565937127848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288974, "value1":0.0033772263476846164, "value2":3505644260463770547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.989217, "value1":0.7234413065529468, "value2":6102862603791704696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151126, "value1":0.962243743917813, "value2":2796295067408464675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.313076, "value1":0.4725232716261173, "value2":5800110791744456113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.169414, "value1":0.10364598232077965, "value2":8013370755699143486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_4":"d","key_7":"f","key_0":"d", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641143, "value1":0.3874568519220252, "value2":1865317425913683745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960408, "value1":0.6444887127846117, "value2":2021509390881885067, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_7":"f","key_1":"h", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188655, "value1":0.9671102461325771, "value2":4499002439335266005, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_8":"h","key_9":"a","key_7":"e", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438594, "value1":0.23438356733095106, "value2":5880329494012383454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.043142, "value1":0.8137602822079195, "value2":5072736510183548575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566884, "value1":0.3037390436862004, "value2":1427943088173652054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.528095, "value1":0.06744248220950605, "value2":2617279385140322030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.075656, "value1":0.6248261098594748, "value2":3102580216650467270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644300, "value1":0.11077089127760144, "value2":252971746503923271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061068, "value1":0.18796880019702752, "value2":1108553918392505010, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.996996, "value1":0.5752434742054809, "value2":8128574670569848168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015917, "value1":0.06928565567804344, "value2":6283279207443485305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238217, "value1":0.29468486256767473, "value2":4879109468201269125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.302561, "value1":0.16374126420414117, "value2":1267840542849218667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.879665, "value1":0.06081772906299747, "value2":7735358902544562393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.436414, "value1":0.9145077406345081, "value2":725040121642504252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845550, "value1":0.6355646354104764, "value2":2412964869373168042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918386, "value1":0.3307533061620177, "value2":4567564826294647319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310744, "value1":0.5825111373150833, "value2":6000767898591093764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206794, "value1":0.8760021186600452, "value2":3359095928804782209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.237359, "value1":0.7515468979685085, "value2":4507098732286155978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247625, "value1":0.9212103536180063, "value2":1210374309278913705, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.531076, "value1":0.9294917705494895, "value2":6008294356031388873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258474, "value1":0.4801885775502877, "value2":4809296905208303773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687251, "value1":0.5503479981842399, "value2":1159057662539665025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294328, "value1":0.579524047000884, "value2":8591427275775614498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030329, "value1":0.3897745465299769, "value2":8195510768796559399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591813, "value1":0.4311576035013457, "value2":57188656194062821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_7":"e","key_3":"f","key_6":"k", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258460, "value1":0.19968970719511664, "value2":6006706732580329556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153359, "value1":0.6163826197572554, "value2":8075354818413450461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977991, "value1":0.47088424321623057, "value2":1437620300807340754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.554048, "value1":0.1312819884002532, "value2":2639581932761358442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.829720, "value1":0.9986118788512084, "value2":3816923447231661055, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648184, "value1":0.9987080219677139, "value2":7493304631161246772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171705, "value1":0.6479656070812448, "value2":940500361907630226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452962, "value1":0.7379240782521741, "value2":8607242240788556904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422715, "value1":0.29055682202125643, "value2":2609014884729200526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997247, "value1":0.03493075047200949, "value2":1341694184124582251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.454967, "value1":0.4083872840808467, "value2":7327414310312171249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268085, "value1":0.9598378795022791, "value2":9180787761997570552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.185765, "value1":0.869366023599698, "value2":6065348312952390593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954707, "value1":0.29066212545734305, "value2":6500179214577769106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134999, "value1":0.0746058596733823, "value2":3954294580008112843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_7":"j","key_1":"b","key_4":"h", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.056701, "value1":0.08887774822608922, "value2":2624540757429278414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503714, "value1":0.7939975169235949, "value2":1543165523760841555, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.407434, "value1":0.5263350258249397, "value2":6207484012298867707, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.216887, "value1":0.6135120862192558, "value2":6706252188296828465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950767, "value1":0.6258842267416237, "value2":6943474392706233335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409499, "value1":0.39019457036616395, "value2":5543165311657950313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.064967, "value1":0.25118206025688317, "value2":4933404061205221336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503859, "value1":0.6260829223715964, "value2":2553798156705274576, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.239884, "value1":0.0646597153433716, "value2":5357886478310061690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.560239, "value1":0.025368350153306324, "value2":1834804451034467956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021890, "value1":0.4250913620299391, "value2":7901663683744276123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.720024, "value1":0.7102429898685084, "value2":8848853087379978689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_1":"b","key_6":"f","key_0":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721708, "value1":0.267345226146574, "value2":8204105067810932755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.688701, "value1":0.8124970825519422, "value2":5072280932532136578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227719, "value1":0.19432503349760097, "value2":2763726347992488568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398501, "value1":0.6725319245300637, "value2":2029542129845625509, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220707, "value1":0.07663396351127535, "value2":6862161694126124253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464464, "value1":0.8033021261492177, "value2":500776341255510367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.369494, "value1":0.6874411246839779, "value2":5472077652556256405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429691, "value1":0.7205162840867915, "value2":2379554410345555228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.003693, "value1":0.5301205012344029, "value2":7339147645001280473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095495, "value1":0.0031080960946583136, "value2":3948518200809126624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.649621, "value1":0.9796250725140294, "value2":5076285879574984318, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.701028, "value1":0.4664187080974912, "value2":8061147057085676917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024773, "value1":0.7030139999067455, "value2":8201442408389043582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570617, "value1":0.6283295712663881, "value2":7893659755244222642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780271, "value1":0.6223497445918998, "value2":4413418515678745434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904547, "value1":0.28169993275009253, "value2":1702651611066126658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.039402, "value1":0.42131084562328136, "value2":3560241450438323083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.612707, "value1":0.19465390726157364, "value2":3161717756488318325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729159, "value1":0.25669937307380997, "value2":1479715553122219203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813686, "value1":0.049715056995271165, "value2":2079826454687289078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747376, "value1":0.7275373884923515, "value2":2435962270026982603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354388, "value1":0.8636816800782944, "value2":6027722589314572513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505532, "value1":0.1663611987871669, "value2":668594150031699725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087699, "value1":0.09888054207021492, "value2":8612173434517336899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.118211, "value1":0.9755664430251321, "value2":3670406131876856616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918355, "value1":0.2320424600489978, "value2":8375963807463485059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106082, "value1":0.2496270382815134, "value2":3218140633568902603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.356200, "value1":0.22905701139904852, "value2":4580449910037652861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272156, "value1":0.08246894655199775, "value2":8520120895086499443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.445288, "value1":0.13062843592283024, "value2":1544733522378030955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189215, "value1":0.023728801354414537, "value2":7950194843845547325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525278, "value1":0.8120711270842138, "value2":6926776820889632931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532234, "value1":0.2803691083519268, "value2":2750609942680894663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169390, "value1":0.721935860273012, "value2":323165999073131824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465184, "value1":0.14065806381420595, "value2":6373433541582043482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211347, "value1":0.5119890992803398, "value2":437881008737730578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.326310, "value1":0.8982944342798628, "value2":7550941116286319434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767054, "value1":0.058056603449873775, "value2":8172199064443221808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098754, "value1":0.802732462605452, "value2":5531373846445331861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.626701, "value1":0.9658445273400498, "value2":6144529458195478799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382019, "value1":0.6200649361393201, "value2":74370737764383341, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.162075, "value1":0.773701976515762, "value2":3415021911489973103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452128, "value1":0.4165931264471384, "value2":1592331302702594637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546168, "value1":0.9818897837453856, "value2":2770075967419206536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794037, "value1":0.6329308941123749, "value2":1855788356162220067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267186, "value1":0.5183944510585616, "value2":2971715621482767583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.386007, "value1":0.7925028746106562, "value2":5519530031313056293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194807, "value1":0.9560872102829179, "value2":8899546063621825706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786449, "value1":0.25289379431092024, "value2":3323685962729555247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722033, "value1":0.791566877264509, "value2":8569419440028565177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.527726, "value1":0.5123053424894405, "value2":5410088201011323097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991248, "value1":0.00835285017592764, "value2":7876085652270552879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971469, "value1":0.7516893928865965, "value2":1615677872423733394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457203, "value1":0.2802029059377931, "value2":3774880897765407379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429963, "value1":0.4064794421282188, "value2":7288375695455796287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807745, "value1":0.1346588472416809, "value2":7295335243976756666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247446, "value1":0.18437680636683432, "value2":2763564622590822709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505494, "value1":0.36367826246909607, "value2":279237135102501393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570322, "value1":0.807767632694814, "value2":3489957523135662213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.770864, "value1":0.12825646293816148, "value2":559050377523308246, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723654, "value1":0.48061854827645123, "value2":2625015563945376475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939025, "value1":0.31592719165163524, "value2":1661597397006530804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935809, "value1":0.16925319668616004, "value2":8449044609778795224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336027, "value1":0.7017500784708112, "value2":8428599073105193397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904159, "value1":0.7924061448124604, "value2":4266452884457119467, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847458, "value1":0.27361557683078364, "value2":3345769475113460399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378261, "value1":0.7380019020377752, "value2":432938298263940108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799977, "value1":0.25999970547607815, "value2":7119589132321429408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786435, "value1":0.812994267772834, "value2":2642960787391817015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.587571, "value1":0.5078453077544904, "value2":7413095241087719577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_2":"a","key_3":"f","key_1":"e", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189572, "value1":0.33029349839366906, "value2":182717731110127548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128493, "value1":0.3005744554380485, "value2":1813809549706584614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.751426, "value1":0.43021252312166497, "value2":1048575931846703963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.824593, "value1":0.9962985612048794, "value2":3451457313304562908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913497, "value1":0.19667409773488792, "value2":324965724484330012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280371, "value1":0.6558253377067158, "value2":3238482029594886215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953879, "value1":0.11691624934505847, "value2":3461549771600466582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189513, "value1":0.9019750283572279, "value2":7575759643142018621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465264, "value1":0.15533711497405195, "value2":8078121216216233303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857857, "value1":0.2731351567904969, "value2":6433553377051334332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.114548, "value1":0.9868338794536605, "value2":6036587149016739858, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734426, "value1":0.4227117888273388, "value2":1455241552057769722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075679, "value1":0.2701709613339434, "value2":3857058056766739646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104255, "value1":0.981970010025583, "value2":3682624724640104305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653572, "value1":0.008103832880105273, "value2":16321050615205215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.790159, "value1":0.6178089725023603, "value2":2820923101957050608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274981, "value1":0.28628503444016773, "value2":5532572594206077059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274235, "value1":0.01138331892670461, "value2":4146948379716629700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280495, "value1":0.19902196639234795, "value2":7564853449905093412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272293, "value1":0.017257126721765374, "value2":91860645774780383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447052, "value1":0.6776141143437516, "value2":5897324288007611430, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.209396, "value1":0.14657117940198489, "value2":3976211106646613048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850761, "value1":0.5201711080016932, "value2":7354069105129952055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.300233, "value1":0.1860571279843043, "value2":470481327297902326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310707, "value1":0.25674808205958144, "value2":2662172304685648211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766353, "value1":0.3692045132202804, "value2":7757321942348800299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214351, "value1":0.6936702854905948, "value2":8248415695462939576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.005940, "value1":0.9564433372149446, "value2":8800540347032950677, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009379, "value1":0.5129617748107491, "value2":2230603715203037945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867507, "value1":0.6850688842908961, "value2":4341430047057401636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640527, "value1":0.8542276865628686, "value2":6337085201397795074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186612, "value1":0.2520653493154805, "value2":3597439324405314019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168374, "value1":0.9686674775113324, "value2":5086009893745928969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.930110, "value1":0.9065446675547199, "value2":9084895148279172527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385337, "value1":0.5310871446244799, "value2":3634604672435791412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_6":"k","key_1":"k","key_4":"a", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273397, "value1":0.6978323941286791, "value2":6270777752769360572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.644440, "value1":0.7390124331493296, "value2":5633558971491993000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692014, "value1":0.5735092134024398, "value2":4018757059763515593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307407, "value1":0.9523210688679733, "value2":4664291077378357176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.864207, "value1":0.2504856063622225, "value2":5125796995165750818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.891946, "value1":0.8905322399035085, "value2":3790600090207208565, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.196262, "value1":0.02637661052612273, "value2":3345947703698603019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609259, "value1":0.9964561610316119, "value2":4297199572790537149, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.479133, "value1":0.573177630994298, "value2":4352137818057928535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.212122, "value1":0.9374174976714222, "value2":2883686750556684691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_7":"i","key_1":"c", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143194, "value1":0.5493618690016547, "value2":8675397367930704936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.911116, "value1":0.8323148117854684, "value2":3727739579448171557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.100156, "value1":0.7882098153425944, "value2":6270723454145661182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159904, "value1":0.5891414895820671, "value2":7892306327256327213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.561342, "value1":0.575909380567069, "value2":2087294011397500886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.542768, "value1":0.7419123808449162, "value2":5892135507379591161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977295, "value1":0.2537719509110862, "value2":23083079770681507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.690704, "value1":0.08136832639928646, "value2":8018440792149947537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.681388, "value1":0.6173470752976815, "value2":4725502695033185312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045833, "value1":0.9052130764103621, "value2":2335561799986333486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.621313, "value1":0.5578705772605427, "value2":5423853801041319612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362143, "value1":0.058506006415061124, "value2":1952601720041809520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.019544, "value1":0.2454839257683829, "value2":5439837990435419484, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414507, "value1":0.8898999762410835, "value2":9399980232748751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.946393, "value1":0.5231814332055934, "value2":8044307915684488670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640533, "value1":0.1401932575466956, "value2":3489584691123939236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.641620, "value1":0.7793690564908118, "value2":6745229093735363060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256954, "value1":0.9792655297711959, "value2":8340241164033802673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.578274, "value1":0.6786422376056035, "value2":2268577288780537910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168131, "value1":0.7011560783500894, "value2":5754899077173699372, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261168, "value1":0.41241864943729767, "value2":6926145892542625665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.142353, "value1":0.026897262060108916, "value2":3407492707727317952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453717, "value1":0.9480329042934568, "value2":5909842310603335433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_6":"d","key_9":"b","key_4":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.498253, "value1":0.2696408414861873, "value2":9093122739488143709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152193, "value1":0.829008206269259, "value2":7872930012981859271, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147195, "value1":0.40564509426795664, "value2":388387455399110782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.756476, "value1":0.08296241719338777, "value2":8819867312676478909, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472139, "value1":0.10622575406945087, "value2":2389837147036501120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553362, "value1":0.4640810980824681, "value2":9129773762938561987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138568, "value1":0.27153851873742235, "value2":3922539618739320056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629854, "value1":0.06560725045609712, "value2":5242856550347246211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532533, "value1":0.6685532806932011, "value2":4403464406211940116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.102100, "value1":0.30269603070417783, "value2":560972484995423105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.594212, "value1":0.3436785616541505, "value2":6118258015340076117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182345, "value1":0.5829131946786057, "value2":316161802601703652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865350, "value1":0.9510270552078569, "value2":5815202444060769343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.417157, "value1":0.07805271825644472, "value2":1671886271645798176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453428, "value1":0.07722736953435244, "value2":5062577774198478117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923875, "value1":0.3869476521576109, "value2":7644772358804477804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170062, "value1":0.771360835653392, "value2":8029933370964328416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.341390, "value1":0.538519995660261, "value2":6637857221246165272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267222, "value1":0.4666279704151431, "value2":716146669296183590, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562383, "value1":0.35102191528024995, "value2":8428363533739190009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767969, "value1":0.6173626174937589, "value2":6290463280158957319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451339, "value1":0.546379610337649, "value2":1547962669719183472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464199, "value1":0.24671185662577808, "value2":3561721323195761229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.154391, "value1":0.45596856325620033, "value2":6560907831281448338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.709403, "value1":0.6109692366076535, "value2":3486468545935536219, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570798, "value1":0.7335030585660903, "value2":1572258901091495894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.198469, "value1":0.7425214958348296, "value2":3904425769094656355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315912, "value1":0.9245437278546984, "value2":3047808486948520796, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186365, "value1":0.8721078635865479, "value2":6305488287785962360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759050, "value1":0.3487806615217454, "value2":5791750426523247447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_5":"j","key_9":"j","key_2":"f", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991504, "value1":0.5895957831739977, "value2":6282966318152098641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355229, "value1":0.507857444797784, "value2":295494213133771851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551655, "value1":0.4307208831464999, "value2":7265410065885482832, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525491, "value1":0.6404292728685821, "value2":1060635693413562315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008415, "value1":0.6916001000931801, "value2":2353592246165483773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400680, "value1":0.36661865825780576, "value2":9104767548881126156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376594, "value1":0.3981424874885838, "value2":7118511131751820911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553143, "value1":0.6460744131761798, "value2":1887476981066696157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473253, "value1":0.8348910772414322, "value2":7455007366058138980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457579, "value1":0.4708610456466168, "value2":4593338028669686470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433566, "value1":0.16598645827046257, "value2":5257287213674383265, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.357007, "value1":0.7860515830836307, "value2":2972497673348348004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.461293, "value1":0.35370822925539525, "value2":355204159426372577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.975565, "value1":0.6491904274322989, "value2":2226145731856801788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951226, "value1":0.8792346577420341, "value2":5756323624906456991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143238, "value1":0.9818710036776283, "value2":4180983964934518595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874423, "value1":0.3287251148209575, "value2":381928105696021842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648299, "value1":0.1842978254216889, "value2":5567179544229053293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055528, "value1":0.2649298342733898, "value2":3063386077105020906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551525, "value1":0.008929798118195076, "value2":1932022596647743318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078106, "value1":0.6249963076831733, "value2":3775614794809061222, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595048, "value1":0.6556888188689401, "value2":263555282139003693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270487, "value1":0.15652423199780005, "value2":8005167724994681049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986526, "value1":0.44168299905409525, "value2":3616684978189935787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648015, "value1":0.6935400265504532, "value2":3554010850084269714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082643, "value1":0.4464596047653067, "value2":7393355921025489008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.297812, "value1":0.8411130245970256, "value2":3060908524323233968, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.023535, "value1":0.8979883834169406, "value2":3955075066530261651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775710, "value1":0.803115138047979, "value2":1956023105923958421, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276355, "value1":0.35534087510852963, "value2":70554362711904210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227561, "value1":0.3475937884638202, "value2":5770217876057854108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.997058, "value1":0.9920212102015171, "value2":4179786645072950325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_9":"i","key_3":"e","key_4":"j", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330001, "value1":0.7117987721576648, "value2":1416148178901813673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.859492, "value1":0.07088427876105753, "value2":5628499584516778303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.758514, "value1":0.5315142010423931, "value2":3999275359211756172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567291, "value1":0.5976846753929923, "value2":3568009536081310459, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914132, "value1":0.8979176888984427, "value2":4778372357738152583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.470387, "value1":0.2136647847272661, "value2":4632422950916722092, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.180567, "value1":0.7349455061346736, "value2":4238367012667671950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444766, "value1":0.35029664162897833, "value2":4149159141941833270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.351876, "value1":0.719839928132912, "value2":8311035304279403168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.343283, "value1":0.5403744774398413, "value2":7831018924301288402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202600, "value1":0.07323861368397915, "value2":7512140161978615708, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953729, "value1":0.4072430540967121, "value2":6162477020927495810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206750, "value1":0.7276384026815652, "value2":3867378392166470570, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313521, "value1":0.19760060640640484, "value2":1845943836170488560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774075, "value1":0.9632484532837077, "value2":2186178755453106945, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228120, "value1":0.1138891785769252, "value2":2119625663561385331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355712, "value1":0.12321943936422765, "value2":5732137663228356419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.004820, "value1":0.1110884226197388, "value2":2265343653685159151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.630751, "value1":0.6269969213669998, "value2":3359610757917670818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.015500, "value1":0.20581219173107831, "value2":452968813923389028, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646040, "value1":0.14066092038312122, "value2":6343927209062846133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220902, "value1":0.6865529990306172, "value2":1648919416099165702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.700811, "value1":0.4621690976315305, "value2":3551937779376072568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607198, "value1":0.8914891421096152, "value2":246252330281719887, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.574035, "value1":0.7865743679720645, "value2":5713754277685792561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_7":"e","key_9":"e","key_1":"i", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473497, "value1":0.13845795308548414, "value2":259488391880499746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944779, "value1":0.9609062478569093, "value2":5705930854457323614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775443, "value1":0.1259913054905557, "value2":5080990926603035088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010365, "value1":0.4903104589413477, "value2":3314614042137410585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913172, "value1":0.7864192987989552, "value2":8344919384469197086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.796557, "value1":0.037094581426906255, "value2":9011770804417776349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.314769, "value1":0.8877112795540667, "value2":6683425457896895076, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.806104, "value1":0.5962447685862431, "value2":5435268152791547936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579388, "value1":0.6885293468734471, "value2":9064761548833010698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.391022, "value1":0.5900734892123236, "value2":4669259037774416678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762980, "value1":0.7685532202125763, "value2":2315953471058650483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.135687, "value1":0.32474681440848086, "value2":5755607255133419339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104934, "value1":0.14589803567481788, "value2":126178294464524762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.052690, "value1":0.593421743400015, "value2":5119197034484564414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870007, "value1":0.4673411129128983, "value2":2860543686633489085, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629308, "value1":0.4498616996591619, "value2":5328613189051877451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711415, "value1":0.8849559918426787, "value2":5903543494874730971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901063, "value1":0.5349473283169682, "value2":4797032563086773247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.651731, "value1":0.3127106327077303, "value2":5700343521566497813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825358, "value1":0.7283436835171048, "value2":7844311848127264742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.956544, "value1":0.8853976152911399, "value2":5661939812745981161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235737, "value1":0.26141150008124164, "value2":1680083033155058708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.537374, "value1":0.4021256295776066, "value2":412217863647126208, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238448, "value1":0.20048610554803628, "value2":6383291537020572135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506422, "value1":0.3812390668975348, "value2":184701845674968962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.456772, "value1":0.12479606082790685, "value2":7770738457381111114, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.789484, "value1":0.7412786023699602, "value2":492664655073083816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931307, "value1":0.2536425784942127, "value2":3030103981590386173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610031, "value1":0.5842291904640166, "value2":5028658995770055300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.068107, "value1":0.6578126156370956, "value2":4352952521986270134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477620, "value1":0.16120256759137608, "value2":8502715215777515370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.124418, "value1":0.2689168702571952, "value2":404701828083799050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.779829, "value1":0.4388821199327946, "value2":4899529018433185166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800101, "value1":0.2714233939432663, "value2":584157730559703393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_7":"i","key_0":"g","key_2":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860807, "value1":0.4135766617320191, "value2":3872756991712201227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.917667, "value1":0.2149289211585331, "value2":1070350813804397058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214604, "value1":0.6792613272879433, "value2":2691750153580462441, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245813, "value1":0.6913342601180135, "value2":6216928639197666035, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087487, "value1":0.9301668438033497, "value2":6232780192940116715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.069143, "value1":0.6293446581971062, "value2":7108419812354921929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635591, "value1":0.9539262040674916, "value2":2954917330417044389, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672064, "value1":0.8389645710809651, "value2":3571122666192720690, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818753, "value1":0.0576648702049322, "value2":450051931472461981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270657, "value1":0.28617962738097397, "value2":7674542934415081242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726745, "value1":0.2340744755556526, "value2":82189729669291091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.897029, "value1":0.2638345630149327, "value2":8238518585440831118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721815, "value1":0.7091199604872773, "value2":6366310396364171134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.130557, "value1":0.49724111456500786, "value2":5852311186811706792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447829, "value1":0.6230142507141527, "value2":937535691807945647, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.802396, "value1":0.6394461194574114, "value2":7242948197847068875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.424144, "value1":0.6000896127174374, "value2":8028530164864078540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.426986, "value1":0.7003205100320555, "value2":2322106140405745021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942361, "value1":0.016353745384231323, "value2":4629676907790639064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653679, "value1":0.37062297658466353, "value2":8208981100462487617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.992030, "value1":0.7072168752768998, "value2":3820991654654691772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.346953, "value1":0.4620611667835753, "value2":4760146202938852105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321463, "value1":0.9135126280713665, "value2":8417775919956392807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168908, "value1":0.05729213648419359, "value2":8181061490241314804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.230852, "value1":0.17085551083436254, "value2":3688016011930153595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.066979, "value1":0.33115248245947465, "value2":398734641986366420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825110, "value1":0.29481497745263574, "value2":7623192343447837676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.979130, "value1":0.18961515202216345, "value2":6037869636552702023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692641, "value1":0.4102441559202999, "value2":2079215473977001869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459527, "value1":0.38418621904384953, "value2":8455919021737289595, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924379, "value1":0.831174718392, "value2":8455941435197243693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433952, "value1":0.6847864876960775, "value2":6021961695231181621, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195096, "value1":0.9137826039265785, "value2":1951159443698721147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.392032, "value1":0.7140691740516873, "value2":7387270195217478014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711603, "value1":0.30553043110809663, "value2":5447610537284294754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024988, "value1":0.636153947206049, "value2":6011662688198487566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140076, "value1":0.9829818790209834, "value2":1992400557046890911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591682, "value1":0.734180708126354, "value2":8165041750078897088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.254962, "value1":0.4530139927706582, "value2":792701874387399785, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653418, "value1":0.6853469695508817, "value2":2445415548541311714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.406935, "value1":0.49945705352987035, "value2":4326827190201670587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140154, "value1":0.19200910536315854, "value2":7788114053604406936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774910, "value1":0.775205817556658, "value2":1778011502106771934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224005, "value1":0.4109368947961367, "value2":5090636250181538696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025070, "value1":0.8639998386130291, "value2":4097020029155871346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.110800, "value1":0.46475105418540896, "value2":7930781849523032659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502796, "value1":0.19123829931308983, "value2":4926709577891896623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_8":"g","key_9":"c","key_4":"e", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.483311, "value1":0.21080006327277256, "value2":1394785449775081074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194479, "value1":0.7780668170411426, "value2":3729639098545975029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_1":"c","key_4":"b","key_0":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385848, "value1":0.4299644347219865, "value2":7191790268699040877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211519, "value1":0.32521468058588077, "value2":2791560527192035681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228801, "value1":0.83757037869911, "value2":7191574037900046957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_3":"e","key_7":"e","key_0":"k", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.752930, "value1":0.8040696176893675, "value2":1942969218455296906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951018, "value1":0.07747976080254226, "value2":1630225104393013299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_9":"g","key_4":"k","key_5":"b", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.750256, "value1":0.816008410350239, "value2":6724933613410762759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583564, "value1":0.4508316797935446, "value2":6917556384524072348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697470, "value1":0.8171672522091267, "value2":2060386169134326462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.974221, "value1":0.7951082158998446, "value2":153021997721922670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134841, "value1":0.655263428147034, "value2":6943977496854991893, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_4":"b","key_9":"e","key_1":"f", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.673465, "value1":0.16441674918821836, "value2":3664243165148767961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152562, "value1":0.26528241288165916, "value2":1119343360475809102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.420604, "value1":0.2336675869017092, "value2":3279324159460647609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.622036, "value1":0.6846769209119129, "value2":7351022317760422092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687001, "value1":0.06651364619191985, "value2":4264970580734046753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106345, "value1":0.7730270909594562, "value2":6259010316309812350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018243, "value1":0.39163099713401783, "value2":2052648194611616106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576138, "value1":0.9602266945474953, "value2":5355810905366063269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398686, "value1":0.6503809877595728, "value2":8153069195959796847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.367754, "value1":0.6043902819598663, "value2":3335160234939115824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.089576, "value1":0.7877453973247192, "value2":7617166012064737014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.112529, "value1":0.42004273729990893, "value2":2084869905228279863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262886, "value1":0.6795217484416415, "value2":8937461554206260283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652433, "value1":0.15404027924570762, "value2":16114496573329591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.285612, "value1":0.36648611710364465, "value2":4517711436918873094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.213223, "value1":0.283682948749034, "value2":1960872184236727731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_9":"a","key_5":"k","key_8":"j", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096677, "value1":0.1578537002758255, "value2":7008414275581832390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_6":"a","key_7":"d","key_1":"j", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.900633, "value1":0.993822419434507, "value2":3303474850432526688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862995, "value1":0.2925883436045491, "value2":4890951234376924728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.038886, "value1":0.18850439286651008, "value2":7964663822924622191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_8":"d","key_0":"e","key_6":"f", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742880, "value1":0.8063312642154098, "value2":8220352304335077837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_7":"a","key_9":"k","key_3":"f", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545517, "value1":0.023471717745134867, "value2":5984820965643364202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.079979, "value1":0.6088556886509169, "value2":8961288376687773441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409232, "value1":0.6445899053375183, "value2":992192951010465319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224358, "value1":0.13573773649022736, "value2":1165195770092489984, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211580, "value1":0.8798345576210722, "value2":2259526712170036956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.137602, "value1":0.5470546586447269, "value2":2196698895749095736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950011, "value1":0.5109493788687889, "value2":7352128446573662528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098941, "value1":0.972587647595324, "value2":4702835811693986573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036022, "value1":0.7940355478266105, "value2":3489006783771959904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.257286, "value1":0.6629731808259061, "value2":6607686064851284173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283256, "value1":0.4735441861177576, "value2":690002256366468336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729363, "value1":0.5152988869648636, "value2":7191544072945571987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.671685, "value1":0.7719299111978523, "value2":8611060565403129118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585682, "value1":0.23795953447079246, "value2":7946332098229864997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.187797, "value1":0.09090886526352324, "value2":470342401691160594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.617428, "value1":0.5223785659162559, "value2":5891162691146013349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.053786, "value1":0.9753973493719631, "value2":1732499838549362952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.830052, "value1":0.2400470925507307, "value2":7569204709479310112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566485, "value1":0.9031315965779712, "value2":1637870392526502935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104166, "value1":0.8068833505635166, "value2":5557577539110699594, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457581, "value1":0.31154508147136156, "value2":3306826589254236098, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740978, "value1":0.6517923386066168, "value2":6929527434345999196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398554, "value1":0.7163746669385104, "value2":6895927508923594471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174253, "value1":0.37937402384141433, "value2":1418065909790834443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168613, "value1":0.8552112788507821, "value2":1201668238332263018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_6":"i","key_1":"k","key_2":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376103, "value1":0.7428463858250676, "value2":9210155210384521495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.798751, "value1":0.16215960908856522, "value2":4916478620663149838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.588227, "value1":0.7494081995223059, "value2":1969994590907491188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270909, "value1":0.8815916954125361, "value2":2264317244685656881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.466310, "value1":0.09496768292246209, "value2":5092383460942749308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543412, "value1":0.9329719667021983, "value2":5844328151679816255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.834516, "value1":0.9248915367132075, "value2":5869247588035820642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.916922, "value1":0.052803170907048405, "value2":105752679780838825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583307, "value1":0.04534276867452472, "value2":1081545276806182474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.957758, "value1":0.1893001037569564, "value2":4530859851365168030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723091, "value1":0.22925973916873355, "value2":5896808329469448104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.719902, "value1":0.5038773612977684, "value2":5428690516490754673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025587, "value1":0.7184099655587061, "value2":4108913234892685798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907580, "value1":0.24492386825368684, "value2":1156962833669313399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958121, "value1":0.9818175985414233, "value2":3857893912179537366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.657555, "value1":0.17952027005330515, "value2":3409773579631767064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829662, "value1":0.9863896179595923, "value2":7017412361402418209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382356, "value1":0.13377486891358994, "value2":2261611253942664564, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.735596, "value1":0.3414280491857885, "value2":1656516704640942493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.365266, "value1":0.08360581432684039, "value2":751089660510969379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.236047, "value1":0.6527140947106955, "value2":7257801694535836049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_8":"b","key_1":"i","key_2":"d", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.563096, "value1":0.4425122630203769, "value2":8109435442419953734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870160, "value1":0.7097748830786019, "value2":6394016408878658736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853311, "value1":0.7865716590956611, "value2":9110725490965710227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.103882, "value1":0.9153251610630555, "value2":5340001815935707095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.319691, "value1":0.32338872565142723, "value2":9182777252636363483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971522, "value1":0.5621974159988574, "value2":6803454508761360177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893225, "value1":0.5781000116088169, "value2":3230113190945788558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.910026, "value1":0.6955188069638236, "value2":3766609848094762934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.680766, "value1":0.24230859153208487, "value2":5617809008491748996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.493463, "value1":0.1697890841270668, "value2":8556153147754124462, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.849699, "value1":0.7552917967316483, "value2":2395400148768820203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091646, "value1":0.5496353344673094, "value2":8512525001992418076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104484, "value1":0.4174390270830882, "value2":8422771009020841833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210780, "value1":0.8954537390033984, "value2":4547239076695924457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.882314, "value1":0.5142219879570834, "value2":7919864591569880176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.983791, "value1":0.42885136319724737, "value2":7969816758490279476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747300, "value1":0.4699241249924091, "value2":4491569276746468237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939701, "value1":0.6800950779540557, "value2":3198358625582737117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261442, "value1":0.4575968148914831, "value2":6477186909414818810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256111, "value1":0.693918133492274, "value2":1509539897942915838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893394, "value1":0.31628967903294547, "value2":5501152751179175693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742786, "value1":0.9162968056386509, "value2":9070857057311832045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.642040, "value1":0.9183764345745785, "value2":1183434137529208194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045625, "value1":0.9635794700047972, "value2":8991017817592303034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847013, "value1":0.07723071092189422, "value2":6180620156015906931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532846, "value1":0.5943541460929466, "value2":2462231763403080171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514157, "value1":0.7898774054205837, "value2":9059207414611927294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_9":"h","key_0":"i", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.044776, "value1":0.9033139820464094, "value2":6478344621702437696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043799, "value1":0.8974102840071708, "value2":4685725453168510298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138572, "value1":0.5940072973710391, "value2":9150962581658325077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.215236, "value1":0.35464434283570645, "value2":2728457088873951162, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545063, "value1":0.4562894763140951, "value2":3160291795997224895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.277361, "value1":0.9620522510409097, "value2":2528750129940465586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477478, "value1":0.5247853848955957, "value2":7252619165572966740, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.940911, "value1":0.4288772636544275, "value2":1739941452377564494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.488981, "value1":0.07801581197604958, "value2":3785496084335402474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.037817, "value1":0.20985274999200432, "value2":3757407018866875990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477559, "value1":0.26133928393072436, "value2":4699711244003996922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.107051, "value1":0.5035109860681579, "value2":2406818127100942026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692353, "value1":0.44174492856895586, "value2":5849623391254459092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.948736, "value1":0.6131752371911697, "value2":8355568973826156875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562536, "value1":0.5314554437999132, "value2":2325922727970832181, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336595, "value1":0.8477764608771982, "value2":2899112579581457318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609668, "value1":0.7984731186839654, "value2":1035702079046481045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.446414, "value1":0.8536378880203014, "value2":1450863797869138094, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.345841, "value1":0.005289676671501025, "value2":9055262701472338992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.665956, "value1":0.5725507407225954, "value2":2388492838265413703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.972916, "value1":0.6568884959214099, "value2":4110169872571045957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.370968, "value1":0.9171054847132539, "value2":9195948027770005199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109600, "value1":0.6976295242986092, "value2":3260150427416206815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722444, "value1":0.756396263144578, "value2":5473984714896425934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.011414, "value1":0.35518020333634837, "value2":5432640933038728748, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.442479, "value1":0.9135774445018765, "value2":1658872104403529722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860707, "value1":0.1419422977742929, "value2":5745627115489673481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.925502, "value1":0.19711051262279727, "value2":4597556979704859536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.636414, "value1":0.8593669141574183, "value2":3782467267185587990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157759, "value1":0.4115235495685266, "value2":3223572108511746336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399657, "value1":0.24629418245237128, "value2":4491685955930394692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.941994, "value1":0.17062270403788604, "value2":3592023630547953731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282191, "value1":0.8843261061566057, "value2":6691527928807377675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938895, "value1":0.7167094005927336, "value2":1624441217258843945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492025, "value1":0.09243530358924243, "value2":1564407566445130438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837030, "value1":0.7918808546682014, "value2":6330206212330805579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.123530, "value1":0.594358060271596, "value2":1086840755761366155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.515405, "value1":0.8774866546466452, "value2":6471463500843130242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_7":"h","key_0":"b","key_5":"g", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814527, "value1":0.46973250770262637, "value2":617895520080191082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.712491, "value1":0.8366069393700927, "value2":8124214803409242134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.598705, "value1":0.13529834276697159, "value2":4048137400971693133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.115099, "value1":0.013374612744122648, "value2":4497239763994228002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.871918, "value1":0.5477513867264745, "value2":6929880807654898865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.600230, "value1":0.06263242199896547, "value2":7099533171627178830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.832053, "value1":0.9159517384120022, "value2":6068004913866310526, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497324, "value1":0.7077689276515862, "value2":8496454581274124817, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912067, "value1":0.9966402570764005, "value2":6508382435383037912, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394175, "value1":0.4056622098754871, "value2":7483291783499825043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931738, "value1":0.7229091089946994, "value2":6319491958713592651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.778862, "value1":0.3546846056067059, "value2":5823244617744051085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.571315, "value1":0.32603322678558494, "value2":617900084765432670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.022106, "value1":0.09634190796880056, "value2":1538435119318752571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777759, "value1":0.9991415497234644, "value2":6469274665586859562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813159, "value1":0.6009731720422214, "value2":5681496788640905646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990295, "value1":0.866629146873839, "value2":454920947769684349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577898, "value1":0.4068627360956291, "value2":8354382964548721361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652554, "value1":0.25976343730256174, "value2":4258649045980411819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031275, "value1":0.4835922460947118, "value2":1965749848088972278, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535728, "value1":0.0741614372394035, "value2":2365543114658538202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.725451, "value1":0.08897749963143643, "value2":2568717794069312737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517633, "value1":0.31412074219954, "value2":3756199754285644161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170595, "value1":0.31501680305424884, "value2":835864889280146724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.839334, "value1":0.8860221287236891, "value2":7263801537900469082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874807, "value1":0.37435025928000243, "value2":7922739501753662605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914081, "value1":0.7766285758384013, "value2":8262805777914063347, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.668141, "value1":0.6740415300239384, "value2":5791687526396481300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.302177, "value1":0.7738583815242437, "value2":4861664383009090324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711096, "value1":0.314472985957645, "value2":8509002774444373125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321149, "value1":0.44826985902573663, "value2":5704588376153110492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.438786, "value1":0.9611261969930821, "value2":8417802456083541652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894152, "value1":0.47571072484930443, "value2":22587301710957954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939556, "value1":0.9226156153333962, "value2":5568759137623866023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159870, "value1":0.6737683215814319, "value2":23284244914642959, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273159, "value1":0.7150778132424469, "value2":7795879578675436289, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.116050, "value1":0.7176538032312242, "value2":6236044932175094049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_4":"c","key_0":"k", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274847, "value1":0.2989651835291124, "value2":2096206477458603718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764340, "value1":0.5947263013675924, "value2":949703240129275173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_3":"i","key_1":"a", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.073476, "value1":0.1345576666087966, "value2":2417122739684294792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418445, "value1":0.8647442373572726, "value2":4787011153359227960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721780, "value1":0.31241733378602804, "value2":2882725760348945089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338609, "value1":0.30106180327338455, "value2":4145007555047407199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109899, "value1":0.7817730958310003, "value2":5914588161487452188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.547269, "value1":0.29274410504915255, "value2":3689334620249634917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741862, "value1":0.6895772402009777, "value2":4687778247837546339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_5":"b","key_9":"k","key_3":"a", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.744160, "value1":0.7533050071878981, "value2":183319309026503278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.880890, "value1":0.8337197335863336, "value2":5434700933804867558, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467450, "value1":0.7739450951598513, "value2":8969122970020127825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036765, "value1":0.05604892859180446, "value2":5414621267015229038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458853, "value1":0.3999783567682559, "value2":7084993602991838778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.144253, "value1":0.945939865721221, "value2":9151551595196013169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_3":"b","key_9":"f","key_2":"i", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.519078, "value1":0.28562220040037745, "value2":3466327056358787385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.425448, "value1":0.1561471042921615, "value2":8555609213379362480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.071015, "value1":0.867937414704974, "value2":7843796710543352776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936460, "value1":0.8572536817998362, "value2":3953769786680934648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886358, "value1":0.2289778721765615, "value2":7617650297619769040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.029905, "value1":0.0015729191075447354, "value2":2294660572707247909, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777761, "value1":0.9646894479758846, "value2":530724906601952281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_6":"e","key_3":"i", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294885, "value1":0.6419477950120894, "value2":3626154680940200317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_6":"g","key_3":"c", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210708, "value1":0.9629469371323777, "value2":1825251501924013732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354794, "value1":0.9560891897559984, "value2":7402084087841622412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.007599, "value1":0.8488357845508476, "value2":9129548229641808509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.949198, "value1":0.3153555717941824, "value2":7434568718980757764, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373802, "value1":0.3277196318709405, "value2":3450473392216255589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.815211, "value1":0.19640953195392433, "value2":6991753246444170561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532006, "value1":0.8004824485602414, "value2":6264834568239277372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.745472, "value1":0.20136170467216696, "value2":5514377914099787271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.296097, "value1":0.9552161212557331, "value2":1452145138149816534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350623, "value1":0.4834905958063239, "value2":3177892637454509046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894862, "value1":0.26274803670489, "value2":5904879356296841126, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663989, "value1":0.4617229653712207, "value2":5839673609391532826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920142, "value1":0.4202645422145252, "value2":6322617270554106528, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174524, "value1":0.7616258127751694, "value2":465160921360548556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829492, "value1":0.9553481357587528, "value2":1445392053101953093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.419698, "value1":0.7725216991307152, "value2":4285351248890366822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235468, "value1":0.9784345273084265, "value2":3668509819811778630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935515, "value1":0.5182554276152802, "value2":3295309486606700899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539034, "value1":0.37544115144770396, "value2":7306450144895271272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729718, "value1":0.8597111754781319, "value2":3238304594948466970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538038, "value1":0.32408768170822555, "value2":2186207806521299657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566784, "value1":0.4892156459829451, "value2":8596226688481136458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.980628, "value1":0.8327067880954203, "value2":8561137068463144173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153522, "value1":0.15467129518207473, "value2":8602088790336601513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748640, "value1":0.18204671422495655, "value2":6797988693835752844, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247682, "value1":0.09249535842888819, "value2":5067988675442272893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936986, "value1":0.9192193881284415, "value2":5123985662043884348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378293, "value1":0.815846067412878, "value2":6070693823656037795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734247, "value1":0.49161995996865615, "value2":624464230830574462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.661797, "value1":0.2056464067499432, "value2":6156118294329348494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.033052, "value1":0.35405594685495784, "value2":520776464703947921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.575933, "value1":0.9335712679806217, "value2":425476291317360309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.732828, "value1":0.4700815629744225, "value2":8550463063060596020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.899613, "value1":0.4385944862087238, "value2":7654768751774889730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165683, "value1":0.16899343532059313, "value2":5688582074969409894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986964, "value1":0.253512537787597, "value2":8166110434329136902, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766787, "value1":0.6248495968383331, "value2":8035594691046262470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338227, "value1":0.6533144293221477, "value2":3922786069578896774, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912364, "value1":0.888149605868496, "value2":7372081699617979857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_8":"f","key_1":"a","key_4":"a", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.223399, "value1":0.21329948027080436, "value2":8415945829680890770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.922973, "value1":0.6987095535905381, "value2":8304144963505893865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.484292, "value1":0.6786625244755433, "value2":595838710724334047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977202, "value1":0.8656898577077133, "value2":3529785897914282727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.231129, "value1":0.9035229811237685, "value2":2477457900157688998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.565357, "value1":0.40499024658110055, "value2":1356497801667980388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.691153, "value1":0.5098765403884086, "value2":2450630350992474672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459909, "value1":0.07976566115765178, "value2":3732709013710166299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104179, "value1":0.3445131016649922, "value2":528513889221012225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764200, "value1":0.1348858138724795, "value2":409727276676244812, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288182, "value1":0.46853874054271116, "value2":6728001446241512599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502298, "value1":0.31577497007213595, "value2":8859554815564990855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990152, "value1":0.3102588503519357, "value2":5723742327078842798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520176, "value1":0.15090023272414005, "value2":5438285255049886987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.155921, "value1":0.6850691286515489, "value2":7357616240486203605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350044, "value1":0.8819289168133723, "value2":4656926419487785926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.383040, "value1":0.8254228746494352, "value2":2036646364854508134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451480, "value1":0.9507122214170446, "value2":4089393907406910121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516899, "value1":0.7707382433506376, "value2":5274734231531309553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467177, "value1":0.603964532592086, "value2":1716897544081104224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264870, "value1":0.9573825263948318, "value2":3412285010274210994, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.556842, "value1":0.17542611082882992, "value2":1043370942216175800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009549, "value1":0.21407922779664104, "value2":5715745607073532896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462287, "value1":0.8132271496462169, "value2":7467232826614980432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538632, "value1":0.7627133328751289, "value2":2219523491939336599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.217749, "value1":0.9344741716490049, "value2":8166313022724995390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811552, "value1":0.9190437296914767, "value2":5885018846694177935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814548, "value1":0.8390440310277583, "value2":5493133492272371657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.817102, "value1":0.6598894636593393, "value2":8537596977440503466, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.067200, "value1":0.263575315223174, "value2":2125629267937428501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_1":"i","key_5":"h","key_0":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082507, "value1":0.6268250434274911, "value2":9058686535109969227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734195, "value1":0.3167788157158676, "value2":7432987222538757392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.795085, "value1":0.4488172699242242, "value2":4473525864538142538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_6":"b","key_8":"i","key_4":"d", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.332775, "value1":0.8505535842615966, "value2":6047205731470921861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727338, "value1":0.8876718108712484, "value2":1531222095185940701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.387878, "value1":0.5788599279069843, "value2":8604920509885625807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.233954, "value1":0.6310172938155442, "value2":8925576745188878375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.805738, "value1":0.9330396354121838, "value2":1424947937953535481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726267, "value1":0.32286499179099665, "value2":3766446177817631940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229570, "value1":0.9916323313225612, "value2":3888482529661188248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167559, "value1":0.08178718223938254, "value2":7520256881259787943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845474, "value1":0.6808433459458884, "value2":7844247683138918907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400844, "value1":0.39452808620697233, "value2":7357453563119638881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206854, "value1":0.9379245513406866, "value2":6598367899784187154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640341, "value1":0.11560748654575319, "value2":414342029075207599, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497616, "value1":0.26256799403609016, "value2":7366324474884701561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.822083, "value1":0.6049405008185211, "value2":6275944038502753172, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.819954, "value1":0.373947872013572, "value2":3355608188213748317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739894, "value1":0.06143600864431769, "value2":309096149476303416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.684612, "value1":0.7692595721499597, "value2":2629612954063953162, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958165, "value1":0.6185066606297357, "value2":2824340682692220488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944290, "value1":0.3864309642344103, "value2":6500366333535522458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576232, "value1":0.6603219832087598, "value2":7174737056650755203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.047735, "value1":0.4501836136879949, "value2":7043766590114442160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282665, "value1":0.3628575018664628, "value2":3673366331802014852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.590235, "value1":0.30332064856581503, "value2":5473820163155790307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995108, "value1":0.1854914610326227, "value2":3604467487566582522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270312, "value1":0.9632251653093951, "value2":4816282908135882739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541752, "value1":0.2581250654809775, "value2":6451610961696076892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535737, "value1":0.3163976646275719, "value2":2322386485534186745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610988, "value1":0.49088683208094147, "value2":4479251775302671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.111345, "value1":0.016089752487428322, "value2":4062002565116218346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.380428, "value1":0.8878960076695425, "value2":7482838775142667379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_5":"a","key_9":"i","key_4":"f", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.286191, "value1":0.5281244570210905, "value2":4742826162899756346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.633864, "value1":0.28655049708989444, "value2":2281246548041472554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517718, "value1":0.8760337085208192, "value2":6876760617398624261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275264, "value1":0.8763572333888452, "value2":2040006649964380357, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.133688, "value1":0.03683553631174895, "value2":879315984438783745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.178775, "value1":0.7921271023115032, "value2":8974452178625132408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.559621, "value1":0.6842414958291543, "value2":4473030242509416772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.933147, "value1":0.42656042046254644, "value2":6875784776532081765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.265632, "value1":0.6577298952553573, "value2":4117148991099157160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_6":"e","key_1":"d","key_5":"k", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.291705, "value1":0.35063088013808896, "value2":6022154315116551060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.486399, "value1":0.9066996869066141, "value2":3890435706555388518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539080, "value1":0.6597581253474917, "value2":7149394823928068405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.172124, "value1":0.8972702809236033, "value2":5889011931565346539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399820, "value1":0.22420487156545466, "value2":3632173066141869984, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857491, "value1":0.658023411947534, "value2":5044439445561970794, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.960983, "value1":0.8072960706443908, "value2":4799697070254932404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672380, "value1":0.6734321372300892, "value2":7080832382541851744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.699212, "value1":0.7095725169760027, "value2":1829131464356530776, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893830, "value1":0.2302554104612147, "value2":9188062021573390442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520337, "value1":0.16075557090608333, "value2":219133269848494495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173477, "value1":0.6892053752715795, "value2":2151318569219964336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229017, "value1":0.9691975492787838, "value2":1366609608627051856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659208, "value1":0.7460794334063994, "value2":817106321355418532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.361925, "value1":0.05687743883366658, "value2":7921175806067397073, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.523858, "value1":0.7866547024502999, "value2":4071070330904780510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920001, "value1":0.5740949320374887, "value2":487306006337280717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775196, "value1":0.1341448972829432, "value2":7272783478294294123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000330, "value1":0.1260519175980482, "value2":9041625229579242321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313318, "value1":0.4146999970169919, "value2":6937035767989914959, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147057, "value1":0.5790734522844233, "value2":1184517563846638764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.331707, "value1":0.5826950424479324, "value2":7357100402735593295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.476945, "value1":0.09337572406541937, "value2":3528304154285323988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_4":"i","key_5":"d","key_2":"e", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.660357, "value1":0.9211165869524813, "value2":6490630229378478544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.978786, "value1":0.24445422171222897, "value2":6316462580906113749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031375, "value1":0.9435579557750952, "value2":3775564649397357096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786489, "value1":0.19831604296196023, "value2":4519338814156406008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845707, "value1":0.0733198408835947, "value2":6043258187561628613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607379, "value1":0.49574791837525684, "value2":2511607289885086712, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541151, "value1":0.9239913172285873, "value2":8906754784907447893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409187, "value1":0.347121325872609, "value2":4022279358676190140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179171, "value1":0.9901703571238136, "value2":1971122679978818549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807000, "value1":0.9794430062314226, "value2":4140173177331652200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514307, "value1":0.3368836183017325, "value2":6646214711319824003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_8":"d","key_9":"g","key_6":"a", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.430324, "value1":0.6783643243115747, "value2":8700387930430110357, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245716, "value1":0.5777847683942137, "value2":1772788621154859179, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825752, "value1":0.8229486912139793, "value2":3218217186376093534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.550373, "value1":0.44484600976320454, "value2":1214217372893594474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810847, "value1":0.6410630380778536, "value2":1107368725861373561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_7":"e","key_3":"g","key_4":"b", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258640, "value1":0.3699636915738955, "value2":3778046401086414746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354190, "value1":0.2787621713653213, "value2":7905127146501563870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276300, "value1":0.38144875313061755, "value2":279189270439050245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030267, "value1":0.09524338073167776, "value2":624319768896255519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395031, "value1":0.017165936825319472, "value2":1839096275149454289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_3":"c","key_8":"i","key_1":"k", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920079, "value1":0.3695378285006256, "value2":3642800979833317953, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_3":"f","key_9":"f","key_1":"e", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055296, "value1":0.5098042037064671, "value2":1508579590160432139, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128994, "value1":0.8793028393199556, "value2":5718919044160675205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418223, "value1":0.7389480744126634, "value2":5430202156875304420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458004, "value1":0.7589090833562657, "value2":7035246718540970070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_6":"k","key_0":"f","key_3":"b", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492634, "value1":0.3450062925676948, "value2":289715553515433637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965681, "value1":0.9053649041017894, "value2":2294309054504906499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234931, "value1":0.0732565141042219, "value2":2755939599945126397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939989, "value1":0.8451432422468432, "value2":2656355912924482468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862255, "value1":0.3071521473170743, "value2":4029052487902069438, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027859, "value1":0.5418227496728708, "value2":5130775345772381962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016707, "value1":0.5241900456864456, "value2":1703509093458181611, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.129222, "value1":0.45379187899537626, "value2":8646708304907761402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167400, "value1":0.4118569437426427, "value2":1933032658312744458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.448154, "value1":0.9880224154363099, "value2":1958598830665944865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095756, "value1":0.7737922723291526, "value2":1554270443791925823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.669733, "value1":0.05998044134164805, "value2":4233304263515802857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360452, "value1":0.021777318888489317, "value2":1462350357926539483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.791525, "value1":0.24698569461323847, "value2":6782513953965205791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.136814, "value1":0.13312227926717296, "value2":2973379075607635624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_5":"a","key_2":"i", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.611322, "value1":0.06800531937511199, "value2":2253428998305264546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000702, "value1":0.40505920041369187, "value2":7444308564790803777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.317483, "value1":0.5921456568045917, "value2":1159233331687655633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.207714, "value1":0.03623125609776327, "value2":4050612595565072329, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739699, "value1":0.1967627573726683, "value2":5128621003255593133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938178, "value1":0.8074789236886436, "value2":5329361240090086591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075581, "value1":0.36183126859676484, "value2":8380824340175200195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.840434, "value1":0.36564465907617033, "value2":4138989352760782569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697538, "value1":0.8089117668002023, "value2":2706669265618979267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810618, "value1":0.9822561514547918, "value2":6363847253975708775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453492, "value1":0.7521891954121415, "value2":3415954419502830788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.246679, "value1":0.291250645374521, "value2":6787758008675320095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.717578, "value1":0.21885147116003636, "value2":8249910093492361560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.584635, "value1":0.267173208281922, "value2":6350566579717368150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288596, "value1":0.8034133849819789, "value2":3057165042866020348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043225, "value1":0.8891915956366093, "value2":1172085376591869222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514557, "value1":0.7993701790896678, "value2":3530709071532758425, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_8":"h","key_3":"k","key_4":"d", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727549, "value1":0.7581402483767911, "value2":2882149021889454970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944767, "value1":0.03125902081908963, "value2":5930819025045052350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_9":"c","key_6":"c","key_7":"k", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635131, "value1":0.5563692033064526, "value2":3139117266540547008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901432, "value1":0.08725832596843655, "value2":2516400281682537507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965447, "value1":0.24555624361218226, "value2":8420031154067673423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653005, "value1":0.6015476287649423, "value2":7179718486091286242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.251270, "value1":0.6303898680513278, "value2":1976335650844570524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.902243, "value1":0.42829690884430943, "value2":7763758072113593479, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748351, "value1":0.5394339139974091, "value2":1947184765867041268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.496788, "value1":0.0776914163879659, "value2":4120402058111565140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_6":"j","key_4":"a","key_5":"g", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202592, "value1":0.3950986449717796, "value2":7867550414487529994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.475860, "value1":0.5847506627916105, "value2":1395577688027634835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307279, "value1":0.66953692642835, "value2":1884810173101069714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447867, "value1":0.30049434187635843, "value2":2021494635928386331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371456, "value1":0.9583003775042523, "value2":8408660678843228458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321951, "value1":0.673208957772375, "value2":4484934316922285119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.654317, "value1":0.34301031853223307, "value2":1377167639028225502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258833, "value1":0.5754272342729656, "value2":3672436572873719874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894800, "value1":0.27153721678184795, "value2":6771295009871301863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886915, "value1":0.4211098948169063, "value2":327314688390276361, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464071, "value1":0.8401442859431041, "value2":7237906700870505791, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179548, "value1":0.35527262545135796, "value2":6929428209196710992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348853, "value1":0.7390477554491065, "value2":6017303276519989534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228799, "value1":0.7190691653435217, "value2":3466847624797872404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.676189, "value1":0.8701176990093307, "value2":8139200890464740272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780301, "value1":0.31331610868424237, "value2":7970934533227613450, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.763684, "value1":0.6117316228567153, "value2":1342308618516991103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.510130, "value1":0.26253831176449904, "value2":223167711418991037, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.054906, "value1":0.30113096258283606, "value2":9029709188884415357, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.101830, "value1":0.6535137630775729, "value2":8047896116508601877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.816267, "value1":0.6779088610569585, "value2":402736258268648092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794465, "value1":0.6385825716360082, "value2":647146760807700438, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.342961, "value1":0.08518129482765982, "value2":3079348210933738649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503723, "value1":0.5000975675164184, "value2":3087069466664576618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.552740, "value1":0.46623198603217897, "value2":5991311418948024851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.634526, "value1":0.1623989693910568, "value2":880550310997257962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409701, "value1":0.18776946541701378, "value2":4672037814003639717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928628, "value1":0.5544683022868725, "value2":8786609556345104074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422305, "value1":0.3339761609696324, "value2":2037434470333278554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497260, "value1":0.21504973222613463, "value2":8744267736203582723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348792, "value1":0.7267165846871734, "value2":2029452385366977955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767788, "value1":0.7268443539701483, "value2":5120204611450631315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313022, "value1":0.3937734460488266, "value2":8789465407671191549, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373011, "value1":0.5815734596824621, "value2":7433149153695507174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.403959, "value1":0.19547385196106162, "value2":4536904833017601954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702004, "value1":0.787480554646893, "value2":6826871522585433391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238346, "value1":0.3158260333418349, "value2":4890876749664266809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_7":"h","key_9":"d","key_2":"g", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.184806, "value1":0.1658734659451667, "value2":2103368389172466197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646125, "value1":0.7611256397426871, "value2":3779878057324979416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.963683, "value1":0.6784369755611176, "value2":7198319968843792884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264784, "value1":0.6944677799630911, "value2":1361718740081763677, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.606235, "value1":0.9767553574110414, "value2":8172694375569168120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.555524, "value1":0.0757100455425729, "value2":6369412142150141094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.683577, "value1":0.23488908846392256, "value2":7496440285542052380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741203, "value1":0.6152800537206294, "value2":4336041330603516829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.076957, "value1":0.46349128481126883, "value2":1995149205719248029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360568, "value1":0.18125853395781905, "value2":5023669000017633099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.127735, "value1":0.2064987900720652, "value2":1090691939971047635, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182440, "value1":0.18035240548503653, "value2":2377567143003176008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173358, "value1":0.9386364543056086, "value2":6498536821606623122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_9":"k","key_5":"j","key_6":"d", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.749875, "value1":0.9851379736422945, "value2":9068316297781013801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262923, "value1":0.6215747239294878, "value2":3444654966165617334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.471762, "value1":0.9687744980352465, "value2":4008980207107557770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.449800, "value1":0.43369424684152247, "value2":1796590317391088733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799032, "value1":0.7408688395558841, "value2":1701592597588021265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727561, "value1":0.14048934174685151, "value2":1964669731520342830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393461, "value1":0.22927403588573142, "value2":6637169452408438633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577492, "value1":0.9899876418682123, "value2":417258805492276216, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.149873, "value1":0.2451895383714133, "value2":6783012468215477739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794818, "value1":0.2978983436952165, "value2":6898137216027059682, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826276, "value1":0.03031011408512658, "value2":6243267082953045335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853432, "value1":0.8319004837974954, "value2":6916103698866916518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336756, "value1":0.6573141098461979, "value2":7552682537825884423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280458, "value1":0.9944062499165491, "value2":4515137518875248587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723258, "value1":0.8389015105022624, "value2":4385100343250672688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.518510, "value1":0.262846594625773, "value2":8956848533096958437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398441, "value1":0.29945983222092276, "value2":7583507905789284927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169305, "value1":0.3509582334049355, "value2":4486670807718872528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229006, "value1":0.9454115215233667, "value2":4148482506114005261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.108860, "value1":0.7792992725899739, "value2":6662551108461731004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270948, "value1":0.22595556811111142, "value2":4858846807835096111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_3":"e","key_6":"b","key_1":"e", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739267, "value1":0.10553288145788361, "value2":3903915559852076766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.771104, "value1":0.2713630569195371, "value2":9127087478037486061, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234974, "value1":0.4512849051156243, "value2":388475201207159224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021067, "value1":0.8525575906494762, "value2":6112677337472259317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360331, "value1":0.5466498354493777, "value2":8416483456048971785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.083431, "value1":0.6268377803161493, "value2":6874204752716819531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543079, "value1":0.356809340191254, "value2":1479162719217476511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543913, "value1":0.1823988812868871, "value2":1772497490870724163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697354, "value1":0.6008728382547858, "value2":4808659248566150951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173629, "value1":0.11694918608631227, "value2":8374732016838634205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.478338, "value1":0.6523910738528493, "value2":6127719310482278347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_8":"d","key_5":"k","key_7":"j", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195226, "value1":0.11728848022547718, "value2":2692984308355874383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472792, "value1":0.3849335854410186, "value2":5551198795362400796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.088581, "value1":0.6479908764028293, "value2":6646669667589759935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.051281, "value1":0.8063603427465644, "value2":4677280644621004022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.421107, "value1":0.8098711885881693, "value2":781451797112683390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702263, "value1":0.7459154166325007, "value2":8763722224980194522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.105837, "value1":0.6082412929946549, "value2":9076065007808782827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229179, "value1":0.5208679934438682, "value2":1614625428456495501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452972, "value1":0.6602375068241643, "value2":3665988082745333789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570733, "value1":0.8539881179697112, "value2":8291534021329215591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.854105, "value1":0.19867014748060782, "value2":4707537791933933479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134473, "value1":0.013248508886480496, "value2":8975225813346771480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.401924, "value1":0.8131686031327187, "value2":1570626633965770939, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462042, "value1":0.33875887576127867, "value2":1531594524235268547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780840, "value1":0.6167438470784343, "value2":3111876176210806342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157314, "value1":0.024850360062745134, "value2":2155248434369533975, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.892341, "value1":0.29151476400849524, "value2":3929095109066478763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811920, "value1":0.3960145984061985, "value2":8079803277636061346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010049, "value1":0.1522516905011452, "value2":3887799589835913190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.511030, "value1":0.1096312618999367, "value2":4683382419730786111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799775, "value1":0.8107638537946521, "value2":1548400788132615580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.034128, "value1":0.8145920212010633, "value2":4095930474475755353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506330, "value1":0.8145793344024634, "value2":7248985564172971873, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.632617, "value1":0.530965707458104, "value2":6582545459218253298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.427368, "value1":0.5488520970953542, "value2":7764537337432663671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128543, "value1":0.339288607631015, "value2":8135936826126620967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.783911, "value1":0.4072868133390787, "value2":398383887746388268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030437, "value1":0.40039567718797253, "value2":5009000449434898292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.301665, "value1":0.2851345919898927, "value2":196381052074952545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740740, "value1":0.481454729816193, "value2":2929970498837276050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867428, "value1":0.8246737181844912, "value2":4477437696490734479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.708771, "value1":0.44011710908297325, "value2":7995836793967148678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663309, "value1":0.07002506566825563, "value2":3582676294388926669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.628344, "value1":0.9801112824460718, "value2":196546046309844748, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330627, "value1":0.6390220685465513, "value2":5252094592319160612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.014786, "value1":0.4666494400039896, "value2":6746944521794840076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.736397, "value1":0.1650175517955452, "value2":4725786271252183916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971200, "value1":0.08835643831453004, "value2":1662264677868065737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.804675, "value1":0.3583297226255054, "value2":6764608743691309744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577888, "value1":0.7784009799351271, "value2":7389846578443042716, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942762, "value1":0.5281041922180323, "value2":225078423814016880, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807869, "value1":0.12159890893033082, "value2":2007944198207870435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393243, "value1":0.9174108973827193, "value2":5956246567273801920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842305, "value1":0.6881483813172635, "value2":51917468498479892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.625958, "value1":0.7332068726661788, "value2":1595636028517738482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.738053, "value1":0.19091798634208768, "value2":2237197021871432008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268129, "value1":0.8406719661444901, "value2":2842521222444292729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865313, "value1":0.8513846203817129, "value2":3859037819752068385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762176, "value1":0.0431523780543075, "value2":6794482877795635511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.658034, "value1":0.12118411547914006, "value2":7727119685187980691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497498, "value1":0.8736017830767735, "value2":5281383221003217421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016157, "value1":0.45635245796699264, "value2":6588970499031475943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434398, "value1":0.5982494402461963, "value2":3451343743898655419, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995659, "value1":0.41609794124038807, "value2":2577003956591436853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.402057, "value1":0.3168156284522602, "value2":7664962841417781702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874274, "value1":0.8588869341047012, "value2":6471105672211793135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.193561, "value1":0.7716291581375219, "value2":6758054142720243683, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.298108, "value1":0.8739782093698222, "value2":7600130946423390938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008667, "value1":0.15272944664848967, "value2":28862629936648465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924564, "value1":0.2262144335029645, "value2":2100662153491567952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.250052, "value1":0.2945787294595236, "value2":6018457540699249059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395074, "value1":0.2290685251999151, "value2":4865312816831141048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414183, "value1":0.2892057090700859, "value2":3284381379197405662, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800711, "value1":0.20023612385737624, "value2":5612718418445245977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096882, "value1":0.6190652975917784, "value2":4456707593278158335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811596, "value1":0.1974893316007636, "value2":8350883557246751407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.328602, "value1":0.3577823211406552, "value2":3856849717607100839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165294, "value1":0.1558044782512875, "value2":2819862014599071323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.934329, "value1":0.6967191883920082, "value2":8317801698439570519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826310, "value1":0.5889684966352713, "value2":8904148697557725595, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.544062, "value1":0.3036793591749508, "value2":1172054639314476559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104925, "value1":0.0006382055160369547, "value2":4527189243213341516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577496, "value1":0.8262646875704296, "value2":6857852785226312453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.304163, "value1":0.23448830223910028, "value2":118297432460308257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.201626, "value1":0.8649386204518231, "value2":2335277383532252247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.072447, "value1":0.305504173455374, "value2":8879950191665801641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635410, "value1":0.8646407642266409, "value2":4794888988052345760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799765, "value1":0.020261599778331852, "value2":6569370350090311777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_7":"i","key_2":"i","key_3":"d", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.643397, "value1":0.25580026537842243, "value2":8727368389050428851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886737, "value1":0.709498290535563, "value2":159638875141661569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027896, "value1":0.4252750468577108, "value2":5985063599490557923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659322, "value1":0.8283450566415441, "value2":3829413923759281391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_2":"h","key_1":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.323182, "value1":0.05823156526332208, "value2":158181159624437675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721491, "value1":0.058149447228806335, "value2":4425192040964093188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.715378, "value1":0.1468510162162874, "value2":957193015456701660, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759517, "value1":0.11145345073332971, "value2":9085390279667186751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179380, "value1":0.22290378424729548, "value2":4452524214324184228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018314, "value1":0.08677941054726214, "value2":6248669636718030352, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579864, "value1":0.1843770522783152, "value2":2395237031433429826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907863, "value1":0.19391713305151562, "value2":1831578948753070965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567206, "value1":0.1200213703180807, "value2":281170079691225837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.908312, "value1":0.16873762365247053, "value2":3349277004368724008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.203346, "value1":0.3461718122656804, "value2":2053673338515188112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.252421, "value1":0.8594831603279349, "value2":8129734681001143206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862108, "value1":0.8625875861161819, "value2":266750263503917399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394606, "value1":0.1284747365418792, "value2":511162186129513778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104334, "value1":0.348212621993861, "value2":8809834773329001166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.390817, "value1":0.056538788821091236, "value2":959306944656093342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371946, "value1":0.0354556141213121, "value2":5315204481196619882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027057, "value1":0.04051346889125732, "value2":19281603192892033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315646, "value1":0.29470865205235175, "value2":3350853301006328333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516736, "value1":0.32421705083072555, "value2":6409364233498955235, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409843, "value1":0.8431693191414327, "value2":2309002455167863238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.290066, "value1":0.769846734088458, "value2":1354615135861452763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025458, "value1":0.04383062091902626, "value2":6270990659808009038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422265, "value1":0.4215091845870574, "value2":903506811871753839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954147, "value1":0.1703656776121607, "value2":8625109899283360680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468022, "value1":0.8656906467989927, "value2":8844353092623218329, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_6":"d","key_2":"g","key_4":"j", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.572694, "value1":0.03657142931464268, "value2":7269928716214871698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.035238, "value1":0.2174690883774512, "value2":2760303323859671187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.500319, "value1":0.8848932111671852, "value2":124147493136831346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.324572, "value1":0.32024379123233365, "value2":8082835280902735033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585321, "value1":0.18246288567364422, "value2":7470239488455160357, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726932, "value1":0.8209711897357084, "value2":4430579260139045397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.851627, "value1":0.9988163498901884, "value2":7782482596272329873, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126247, "value1":0.7338638289875997, "value2":2035846164975007267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923987, "value1":0.31023714156918913, "value2":7969614988653223555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.797525, "value1":0.9308157523651853, "value2":2008465742081738773, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595532, "value1":0.6306393412084162, "value2":1794355738915834222, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078930, "value1":0.46045211861703167, "value2":2231560481693103072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.835058, "value1":0.49083826368226746, "value2":947523171682771279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.757956, "value1":0.5783027668846169, "value2":7939213319454038876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.431201, "value1":0.249271140930431, "value2":7943702047022612152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.674800, "value1":0.9050148884978706, "value2":4701267498128748767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294430, "value1":0.37838796407920183, "value2":6812226255195471591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837164, "value1":0.41192774227492235, "value2":1535119829477772546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_2":"d","key_1":"j", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464005, "value1":0.1600487895075015, "value2":1434123145157650204, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546806, "value1":0.6061159682210883, "value2":6208205100336254753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096572, "value1":0.8332138223552116, "value2":7164548384493446541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.325386, "value1":0.6865864080335002, "value2":1198200374614070582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179713, "value1":0.590117470581001, "value2":8089464430800358371, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273193, "value1":0.6452720470474251, "value2":1981356061798226474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330589, "value1":0.2830579363626061, "value2":6390191335007008364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468768, "value1":0.2265890199157533, "value2":4363523400851143524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901505, "value1":0.3297489212069802, "value2":7168425838004946986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850173, "value1":0.7443546478795617, "value2":1215583272150299781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.968362, "value1":0.6932020029271382, "value2":4461729221210949302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210225, "value1":0.36249305348248906, "value2":7617198529682822125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.001816, "value1":0.46528315796295283, "value2":553457115403327004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.564478, "value1":0.42774941008998635, "value2":6190959325547605268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126782, "value1":0.6691920311814687, "value2":8190037233750158351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.125382, "value1":0.3161902537409363, "value2":2607443448109856155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.782262, "value1":0.25500427404714787, "value2":694236189793698587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635388, "value1":0.4760006261995292, "value2":7236218797911995877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.481920, "value1":0.5382100231194429, "value2":6783652184521772255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818398, "value1":0.6906166285200835, "value2":1593698347526099227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283475, "value1":0.9162718353521024, "value2":131978695455153917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434017, "value1":0.8865622180830747, "value2":5499929802234516147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838878, "value1":0.8831606268003614, "value2":4413611476125545911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451279, "value1":0.9923758573970507, "value2":7842864722613769419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362386, "value1":0.09704282298545679, "value2":6868263842667298833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274837, "value1":0.5887807861238907, "value2":1280869300914331508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.915666, "value1":0.6149407272649627, "value2":2848990718411372775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024227, "value1":0.1623145961115003, "value2":2455593750882693548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.695944, "value1":0.6887899812661492, "value2":533109273214059474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.528843, "value1":0.5075436862092302, "value2":515274582595904032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842262, "value1":0.5447726756017138, "value2":411822948007690047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444248, "value1":0.5454758220348956, "value2":1028316813231771868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.604063, "value1":0.46187992572513836, "value2":8158929971855121147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275994, "value1":0.8154001283675578, "value2":5366649133302343998, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348933, "value1":0.903584155771458, "value2":3421400845314883405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.041062, "value1":0.1190413277495334, "value2":3602483838893891944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928372, "value1":0.19081223162638847, "value2":4430971211717726323, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.677971, "value1":0.7830672620168732, "value2":4412058601784711710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152188, "value1":0.6024595453233776, "value2":3457200300695401373, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838789, "value1":0.8421167420494099, "value2":690533435719409387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.578903, "value1":0.6328751611123744, "value2":3916711663864490625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.896839, "value1":0.9070308716530836, "value2":5545334900650073897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.712901, "value1":0.5273783009751816, "value2":7859089933688216880, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884939, "value1":0.07741816375541602, "value2":2787557417309799857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944440, "value1":0.9501843438293957, "value2":4306730282944265532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814029, "value1":0.12358405285849303, "value2":6299011328560473443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965702, "value1":0.9397419047313924, "value2":2182258508456140601, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045751, "value1":0.6945070980136847, "value2":6892165864647238687, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821781, "value1":0.27908605223075134, "value2":5742625392746876485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514196, "value1":0.8720271384105083, "value2":6651696455700077053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.705476, "value1":0.06250737971386458, "value2":1193722866498852150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.603276, "value1":0.8453600274943766, "value2":31981742286968982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.682527, "value1":0.2427465956249123, "value2":7462455737298824829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.605338, "value1":0.7488120008677794, "value2":4273030954599275940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893961, "value1":0.0012093187527627421, "value2":3798553194034218364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.593870, "value1":0.4389808952680249, "value2":4760750743940894178, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091153, "value1":0.8390848719802647, "value2":4663558817694651969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.014239, "value1":0.6803598889667898, "value2":4768916054002910461, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478582, "value1":0.8497920155872134, "value2":6014547020899484560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895949, "value1":0.2900471481778915, "value2":9139898160138156870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.613215, "value1":0.18238778666189503, "value2":1572248498405458162, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.817510, "value1":0.7988916675957994, "value2":5551320908811649706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881135, "value1":0.18477673938398376, "value2":5754586889701314224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128674, "value1":0.2730542267310359, "value2":3723967523716120986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_8":"k","key_0":"a","key_6":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392852, "value1":0.5342690346158169, "value2":2053063244515016190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931934, "value1":0.712284281615868, "value2":5743978094943818789, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537147, "value1":0.023259389444222435, "value2":5526408740424742448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.216805, "value1":0.8594083029021016, "value2":5158284428621389401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837342, "value1":0.5180882037372726, "value2":7808493880824756093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144558, "value1":0.054388678344141024, "value2":7174534666209181912, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599909, "value1":0.09725816948703289, "value2":4397869661367210515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.880028, "value1":0.4672046328070093, "value2":7512677570982024303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271509, "value1":0.8816366985657997, "value2":7431242132698500082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.752197, "value1":0.6078880045612306, "value2":3606538184952837708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_5":"b","key_0":"a","key_4":"j", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392689, "value1":0.5702352441472875, "value2":327376335260250215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.201991, "value1":0.3880870455267555, "value2":2539019737033922024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.894353, "value1":0.3785370438906491, "value2":8100099805481858212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_4":"e","key_7":"c","key_1":"f", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.193824, "value1":0.1316387900370704, "value2":2566382490326741291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.735887, "value1":0.9017592259731758, "value2":3204066872434851855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891488, "value1":0.8557864633508439, "value2":7321454426994500720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868424, "value1":0.21022389029059355, "value2":4239400393420255730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.624822, "value1":0.8941824508080144, "value2":4130019486643335780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.292285, "value1":0.2379306348830461, "value2":19245718469181538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690737, "value1":0.07804300554340773, "value2":3416010458091576918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404288, "value1":0.7382908404371364, "value2":4673144760418942783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.333079, "value1":0.7119882858599182, "value2":2804563282517597579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.472391, "value1":0.08143361630646322, "value2":8016722515581298003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_3":"i","key_7":"a","key_0":"c", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.373563, "value1":0.6580401651760573, "value2":6974099035575699967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078367, "value1":0.04245208361261986, "value2":3309677777480755698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.312294, "value1":0.08781290868459149, "value2":410348162808240186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768785, "value1":0.8804562203614901, "value2":3554533373089833157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636131, "value1":0.6764401768679541, "value2":5380729408367504508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041277, "value1":0.013798250982638604, "value2":7112024095753417757, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867267, "value1":0.8747601920631648, "value2":1340507463961878602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776242, "value1":0.08520211967435457, "value2":4488809966272125582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634645, "value1":0.8571991096706362, "value2":4813432822336330164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415856, "value1":0.16181998754042196, "value2":2843492259629836672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466761, "value1":0.6961378523412571, "value2":7745491307176492894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.594554, "value1":0.04099735827317489, "value2":8913619907287518073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.170785, "value1":0.5648256194335608, "value2":476929125931913132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.184821, "value1":0.7825477180517281, "value2":4452433023228691120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906243, "value1":0.48861587452050464, "value2":8430953017453652958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.667172, "value1":0.06605403171722925, "value2":3065000063305702983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.192260, "value1":0.12827053824062132, "value2":4944252317941317234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400652, "value1":0.728695995485893, "value2":1367272622265070542, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535047, "value1":0.4567196818143048, "value2":5331291578601537074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466316, "value1":0.5659706744457615, "value2":5221382170536673746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487405, "value1":0.8907853085096481, "value2":2583510249445277455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.375762, "value1":0.8536987085513627, "value2":1666025213570716259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.165432, "value1":0.43405723604720065, "value2":714036789144069944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175401, "value1":0.9859093721286644, "value2":4617822411248224956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.059065, "value1":0.378208510942557, "value2":8575694693763974484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_7":"j","key_0":"d","key_6":"c", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.073374, "value1":0.7874709956095631, "value2":3297271830714214500, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656159, "value1":0.11136055751977454, "value2":6742238162891439594, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102008, "value1":0.8937221842663475, "value2":6378107019082012182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.177632, "value1":0.36894176860576355, "value2":7704992361178834991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335734, "value1":0.6980181667187008, "value2":260944621673155935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.855348, "value1":0.9199066328884838, "value2":4741986190169422412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965490, "value1":0.12178040424602839, "value2":8326386976538257273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090240, "value1":0.23602165280654655, "value2":9043605027949953616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.028464, "value1":0.05960386743156977, "value2":3549887217756566974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.956988, "value1":0.5498920550278433, "value2":6597949147890816377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.171316, "value1":0.0904948299110727, "value2":3471748119473561940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_6":"b","key_9":"a","key_1":"g", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.523091, "value1":0.4454856417468202, "value2":3973537432672533989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453397, "value1":0.7881010562880141, "value2":448816938066732131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384576, "value1":0.9045802469513693, "value2":4349360112470402658, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.929660, "value1":0.1904824371420524, "value2":4775599930437109515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642052, "value1":0.9792443542954197, "value2":6092407690974464405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194609, "value1":0.8726282980309354, "value2":7591391705732812177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095352, "value1":0.384168214137503, "value2":7516656275093877057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868497, "value1":0.924966560225317, "value2":4429888702948279946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.146490, "value1":0.7150691128822396, "value2":205950019316022568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009297, "value1":0.4940665543165739, "value2":924283179287111091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904057, "value1":0.23580267125603968, "value2":531910251609622124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728011, "value1":0.8321936268339676, "value2":2170557262229675184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944000, "value1":0.7690990294985828, "value2":8227626091745901, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.871354, "value1":0.21098039407901223, "value2":4374012870679696344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.538323, "value1":0.681484837485245, "value2":6775701390216802286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230276, "value1":0.36173491575946426, "value2":8235898214428447094, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552468, "value1":0.8892513220522287, "value2":612692529250917070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782746, "value1":0.3267438552083453, "value2":5821122346950557164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908039, "value1":0.8750317891455761, "value2":6904198848902253780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124465, "value1":0.27489616104124, "value2":1813939809215341473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.739988, "value1":0.11884426917626255, "value2":6375710418065432574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246847, "value1":0.065994349338974, "value2":4501641519889306838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.675544, "value1":0.4346958284003153, "value2":8656554552944846961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907951, "value1":0.3861999534665598, "value2":7376195997991690698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.601240, "value1":0.09029192218287085, "value2":4084656981993173700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327311, "value1":0.46909482167142375, "value2":6484192245523357545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453608, "value1":0.634255139121383, "value2":8880783116370612057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562417, "value1":0.4260014133745824, "value2":1052849307102914712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708174, "value1":0.05688677429464904, "value2":5475041244330586198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.774531, "value1":0.8132424157243254, "value2":1362266175582132598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.106500, "value1":0.7014281430749205, "value2":7134854011640912176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.486559, "value1":0.28063120575400446, "value2":3192436055979504417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.864997, "value1":0.3477928996994262, "value2":735248433592549787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.657367, "value1":0.65265411185833, "value2":2443852584375783883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.172271, "value1":0.4417461489558258, "value2":7942690633839505457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419664, "value1":0.16011001601248667, "value2":6549588207213967629, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416761, "value1":0.6429956000093604, "value2":8493895113638046625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564282, "value1":0.4457588529183665, "value2":6725016002434781716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430750, "value1":0.3037956469116298, "value2":7998283396935875899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827125, "value1":0.5250484976450595, "value2":6524724358656282389, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395896, "value1":0.12287086144958013, "value2":1930681199692368788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851159, "value1":0.23103414004696357, "value2":4914748620478090577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.736709, "value1":0.6408817571256187, "value2":9105122753792897591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984808, "value1":0.2905624937515174, "value2":7851451601162680954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487306, "value1":0.605006354442018, "value2":4276465105843409598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790850, "value1":0.6268022197129945, "value2":9220780769142386741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185700, "value1":0.566519118535341, "value2":3705647574979636584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451877, "value1":0.0460137249105004, "value2":7915985022414197030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150718, "value1":0.35693559742029707, "value2":5014219743291793375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.294260, "value1":0.9247064360015096, "value2":5127524279022159080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995623, "value1":0.21178547000119013, "value2":2431494318968387519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.256959, "value1":0.8916548214121515, "value2":1002635579417294841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_5":"c","key_1":"c","key_2":"g", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110758, "value1":0.9737478776363947, "value2":7247818685878473810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825871, "value1":0.2917878380844755, "value2":6516114778766465158, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910566, "value1":0.3957632746819297, "value2":7166657573372241684, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.178800, "value1":0.5649796089766207, "value2":7775035464817162126, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634489, "value1":0.6090054970934514, "value2":6647498876825886588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610499, "value1":0.5292322793488414, "value2":8830154316280741019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574572, "value1":0.5304362659437236, "value2":3949756347058852883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_3":"a","key_9":"i","key_0":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380651, "value1":0.956019198700713, "value2":9135146156795796096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778218, "value1":0.2121518375707717, "value2":8667556184145489676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078961, "value1":0.06325131155030916, "value2":8555129638818277403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.047017, "value1":0.1027203958612216, "value2":5246272405737501952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439829, "value1":0.7417681560989317, "value2":7749536771402273968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203072, "value1":0.5621215132192839, "value2":6118954075279232029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758671, "value1":0.48389569392567205, "value2":2705225096795559234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_2":"g","key_1":"c", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665522, "value1":0.8426429373047708, "value2":8279168363918064894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_6":"b","key_8":"h","key_1":"a", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.345535, "value1":0.20088437097171843, "value2":4950747031440638845, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124655, "value1":0.41827026587025645, "value2":3430038763553236823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977760, "value1":0.10421019819450697, "value2":733644826366835807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118861, "value1":0.7989242448499801, "value2":7548777360232504083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318842, "value1":0.009939081834750191, "value2":8375328698470876119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590619, "value1":0.03636791737366475, "value2":7582770650474370737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045468, "value1":0.7763100607726895, "value2":2803435127241577664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.553650, "value1":0.9987121300161536, "value2":3948733587296676032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790022, "value1":0.793158890727088, "value2":1174928966366390949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.232763, "value1":0.07728571888784644, "value2":3240453509192284959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074815, "value1":0.5297469977127046, "value2":2019972453697918898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.126329, "value1":0.8018605847489219, "value2":4857597305010910280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.465083, "value1":0.1752053660866726, "value2":7163605506157251603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.714319, "value1":0.8220755414438086, "value2":876552025810846680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392782, "value1":0.6635667135366609, "value2":6624504880778850428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404283, "value1":0.5416570514867133, "value2":8266175224671672272, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.734325, "value1":0.7575887007274229, "value2":8864761125492843547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.164845, "value1":0.7328611642697666, "value2":1939960544899646141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.087713, "value1":0.6472001702780127, "value2":7064434641814632478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636375, "value1":0.263542227747472, "value2":5091288498651627633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439658, "value1":0.7518898990734011, "value2":6126681364245747402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580431, "value1":0.28444432068121084, "value2":8147884504685238694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408258, "value1":0.5563100440073113, "value2":4573231178010278468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923759, "value1":0.7035415767681118, "value2":4284377617122095281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069625, "value1":0.7755797987036898, "value2":7005063200108318654, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475808, "value1":0.7602915679203158, "value2":1889690102234767528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045338, "value1":0.4860219702134258, "value2":5422255506416156729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.985908, "value1":0.7866996153890412, "value2":4784461751340030748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937856, "value1":0.5239212561479751, "value2":3109818040142871590, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.376519, "value1":0.2587179526287473, "value2":1263076833301844915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943956, "value1":0.6806333311736701, "value2":6002705815114995201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.653771, "value1":0.7947376720839073, "value2":2247611778314354742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290979, "value1":0.13605627013098143, "value2":859676226210675359, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166492, "value1":0.4455170530582711, "value2":3676647784290217622, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.625732, "value1":0.26531203043868007, "value2":8360465694110411228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033924, "value1":0.027884795524999093, "value2":4290921120785429469, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.199865, "value1":0.6930021146351104, "value2":8033400918877373777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.387257, "value1":0.5315645824113636, "value2":8487597923574953049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.385317, "value1":0.687478534046926, "value2":5788685556231917492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.632370, "value1":0.4013559424653022, "value2":8640639439561304540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226297, "value1":0.2440055184343977, "value2":5133913946019355967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.763037, "value1":0.20026459422324144, "value2":8670140335638909516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695304, "value1":0.16276770698378185, "value2":1188100569829740721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595366, "value1":0.4506718132549764, "value2":5734081257378041711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.833543, "value1":0.5734402814385565, "value2":922425738240504967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709630, "value1":0.8692041834415107, "value2":1960359817596413212, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.732788, "value1":0.23444382672590003, "value2":2854071547092864254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.412896, "value1":0.19053537293020936, "value2":1082762509249072310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988462, "value1":0.4293280866481845, "value2":4780102507614458134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_8":"i","key_0":"h", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823600, "value1":0.01981381195792859, "value2":8743022558864727512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.092886, "value1":0.9862388684646872, "value2":3622022818199205736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.018231, "value1":0.4242288910229569, "value2":880886547836969584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062605, "value1":0.9179292075855371, "value2":3743117943661894467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.759025, "value1":0.5166884948154693, "value2":739459076946893696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907932, "value1":0.2882739675567565, "value2":3650155370836487214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068172, "value1":0.4347373436897239, "value2":1372308133830179280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514459, "value1":0.3578934602691126, "value2":1369826174473414234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282115, "value1":0.4836091160007032, "value2":7304317831220994830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652653, "value1":0.7715198420930932, "value2":1854799303037917400, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.873426, "value1":0.6674125872681426, "value2":1713253808702182629, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.253882, "value1":0.35618452262868266, "value2":1451362150616905038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058435, "value1":0.11619135407507769, "value2":729304989898004351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711979, "value1":0.19795958240207612, "value2":5044852283450257784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.620015, "value1":0.7494674771784868, "value2":9072157623838715380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265268, "value1":0.6332467233229876, "value2":7660280405186635297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471797, "value1":0.9586561875915051, "value2":1400450692063171507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715350, "value1":0.28992759308570154, "value2":4420882374996204645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447264, "value1":0.32845931032025727, "value2":3764283351468138749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_7":"g","key_1":"f", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799817, "value1":0.8344281598160882, "value2":3194977322046448615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225092, "value1":0.832673789719215, "value2":8242369633095170423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261842, "value1":0.1783779388932828, "value2":7591102613464952913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885476, "value1":0.33420965888644627, "value2":2231088099862071194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015564, "value1":0.23747285224855552, "value2":2211582269309113296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415709, "value1":0.43459757217905465, "value2":2297522408220839681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671302, "value1":0.5521783843346304, "value2":8797575493863836447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809774, "value1":0.3232411464090472, "value2":689408551095099282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391492, "value1":0.44145120690127054, "value2":741707985323247529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593752, "value1":0.5004004561210201, "value2":7899920698831884885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090704, "value1":0.9099972336756517, "value2":8993930752293016892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.365212, "value1":0.17134030372562564, "value2":6300597784300096913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.458419, "value1":0.03537414872175956, "value2":4931151600790868120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.266288, "value1":0.2710772740926763, "value2":5168762762336913335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.846075, "value1":0.8530137052878358, "value2":5846881912829979057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.441246, "value1":0.895274800378327, "value2":6555884777228809545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181487, "value1":0.09635807954147779, "value2":5139650932097495606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_7":"g","key_3":"a","key_4":"k", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.283526, "value1":0.6765717041541169, "value2":717191993149931255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690428, "value1":0.5868291690096701, "value2":4142969563289744689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461183, "value1":0.7388746066577521, "value2":3907321818804232131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078559, "value1":0.5509982085563974, "value2":8489285868587130769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.006442, "value1":0.17964435576390814, "value2":6385046429151027537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803653, "value1":0.007944203568736737, "value2":8630069900982836811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843384, "value1":0.5245784602574561, "value2":1611187771800002701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.322734, "value1":0.26583004470195515, "value2":5891979565833996742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019366, "value1":0.9471429425977943, "value2":3105898999363337716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583535, "value1":0.3000091753953168, "value2":1671214988273742274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_9":"d","key_5":"g","key_6":"h", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.528263, "value1":0.6167724953698004, "value2":3571326627931246323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.353615, "value1":0.09339691407017821, "value2":8955717188172767409, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.638995, "value1":0.5498958167694153, "value2":6537844553335776020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327054, "value1":0.5183869093313511, "value2":4021636019817176776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391759, "value1":0.6107482510959784, "value2":6639243589639002168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.083515, "value1":0.6947157647744896, "value2":3642097726125624072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.586405, "value1":0.683079171453159, "value2":414358593011525856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847208, "value1":0.004363913601366179, "value2":6293076524070827579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583066, "value1":0.38103391426791644, "value2":7168389465606666058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.361829, "value1":0.0757878691848108, "value2":18451857992724418, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_5":"i","key_0":"k","key_3":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481695, "value1":0.5254185806253874, "value2":3155631786417589238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.072077, "value1":0.8787002152127282, "value2":5274515234654561765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.267961, "value1":0.8648926070157235, "value2":4125897469271708547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.245076, "value1":0.06458749862097861, "value2":3727526843973668028, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602905, "value1":0.9902842013208221, "value2":3395310217472956562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776585, "value1":0.7219469790990183, "value2":77676005392087147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741224, "value1":0.9110757287517638, "value2":3235394638378837669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.616438, "value1":0.32846413845355765, "value2":5558119133284488407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.617955, "value1":0.12562192604901234, "value2":603576184957547937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.651221, "value1":0.8976398132733482, "value2":7308768939039920124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610616, "value1":0.3225584230939945, "value2":8416609441712941807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.688197, "value1":0.39642287994645425, "value2":7290339184191009077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599842, "value1":0.6105446645640619, "value2":1208112926522863577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867582, "value1":0.1840371797963492, "value2":2177170750841832105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098316, "value1":0.966593809823329, "value2":2548324970162406510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694537, "value1":0.3376604030915601, "value2":607328136959332742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748191, "value1":0.9957584594865343, "value2":67877367036436991, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.064163, "value1":0.7248743828192261, "value2":2749527480847272451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502667, "value1":0.16495830272715267, "value2":9030464128428456991, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990661, "value1":0.3949843558188797, "value2":7201429976848106189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987854, "value1":0.29870559261727064, "value2":348149959926675176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481753, "value1":0.06548607806382464, "value2":1031140396971019018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.630820, "value1":0.8992403438153617, "value2":5146298573769468519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_7":"g","key_9":"h","key_3":"k", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804082, "value1":0.2519724232639516, "value2":1298310485258981735, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_4":"h","key_5":"j","key_1":"c", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368623, "value1":0.647450377107345, "value2":1284622928379296522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.541781, "value1":0.3261143352667153, "value2":130140210100844559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543424, "value1":0.06226681703581004, "value2":726618431953897465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988533, "value1":0.49745860881748044, "value2":3406264037344093572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080362, "value1":0.7895574305241833, "value2":152509341700687015, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150372, "value1":0.9343525151823494, "value2":6076357966504332576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489577, "value1":0.16062395564859527, "value2":3405065082227979672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225570, "value1":0.28122658630857733, "value2":8458136042003760574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862952, "value1":0.7649680950314343, "value2":1756495878088169871, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009186, "value1":0.7999050849241861, "value2":3718209620537893555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327636, "value1":0.23708513628211542, "value2":1522368463913294016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.551176, "value1":0.009439114458257377, "value2":2361018304142309387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.633123, "value1":0.2821310200838197, "value2":6052402151738260293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128752, "value1":0.3359757178563009, "value2":7472471602725669166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.456386, "value1":0.46133506790277695, "value2":2296093836300686089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_3":"e","key_4":"d","key_1":"g", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420349, "value1":0.36664549905260974, "value2":3546467223240204480, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.938528, "value1":0.007977829048726685, "value2":4244809297577031047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226579, "value1":0.6725646472034443, "value2":913735509091846801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.850850, "value1":0.23068766044782393, "value2":2300681752566773332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285218, "value1":0.6989337075616671, "value2":2607116938388175821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.221331, "value1":0.11523512208048978, "value2":8241016534243475750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886294, "value1":0.8815954846610522, "value2":1129122380251841667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010095, "value1":0.4468942822052692, "value2":7305659252479204819, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007461, "value1":0.3287186449948172, "value2":8533612311279585337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841878, "value1":0.06768912562638665, "value2":3915275733365598237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239318, "value1":0.2752287175449378, "value2":6104494420782937095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364905, "value1":0.7269412172343553, "value2":7167814392288649336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030862, "value1":0.35386987370196926, "value2":568029767114417610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945737, "value1":0.8998415941144967, "value2":2346717810373152754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.093117, "value1":0.8244480275524787, "value2":3248267576826263944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.422568, "value1":0.5719502852011219, "value2":8260911644483203383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823166, "value1":0.2849312986012331, "value2":3755798306625123375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268641, "value1":0.33070121758142046, "value2":3503668812330101597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564550, "value1":0.4481321012388454, "value2":2552310395319729245, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417743, "value1":0.047809954457076996, "value2":1081553625885385604, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595860, "value1":0.2921995022802213, "value2":5059986207137621431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.505814, "value1":0.9601390700122682, "value2":1325398133306866350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357820, "value1":0.3304365801280413, "value2":5155828582049823863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963899, "value1":0.8305280855606907, "value2":2952790462431625685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395768, "value1":0.05940274489209035, "value2":2206736622008483109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_2":"f","key_5":"d","key_0":"b", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.994837, "value1":0.28131858851364544, "value2":6377323293251471217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825885, "value1":0.9524553792547288, "value2":5686625986396675519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962844, "value1":0.7824414856459302, "value2":3176085690721005555, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525393, "value1":0.2894500361431452, "value2":9170274733300331931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066894, "value1":0.9750264781896856, "value2":6481943120958967565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118777, "value1":0.49186253332263474, "value2":196535145535309391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.011455, "value1":0.25527316956832247, "value2":23216573243095801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.516739, "value1":0.7031419417768016, "value2":2051319192678332676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621518, "value1":0.7948563798954263, "value2":8394265809911073481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.008794, "value1":0.671348308630362, "value2":8908504416251388849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804142, "value1":0.34185386259040573, "value2":7569056971355045232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971215, "value1":0.795050202357287, "value2":7291081553392495608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285984, "value1":0.6233623903797153, "value2":6444347877884163801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.436168, "value1":0.122788509000057, "value2":2542694660610691923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545038, "value1":0.897272884042895, "value2":3518180018317716924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.377957, "value1":0.26500726028503324, "value2":918021549502007270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186659, "value1":0.8861606285044316, "value2":7428746411605199070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.706851, "value1":0.8539941323885553, "value2":5095874822126979293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552666, "value1":0.8209937326987259, "value2":7918997962757156260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708678, "value1":0.20172873469302396, "value2":473140516571282958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.588929, "value1":0.8992364975321605, "value2":1107792164310220135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.396543, "value1":0.9159358105409485, "value2":3240110351377657654, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949484, "value1":0.3395753649788759, "value2":1556081794159850294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627633, "value1":0.6064813434025144, "value2":504106831033400051, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760461, "value1":0.027052105627838302, "value2":2547821024594975963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690642, "value1":0.29842739674487334, "value2":4347369482397818828, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271622, "value1":0.9232891397814524, "value2":4902825584314369050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678674, "value1":0.8612335750248717, "value2":2676726884600182684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.789269, "value1":0.7153455595388135, "value2":8629094676547921616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_4":"k","key_5":"e","key_3":"h", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.274447, "value1":0.04852479796186674, "value2":520491486142586388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517924, "value1":0.5065900646461697, "value2":1305736770624753229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440510, "value1":0.8079290707038177, "value2":745163355638334416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.992658, "value1":0.33022695610085456, "value2":3745647814397920317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.142855, "value1":0.7836474840216356, "value2":1701655973854226818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.718918, "value1":0.9106879340813463, "value2":2642697174847166539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358697, "value1":0.6889718382775613, "value2":2396285724783610411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782243, "value1":0.10018629733131312, "value2":3557685057170687759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290843, "value1":0.5213600910147824, "value2":5522593432675792373, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243393, "value1":0.9922299657128142, "value2":267617765686382025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827775, "value1":0.17424595518809072, "value2":3247150599498567700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954439, "value1":0.6395234710891515, "value2":3700591746218368196, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543518, "value1":0.7123005069599985, "value2":8170425126276779400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_9":"c","key_3":"h","key_7":"j", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862097, "value1":0.7928466963741031, "value2":1388049342872581158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861104, "value1":0.44830540845614686, "value2":3638392887423846691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996353, "value1":0.9606274415719652, "value2":6260542395914742363, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269620, "value1":0.27973905679528643, "value2":1556926452504115363, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118090, "value1":0.5428339606791369, "value2":1853095038687144617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843203, "value1":0.15162943121847733, "value2":978719322151022878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.168392, "value1":0.30548225941021784, "value2":7137186554911304535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.197151, "value1":0.3975385059702385, "value2":5863175014471717006, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565354, "value1":0.9234247531019719, "value2":812121677319261736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030909, "value1":0.8792093662826526, "value2":8559138722753842843, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.484600, "value1":0.23944552081323173, "value2":7751158766892162613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.320620, "value1":0.12230291625693078, "value2":6192516194919614511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.935691, "value1":0.28006447226692, "value2":6784015217669799303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.336215, "value1":0.23094104388429443, "value2":5569809866177597446, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988466, "value1":0.5462607441657077, "value2":8311104273733417101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447804, "value1":0.4867937694484817, "value2":5062168010270449210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.190570, "value1":0.8283372201936341, "value2":5874711067636169206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904281, "value1":0.8553852749887187, "value2":3311030506444553065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.751439, "value1":0.4266991113414808, "value2":4999173498909581042, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.125309, "value1":0.5563950948301023, "value2":1187994145431678821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.973142, "value1":0.11588338763957572, "value2":8318570385943791860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.116231, "value1":0.2526143185693312, "value2":782968109827902547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.187535, "value1":0.27872170170772304, "value2":8766589428595489268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945637, "value1":0.08854405995381304, "value2":3412460749402337910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.270152, "value1":0.3638550762746485, "value2":2218032724303888987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051113, "value1":0.40794935925664466, "value2":7831097119693907093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265496, "value1":0.23954206323753308, "value2":63253225412010244, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416624, "value1":0.17710696129812786, "value2":3803821152719869396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607195, "value1":0.28582291116541503, "value2":7284275038719896556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.326362, "value1":0.17094485896565303, "value2":4663361033102261072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438839, "value1":0.3486395787421312, "value2":8809206276934706557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033597, "value1":0.8575270834812074, "value2":1940304743017854051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.065029, "value1":0.9427613360719956, "value2":271108060306002877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.488385, "value1":0.12779971800108084, "value2":3316011689829646532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961814, "value1":0.5428137306648136, "value2":1514601534044579978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263207, "value1":0.6949685012843101, "value2":895006576805426648, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282635, "value1":0.10777955836055363, "value2":564953042266607985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.001269, "value1":0.042927028237879465, "value2":1466958494603226347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682457, "value1":0.8478178196386861, "value2":107121152123493591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.159543, "value1":0.9604877997752348, "value2":7593923445754669936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015347, "value1":0.7626139882833062, "value2":5647986907617063275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209254, "value1":0.8450839759887067, "value2":1886374721985796728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066244, "value1":0.07791428229072243, "value2":919090579972011287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.699052, "value1":0.06643124534628293, "value2":2711140899074023727, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.631133, "value1":0.143383362490665, "value2":971805873892273782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426673, "value1":0.7738540415601494, "value2":4397130911534361286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.389568, "value1":0.33397631468501043, "value2":4052528315967262589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.622610, "value1":0.24246003195798918, "value2":5479722481130894419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904392, "value1":0.9774495314687007, "value2":8793165354063487461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.793581, "value1":0.4190858782174469, "value2":4926610192238705598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110855, "value1":0.2884411940593678, "value2":5227068360933879219, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_7":"d","key_0":"k","key_2":"f", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135653, "value1":0.402267966757566, "value2":6977955000760069723, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.307152, "value1":0.8927574650950808, "value2":5803177489389723115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.737685, "value1":0.21404400943023025, "value2":6732954103348577362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.121533, "value1":0.18872391644762126, "value2":8989825709899786188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.972223, "value1":0.023522846364320095, "value2":7146680715055172496, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923388, "value1":0.8860383778342971, "value2":8303048822044107475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851782, "value1":0.32631560891796313, "value2":1989916996789168449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391839, "value1":0.002802043190783618, "value2":8629895271323501696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_1":"c","key_6":"c","key_0":"f", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066714, "value1":0.031995170191600183, "value2":3319906268065732292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421535, "value1":0.635220505499552, "value2":7200157785718775016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287471, "value1":0.8948051177891431, "value2":2176610475830643902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804659, "value1":0.28415985878229416, "value2":33052675971427371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996360, "value1":0.6152217383986905, "value2":544588703505765788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.798458, "value1":0.8665558307064753, "value2":8115268483760058936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448744, "value1":0.3917339620374744, "value2":9198148727913546136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861092, "value1":0.9826532618696505, "value2":4765520374138069290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.202532, "value1":0.0052591320469092834, "value2":336652236719043103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.524950, "value1":0.8143167251197552, "value2":6529413407694528834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.485366, "value1":0.5907312441951306, "value2":3337798525793799244, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558504, "value1":0.3641804008571667, "value2":4575793560178915745, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_3":"b","key_2":"k", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642936, "value1":0.623697886602654, "value2":7776504717328671778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043344, "value1":0.9623993994613507, "value2":6602176914068250275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.241555, "value1":0.23253929846791072, "value2":685349515600255534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926727, "value1":0.21223983767190985, "value2":3951472227334441759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.131046, "value1":0.2851063666145672, "value2":6180530959986721149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.247299, "value1":0.5130082295270245, "value2":8139956405448602641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758053, "value1":0.49876246690774434, "value2":3786706615107859566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323322, "value1":0.4313958567662574, "value2":8854072512884634153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748573, "value1":0.9590027921453678, "value2":3839028144294969465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384213, "value1":0.010562139959872605, "value2":6878855550977960785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180644, "value1":0.6131439659014073, "value2":2706285599198878081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593573, "value1":0.4120750238333436, "value2":1200854625465685501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995840, "value1":0.297035809872462, "value2":6168541710158809840, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_9":"k","key_3":"e","key_8":"d", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.511756, "value1":0.006206978177219643, "value2":1958577823681147315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565859, "value1":0.4442276087687805, "value2":39986647781639917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715853, "value1":0.9381008994826828, "value2":3556376195988394411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236468, "value1":0.20999691756150096, "value2":9161072742831597145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.161371, "value1":0.738745986035695, "value2":5138216362948975345, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445330, "value1":0.14739850543555172, "value2":8667736730671884444, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.982188, "value1":0.40813076015564426, "value2":3022991256121544350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984748, "value1":0.6214792221342914, "value2":5933123653156672217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562273, "value1":0.8568295281307874, "value2":5414869267292754213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476811, "value1":0.49675643916102746, "value2":5664619325942193065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891490, "value1":0.8866201621688181, "value2":8214607251036840472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931525, "value1":0.8288823187078326, "value2":3069955632009854668, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455511, "value1":0.8774391931652025, "value2":3046641762023967525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400651, "value1":0.5254465108261448, "value2":4999803900412377548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666971, "value1":0.9993076586927376, "value2":4982431527333428358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.8618210720028192, "value2":761627529379782309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.915374, "value1":0.37053901712348225, "value2":3944125570135845867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.960156, "value1":0.9374154078802892, "value2":4747973404867949779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210720, "value1":0.8468677886051782, "value2":4852924773263065270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.332500, "value1":0.6465642869647131, "value2":1008458018130353532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.788169, "value1":0.3773654642642006, "value2":4191084511110280464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545303, "value1":0.25852749290398136, "value2":2783025552475099417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.143623, "value1":0.36024800648413563, "value2":4932311496806548620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895647, "value1":0.0053042457769262655, "value2":6923819500937313799, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_5":"g","key_8":"i","key_0":"e", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932434, "value1":0.659079228202596, "value2":4313811336647625065, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.260771, "value1":0.403057123436948, "value2":4416521488550693116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238541, "value1":0.2952177348030342, "value2":1538460010819351929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958540, "value1":0.5291106177059647, "value2":1249052435364791122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430589, "value1":0.8147197427498966, "value2":5526371695265553363, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431544, "value1":0.9355303369766881, "value2":4469095563720030593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640478, "value1":0.48586313377391904, "value2":7959068707072142404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144376, "value1":0.00848981726968387, "value2":5068460906328461885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416953, "value1":0.6734644377643653, "value2":2766510942957347250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885086, "value1":0.21498485778798626, "value2":8278007933868416694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537244, "value1":0.3154976316557619, "value2":2639365305979929980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_8":"b","key_1":"b","key_3":"f", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.020210, "value1":0.453949847740753, "value2":2024272373950233485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514022, "value1":0.3475698755294227, "value2":1372329110659095675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806258, "value1":0.9274447352712736, "value2":3920648533163431087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209427, "value1":0.014587282647209867, "value2":8695997195066251972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971326, "value1":0.2797692384280631, "value2":6419079357930277367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652286, "value1":0.5905911533568637, "value2":8712000656540499616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878040, "value1":0.48362174266483915, "value2":2926563391771284063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990329, "value1":0.5049151964579438, "value2":3650817862757599534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049823, "value1":0.838067458662296, "value2":8854264061775325606, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017442, "value1":0.8431632474037919, "value2":8852347705624487512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.544821, "value1":0.005649842126714668, "value2":2282778264986625535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.597518, "value1":0.2045781468759336, "value2":8575639431654539646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612206, "value1":0.3075892336013964, "value2":7673450384556490732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471060, "value1":0.40918086270080856, "value2":6747291739813622106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.513402, "value1":0.7422854995723118, "value2":4083993911078701361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965008, "value1":0.6372798813181509, "value2":1707224457136697946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.354283, "value1":0.9108473180297934, "value2":1020266209800871174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806140, "value1":0.14778437030634853, "value2":6105158525255417708, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194962, "value1":0.3473291379662418, "value2":3795992189004973455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.974776, "value1":0.09118542426518424, "value2":5134289787707010070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.339938, "value1":0.13774510006333723, "value2":562171884148417195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.550551, "value1":0.384009626532502, "value2":8609730106908869554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.654960, "value1":0.2639517045663951, "value2":1862456670074602807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030932, "value1":0.5143312749002444, "value2":1494847764422163119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695169, "value1":0.2423970054677799, "value2":8509443740065644494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713656, "value1":0.34108742339653203, "value2":4598279459368093139, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_2":"f","key_0":"d","key_1":"e", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.794261, "value1":0.9458347874797615, "value2":7302724440323717945, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364772, "value1":0.2365974372946532, "value2":3654435595389536462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395099, "value1":0.15838919107992194, "value2":1939247105684031089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335629, "value1":0.39545025723069543, "value2":2672917998276624419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074949, "value1":0.7876350399335282, "value2":625303995684394443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536138, "value1":0.5616610785403638, "value2":8164874977498368721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697782, "value1":0.23834340441765955, "value2":3951692500137347815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671649, "value1":0.6501720201004972, "value2":2955880776784904551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.787079, "value1":0.378893964304627, "value2":3193906322405958964, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.031359, "value1":0.29145414339441095, "value2":5615459484207384702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.306294, "value1":0.34557951146905197, "value2":2931839249030464715, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_4":"d","key_7":"e","key_1":"k", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357655, "value1":0.3923960009663459, "value2":8112783273863341075, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961896, "value1":0.23361672067408637, "value2":432132564805318676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098797, "value1":0.18420600660031175, "value2":4984332870487251645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.071876, "value1":0.6659197757567412, "value2":1553450733606247594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186866, "value1":0.7000117639690743, "value2":7329872261968800728, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406584, "value1":0.13046586013887515, "value2":1622087582581823179, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440301, "value1":0.11646408872647017, "value2":1011606322554570900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.136166, "value1":0.4328392754883894, "value2":2919499144416670917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318957, "value1":0.7075576973107754, "value2":398072267285532828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.012179, "value1":0.5668221134322494, "value2":6110074131405385081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830949, "value1":0.3334949924022553, "value2":7572722630631231675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_7":"c","key_8":"g","key_0":"k", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145965, "value1":0.365420169950548, "value2":2535895288587113465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.649060, "value1":0.16445860625759065, "value2":6921600715877525051, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517227, "value1":0.07192874427493294, "value2":6297114124078920008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832587, "value1":0.5150936291796077, "value2":181711691089923280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291455, "value1":0.5623557100802268, "value2":8762626146094505466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_5":"h","key_8":"a","key_2":"j", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.149447, "value1":0.5534139568709165, "value2":2510872908080100775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476658, "value1":0.4486907399118968, "value2":8992668719445662670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236318, "value1":0.49075032225313237, "value2":1924725427300127433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.561765, "value1":0.9668124792198439, "value2":990391456732695590, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884668, "value1":0.0011015591734681801, "value2":5327656656973278324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175200, "value1":0.9826652864577513, "value2":2876633651705447757, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379413, "value1":0.15362330796009416, "value2":7353421640256094445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.319517, "value1":0.5202924305313171, "value2":7456692117889227019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.122103, "value1":0.7187092003738956, "value2":7131563361103346240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_3":"a","key_9":"k","key_1":"a", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678103, "value1":0.7543751799594591, "value2":8526553775473008293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.643972, "value1":0.19798980984364278, "value2":5929329615236343098, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.831121, "value1":0.1528427519418499, "value2":6184723596853858850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432219, "value1":0.5515160206615604, "value2":23121424938113426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058751, "value1":0.6338015087216923, "value2":7816911191202995864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790739, "value1":0.8762741406918414, "value2":4555556160832468314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797544, "value1":0.8139645999303083, "value2":2761547225024668528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404353, "value1":0.9860244138424634, "value2":2908826677832473108, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016900, "value1":0.053160518991546266, "value2":8380107749249883562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.407973, "value1":0.7843730354485248, "value2":6824583973593836053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987174, "value1":0.7619687243629265, "value2":3319925462827235787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577835, "value1":0.20813952139985942, "value2":6437982890007555330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755320, "value1":0.3636597821442561, "value2":5470963049552133157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.897987, "value1":0.07151356373469021, "value2":1477083928227057783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388580, "value1":0.20932748154080844, "value2":3392543048891869159, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_9":"c","key_4":"i", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.025802, "value1":0.38577745314564454, "value2":868566582237921681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941032, "value1":0.17111301155402217, "value2":6961085515450408974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947055, "value1":0.9780343555878157, "value2":133683558419674847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954822, "value1":0.08800460018752372, "value2":7497584262448035095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063420, "value1":0.8100121364093217, "value2":3755200469512293547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438294, "value1":0.3424655143965952, "value2":4316249625206240732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809627, "value1":0.9673131880891396, "value2":4132748723398010856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439925, "value1":0.8075664782512888, "value2":864263057207692643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.084104, "value1":0.3818348864598292, "value2":2553923579282526899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.647912, "value1":0.46216103925581126, "value2":4639093581287452200, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369627, "value1":0.3422458784570339, "value2":4254456412295026385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.614963, "value1":0.5220645586983701, "value2":6072161543938690608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.321895, "value1":0.3049038907635511, "value2":6325538472569574229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464290, "value1":0.22336524838033933, "value2":9076126601794953657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166932, "value1":0.3553203900309477, "value2":5982106259976109422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803898, "value1":0.5280548625824253, "value2":4063002262098608883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.927974, "value1":0.17895618154031318, "value2":547646730434307817, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.303832, "value1":0.5511007572679634, "value2":2862350870063686751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520504, "value1":0.39724544886891017, "value2":6931266301104819935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941095, "value1":0.14786172741452286, "value2":3205668266543533832, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656997, "value1":0.32075936991205234, "value2":7391478133987747835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120921, "value1":0.3939864396610821, "value2":2909689659050189847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958790, "value1":0.3269314306247572, "value2":6783238272107071770, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.386136, "value1":0.861947664801586, "value2":5805659227499250652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.449365, "value1":0.5799629274092946, "value2":9193070830904141987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_3":"e","key_9":"h","key_2":"f", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.785656, "value1":0.3165816948400473, "value2":3116393083198778433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520711, "value1":0.6656585396817016, "value2":1017823568348313071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.650203, "value1":0.22215575308437716, "value2":1902945447598147226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249002, "value1":0.2961338302748422, "value2":1415432970559418672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_5":"k","key_7":"f","key_1":"e", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694015, "value1":0.7506751146797324, "value2":6142650591148786794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.304466, "value1":0.37747841635497736, "value2":5375036774405769501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.641004, "value1":0.46446471911344284, "value2":5955856288590241375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.684973, "value1":0.907579891922439, "value2":5346005034595738366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.111153, "value1":0.49826719629056415, "value2":7497650141562337241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830722, "value1":0.28960521156096847, "value2":6154503575103623089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392721, "value1":0.6834318849474948, "value2":2628637152649849276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969704, "value1":0.15429907011430974, "value2":5405475101757554186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536252, "value1":0.7000830425153579, "value2":406966410660924195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.807572, "value1":0.2012419938894414, "value2":4358282132524848204, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.820107, "value1":0.5076022682087846, "value2":4017461578732969062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996611, "value1":0.7422015873263831, "value2":436292068764222211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.922123, "value1":0.051072363737232486, "value2":268646986926660655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.729895, "value1":0.8676962023633001, "value2":2065960180781249750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636047, "value1":0.4996316174329004, "value2":3156670443682802457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431520, "value1":0.5204212362312931, "value2":8019726224533633908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.903149, "value1":0.5859452724568418, "value2":2288060944551638278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.070771, "value1":0.9932429756726906, "value2":6690408728176272233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.314453, "value1":0.3472880048665214, "value2":58548757977887750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627243, "value1":0.7139819549630214, "value2":7080915148891558502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755294, "value1":0.05408051807422248, "value2":6931097273017939069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282331, "value1":0.3203397893756164, "value2":8256062300842673383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.034396, "value1":0.9531256831279978, "value2":7765396458031448915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.115811, "value1":0.13116244832703935, "value2":3543488806097224100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666851, "value1":0.6887850450312126, "value2":5462261380989469479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525431, "value1":0.3139759113605816, "value2":8160395548303928540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036488, "value1":0.7437550776292858, "value2":905546975072552710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246868, "value1":0.7173148070188747, "value2":7273545640281703287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840343, "value1":0.20321210783475557, "value2":336636579624758589, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100735, "value1":0.8393194019944642, "value2":291716047034557387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853981, "value1":0.5202422200638643, "value2":5204943321202939039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512132, "value1":0.018314329801792806, "value2":961979132289563000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487612, "value1":0.7033581894769818, "value2":4186345519226619201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778169, "value1":0.7839413953257911, "value2":7874569661704504858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.587363, "value1":0.24066050930994098, "value2":2238921997040090404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.863823, "value1":0.22725313032865457, "value2":4922946319365439720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.462195, "value1":0.6013044688497777, "value2":137484525088955387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.967535, "value1":0.21528129199568857, "value2":5907496598149934999, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263462, "value1":0.3398984210102334, "value2":3627872884296903263, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778010, "value1":0.2649025290720317, "value2":2665113842043817940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895866, "value1":0.8505969892692369, "value2":1773042500322059709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475234, "value1":0.08631156102550058, "value2":519973765404949203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996762, "value1":0.6378946736901862, "value2":9195196376217348512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537168, "value1":0.2006763700535789, "value2":9067766312454029883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.585179, "value1":0.8887141585928169, "value2":1447763822787404407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937668, "value1":0.18798426398408305, "value2":5531072459849624180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417573, "value1":0.018969474714960195, "value2":8739232693787930523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432784, "value1":0.6772697684929758, "value2":3670574248400915486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945707, "value1":0.2017640539145947, "value2":6866599884426415644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715992, "value1":0.2770810284692845, "value2":6065196798538107388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724512, "value1":0.7694937233145257, "value2":5461354547379838063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555480, "value1":0.4116895522673444, "value2":6999095526200651536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357696, "value1":0.9694808723879291, "value2":2308259417403012034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.925321, "value1":0.9804626384591183, "value2":1344780667969714366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340972, "value1":0.8564397089342566, "value2":3335346544187347307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634021, "value1":0.2824134389967874, "value2":827846054793909627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535849, "value1":0.8383622181777869, "value2":3210193933415043724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949231, "value1":0.3423558587998688, "value2":8018588983650318493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.604636, "value1":0.6545573982070936, "value2":8784079580099061471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041080, "value1":0.6815494814472329, "value2":2413918152267969152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206623, "value1":0.7407991857310414, "value2":8688412282907699410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697192, "value1":0.9920334806937902, "value2":4058886921829417026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748394, "value1":0.3435945058384803, "value2":6002972956903494678, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421611, "value1":0.8151473391165547, "value2":261810376504255149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.027674, "value1":0.32343755952478886, "value2":8467224123375485048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205103, "value1":0.26583420276460434, "value2":912318083515976688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362886, "value1":0.6918507204150298, "value2":3042551716463426665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181414, "value1":0.9588817871484018, "value2":5500996639169097947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607786, "value1":0.1555201931458164, "value2":3667544046344919475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447991, "value1":0.6322575842993106, "value2":6581489005452708007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_6":"k","key_7":"k","key_4":"f", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602539, "value1":0.3803193364932477, "value2":501846031619014460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224149, "value1":0.14287189157740537, "value2":4390766248301994532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019349, "value1":0.9275903306872896, "value2":6065489708075344215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.753905, "value1":0.04103970388034392, "value2":3647389822252762296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.808552, "value1":0.6912948158348138, "value2":7168224379303111769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.383522, "value1":0.7213839418172744, "value2":8917533656021329315, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988681, "value1":0.4652567645908903, "value2":6088381044289659824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426511, "value1":0.8194664026578082, "value2":3767723603035206676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291449, "value1":0.11519619446576408, "value2":7042710433748403097, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.495051, "value1":0.7541019265512129, "value2":4805842413127851386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.498957, "value1":0.7657156651926107, "value2":4284045357936772257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.920671, "value1":0.557277837350798, "value2":7618404853124250629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.325130, "value1":0.8555228162118138, "value2":5568850179022711136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619588, "value1":0.8294092067729972, "value2":1498942661428438729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_5":"e","key_9":"d","key_4":"c", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932410, "value1":0.9716095495296341, "value2":6631794618463446087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.257618, "value1":0.6931656613099204, "value2":4999381616225061323, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.328148, "value1":0.3114535269637382, "value2":7689977371490904120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.280989, "value1":0.9111540107476489, "value2":2001844404871692409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722254, "value1":0.17067710980514048, "value2":9024628709036843159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367790, "value1":0.7784645051385055, "value2":4706281866949049190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368237, "value1":0.6741959290593132, "value2":2438020324428284656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885688, "value1":0.4581933301264591, "value2":977306309852850973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.856287, "value1":0.24855090559153642, "value2":5403228793051986733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.305675, "value1":0.6170232598989743, "value2":274658269233518106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908698, "value1":0.5064010761390361, "value2":5817465185742290417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757988, "value1":0.04843336165427696, "value2":5801707099278996481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.933565, "value1":0.22461787050531695, "value2":6104462847804923186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.720615, "value1":0.12587001891222954, "value2":7755893877648671182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642145, "value1":0.4752800242516813, "value2":1097468219850109153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975649, "value1":0.9950222857947861, "value2":323860279126996239, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.571701, "value1":0.13176848875241834, "value2":8974250013315108425, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841017, "value1":0.5108584547409708, "value2":1015654371149607255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.499539, "value1":0.7323245432905996, "value2":866168553464820252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.137678, "value1":0.9837633616038727, "value2":1980815750999371252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713500, "value1":0.6844141285754438, "value2":5215363796900492290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564182, "value1":0.6959446753092476, "value2":2124568431454851658, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.548476, "value1":0.8904291651425387, "value2":8960234619080614404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.334127, "value1":0.5777624482637982, "value2":3890652700139772457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.141095, "value1":0.3019838758248612, "value2":1159300889479734121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_8":"g","key_1":"d","key_4":"a", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800239, "value1":0.4131750973298026, "value2":2344128018741531119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369249, "value1":0.1589896462920189, "value2":4160545898250072075, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.198713, "value1":0.08298578373908365, "value2":50980726393447907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.773734, "value1":0.7496268790368541, "value2":7804146534404388324, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237426, "value1":0.27543463085841935, "value2":3694238253557813150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577321, "value1":0.6586006530161179, "value2":641077213271202791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368897, "value1":0.6785289242869411, "value2":2009007121053273154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.801586, "value1":0.7989756868311243, "value2":3470745118950983568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.035889, "value1":0.4089400550263047, "value2":3320771957585601823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760511, "value1":0.7386595086100225, "value2":3332011099556778058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.359596, "value1":0.8203562587923371, "value2":574506838747954548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063436, "value1":0.2579978917934866, "value2":6993045686428025271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369331, "value1":0.4769488047439803, "value2":3318100445225134934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795599, "value1":0.9012136284147806, "value2":5770204186306544532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.219070, "value1":0.20974721576307698, "value2":2356228020734101377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537221, "value1":0.1725299480275798, "value2":5073957380468101443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015458, "value1":0.4025573823516011, "value2":3282394548862748845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580078, "value1":0.9823965911617399, "value2":8476500551084420415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.979266, "value1":0.5236855845499949, "value2":680393934209420766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682327, "value1":0.844770708190905, "value2":8298068650764370173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350010, "value1":0.5806359147839931, "value2":1527219564263667157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206279, "value1":0.4978598735396454, "value2":8931241509064706089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135137, "value1":0.7138645301142236, "value2":2633600346218814000, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.981064, "value1":0.7353726130780065, "value2":268395459435449482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.526357, "value1":0.3584205029962212, "value2":7926609514114360701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145680, "value1":0.817655724537724, "value2":2809466879616878308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.877503, "value1":0.17147641272949588, "value2":1475184649319875768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049475, "value1":0.3904708182990629, "value2":8178189937335677971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419221, "value1":0.2498991121797316, "value2":2019493757980421565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646852, "value1":0.2981893785163364, "value2":4127223489573331187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194796, "value1":0.9706301430727536, "value2":3907775771119307356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135232, "value1":0.952467511486142, "value2":6824235396463011996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392387, "value1":0.19563387556471776, "value2":6531634041022343859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.133702, "value1":0.7965289882627238, "value2":6086508658030257940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.691192, "value1":0.21849926406658296, "value2":2500546592704430602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238250, "value1":0.8547288670113503, "value2":1751658935211452093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.259613, "value1":0.7399127039208734, "value2":3758431842842501346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.959851, "value1":0.06348104065773419, "value2":5803029266892181407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350885, "value1":0.07399715612950311, "value2":9184522919237034506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821915, "value1":0.9143789840874811, "value2":3150076904171453287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.554775, "value1":0.406185744411068, "value2":4390857727102845864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.026241, "value1":0.576960195810922, "value2":864775792792522430, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118029, "value1":0.6740072102143805, "value2":2738633295377346249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.669898, "value1":0.6063573387513779, "value2":8118600251510115903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406027, "value1":0.5408212514385392, "value2":836959541970766944, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_3":"j","key_0":"f","key_2":"b", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205752, "value1":0.9617267357194531, "value2":2302570688726041304, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.683346, "value1":0.7764943186065553, "value2":1516613683034572821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535753, "value1":0.8501812602308251, "value2":3781935852497505404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562975, "value1":0.5762183849456854, "value2":8803839397694965730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_9":"j","key_1":"c","key_8":"i", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224509, "value1":0.6945846689247714, "value2":8790065402136541035, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120301, "value1":0.5693297769803048, "value2":8817337009559896419, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535111, "value1":0.7168638232585879, "value2":1459446787633025168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590881, "value1":0.052466731276145914, "value2":6984133752001938547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800340, "value1":0.19161367226547096, "value2":8470840993920678933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797252, "value1":0.22382372006161758, "value2":960932677896506756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502545, "value1":0.4567752766501535, "value2":9200307528794857259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.919995, "value1":0.3074462546242729, "value2":1729230620360441809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.273301, "value1":0.5542454650469373, "value2":2007525431527262950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249396, "value1":0.30289020495328595, "value2":7647941374943306916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_5":"i","key_8":"h","key_1":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847062, "value1":0.8617179899648977, "value2":2309423222399622901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.770003, "value1":0.8185102377578417, "value2":8900535665947339098, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612076, "value1":0.2520488653737615, "value2":534199497930559417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867000, "value1":0.3056461531330615, "value2":8074358922929649800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543352, "value1":0.1267819654500433, "value2":2282337611780682090, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771337, "value1":0.3860477352422603, "value2":6614019503509668230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724939, "value1":0.35759761476037216, "value2":7730807476801277254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.769189, "value1":0.06006911990731757, "value2":6135075757488436084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.750837, "value1":0.22593301727381723, "value2":815096043795563480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.733823, "value1":0.37434887115879495, "value2":8767359811379324612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_7":"b","key_4":"g","key_5":"a", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110462, "value1":0.7613769242381963, "value2":679247224492133418, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.207493, "value1":0.26270378513285625, "value2":6127960652597929191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757551, "value1":0.17246010749021554, "value2":3746123411387113183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476665, "value1":0.530111421321598, "value2":7929787788154212077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080232, "value1":0.24610687758058403, "value2":4382073893358318099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.518795, "value1":0.13825391880428892, "value2":6023983814814871189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.411487, "value1":0.7443212678339678, "value2":3163265105788950613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_8":"g","key_4":"b","key_6":"k", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230729, "value1":0.6799575270403697, "value2":6569848391065499405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.251807, "value1":0.23368831551353111, "value2":9121098592064787356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450371, "value1":0.8049793990951134, "value2":6271606339009618863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268506, "value1":0.7393978258409317, "value2":3084814221880218023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453066, "value1":0.7167056491982889, "value2":8359177712079217325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840677, "value1":0.3999771937764131, "value2":5907070339651841877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853610, "value1":0.4459636741761099, "value2":2591090489482088505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.050602, "value1":0.9347582889921432, "value2":3586868729035531903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.460673, "value1":0.8573683555697056, "value2":77436103148660020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.791642, "value1":0.7214035249481627, "value2":4712458830503182070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711339, "value1":0.7704360355595858, "value2":4818602735855589639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_8":"e","key_2":"g","key_4":"c", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709838, "value1":0.980850892357579, "value2":5814687934592979738, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448259, "value1":0.6911929910350553, "value2":2331566896810690473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069070, "value1":0.7715792084831908, "value2":1237753144790060173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728905, "value1":0.6453677424318518, "value2":4827712214727464044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_8":"e","key_9":"b","key_7":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.581880, "value1":0.2677360178055146, "value2":8844278477287710309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969799, "value1":0.5783967942752313, "value2":5905445599124832757, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741372, "value1":0.012956038256400619, "value2":914084143172531236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862619, "value1":0.5947373167351414, "value2":3630517863485151782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.139536, "value1":0.8372408825864636, "value2":6952590048486505005, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803976, "value1":0.43722974429572603, "value2":2176830109855416805, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.534405, "value1":0.7241541542820893, "value2":4511693404596905586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_6":"d","key_5":"a", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574402, "value1":0.7055278131957177, "value2":1238394241461479434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.06494326977178205, "value2":3563720913525659633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878052, "value1":0.8371531564279497, "value2":587508487226380908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.672424, "value1":0.26126599404383394, "value2":3460643419757907518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408361, "value1":0.5841391902007207, "value2":8050508010125067469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_6":"a","key_0":"e", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.743507, "value1":0.5292956990399417, "value2":5743834222350004253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.024743, "value1":0.5111340297086123, "value2":4739422872896352154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053127, "value1":0.41791773967804086, "value2":5388222563610877961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489667, "value1":0.6344084078946665, "value2":6127655871996983541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.876031, "value1":0.49328901667737785, "value2":3143559155074364531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.278465, "value1":0.23241882776321474, "value2":805438734033729133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380252, "value1":0.6353489769585325, "value2":1619270428062881618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512159, "value1":0.8152126527698539, "value2":1368040827327524990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.317649, "value1":0.9550761539015656, "value2":8914758126172814906, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.723807, "value1":0.7854700255052053, "value2":7095402949367847209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.660856, "value1":0.4053530100702817, "value2":2145190666873464855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.659680, "value1":0.12646331821035406, "value2":1576238191679234703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.993554, "value1":0.8499325393155558, "value2":5900620451042065379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091338, "value1":0.41503145937749936, "value2":2042239402577329647, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.299870, "value1":0.687218145400586, "value2":7560464390267142060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.731227, "value1":0.15009967640134209, "value2":1886018024243215805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.348963, "value1":0.6737669308977404, "value2":5404336119559355867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162168, "value1":0.813527799284827, "value2":4016392262793002099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086162, "value1":0.7399808700194177, "value2":8264192680187392458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975595, "value1":0.11433087569708594, "value2":4671281305443756113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799363, "value1":0.3067546199025452, "value2":225546722481809384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.113667, "value1":0.6424657658728049, "value2":8718201548571703569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904642, "value1":0.5465983180036486, "value2":4877073522652743329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.467203, "value1":0.8662610389904551, "value2":399387690820391383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082862, "value1":0.5088605240671744, "value2":1531739105872671512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.039708, "value1":0.10189606169579954, "value2":5185895909677369244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068495, "value1":0.2805843145287995, "value2":486469505433183294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357082, "value1":0.75740796150096, "value2":2690075476895328471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492715, "value1":0.3004718942084548, "value2":7383744941741065497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.968147, "value1":0.5004304919533921, "value2":1731582924085558465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.991776, "value1":0.7394075904771168, "value2":4141740040514370033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_4":"k","key_6":"a","key_3":"i", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420254, "value1":0.45767543208547934, "value2":1964890506496503777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285702, "value1":0.1366965878727093, "value2":2681549871870198885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492204, "value1":0.9414045318691766, "value2":1575036830396652846, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086526, "value1":0.5271825907095232, "value2":3742001028067515529, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.296194, "value1":0.6916437451586811, "value2":384054583246095269, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478834, "value1":0.5030459095566234, "value2":3918028956922111738, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.029769, "value1":0.2164261260808611, "value2":4102211333190644591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102553, "value1":0.42618668443550417, "value2":7568518633836620405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.681434, "value1":0.7225843033115362, "value2":8160060706717748819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_9":"b","key_5":"j","key_6":"h", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210135, "value1":0.21812369739137172, "value2":5702739804858144278, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.786060, "value1":0.33904050622912296, "value2":1017560684015420231, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558541, "value1":0.14437656270299287, "value2":7701298697010262063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.818721, "value1":0.15017675533914024, "value2":7186230665665602451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910629, "value1":0.09672885609565206, "value2":6203887741849145968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069131, "value1":0.6075489323724108, "value2":9025102416378601029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_8":"h","key_1":"j","key_7":"g", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362866, "value1":0.8018878313034695, "value2":8037732655719146713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682018, "value1":0.5433657256031907, "value2":6040793575397278438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797323, "value1":0.5032038038329079, "value2":3803498450979968327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.829279, "value1":0.605237212493161, "value2":4367127372886022859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.999162, "value1":0.5082854850214861, "value2":5031512432939201812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269625, "value1":0.0007161270371379666, "value2":6545591527308584857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180859, "value1":0.6040920298819816, "value2":8926725137631299725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090602, "value1":0.5326464117623615, "value2":7752649715252147157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.191121, "value1":0.12836456615806674, "value2":7198874487475192980, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.686863, "value1":0.708342981200594, "value2":5465607373577957408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.527995, "value1":0.10447991595171255, "value2":5974156320762011232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.123930, "value1":0.6200031138885586, "value2":5406077264893957847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.356501, "value1":0.44396993176396354, "value2":6255319076363219685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.048169, "value1":0.08153644488046811, "value2":8582968949378456970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.112743, "value1":0.6933778275610673, "value2":4587548588265206284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323148, "value1":0.20142008973704545, "value2":4527540178335323769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.934744, "value1":0.08093337102458979, "value2":7857820814429250360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962496, "value1":0.6124995000870105, "value2":6489203228482067372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.160517, "value1":0.5248710269553188, "value2":5026165991297404062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932200, "value1":0.922378624964946, "value2":7930339398930216623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.900245, "value1":0.1182228551739615, "value2":1373671224306855417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790920, "value1":0.31880287032595506, "value2":6084131296995295782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237291, "value1":0.13557228914344616, "value2":4432801334887160283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.914465, "value1":0.02040606137133301, "value2":1808915457173232015, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941296, "value1":0.9439262294487866, "value2":6574505025862591303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.822459, "value1":0.9942489150539108, "value2":3506354754330173943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239733, "value1":0.4989511485969244, "value2":7173139334014716167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.444066, "value1":0.7594335589002584, "value2":3216271266780228569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866271, "value1":0.5931978644398052, "value2":3444180839017636171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421087, "value1":0.6530151828739431, "value2":19805620992839207, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771459, "value1":0.8073198535663951, "value2":4810887667402700825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832482, "value1":0.734748250215754, "value2":1123934068717543450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450110, "value1":0.15409548849312346, "value2":418445000125934927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.234597, "value1":0.3684594320773471, "value2":3167333011558865991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.403637, "value1":0.891873955682516, "value2":6523321617886226804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017496, "value1":0.5380231504237305, "value2":1565703384693902706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577818, "value1":0.942234653476486, "value2":6854337928979531257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886845, "value1":0.561600684880549, "value2":6700870407358100744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.493299, "value1":0.1842060640300563, "value2":7660717134967306529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_8":"j","key_5":"i","key_6":"b", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043840, "value1":0.2702623264568812, "value2":7040356564551372493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051963, "value1":0.5871637139837996, "value2":4534754735986232109, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.568830, "value1":0.08173891497184316, "value2":408699021018747548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694578, "value1":0.8818619724910839, "value2":5720963057839829442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095000, "value1":0.9081401378248263, "value2":8656658880665915661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977416, "value1":0.9899458292085819, "value2":665964700424941220, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.670945, "value1":0.6733933124715158, "value2":2460152217631000428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636539, "value1":0.6931186653224365, "value2":7451786524160575556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.398486, "value1":0.8808674761773307, "value2":6001178466503449076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.085292, "value1":0.9258669929119259, "value2":4480969611449139348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451321, "value1":0.8902429562501767, "value2":2399440975743970404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100110, "value1":0.4539977466656097, "value2":4906418767346623646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.725438, "value1":0.8779089763048636, "value2":7986118613837183355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.559619, "value1":0.42482197030409785, "value2":921145061767776729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885834, "value1":0.32962036175845005, "value2":5608652380611657042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619049, "value1":0.4697848554943135, "value2":6930715352479440739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768456, "value1":0.55002846829011, "value2":6856343919804361537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.052488, "value1":0.9193313692023829, "value2":8075338150518594907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.346748, "value1":0.7969202775153313, "value2":3854682510572365821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.668529, "value1":0.8901520814855493, "value2":3780299539456604836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.680780, "value1":0.9128377273315654, "value2":655380872616480476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.298499, "value1":0.8901980009691262, "value2":4124662751920860689, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.061822, "value1":0.922218588720657, "value2":6935461410888938736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.220215, "value1":0.8707922044422206, "value2":6415853705476229931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825872, "value1":0.6145357419122679, "value2":3712577567679334743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287572, "value1":0.7153750220855114, "value2":6766565708182723447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.258723, "value1":0.8623529521546892, "value2":3906675258214843555, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417686, "value1":0.6071420867010977, "value2":8210404273898330224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.939674, "value1":0.018125841559591738, "value2":2686797765813242922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062829, "value1":0.19220806032635843, "value2":4009115073519468001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.761409, "value1":0.282044445283748, "value2":2659364730071661456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340677, "value1":0.7867304449409817, "value2":4726626950116268051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.848210, "value1":0.7451256645145572, "value2":6540697575244319830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082209, "value1":0.08477415388397415, "value2":5473839595744351618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562389, "value1":0.08808192976810916, "value2":1578232710018420800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445878, "value1":0.7573342205320885, "value2":3400644244684606362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.603775, "value1":0.183625034999939, "value2":7793796792648526034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036048, "value1":0.8359424936170327, "value2":5865912253243965224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866722, "value1":0.3949434009829113, "value2":3290170127727456353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697995, "value1":0.8423729542207233, "value2":4861550929219709907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.211200, "value1":0.36512924472577757, "value2":2972205895387325421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227557, "value1":0.22341834017020207, "value2":4806018666421767946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574952, "value1":0.6007129713703638, "value2":3849253119017333239, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741721, "value1":0.5702349573898813, "value2":897876839681174700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379915, "value1":0.6564162460066056, "value2":3131015984288737246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.229369, "value1":0.456646560158155, "value2":4778476677749084799, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.242697, "value1":0.4142421381714439, "value2":7101584169529051655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.212755, "value1":0.39066964505760526, "value2":4253776777664568253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_7":"h","key_9":"k","key_5":"b", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837272, "value1":0.15824518985564762, "value2":6812701504471384455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.104360, "value1":0.9940983043766688, "value2":2203777894935310413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391080, "value1":0.5579038729371453, "value2":6131596744931664143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265096, "value1":0.7511430046090084, "value2":4283281105409640096, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881500, "value1":0.8197410181139487, "value2":2167116001911138380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964911, "value1":0.6839484284574476, "value2":2392821775915043629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_4":"f","key_9":"c","key_0":"j", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906639, "value1":0.12232760875324074, "value2":9162174110871211714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562264, "value1":0.9910780181559037, "value2":1543791401715243110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016345, "value1":0.3277590263870748, "value2":3851358761838506924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963516, "value1":0.13686033235694836, "value2":2786944428195217311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.446494, "value1":0.051498079678560585, "value2":1973784887996536653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350210, "value1":0.13860925037657423, "value2":1354426546825223222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358378, "value1":0.28813722807183184, "value2":4138306657967174744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455527, "value1":0.9787455514136848, "value2":6829598111029855692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.208210, "value1":0.4291478612877578, "value2":749843861604187954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.330867, "value1":0.6408478280255169, "value2":8395361043273182125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.176999, "value1":0.9117058417477248, "value2":8428888804090519109, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091618, "value1":0.8122911740955318, "value2":1478086552909570392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243584, "value1":0.06727976096288107, "value2":4178958373440436949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.127002, "value1":0.8012772426876036, "value2":2149729620077994596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_5":"j","key_6":"e","key_3":"h", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461395, "value1":0.21653382210662625, "value2":2558734709934645897, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932379, "value1":0.43744123497894677, "value2":5779172669858974850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180562, "value1":0.6091002318007619, "value2":2233652952119987641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_3":"h","key_9":"c","key_0":"e", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.738334, "value1":0.8147115963605253, "value2":1038206878786066414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943138, "value1":0.1805265868912326, "value2":8231024461542686979, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_7":"g","key_0":"j", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007238, "value1":0.30166077327328183, "value2":5211541986455490237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926840, "value1":0.7197646392974351, "value2":6170887305282807365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.976392, "value1":0.02561261869316712, "value2":8003552327575750503, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665296, "value1":0.38463860662667215, "value2":702575998172013860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.957627, "value1":0.8365108478779814, "value2":3172439569974822855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.779959, "value1":0.005457784803842251, "value2":1217373846044030405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814440, "value1":0.03586677782979768, "value2":8668868565483927092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590538, "value1":0.7998407978822464, "value2":5953384113011158992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.381146, "value1":0.6861901618066985, "value2":806723358919392729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.324953, "value1":0.09051509245266014, "value2":930835793453842933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.309761, "value1":0.44326721766264904, "value2":887788347593465760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722069, "value1":0.08849038862609183, "value2":7352758650578649855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558827, "value1":0.3337734956738152, "value2":8685182570627965041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715649, "value1":0.23109099835442082, "value2":1083379478056182665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_9":"h","key_7":"b","key_8":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964540, "value1":0.6330159342037405, "value2":1167652807667781397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795126, "value1":0.5778697926785736, "value2":9140994998625427103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236481, "value1":0.5802899488963301, "value2":2895083535971839167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996259, "value1":0.2785697344481934, "value2":4913023416457428502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.477938, "value1":0.484568023326054, "value2":3495500670092878690, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261065, "value1":0.11116858735231369, "value2":5869730879322934574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_2":"f","key_1":"a", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185846, "value1":0.7126003929744608, "value2":3434450920636329132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841676, "value1":0.3736127841617722, "value2":7802733756976416976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350997, "value1":0.582318689502649, "value2":372190874054145543, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555394, "value1":0.12831996488184216, "value2":4582628444562419674, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.155140, "value1":0.3906336986690914, "value2":8237405956495459279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367910, "value1":0.741845875474543, "value2":8082126523839106878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.950768, "value1":0.6652471104213182, "value2":8268690822588065094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.433269, "value1":0.7980991574588373, "value2":8513183309513339766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.874369, "value1":0.5464494520072651, "value2":5885072115794893058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537599, "value1":0.7405427933790399, "value2":6635799236257663587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.955763, "value1":0.8778283555530513, "value2":6592488434360400176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.582015, "value1":0.879034635958457, "value2":7219576376735925925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388984, "value1":0.53843340060884, "value2":2254745170299712210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646131, "value1":0.5101339977981793, "value2":3292039546456501555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033442, "value1":0.6456192738711094, "value2":4906576027749942202, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.836645, "value1":0.48635849410277077, "value2":1404791026873124424, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464295, "value1":0.5617144285499786, "value2":7990958033367681704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010923, "value1":0.9807071857311617, "value2":5683359887238098850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438005, "value1":0.5935141575872156, "value2":4865067817966075444, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227310, "value1":0.6681961275740043, "value2":8551103550465421536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186394, "value1":0.25048622317521113, "value2":2888181774636358576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053227, "value1":0.10049698552895067, "value2":4557545020451680363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621258, "value1":0.9388516977567649, "value2":4055003743201350923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227442, "value1":0.7022685194913532, "value2":3740668486633940756, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640300, "value1":0.06434066744840804, "value2":1646848528935407928, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162609, "value1":0.71410216780028, "value2":7370378990581242781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512792, "value1":0.19345496541059937, "value2":4424425935560475985, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203787, "value1":0.9476247108925144, "value2":264480497401916863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.076408, "value1":0.13927961349104323, "value2":1149972870879073163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806676, "value1":0.6172364818942355, "value2":4996985075367830737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.313484, "value1":0.6715999817727264, "value2":5237271908928184120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947617, "value1":0.1400224644800571, "value2":4212016629330280660, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.695969, "value1":0.13461968177274677, "value2":8209146459721882476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466156, "value1":0.7851823238051198, "value2":4609596535178738402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047192, "value1":0.6615735622970099, "value2":3688971736191236071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179244, "value1":0.5525116533205023, "value2":6958696123355987767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.836535, "value1":0.8752088579621973, "value2":1772355477524999885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.385661, "value1":0.6669392703121872, "value2":5026611864593335422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.122798, "value1":0.5624470316374681, "value2":973613507611825241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663323, "value1":0.2836760055083927, "value2":8274369123408107345, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988003, "value1":0.07587743066420682, "value2":4947294410032829574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.240844, "value1":0.4161174869019418, "value2":4077015391953706312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.945230, "value1":0.14944194749229664, "value2":7660491006096000021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890111, "value1":0.20296484407777726, "value2":1824051875710962915, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522740, "value1":0.021855326277136235, "value2":3218235817889269142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.679843, "value1":0.5055217891288131, "value2":4822706751533920431, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191340, "value1":0.9959364873278129, "value2":3932069232230299078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771462, "value1":0.4557053136591391, "value2":184726250007256859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581655, "value1":0.689124829253459, "value2":1817463261736430161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.894089, "value1":0.9069576891707435, "value2":7318133204837024663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.307941, "value1":0.4227283465946924, "value2":356860997794924081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085219, "value1":0.7548908197263309, "value2":2736908468355530081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510864, "value1":0.6045243894505045, "value2":3203823484629938988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194038, "value1":0.2945791412290687, "value2":8542013145729544847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.822784, "value1":0.7046664475635507, "value2":6531211714773873879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958034, "value1":0.6210073688384752, "value2":5171125204919187937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245514, "value1":0.07787807249020082, "value2":4796876093300411441, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922409, "value1":0.026305095785565266, "value2":4204774996273721582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618949, "value1":0.8021374519845728, "value2":7055530836155104119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_2":"c","key_7":"d","key_1":"f", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.842155, "value1":0.9078167657864898, "value2":659406508816846003, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.429827, "value1":0.6861191669827441, "value2":3716781946259346478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.721875, "value1":0.7721427864001064, "value2":1559707715553693565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.667085, "value1":0.41573209008012846, "value2":770484632734171477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567897, "value1":0.2688265887173671, "value2":8731672977339492909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716882, "value1":0.20533155732399708, "value2":8158121924487423466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.450278, "value1":0.45425248453866135, "value2":7472174068145252016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700766, "value1":0.38900719316872584, "value2":9070338695929499459, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296131, "value1":0.7588298197955197, "value2":1078200536047373074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644620, "value1":0.22063287558750297, "value2":1455208095653529502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826663, "value1":0.9059127191998474, "value2":163331002989197798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.375920, "value1":0.888489428778294, "value2":3478637008435710026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795211, "value1":0.7169431218180555, "value2":5240161460250645540, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686886, "value1":0.1615738416501897, "value2":4522663398730909152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.641311, "value1":0.5919216049468233, "value2":4264524771451405531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415500, "value1":0.8013193758641248, "value2":4655946330615841665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310579, "value1":0.03464731335525151, "value2":2648033740620709930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.135452, "value1":0.663217728481436, "value2":6656249157262608643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036458, "value1":0.7592021704654797, "value2":4708550885998497867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_7":"j","key_3":"k","key_6":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196106, "value1":0.9109556832409251, "value2":1182178953302502536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897060, "value1":0.30582754840738535, "value2":5110613128180106106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.386690, "value1":0.17482144522900334, "value2":208181032234291542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770888, "value1":0.5775727956988672, "value2":7308314003124624706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710417, "value1":0.9182102368115501, "value2":8001324021984156096, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.133639, "value1":0.8132459830268748, "value2":6362172937464044603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313132, "value1":0.6425828291871425, "value2":3812023207458774027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816183, "value1":0.12876194798023052, "value2":353578462778793882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.537076, "value1":0.3677093772206428, "value2":3410314947529692873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.785801, "value1":0.5650444568188182, "value2":2518267372616726657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793640, "value1":0.36840754337253134, "value2":6490561754060041624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277029, "value1":0.5742325627742796, "value2":4576556191237422408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086053, "value1":0.635715694469195, "value2":3276323627400814300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622335, "value1":0.6664883465636151, "value2":6846298203429885729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784935, "value1":0.8345083778637773, "value2":2989860444276517490, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394440, "value1":0.3881444024090245, "value2":211938452740442471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.150923, "value1":0.2049623118713732, "value2":7830583450265904275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334186, "value1":0.8877162218809062, "value2":5530254516648658024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919131, "value1":0.645089444938841, "value2":2752404297334193415, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.550415, "value1":0.8672407553696915, "value2":7296354502501329771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.251810, "value1":0.12582294112213796, "value2":3850461942759838134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.768322, "value1":0.5089660327127362, "value2":4637000702345102317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.5790505163445078, "value2":2746113057580625956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310041, "value1":0.7419516660136578, "value2":8573283812679768621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567732, "value1":0.20310332375311568, "value2":1684484347666739261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.149277, "value1":0.8537753954365711, "value2":3792414057901721222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019813, "value1":0.3193924681887993, "value2":4416687297621574996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555536, "value1":0.2678716255116121, "value2":1978875629658477083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.676000, "value1":0.268194488154906, "value2":9208242650214533822, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603232, "value1":0.7475788122275303, "value2":6003796300446172845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272526, "value1":0.7633068823447601, "value2":5387009257178750855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288629, "value1":0.0999015536587548, "value2":5230082044659723490, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198708, "value1":0.5411261720205245, "value2":1324824529248190153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754319, "value1":0.5181119376899533, "value2":7586548988147520888, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_6":"c","key_0":"d","key_3":"d", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.974535, "value1":0.6605961675334583, "value2":4149492144268522705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620393, "value1":0.0400929610839578, "value2":2611761419308325853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489379, "value1":0.20432994152077563, "value2":4859071163254701004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.497240, "value1":0.21587046902788842, "value2":1674028941320053082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227611, "value1":0.31207060679536985, "value2":3782150896328417443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.813479, "value1":0.9346689305684607, "value2":4170814668135359561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832568, "value1":0.40374034609384823, "value2":4890680963042405726, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.868761, "value1":0.7456764740945415, "value2":7985095078472385264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_5":"k","key_2":"f","key_3":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566095, "value1":0.11155632539641938, "value2":1650668369012843098, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516140, "value1":0.2921171335805605, "value2":699561345288206492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.009998, "value1":0.0698415110112407, "value2":2175710360877139700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472788, "value1":0.6894980351369026, "value2":5185256568181045963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013027, "value1":0.47916067130802836, "value2":8339740169449446777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947875, "value1":0.15641692100908322, "value2":1874863435098609183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834625, "value1":0.4725529822941945, "value2":3313442976129641776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032027, "value1":0.11526706346113103, "value2":4474958127457336496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.643426, "value1":0.2365209546681749, "value2":2452872778362311452, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399009, "value1":0.10587590188915917, "value2":5343568589706680846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_2":"j","key_6":"c","key_1":"a", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464026, "value1":0.23089518212237822, "value2":6365432144590215713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.573860, "value1":0.41263343273878106, "value2":7100420790565954895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575295, "value1":0.15635829479176244, "value2":972701597090717176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_6":"f","key_8":"j","key_5":"b", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685505, "value1":0.3500577640159365, "value2":5296267831822852754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382859, "value1":0.8962437052233099, "value2":5000561412494049933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793198, "value1":0.03594880833972008, "value2":5558834996866286026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296341, "value1":0.6801030998229524, "value2":9104706563374398078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.044529, "value1":0.43868239396683095, "value2":1119681077532090572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977458, "value1":0.7636375641015349, "value2":3738349085487316798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.769690, "value1":0.38941161495083887, "value2":4119290182206919942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604117, "value1":0.06703731437988915, "value2":2776279989058320233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.107061, "value1":0.9783306976372779, "value2":6724603539014036116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.243947, "value1":0.3058624284281911, "value2":8607064800338196111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784300, "value1":0.47413357934186534, "value2":6062775812546347462, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053929, "value1":0.8033894774680731, "value2":3798712131176287721, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997633, "value1":0.3495480156862711, "value2":6358034341550557079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399888, "value1":0.5577617871085867, "value2":7606548109917229902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_4":"k","key_2":"k","key_3":"i", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.343972, "value1":0.37033984748970067, "value2":3823813661350967339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741859, "value1":0.15725835002679114, "value2":7931796757818180793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118790, "value1":0.8710092300018802, "value2":8432660813300591545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_6":"k","key_8":"f","key_5":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.514050, "value1":0.3589657867466283, "value2":2807064518724813388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105213, "value1":0.48313119234822866, "value2":9012983220702483909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.289587, "value1":0.28297188108556603, "value2":8293007746329622374, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.682523, "value1":0.2601723288341452, "value2":5132563496112960923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.521609, "value1":0.17923659255236565, "value2":9080059564709771845, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.866470, "value1":0.8472875420381546, "value2":7662519070278706881, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139688, "value1":0.701824998288288, "value2":8626519461512263205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693655, "value1":0.3931946923075373, "value2":4088184802361327943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.733184, "value1":0.33562267701400733, "value2":5057125078949459009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530556, "value1":0.8546701921585885, "value2":7172802034424433024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890826, "value1":0.9156630138494082, "value2":1106383994017980379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736946, "value1":0.28531565085928856, "value2":7492657224478970683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184271, "value1":0.66321078827276, "value2":6482907214355252145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771765, "value1":0.8506763394943703, "value2":2997501176379232751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621435, "value1":0.4219907917118007, "value2":2371448704479482567, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522551, "value1":0.4559374479237847, "value2":8705737330976472895, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579601, "value1":0.3374291561460269, "value2":6118047213487455086, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.911838, "value1":0.24595613407282738, "value2":8543292459890755038, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.461163, "value1":0.20032563894255778, "value2":8271655059958021803, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480016, "value1":0.537301255549281, "value2":131039455325957900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264510, "value1":0.8207403430077567, "value2":448751349067313394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.908904, "value1":0.45150207048162533, "value2":1484572129461087565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279527, "value1":0.48678886025337537, "value2":4675520724179222107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.983786, "value1":0.5074627713755523, "value2":6761505393797376999, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407769, "value1":0.8974462165990488, "value2":4427875231986566903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906297, "value1":0.32841731679412833, "value2":4853751460801403413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.910866, "value1":0.22248621009204866, "value2":6153217292256934376, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481193, "value1":0.07256346779714405, "value2":8357563464914791440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819751, "value1":0.7821155606881222, "value2":6350148722815418199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925016, "value1":0.08017277463129999, "value2":8127038680866850197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380866, "value1":0.4236203787530011, "value2":6102770882422449664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.687520, "value1":0.18833602677282602, "value2":2416669714696967955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_7":"j","key_3":"j","key_4":"i", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392864, "value1":0.12162108913858492, "value2":1037740633966701101, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555618, "value1":0.5726263308688928, "value2":5340970336126619965, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041150, "value1":0.1866697214008993, "value2":6089316958837860364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.848709, "value1":0.5912728689790879, "value2":8374928084508664533, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.633435, "value1":0.34567925121735993, "value2":6074382604796755677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615342, "value1":0.43884173283368255, "value2":2552875653507489786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.420473, "value1":0.024593195105800023, "value2":6998883295968298787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.493473, "value1":0.517385578986543, "value2":7898967292563966468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398409, "value1":0.9494831815326669, "value2":1043777551847385399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.261653, "value1":0.605095923821751, "value2":5798307460445919671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734130, "value1":0.17339556093076888, "value2":3888875155355529135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.234024, "value1":0.07147953681129654, "value2":2220989894337620932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.552276, "value1":0.46428310479897317, "value2":5429095800661644988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.547014, "value1":0.9160534111766008, "value2":4258800689197649539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084253, "value1":0.06651726425775369, "value2":8202506388672279723, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137868, "value1":0.6230337836484641, "value2":8788058834810483014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.168246, "value1":0.1571304566786965, "value2":2933470821094626337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043250, "value1":0.3445304512204906, "value2":8908780418296865224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542269, "value1":0.007638770771913262, "value2":975662304048103149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016306, "value1":0.44304617075801217, "value2":4700290207925657878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967184, "value1":0.9943193022594318, "value2":7141028805491571993, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522277, "value1":0.8578463560518583, "value2":6372322792910116627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541009, "value1":0.5200416543540609, "value2":5687465452183940942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742322, "value1":0.4918651213341637, "value2":7530197332852248825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.478705, "value1":0.02234257118225825, "value2":381364684161431163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.401319, "value1":0.36716744081601327, "value2":3165286506094547515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.154587, "value1":0.8570612240935895, "value2":440607090836829314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268457, "value1":0.19708890772544582, "value2":716490672588175731, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976461, "value1":0.37921805973698547, "value2":5187015251060220885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.291361, "value1":0.0347133343718353, "value2":4081610772727292868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.831910, "value1":0.20329085836851984, "value2":1212790782212732680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_3":"j","key_9":"k","key_2":"g", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045270, "value1":0.742873126094037, "value2":1857680286082818546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719581, "value1":0.8427487279865438, "value2":1584262554399141329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.878973, "value1":0.7217975560465744, "value2":6936890459625848218, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.786254, "value1":0.7388730574867226, "value2":3591226182562159088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719189, "value1":0.6138101273145392, "value2":4767024689432495577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335758, "value1":0.6941935445321573, "value2":1310712364678635982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333397, "value1":0.7084933562313812, "value2":2679977702860384198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816868, "value1":0.5704940444863804, "value2":3982836666436033897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.806625, "value1":0.7550148139027509, "value2":2819861429300922010, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155981, "value1":0.3978060292285169, "value2":2375324886880679840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.172764, "value1":0.8807292451628046, "value2":1585412394695366401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.127700, "value1":0.8691901922846396, "value2":4006452105841009084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.937137, "value1":0.6079865671377581, "value2":9014517461395944209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601939, "value1":0.5587486969852118, "value2":896670201927694097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147910, "value1":0.7860716932621791, "value2":8828747393971238771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271640, "value1":0.8634829980251869, "value2":7534202338930685599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_7":"c","key_8":"d","key_4":"i", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510609, "value1":0.273735584071857, "value2":7995135221543970576, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.424702, "value1":0.6570665232819806, "value2":7130224512963405605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.120070, "value1":0.4366525687584962, "value2":8321790572084844233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.355286, "value1":0.9376756075719435, "value2":7612697762460955574, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057269, "value1":0.9603884122588556, "value2":5675558649074747456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_9":"b","key_3":"a","key_7":"f", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.396116, "value1":0.3241173308406254, "value2":2149479689060994101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271843, "value1":0.5095285278231975, "value2":5564561021464735924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389082, "value1":0.7189491284948987, "value2":8539354123799965742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780310, "value1":0.7959419420056967, "value2":791327040768245285, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496100, "value1":0.7137676161743793, "value2":7249423453544847575, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.320840, "value1":0.7088322490051288, "value2":2259284238716415015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.588238, "value1":0.2616266339940352, "value2":7616111814012126210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510231, "value1":0.34548313055527086, "value2":9146415948654460714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725142, "value1":0.832268560021817, "value2":8449163586612650016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722343, "value1":0.5127452250176201, "value2":8372677466999750797, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455239, "value1":0.7274771017691084, "value2":8558710521155328448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070961, "value1":0.5108403289338641, "value2":6822737602377250548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.799042, "value1":0.6579145757389928, "value2":6367849243718047663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997752, "value1":0.29826795257630173, "value2":7963803897292403489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019959, "value1":0.5329624946388745, "value2":6166072735161874859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183220, "value1":0.1192005017394279, "value2":1414525889342507585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.216504, "value1":0.10069780302425961, "value2":4147229267124117622, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184537, "value1":0.9674252449470503, "value2":7196918589322334021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.982196, "value1":0.5121645885657403, "value2":1331574421408029626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.807710, "value1":0.6453473320699041, "value2":5880434997959490048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139645, "value1":0.45151332154483154, "value2":5206983362834731524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202627, "value1":0.5085399708189182, "value2":229347926887383465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703675, "value1":0.1410911985436997, "value2":1995545208705825638, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.916538, "value1":0.9588604837943727, "value2":3014447442999476863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174096, "value1":0.8442896590468587, "value2":1669527921851537286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268618, "value1":0.05460739672623512, "value2":6145595254763691216, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184210, "value1":0.08111723273128417, "value2":2941741438359406510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419569, "value1":0.11197378055505244, "value2":7190302664501414682, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.080457, "value1":0.46189706497698463, "value2":3739595486194011632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685402, "value1":0.5911751311740201, "value2":375795342652216999, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584152, "value1":0.842391293090085, "value2":3305999218832830126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_9":"d","key_4":"e","key_8":"a", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719567, "value1":0.7117389665869482, "value2":4436375112558311543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.599224, "value1":0.7507823724841362, "value2":5289861475224212810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298087, "value1":0.03464563720261393, "value2":5487979858306400729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084762, "value1":0.4408096997046924, "value2":8782838705036394284, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153226, "value1":0.8396685378218853, "value2":2305238685769866566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875795, "value1":0.6283427563062458, "value2":1424824369894963651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.841870, "value1":0.2951891910717211, "value2":1229301210242052079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036891, "value1":0.10272064811801765, "value2":8629352909535519019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.475886, "value1":0.47884350384173996, "value2":2002152937943396093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286578, "value1":0.5511122934004854, "value2":6456656081230025536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180709, "value1":0.591154696871619, "value2":1085648256858149575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.549035, "value1":0.9073632737596287, "value2":441521694946529713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644644, "value1":0.42900060724414874, "value2":7252379057595536961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.185531, "value1":0.28679213225919725, "value2":3666625025829964686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858171, "value1":0.07899073830408582, "value2":8847544646507443860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.9086649445849005, "value2":7300771702176978507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.217129, "value1":0.8286760370555245, "value2":437314161092307860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.092457, "value1":0.3257453112147972, "value2":1241192378749271122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_4":"a","key_2":"e", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.318459, "value1":0.6218522470963161, "value2":2601033448014127071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543632, "value1":0.47194572326020234, "value2":330858668685674119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.494889, "value1":0.42021023869128465, "value2":7698787812508472546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173188, "value1":0.6333586665476976, "value2":1031119021359267335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392225, "value1":0.13669516747392368, "value2":2722787133031533732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.418321, "value1":0.8852713566769926, "value2":1145580014369872247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499498, "value1":0.37652715283253, "value2":6023088039348052644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049271, "value1":0.7428393913560761, "value2":4074778390717770530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344096, "value1":0.618674195595606, "value2":8432036346015765748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.504726, "value1":0.8381168181634259, "value2":7490362504268179930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130009, "value1":0.04787900547347569, "value2":8930977988943996511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517353, "value1":0.9123390757414803, "value2":5406757776090744464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.159780, "value1":0.2711186767871672, "value2":6650537343113966916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601454, "value1":0.036596921749510625, "value2":1801819188066798655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740877, "value1":0.9080583731663923, "value2":6623419932438799475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.743094, "value1":0.9900165531318826, "value2":383393257357043043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.939900, "value1":0.8769075090910828, "value2":487693007619851718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.102543, "value1":0.4399163118191967, "value2":5888144134235313914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917706, "value1":0.6339047693941521, "value2":6760479052672151531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108610, "value1":0.06502295112846754, "value2":8902773220268340372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.756394, "value1":0.7104082109611765, "value2":3854497302576556911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914430, "value1":0.39800381262255646, "value2":1413572596816022205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.046273, "value1":0.9197740465700138, "value2":5994606503835738604, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.042126, "value1":0.35565383574544296, "value2":206374188969693152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.753926, "value1":0.14736292648599036, "value2":5720098218146494963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138925, "value1":0.14592833793919333, "value2":5845534364661717380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_7":"e","key_0":"b","key_3":"b", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966713, "value1":0.5691036450918006, "value2":5571037157834825483, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128296, "value1":0.522798351805377, "value2":7845314774564626095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.374869, "value1":0.16619436230032464, "value2":2683834913042811436, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349899, "value1":0.0806742622823487, "value2":5654997107573600799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621137, "value1":0.6389883746416457, "value2":6718840923740903865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505989, "value1":0.5736885226925782, "value2":7660274947298694696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245218, "value1":0.06067210631905806, "value2":7481373150924938933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.961402, "value1":0.40476409107709094, "value2":1656152426641438063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064546, "value1":0.7710994608894077, "value2":5484900997853498427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686047, "value1":0.46617534097616353, "value2":1998746207850635494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195207, "value1":0.8036664106551048, "value2":21823515460647768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890318, "value1":0.49246668518608533, "value2":4290226524505777397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.662549, "value1":0.0648629447687713, "value2":2600958480076392029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867346, "value1":0.6103284095540984, "value2":959086414558873258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036038, "value1":0.22211264871806694, "value2":2970074654495356013, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772742, "value1":0.47254264158251036, "value2":5967291460715844391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.765254, "value1":0.59761349012731, "value2":5045361763860538939, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684477, "value1":0.3727521458297325, "value2":7120290254833886255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782498, "value1":0.5592140587339981, "value2":1560278611367944657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472665, "value1":0.4451202433666956, "value2":7528293771469761549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957028, "value1":0.6722101190310112, "value2":3256843532996983591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_4":"j","key_9":"b","key_3":"g", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.558411, "value1":0.33974258843823785, "value2":9219292136379640073, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686456, "value1":0.662252940687323, "value2":2872679473486076312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.664702, "value1":0.13252769744708778, "value2":3782836160183678190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340522, "value1":0.321610730663863, "value2":1993266611063130503, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_7":"i","key_9":"k","key_4":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738508, "value1":0.03455404630849952, "value2":5265512962120119813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722717, "value1":0.039393801449424636, "value2":8411289366884053387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738985, "value1":0.4374781921483304, "value2":4532229627185520929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.959594, "value1":0.06784833824743756, "value2":2235113542571170507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512001, "value1":0.45518592645372824, "value2":1219663694779720192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699152, "value1":0.9199280757111055, "value2":920010063978439254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870981, "value1":0.7153328294012858, "value2":2241596700674662533, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713651, "value1":0.1871397130196525, "value2":5525165593287246022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608421, "value1":0.2007302921644417, "value2":7082345286303554347, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.898979, "value1":0.4326799832087361, "value2":4510789740667402710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542323, "value1":0.5916017742256197, "value2":7802920355385118767, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245771, "value1":0.3115277557229622, "value2":6710572841494695733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772755, "value1":0.17023256633885028, "value2":4693930877366777744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.247931, "value1":0.4740059660910414, "value2":1391972395280563550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394287, "value1":0.02366479635408087, "value2":7477007108410228541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108172, "value1":0.971283160522281, "value2":6224196629196175206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932465, "value1":0.6431474969443486, "value2":6996302573266425119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070979, "value1":0.4324217725560448, "value2":9202735673442875059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862734, "value1":0.5425012681866345, "value2":8705521243115097026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207036, "value1":0.7438270358882513, "value2":1482607831466689197, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345448, "value1":0.3875837956213489, "value2":2480267181897079015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553288, "value1":0.3343119844132966, "value2":7890867607759034801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559986, "value1":0.2851751818045875, "value2":3231450807250993029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.008285, "value1":0.11082046459395005, "value2":5670330804396842703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620145, "value1":0.27733408115115116, "value2":4537754734930200229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.861443, "value1":0.7565577041788646, "value2":216193021440411568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584594, "value1":0.7717911844132257, "value2":7036437829052540741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.066975, "value1":0.07703464444888349, "value2":8332259918814404840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272133, "value1":0.2064168890828292, "value2":1598424380451243838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725661, "value1":0.5107106855646196, "value2":2095325894892271000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.992271, "value1":0.04606905650348332, "value2":3458383290691046043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988671, "value1":0.9012802494548691, "value2":3265213363984040214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344354, "value1":0.6315603617727772, "value2":1819854718616345426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198753, "value1":0.2443944990728996, "value2":3740319967339712109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294599, "value1":0.7030675431685813, "value2":5821390349436487736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.233177, "value1":0.5520329333372563, "value2":2208761416514593046, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.369614, "value1":0.2829108305094591, "value2":2978362585536403835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.484777, "value1":0.3351273859753424, "value2":1348038406704883991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511345, "value1":0.6897076794113861, "value2":3140482719657458044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780083, "value1":0.056509953898858746, "value2":4983435501891849142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.023345, "value1":0.08633016397064507, "value2":439606973167008387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013043, "value1":0.8750924151560185, "value2":2563435998029443275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929738, "value1":0.6942458871164995, "value2":6742189975038174280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348725, "value1":0.6477096282239411, "value2":614033611645126767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.148404, "value1":0.3122137082601719, "value2":5921845943066980452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_9":"e","key_2":"d","key_3":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.892407, "value1":0.0024857877019035433, "value2":78557787569325314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770110, "value1":0.4854945232954409, "value2":4441111957237018619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969650, "value1":0.028054363655575613, "value2":6620021166465815802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096277, "value1":0.7642864471977222, "value2":7079078098912113925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104248, "value1":0.17068081288522707, "value2":2914828831179785715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.033230, "value1":0.6691862134057828, "value2":5879037419816229961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442293, "value1":0.918687680755694, "value2":3522143514446827859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340520, "value1":0.3316554635970253, "value2":3298849581857165314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.005701, "value1":0.22902534413133244, "value2":4040792449962896607, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.101103, "value1":0.4459003073823555, "value2":8364403783112020861, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.269361, "value1":0.9829136286924077, "value2":305552246857799140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002507, "value1":0.10735284014448321, "value2":2548523454772398525, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516501, "value1":0.3420030652884641, "value2":449036192165412633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914771, "value1":0.6948526439267081, "value2":3274351745805138467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151382, "value1":0.9010351248418295, "value2":6225882894853282790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_7":"d","key_2":"c", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.797287, "value1":0.450517653615541, "value2":6406142276072623664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575226, "value1":0.6947969923557978, "value2":4732050542115525893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.256780, "value1":0.29401472322197164, "value2":8790173514967991405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791599, "value1":0.6204111480891676, "value2":1162220986630177027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615289, "value1":0.17893814852804246, "value2":3187373261649831292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738034, "value1":0.9979754782079555, "value2":6006267840987043849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735591, "value1":0.011768322537597629, "value2":7974963296156276842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.751213, "value1":0.43700619365470833, "value2":6294325004534159296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781793, "value1":0.6112369782572161, "value2":2263959578091534198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118524, "value1":0.4979095180792778, "value2":5142183829715514199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.574787, "value1":0.8076799963798312, "value2":1975720601465651847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814750, "value1":0.45891950977715895, "value2":4074714357288466936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579803, "value1":0.25446438731696874, "value2":7191932871100542216, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722158, "value1":0.19082711905600802, "value2":1422914021938617784, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342798, "value1":0.5097927561705997, "value2":2281093093995561768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164991, "value1":0.06555881689103157, "value2":7152427537795186892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.316516, "value1":0.1580165967231405, "value2":8769573796829946768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548667, "value1":0.8164945933097875, "value2":1223332369807607717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174435, "value1":0.5629514369055348, "value2":2359802095953741925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.561139, "value1":0.9847446485189013, "value2":8748242508292199474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604206, "value1":0.7084164989246987, "value2":5662112818525056068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.069641, "value1":0.9392432577384211, "value2":7977134301111651930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977024, "value1":0.8770253339233329, "value2":114419076017958040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.981104, "value1":0.0262761349528985, "value2":3404389245833544901, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.309389, "value1":0.6700115572733264, "value2":1746800498285488694, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907890, "value1":0.3847355476155844, "value2":1062772964889350348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.557849, "value1":0.8423014863985597, "value2":8693557035023467587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174807, "value1":0.14044078429336324, "value2":2899359413194652810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907265, "value1":0.7213413890977288, "value2":711482054709798828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730322, "value1":0.010569544047203483, "value2":2272852869521847556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047968, "value1":0.8749328675131659, "value2":3313417752475725526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548996, "value1":0.13476734433713453, "value2":5686844894034380593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090199, "value1":0.47038291207998856, "value2":6737581800965808102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372121, "value1":0.4982647921591242, "value2":2960213658998005236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.732328, "value1":0.6794810773912932, "value2":5614573251611720393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195822, "value1":0.7817234737401128, "value2":6738900276029001186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_9":"g","key_4":"h", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173099, "value1":0.7620323539766012, "value2":6195639668903126539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739085, "value1":0.8805144351999263, "value2":6990390850908433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851839, "value1":0.19515034480983603, "value2":5022120185556407326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.764811, "value1":0.07516380443788656, "value2":4953415333432850814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.954192, "value1":0.6068771093686028, "value2":1921210459347583907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.501412, "value1":0.20889031917804166, "value2":7266160086492287738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798358, "value1":0.44429041626860627, "value2":7311639409429341020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200103, "value1":0.5998566082715615, "value2":3968426630272414916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443243, "value1":0.7996897987041263, "value2":2955385724626539485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808129, "value1":0.12934958318030565, "value2":8640009016586784249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890136, "value1":0.395437996374284, "value2":1375026123479284573, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.726245, "value1":0.8383327509398333, "value2":1069495836938595505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464465, "value1":0.7522623057444222, "value2":3745372652823842076, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958198, "value1":0.6062226980863115, "value2":692634818001600691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336639, "value1":0.3569530703633769, "value2":5114963101620697516, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153507, "value1":0.29997973310668347, "value2":35895264299031809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496206, "value1":0.16796883917469935, "value2":2457448015461697922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699773, "value1":0.1320495753882598, "value2":2684330380511330426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_6":"i","key_9":"b","key_3":"h", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086489, "value1":0.01686874394925538, "value2":7266008503964487472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.755161, "value1":0.12039999679109185, "value2":7441080332932356910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481861, "value1":0.21023924595248417, "value2":7215904118134021296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141970, "value1":0.6959505249259283, "value2":5593000831636710131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473262, "value1":0.20484780325876686, "value2":1515840223895186260, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.610674, "value1":0.37794171859625975, "value2":7210398121561053631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784669, "value1":0.8314547953224727, "value2":6889739271959092821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348279, "value1":0.9806070629868892, "value2":6665537531662525499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_7":"h","key_2":"i","key_5":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342228, "value1":0.8292222808866697, "value2":8140150785342027171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049991, "value1":0.20259901011770426, "value2":860224417352173376, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442752, "value1":0.9271842959401567, "value2":8958731671751036609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288552, "value1":0.19606129029896613, "value2":3803062421310719257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_4":"c","key_5":"c","key_2":"h", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851767, "value1":0.8040030606139774, "value2":5022190100385685639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258187, "value1":0.06995124309795013, "value2":3290080872961653036, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967676, "value1":0.08273649902951419, "value2":5115999717780317436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.972416, "value1":0.3332024948916442, "value2":3195696687672274736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273980, "value1":0.9801180643314533, "value2":7124826086691987600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962051, "value1":0.7078535280449856, "value2":2450804349449720576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_2":"e","key_6":"f","key_1":"i", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342560, "value1":0.21100802753268208, "value2":5026066029722000095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582068, "value1":0.9724895772631098, "value2":9045330876857967867, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716662, "value1":0.5239131397726235, "value2":6699936731993788174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.630988, "value1":0.8757044551570439, "value2":6009726873690078427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.617422, "value1":0.029531083291305976, "value2":7932542565063790762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389736, "value1":0.40122875818931175, "value2":7350974819717892233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.470743, "value1":0.286118127257554, "value2":2903264335442728018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_6":"j","key_1":"i","key_4":"i", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118668, "value1":0.42900363662380264, "value2":2786819190490835132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819243, "value1":0.7832376096990535, "value2":6768946688624968401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519924, "value1":0.1784273794737174, "value2":606279348173747859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575499, "value1":0.8580592683181654, "value2":6065349113607731318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.952434, "value1":0.8198422458926743, "value2":6520729939047239531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180605, "value1":0.504162836348111, "value2":8107251100494910096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.680620, "value1":0.9192990596381249, "value2":7916466116307364506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421505, "value1":0.6014385702769156, "value2":6172295401410316312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581350, "value1":0.2695750525680962, "value2":6289934971199391350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.244031, "value1":0.7392367672457897, "value2":7438364572557990950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801331, "value1":0.014860008784846704, "value2":6873217646463864251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875352, "value1":0.3975554355639885, "value2":4390005070727485119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929440, "value1":0.40393323123767977, "value2":1148796038957803851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.817616, "value1":0.5647949954501486, "value2":1024684537861148034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.990318, "value1":0.25287778647718107, "value2":7093365367237549470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741736, "value1":0.023580263258422104, "value2":1518420105714291210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463012, "value1":0.7888469551975262, "value2":6786560784050350679, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449551, "value1":0.2636704318808076, "value2":7884141870151198099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419454, "value1":0.6010995928923429, "value2":224905435491328214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.322105, "value1":0.1331880379524159, "value2":7369022064377940075, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581063, "value1":0.603596835974457, "value2":4386876721702326915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.857080, "value1":0.5842046650044923, "value2":1066043362420191253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191051, "value1":0.4639324153416964, "value2":8874234446483833192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298751, "value1":0.20423387563647288, "value2":8774956872647827067, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924593, "value1":0.0659156088635184, "value2":3634318708597769338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404558, "value1":0.7799254034556268, "value2":4510574800615418821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962467, "value1":0.8168896478129019, "value2":7986325005390816690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622579, "value1":0.0904343638172895, "value2":8283142617455012372, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710038, "value1":0.6909066964841821, "value2":6203472414968382073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032660, "value1":0.21417322011190834, "value2":1054486238769744890, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801225, "value1":0.2765954328390815, "value2":442385272245544502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116566, "value1":0.6031158754932586, "value2":3770751187916956934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.284086, "value1":0.055471202806315845, "value2":7266763851735180811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559271, "value1":0.28967037620921054, "value2":6147439427772857051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224654, "value1":0.5513198770455077, "value2":7761756752100166184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575799, "value1":0.07079456843554317, "value2":4847399039818291257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819125, "value1":0.4806719127445899, "value2":1129446367168224920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730720, "value1":0.6013248235298089, "value2":1610990221268346838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.593103, "value1":0.8196760148078414, "value2":6644540894797627889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.218509, "value1":0.5227005302649587, "value2":2525326877519094680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.292767, "value1":0.09698193120132978, "value2":8960471678202836654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858612, "value1":0.675005592095046, "value2":6028830951911645471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736558, "value1":0.24026583264924886, "value2":1360237166500303266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826970, "value1":0.3893255587955024, "value2":2292740904231475335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128577, "value1":0.3146179907253604, "value2":164469213437964556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_3":"i","key_4":"a","key_0":"g", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003953, "value1":0.33225441021352176, "value2":5713576701192138110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352438, "value1":0.12247814806052679, "value2":976940527664803795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.434520, "value1":0.6138506637398143, "value2":1920264902732481602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.731843, "value1":0.8305061213621728, "value2":4607426852903927950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_7":"h","key_0":"c","key_5":"a", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032970, "value1":0.8273657553543087, "value2":3103821801573462769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833737, "value1":0.3431117940438562, "value2":5529881494203381864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.206561, "value1":0.03180903277463739, "value2":1534192978519074976, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598146, "value1":0.7943659660871121, "value2":2512296846803666990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.830318, "value1":0.9077939825338383, "value2":1289520021940183737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409047, "value1":0.694416059529231, "value2":520453313543014186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.949791, "value1":0.6369046983298985, "value2":371741091848582403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354931, "value1":0.7633140731862355, "value2":8981576909845465787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.061767, "value1":0.7699262325323719, "value2":79439898840104021, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.941133, "value1":0.26448399729496197, "value2":6448749789711445941, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.339821, "value1":0.5246395493337294, "value2":44122001398655312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359654, "value1":0.5954751506379091, "value2":1668054016792098588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.304719, "value1":0.17854657722051143, "value2":5240836642486639588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533350, "value1":0.9102023121053504, "value2":5223044570382986189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.495583, "value1":0.8380393694454433, "value2":2826946501048985900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_6":"c","key_0":"f","key_2":"a", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925864, "value1":0.06057194714527472, "value2":4827880370625631466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200061, "value1":0.3196222334980913, "value2":5536801387533970020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.635607, "value1":0.5926951079597541, "value2":4317404702469315811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041722, "value1":0.2038750977962134, "value2":8603762180622472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.565080, "value1":0.5851130292640377, "value2":8612148593533709646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090894, "value1":0.12207888531940794, "value2":435323305893709131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_9":"e","key_4":"c","key_8":"j", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.071849, "value1":0.7914473708034223, "value2":4833048612300779383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202326, "value1":0.6798241827808086, "value2":5321585738808934000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116141, "value1":0.8646385559973755, "value2":2431723757311861306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.074497, "value1":0.2771958266692314, "value2":2948732814035791763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.453427, "value1":0.9750934919782419, "value2":7598695164797163349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517105, "value1":0.6447380734003394, "value2":5945582855956594632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969670, "value1":0.8792534113647463, "value2":9118825598855667812, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064653, "value1":0.7641915057577142, "value2":376649068007551056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.299983, "value1":0.5328541897726483, "value2":7848443219053578824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814787, "value1":0.5822532477986045, "value2":3517444920674306021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645109, "value1":0.6788316811493683, "value2":2399412759017043209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_5":"d","key_6":"f","key_3":"h", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104970, "value1":0.5150104887448738, "value2":39162375392832614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124629, "value1":0.9415126690810315, "value2":2487867259401435813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847494, "value1":0.8236014719422879, "value2":8522949906604300533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.694932, "value1":0.5232173520376328, "value2":2865775556504465271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.824405, "value1":0.33768880558190767, "value2":1449467820646971158, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263091, "value1":0.800562879218074, "value2":4640064437824793404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.545676, "value1":0.6991321506153548, "value2":504604265126996830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685750, "value1":0.256315076419413, "value2":7805803213074768353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.955572, "value1":0.12777100392825705, "value2":4697380262379050863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.052725, "value1":0.24897441126326256, "value2":2806761823519892967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832231, "value1":0.7123036010256201, "value2":5558652604185663810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_7":"c","key_3":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.779475, "value1":0.42379979395746137, "value2":5877566589967590752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_1":"a","key_2":"j","key_0":"c", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.056823, "value1":0.7557073972848881, "value2":1462009193456216531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179551, "value1":0.33562095061974395, "value2":2905800790209406415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.946943, "value1":0.7895726694044407, "value2":4556871779015432465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.792955, "value1":0.9478490399693646, "value2":6469796413879524879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302294, "value1":0.40539394874343054, "value2":1479528610985409190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.859086, "value1":0.8533535931566633, "value2":8328258195824345992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.912801, "value1":0.4080605346123484, "value2":921699075685228177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026911, "value1":0.5160934551500268, "value2":4192638354189389027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.145578, "value1":0.4697942761887252, "value2":1007860316702817500, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224036, "value1":0.21446416617045144, "value2":2612501104515894392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.083418, "value1":0.3800566396023636, "value2":1387864613630168710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.483137, "value1":0.9606563644656662, "value2":2761802265245174240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.855910, "value1":0.606558104786761, "value2":842530041019109994, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332745, "value1":0.01260394609809363, "value2":2781673523626108476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198691, "value1":0.0952542911286906, "value2":4639530496495690336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207605, "value1":0.29598976404510996, "value2":149815254996856780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.219100, "value1":0.7173110134058144, "value2":207519709019999249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.065034, "value1":0.7683750910989782, "value2":6262446867113607884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362737, "value1":0.5652820007764047, "value2":4138085582551770816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.790498, "value1":0.45469936210178996, "value2":3874789041764013688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581214, "value1":0.8566672554079884, "value2":5701861631669426000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741360, "value1":0.7962411718913905, "value2":3357161583638308077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_9":"k","key_1":"a","key_3":"a", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.934536, "value1":0.07212743130410183, "value2":6319820377917318485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258971, "value1":0.09357358589212993, "value2":5081900353713330302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.436226, "value1":0.5553015007542973, "value2":6124312980620622172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566508, "value1":0.7132604990750537, "value2":8485961090508027563, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298396, "value1":0.4460940570451377, "value2":5709626152776289179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596697, "value1":0.3774598732286396, "value2":1819820653928002535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.758549, "value1":0.49680425119422045, "value2":1371518397741898898, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151965, "value1":0.9897517149153644, "value2":3974511699577597667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.223787, "value1":0.9859897177824236, "value2":6368178524733411270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699805, "value1":0.9240609624872184, "value2":4091684677817576336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.640771, "value1":0.3605713604653968, "value2":2196974552050802581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917234, "value1":0.3769625716544593, "value2":7094294232648752669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541154, "value1":0.07721056009394252, "value2":2450341303960944649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188174, "value1":0.511305577711157, "value2":4677738772360990938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277439, "value1":0.006756936522771483, "value2":2736398221812059244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847832, "value1":0.23362336838695744, "value2":6292362019324286334, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.840677, "value1":0.6137390254843741, "value2":6506004284907452986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.397331, "value1":0.5291020730916409, "value2":766558029100390419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.668711, "value1":0.6677953904066025, "value2":2813624786174976455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.728127, "value1":0.9455687108064199, "value2":2598623203001035108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.260429, "value1":0.5485107722176485, "value2":8238880717060307339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407939, "value1":0.7503222721986553, "value2":8088756690439902034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016860, "value1":0.6281421773683825, "value2":4463756144226416929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.592662, "value1":0.7215969122220701, "value2":2957728488172560734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596916, "value1":0.6365196560861767, "value2":3939747841173152393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.650813, "value1":0.33343709461482135, "value2":8822031185214434530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543088, "value1":0.5660730417813042, "value2":6651394201039088708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.416422, "value1":0.2907903630149851, "value2":5811650096165674198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505086, "value1":0.44660978158387593, "value2":6270587282710629926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.055558, "value1":0.394254430161103, "value2":9153429600073084016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.578079, "value1":0.6160395423413576, "value2":7388869261196091596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.347142, "value1":0.28357580684654937, "value2":7651284402887473938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089369, "value1":0.24785364470389606, "value2":2827626134616355884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_2":"f","key_5":"a","key_0":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922968, "value1":0.9908293125287131, "value2":5006327563319926460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489701, "value1":0.9764768678976257, "value2":3780590525108581330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700790, "value1":0.8524762188330061, "value2":2298426148247437198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.884155, "value1":0.17838161460811935, "value2":707461345329966226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_1":"e","key_7":"b","key_0":"k", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.877319, "value1":0.597182710013635, "value2":66780576193296735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906533, "value1":0.44570088765160515, "value2":1159099942400157753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555879, "value1":0.6225620185948739, "value2":7381136800707660104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.498993, "value1":0.9016213008618695, "value2":8432726482609273880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349195, "value1":0.16850895992044404, "value2":6332393888509943818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530337, "value1":0.8558953203814225, "value2":76710422944574508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345065, "value1":0.8230787557550705, "value2":4201898391845439571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947222, "value1":0.9688974381514411, "value2":2441269560109987314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.287425, "value1":0.20362983597374928, "value2":5349125652009205097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519644, "value1":0.3548258293385043, "value2":1594647307470342039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_5":"i","key_0":"j","key_1":"f", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.027626, "value1":0.9120926865897546, "value2":356446699135295995, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.054561, "value1":0.22396147232559485, "value2":3930608769790828524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.388931, "value1":0.5101134733621842, "value2":6568513065035568570, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567065, "value1":0.79529931520039, "value2":6002480040724485306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449050, "value1":0.9291008790205846, "value2":741995070657009132, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798788, "value1":0.9907520697579564, "value2":2343010721096789225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.927945, "value1":0.8814555609090619, "value2":6729196069806990782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917877, "value1":0.15366203128840322, "value2":2435285873402812217, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.301622, "value1":0.20876721045801763, "value2":1403445590939959710, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.745719, "value1":0.6060177757203955, "value2":2370675136825151538, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156169, "value1":0.40315211694606023, "value2":5606845635294528121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.651978, "value1":0.02572219217737101, "value2":6216284509197748039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.395139, "value1":0.4352070266196555, "value2":2437710317158637331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.132661, "value1":0.3845242564807311, "value2":1179550652058134811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_7":"j","key_1":"k","key_5":"d", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.088075, "value1":0.27124710684397163, "value2":5954367646517477044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_3":"g","key_0":"e","key_1":"j", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213664, "value1":0.9844242730115798, "value2":1430390936289238362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070775, "value1":0.6998833636894136, "value2":8373393772820367148, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213733, "value1":0.44329246485521806, "value2":6509431828213171835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608249, "value1":0.15493446331886743, "value2":4384868943457502322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.956693, "value1":0.9523530599130996, "value2":365504939691815078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.432058, "value1":0.16692183631193722, "value2":4864357898977097624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.303592, "value1":0.934106512625592, "value2":7515104000227729499, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333787, "value1":0.6736717480454248, "value2":5830960931165083327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359385, "value1":0.922695683809537, "value2":6703804740319670530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.211845, "value1":0.13745320161077518, "value2":8688931964841266452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600304, "value1":0.365028686112319, "value2":4172510140835977605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045527, "value1":0.1293058694163136, "value2":4396801118892738860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124929, "value1":0.2927998769649342, "value2":7840410733128704268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.708707, "value1":0.42917010198000677, "value2":190549078586280269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543921, "value1":0.45395915446447055, "value2":4038857743678185292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001958, "value1":0.8799882402246043, "value2":8959405737069039234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834357, "value1":0.9787241750393116, "value2":4713255732772978232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398402, "value1":0.3356690698861162, "value2":8600435717396946203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.532477, "value1":0.5790086501753019, "value2":585720405314800565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995284, "value1":0.019519382747907175, "value2":561302586535494029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.492003, "value1":0.22729751026433825, "value2":2059445045999328988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.357595, "value1":0.2653305071097244, "value2":7325567281409701416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423155, "value1":0.8492553718976581, "value2":5551377313853459727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582438, "value1":0.9616765075659243, "value2":1026983801153932760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409214, "value1":0.1250667617190597, "value2":5921255804215527024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744368, "value1":0.8513042177093529, "value2":6603153879156409175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970658, "value1":0.2306199750722779, "value2":2906238010168281574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_6":"c","key_1":"e","key_4":"d", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520199, "value1":0.9665587748785213, "value2":2458054282507819610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.098618, "value1":0.12617188969368395, "value2":2230070381051915192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279517, "value1":0.4244894688924968, "value2":7931154210918628962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394850, "value1":0.011258876506077731, "value2":3335326787820059838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.672924, "value1":0.7327755721792956, "value2":3880440053684337972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.431109, "value1":0.6741923566632704, "value2":2327920765233612965, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.659350, "value1":0.11708731624685002, "value2":276841347975235584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.115617, "value1":0.43754612855223973, "value2":3365515217954889202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533611, "value1":0.8399047779975791, "value2":2365721152492621420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265333, "value1":0.7911288127976842, "value2":5955361860868770714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966474, "value1":0.3536325699896996, "value2":7032196306810796442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684964, "value1":0.8626527282208771, "value2":7197488766581698381, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559000, "value1":0.010659679933132453, "value2":4115322864019466158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455110, "value1":0.6204915686658947, "value2":6428809732964624602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625937, "value1":0.9727096027334052, "value2":6229477409536583105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.113032, "value1":0.5903167177404812, "value2":4156394072465021166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567689, "value1":0.10316863931938577, "value2":8418749122094896416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.987774, "value1":0.7873883846984865, "value2":2030919041886568670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.811061, "value1":0.4992724231992665, "value2":3986295112752729017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194651, "value1":0.2743659705657923, "value2":583251715501918289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950390, "value1":0.5427606927000886, "value2":2986321212639280881, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598120, "value1":0.20329762636615106, "value2":5612645284463886443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950214, "value1":0.4515603514606786, "value2":259527570321604761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645723, "value1":0.13300625888317788, "value2":9108039921133288290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001569, "value1":0.01777574006312956, "value2":7180543411595530153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036902, "value1":0.19075245150411826, "value2":5889621655868191557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929644, "value1":0.8218270153578531, "value2":6338216490369302155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036799, "value1":0.8121072864023562, "value2":4644698086978403677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099804, "value1":0.41836860978181317, "value2":2663944418369780756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.607018, "value1":0.04603283781595935, "value2":7178332091114199020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075733, "value1":0.9728379954214806, "value2":4310309256078899557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795422, "value1":0.6466237769859894, "value2":344630441945165516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924050, "value1":0.5005809462467858, "value2":4504593684309524057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352892, "value1":0.9539405805618134, "value2":996952878654521350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141322, "value1":0.01789552453746491, "value2":5938050187519951412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.783153, "value1":0.1436758434821978, "value2":6441131001960903605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473891, "value1":0.3495022243472323, "value2":4306013569645375103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814977, "value1":0.38372259399528474, "value2":436380091488564183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716628, "value1":0.6699570981955947, "value2":3523603923463352771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735953, "value1":0.6776358805955892, "value2":5856380565120662945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_5":"e","key_8":"b","key_0":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265185, "value1":0.11607380521583967, "value2":5582686822797693437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516162, "value1":0.2994616004522488, "value2":2261467086953311820, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334873, "value1":0.14715552224004888, "value2":8821110853892469057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.400930, "value1":0.8143295386608986, "value2":1312967392198736440, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196057, "value1":0.5352481987346313, "value2":3859881997560184862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925763, "value1":0.823842231642209, "value2":7076092787875459652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.012976, "value1":0.08033889071311322, "value2":2194666423113168625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656711, "value1":0.979250223824098, "value2":295513105593707440, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542943, "value1":0.34216024465522854, "value2":8482137672426222632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738294, "value1":0.6164314293888586, "value2":7863948057381466211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957221, "value1":0.4133406835193543, "value2":3905154346081616319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047765, "value1":0.9114235810126055, "value2":8926222646986865918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423884, "value1":0.30572384015704446, "value2":2339988500116190675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209371, "value1":0.07849561730590998, "value2":3882974907897625962, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310209, "value1":0.2535587414218041, "value2":7470349035360557369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988088, "value1":0.2879292667105676, "value2":5363951038000132107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947720, "value1":0.958399617505033, "value2":5218176225849015970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.674130, "value1":0.9617762098247892, "value2":5270132257081817867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279827, "value1":0.8686925044217091, "value2":7240549587345744089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.257178, "value1":0.28758053646406956, "value2":7938343410176527229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870350, "value1":0.16917076304978226, "value2":9111760754875505635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.763299, "value1":0.8896924750809221, "value2":6658856857804173307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_8":"k","key_1":"g","key_5":"h", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734200, "value1":0.44507702947189576, "value2":4594583173060659683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155510, "value1":0.8897408774019222, "value2":3124062956444214018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.363722, "value1":0.07362703906856644, "value2":6124606823090260320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336581, "value1":0.7586482338064128, "value2":2691495825421343596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075863, "value1":0.7294045132994561, "value2":8264827508243049447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108381, "value1":0.45788844677640256, "value2":2616045646790504337, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901873, "value1":0.08770240902249962, "value2":6640573405482547401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.438350, "value1":0.25439050102218247, "value2":7286046839968613612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.314379, "value1":0.33872622756746645, "value2":2585280362910555462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736079, "value1":0.12615475607591273, "value2":8133018761204018534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130003, "value1":0.7963354396181769, "value2":6450859077912673686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.485449, "value1":0.39497609142163287, "value2":170013329412671465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.06350907823991539, "value2":18163724507539892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.6759106102023759, "value2":8613373636446320004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522262, "value1":0.9415564313481927, "value2":183076459384497822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.456571, "value1":0.3755600914749928, "value2":1317951286539480179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713142, "value1":0.3684027597862527, "value2":6666246661014977211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332747, "value1":0.2712525974797622, "value2":6685862283475642897, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622289, "value1":0.8099425380336615, "value2":6696254947892698280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099077, "value1":0.4218807485485893, "value2":8151022600373825319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833739, "value1":0.471092674976307, "value2":4241908968763096086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512079, "value1":0.6891384899122228, "value2":2796503546861546091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.405406, "value1":0.9060026283921703, "value2":4773855974745798571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742427, "value1":0.6129480541080172, "value2":3917404693410650609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_5":"k","key_2":"d","key_3":"e", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.613376, "value1":0.3584257069399756, "value2":6068105032813229930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.759402, "value1":0.29210317562134064, "value2":3082388413784210214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090526, "value1":0.4678159387104084, "value2":1605658432958093758, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380672, "value1":0.5086572266239265, "value2":2924449619880684423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.249203, "value1":0.6593022058411281, "value2":727243123088870599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.142921, "value1":0.5973096622193752, "value2":2056855353114653381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041929, "value1":0.03439471294074195, "value2":8729944365498889284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970035, "value1":0.629094069595545, "value2":954705184774295714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656521, "value1":0.4935396027418025, "value2":1513112375773530412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286143, "value1":0.590796213295746, "value2":3085171514913296735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.611048, "value1":0.3503905353305475, "value2":5805006250107454516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_6":"f","key_3":"e","key_4":"h", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265940, "value1":0.8719050400063297, "value2":2334709836967925044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.231945, "value1":0.9482208594424564, "value2":7605532380274260726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099422, "value1":0.6387093062831974, "value2":6416034503475989540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862301, "value1":0.0844793706976894, "value2":1724212296853776891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.882958, "value1":0.45080094639121554, "value2":6596820877226155907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851802, "value1":0.784896015737414, "value2":2267862818127250105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.871761, "value1":0.9219727566018483, "value2":9044274636085577036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704109, "value1":0.6618033501935818, "value2":8585487696070011938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.863670, "value1":0.6882846261541673, "value2":5478374167595454238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183651, "value1":0.21762989261712523, "value2":2054870959404109040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157246, "value1":0.1072057187986972, "value2":2474773268716838947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263088, "value1":0.6424740570966222, "value2":4311694979316615362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227492, "value1":0.6060012408935248, "value2":8262744174339677603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919259, "value1":0.26635737492611994, "value2":146832998544852566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_2":"e","key_0":"d", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.810229, "value1":0.6433893313883111, "value2":6020303671131449049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.021406, "value1":0.5663729785218883, "value2":6847532330100114835, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.331141, "value1":0.03123867674281506, "value2":2370510435982901478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313836, "value1":0.7842831796528092, "value2":1464971588517507800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_6":"i","key_0":"j","key_4":"h", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147234, "value1":0.232833406436683, "value2":5302769737190544079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423280, "value1":0.5646235246180139, "value2":3807301011411244203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.872535, "value1":0.3508318181055984, "value2":8035632162257252295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516267, "value1":0.7376190499237748, "value2":1779371132519351560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.020480, "value1":0.5129193623789473, "value2":6378565762108475442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421514, "value1":0.72712434334131, "value2":3908844968753938444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026297, "value1":0.8255820903801332, "value2":5180017205009891488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138610, "value1":0.5467990020300735, "value2":8134249053330675938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754861, "value1":0.5832263769294902, "value2":8216858329154999597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655676, "value1":0.45804041306603793, "value2":8403704610668639637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.091295, "value1":0.7313452902241809, "value2":6274232113944917429, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370397, "value1":0.7082796943852117, "value2":16646752364177844, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.072185, "value1":0.8009986319191017, "value2":6123195610574092125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068380, "value1":0.3124676060355689, "value2":2569282749677030904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013897, "value1":0.9321729707819904, "value2":2281258981052686264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.963470, "value1":0.9927608945020866, "value2":6050801587231871156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481229, "value1":0.3689051456876514, "value2":1924733693360792633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.295891, "value1":0.44913357839758283, "value2":8069037361024844952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_7":"i","key_1":"h", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409485, "value1":0.18423629567656025, "value2":5066355149165531813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744404, "value1":0.9737949013567385, "value2":5174295290317701561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739701, "value1":0.4976173273573324, "value2":5658855721642744182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.186283, "value1":0.419361633943128, "value2":2053614790636840527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566841, "value1":0.7094596520198607, "value2":2148895338889314160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.215921, "value1":0.34276587375112927, "value2":1645566772308625552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.701909, "value1":0.22008909623955855, "value2":2295629303145372991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002697, "value1":0.1765088617877308, "value2":4201797060324467420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354799, "value1":0.6538892310533279, "value2":4577607391337663337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345385, "value1":0.3843950816957309, "value2":8800458811056296695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988457, "value1":0.5588868714021769, "value2":8433496452370061074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415009, "value1":0.1746188760147135, "value2":6912110645446824396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655363, "value1":0.9949350156565988, "value2":8110347995943103660, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.242069, "value1":0.48932951486152193, "value2":8286237027514157124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085368, "value1":0.0689091306398061, "value2":5859648999386501040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_6":"e","key_3":"j","key_4":"d", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.661208, "value1":0.7864155291083853, "value2":7826276349217880531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.803222, "value1":0.6015539667825235, "value2":1102772758151131198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.458180, "value1":0.3809696927779604, "value2":4439664400270353254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334077, "value1":0.4097523501895673, "value2":7454062761319263955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.885572, "value1":0.548188359160327, "value2":3020054439494862697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489119, "value1":0.6763563343582067, "value2":4242764599158368849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.187849, "value1":0.7091709802238197, "value2":6867639316908364367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663732, "value1":0.42631474518610635, "value2":9001795643794270349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.585010, "value1":0.6237338911978378, "value2":4365340422086338410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.379644, "value1":0.15387674044669916, "value2":818900165755081001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791337, "value1":0.32588514831554516, "value2":1691735796263334884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.082385, "value1":0.5745615587827129, "value2":772277205802820637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086473, "value1":0.8230348349086334, "value2":8156169851471463456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808463, "value1":0.47620124507643474, "value2":2357967788477712632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380686, "value1":0.011323528011005758, "value2":2028993392634762324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.371579, "value1":0.5028955605922396, "value2":7910467950096619453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.883127, "value1":0.20067026957438594, "value2":3625887507362062386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.305224, "value1":0.7123715236402623, "value2":7522375891893442954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443461, "value1":0.7555458976714571, "value2":655917858760911221, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901204, "value1":0.7387538487064921, "value2":7525092307824794219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.376917, "value1":0.7993175855480273, "value2":2807412127532185201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_6":"h","key_5":"c", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740075, "value1":0.3001865006906782, "value2":7190874681083752315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209665, "value1":0.89276344594564, "value2":4778320938543530902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.341730, "value1":0.20200750378616714, "value2":1664606777941171586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.629099, "value1":0.017583528638071864, "value2":2707650519412422348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057519, "value1":0.04492925365315839, "value2":2578667952703739482, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.723798, "value1":0.8454916194785576, "value2":1835422305385826626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.034767, "value1":0.201322613886975, "value2":7356183312830760350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_5":"h","key_8":"a","key_1":"b", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520001, "value1":0.5251237247182675, "value2":134900877343320532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489083, "value1":0.8513994038065534, "value2":2159556869883772494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344496, "value1":0.16316310680106225, "value2":5518614284082110727, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.724230, "value1":0.6875155435793379, "value2":35737216326475935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568987, "value1":0.5138501785175705, "value2":4338888431721237426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598387, "value1":0.4390074014236514, "value2":7041842749721887181, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.626270, "value1":0.2391917526821926, "value2":3091537573368363672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.297931, "value1":0.11690748126426959, "value2":4039852836857346610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.175994, "value1":0.6789300486469183, "value2":1246840956922367253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976992, "value1":0.43367432666147043, "value2":564959136229688173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703741, "value1":0.47707265609489413, "value2":4068705543834170412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.235944, "value1":0.8012892224294691, "value2":1988300147401692461, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499512, "value1":0.9990473134151273, "value2":8394845441054007311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404130, "value1":0.6068400138809874, "value2":1935348003856793271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542560, "value1":0.12776285353397152, "value2":6795264619125251375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693012, "value1":0.2569652780369043, "value2":1475809083419006811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068970, "value1":0.3881338979751516, "value2":1218503134871756052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264926, "value1":0.19791850227382649, "value2":5513222623463327397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.408769, "value1":0.39494174800117104, "value2":1576673230492191692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.081892, "value1":0.9402614771700961, "value2":1589947956431908818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.058425, "value1":0.5327215264224379, "value2":445308327885517649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_8":"d","key_3":"b","key_4":"f", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.838442, "value1":0.9519819634366921, "value2":745087205152711718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064037, "value1":0.23777264651361862, "value2":4675095342249511184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.518857, "value1":0.8856046396694535, "value2":1109051615753148759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077463, "value1":0.8624901705258308, "value2":3793672263940517136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.921342, "value1":0.9049145350717305, "value2":3570805638374912588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157723, "value1":0.6223760989748452, "value2":2996869515277202402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155659, "value1":0.13854963250628805, "value2":6512544911629959850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977987, "value1":0.036771554508911904, "value2":5994549505435027834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265064, "value1":0.3403489462496419, "value2":2814929460502496771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404153, "value1":0.3208642154389825, "value2":5256172233350798960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.312852, "value1":0.011902750850203012, "value2":5526100237606282605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.491995, "value1":0.46992302986974577, "value2":157557322951942167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704212, "value1":0.9425835794283676, "value2":915577115687321604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.576745, "value1":0.7854877804515764, "value2":6382074091209031167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070556, "value1":0.18030779510661127, "value2":6308540602135606131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542532, "value1":0.3384327861708296, "value2":1613795669513593265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511484, "value1":0.8996232864271185, "value2":7405365074827637694, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_3":"i","key_7":"c","key_2":"b", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.562131, "value1":0.10596222820549657, "value2":3306812514398053557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.391786, "value1":0.20647295311593183, "value2":5486022828508870343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466143, "value1":0.6224930113197604, "value2":8535178573473477072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423074, "value1":0.3088852292542975, "value2":3166186037825297943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.762488, "value1":0.33490363470863166, "value2":2912130613463378266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713117, "value1":0.2993000272696047, "value2":494302122350368825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028636, "value1":0.6503466890929638, "value2":4759183611197798725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404461, "value1":0.043678508683084524, "value2":6408629579798949638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077439, "value1":0.2418209601880771, "value2":6820734383602297695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.624831, "value1":0.4718269105814766, "value2":4622812711240963076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053083, "value1":0.6875169297990352, "value2":6119677537083039891, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345042, "value1":0.7324917812164005, "value2":5511749504782560024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.623531, "value1":0.9320016848414698, "value2":744369719732623498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.182404, "value1":0.9674131847138503, "value2":3838480249361897923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.696213, "value1":0.7881956638078709, "value2":4371203118864420550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.628037, "value1":0.17572736986644139, "value2":3461303046815504013, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273728, "value1":0.6916742591847842, "value2":2382468702478522273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335163, "value1":0.9758093684084915, "value2":1680436716344950339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782020, "value1":0.1537134689497552, "value2":4414043374214499577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.134363, "value1":0.5083067111414208, "value2":4982306711291417821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.865817, "value1":0.9406393737656967, "value2":5857776137176686892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716601, "value1":0.1916346431757554, "value2":5910562859970195422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.427366, "value1":0.03546691845658599, "value2":7315776376647279090, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950000, "value1":0.5369396780416313, "value2":7524727304557507728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.325387, "value1":0.06313877213741768, "value2":7980198837673375086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950927, "value1":0.32484655923390965, "value2":8297079571694626407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832701, "value1":0.8101399444767259, "value2":6277528961017233439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969582, "value1":0.5893386387714608, "value2":8824793605581596113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302334, "value1":0.03653755659557745, "value2":2394591637437372554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_5":"i","key_1":"b", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463188, "value1":0.2645292258357236, "value2":7750296583775050063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.774073, "value1":0.7854839946508317, "value2":4045742754979149401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.729676, "value1":0.4706720879040935, "value2":4750321291295032700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618814, "value1":0.2230204325484985, "value2":9003120111243218251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603138, "value1":0.22960374502253963, "value2":6338685193000832935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.412324, "value1":0.4957126042457222, "value2":8735002444024394358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657244, "value1":0.08184942404758783, "value2":7470578294751016176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442850, "value1":0.7229538622773333, "value2":4724836800934702763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156841, "value1":0.7233638365572691, "value2":7858912978560139100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734329, "value1":0.3404208114361277, "value2":3050306358924697552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.465656, "value1":0.38391986849101767, "value2":307361261718394449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.445647, "value1":0.990731352209279, "value2":1207071782335047705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298620, "value1":0.11180836767709064, "value2":3870338901819904571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.095445, "value1":0.2870001965302557, "value2":2415945267551238695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.918448, "value1":0.7277791066957175, "value2":2821193899111524763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541594, "value1":0.09005789780178877, "value2":2925035824598662472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.631726, "value1":0.9959155652844405, "value2":3833701945431571315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049800, "value1":0.8110556334200572, "value2":2218497283809170432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070530, "value1":0.65401432282683, "value2":498838541277181702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736247, "value1":0.7394025858411815, "value2":7148599556072452144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736608, "value1":0.08218047603491815, "value2":3776524585186258439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003840, "value1":0.6757338653122574, "value2":379207741924258587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362539, "value1":0.11651117191245357, "value2":3906526956584011608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.539295, "value1":0.44782322310256395, "value2":2595894391836664475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392973, "value1":0.8520294340743481, "value2":7384112654505361476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.078027, "value1":0.3547792528217613, "value2":8155526662947982478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.776543, "value1":0.6604220360650918, "value2":4011373177447838470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.691748, "value1":0.4862664301051333, "value2":7696312329542980810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964567, "value1":0.11169294328511947, "value2":4727405940623359973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722155, "value1":0.3780569227165093, "value2":3316392130699376882, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457388, "value1":0.9243183084724984, "value2":2432925818286345186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.040558, "value1":0.7743604477646876, "value2":627834147679968207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272066, "value1":0.5917633264655242, "value2":5918933193525430349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200729, "value1":0.7327561219912169, "value2":1393602674371677220, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.905147, "value1":0.8568183299448732, "value2":9034660617242414820, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310139, "value1":0.802088771431449, "value2":3471155770712582770, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.821462, "value1":0.5273591260032361, "value2":5140674777677119228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.384953, "value1":0.4025027387592999, "value2":5097429126997347384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.829568, "value1":0.8045993326905381, "value2":2809186620050229002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294923, "value1":0.5644401567133956, "value2":3718732987675906525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.556732, "value1":0.48820899714237304, "value2":4078290062854715892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.176213, "value1":0.2510405463103239, "value2":635371208287508106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.267004, "value1":0.22999772540025595, "value2":1871342923142731692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_9":"g","key_6":"i", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.506694, "value1":0.4581021751822573, "value2":8794558811813763792, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_3":"k","key_4":"a","key_1":"c", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354208, "value1":0.6088901271960526, "value2":132806552362853695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516024, "value1":0.7664701528892452, "value2":107006237498756274, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_6":"e","key_3":"h","key_5":"j", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.750598, "value1":0.3217472443950159, "value2":8378919480372645762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419841, "value1":0.6663125192971521, "value2":2423610833256438065, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164890, "value1":0.2006090031725153, "value2":1844725035830445264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264867, "value1":0.42671643117928926, "value2":3438912836085300466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.430421, "value1":0.07257562390395779, "value2":6088518193551139346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_8":"c","key_1":"k","key_6":"g", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.673454, "value1":0.6691702132206785, "value2":8625848517577296022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995936, "value1":0.702118704788376, "value2":4086471613439852656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_5":"i","key_9":"g","key_4":"i", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964584, "value1":0.7747761523528802, "value2":1036051996047955870, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.948289, "value1":0.25140012327548283, "value2":788619002191666922, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372555, "value1":0.9999769483157629, "value2":7184076213289982875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.920761, "value1":0.868179497430529, "value2":1697965828697918238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372531, "value1":0.5396817815353727, "value2":7506498166097940651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.278206, "value1":0.17061135686969672, "value2":713580384695452010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600654, "value1":0.6616469498902554, "value2":774519242775197957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188862, "value1":0.5531871959368098, "value2":7422109554403592667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457643, "value1":0.7840548969774126, "value2":7827846462514659006, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655234, "value1":0.76526166835666, "value2":2375166258510616618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370401, "value1":0.021438154414782817, "value2":2771089523399378961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137411, "value1":0.5646878830454839, "value2":5968018682136017050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423235, "value1":0.36529823070220574, "value2":7162022842889925076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272788, "value1":0.219359460529438, "value2":181241653539382719, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.707284, "value1":0.3923075865949308, "value2":2782827649523891195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858145, "value1":0.688162185571026, "value2":5086226934464427172, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867650, "value1":0.6841047492247676, "value2":8098586901309655486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.902390, "value1":0.706932118893774, "value2":9086709640294623772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131998, "value1":0.8578147943558058, "value2":41651666626493230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932017, "value1":0.24496475938485004, "value2":7307212968529901562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.469902, "value1":0.3291978179419431, "value2":6674355534843744608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.715852, "value1":0.5949781040197312, "value2":6585582445453110248, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.439874, "value1":0.25098256408304576, "value2":9009893614730547675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_4":"d","key_6":"k","key_2":"f", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.775831, "value1":0.888090262607404, "value2":7565201070136116477, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480681, "value1":0.47062171981528383, "value2":4853013107516729574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.654578, "value1":0.4096044979675714, "value2":1606198625715352901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.881884, "value1":0.31666628836121824, "value2":5391861566395998267, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028180, "value1":0.2934267405568024, "value2":1598172822680744133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245149, "value1":0.09374074237151239, "value2":6863637452711576117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105508, "value1":0.7783021910779061, "value2":2669034341302440615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474623, "value1":0.5494532131881231, "value2":6109855504680847451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625728, "value1":0.471409564376847, "value2":5293699949847690348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_5":"g","key_0":"c","key_1":"i", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382751, "value1":0.7835774962833667, "value2":5926458098948297057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_3":"e","key_2":"c", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.527286, "value1":0.6322552498331545, "value2":437257708157331646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.166237, "value1":0.6386670258285629, "value2":2053216930342060527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.942234, "value1":0.7212756328576085, "value2":8527112750674030055, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.880290, "value1":0.19477642153523786, "value2":8136028599409004140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.477164, "value1":0.7162729706892131, "value2":3933534960467417574, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089992, "value1":0.47761006384043037, "value2":4843140910052787407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.204970, "value1":0.4917659716382411, "value2":8639172605312608175, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.428786, "value1":0.5707960425927678, "value2":757157967003444236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131753, "value1":0.32321264423119905, "value2":1903644540800881525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.540623, "value1":0.528958932538034, "value2":8999603816652982043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.991648, "value1":0.7098945581193735, "value2":8146175572944228409, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.523699, "value1":0.499825459180699, "value2":1724724639775860908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116240, "value1":0.6561999665129753, "value2":5656111967421346554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618968, "value1":0.2848979660832576, "value2":2049018652807530472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116765, "value1":0.8252075257277172, "value2":4775243296364010632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.410327, "value1":0.06668100857135961, "value2":2183721339943212726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.688645, "value1":0.41653256081146506, "value2":5969120448550176776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657571, "value1":0.9568582636724965, "value2":730216901987394123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043883, "value1":0.40826500145612177, "value2":6407138186851676502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474102, "value1":0.6914723313082367, "value2":469628036375832328, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596154, "value1":0.050104900538538846, "value2":1354999585856659679, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096028, "value1":0.6282483124845963, "value2":116602313358973711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568083, "value1":0.07619222667132636, "value2":6984709326920626754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.984466, "value1":0.3323790308005344, "value2":675504136493878123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553260, "value1":0.45304427452622936, "value2":8153841508229333988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781038, "value1":0.2512350719527022, "value2":1697664403294613401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230875, "value1":0.8575694561008984, "value2":1340681872774498770, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180011, "value1":0.7159258322890252, "value2":7263307563948982502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318368, "value1":0.7865256606198752, "value2":7176709659967207104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.594668, "value1":0.9818689490949409, "value2":4816922436179228865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.178515, "value1":0.9620215494327841, "value2":7544435408144975228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596427, "value1":0.7405089300246521, "value2":6613451887211746808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317317, "value1":0.8954816734384349, "value2":4023989770168105300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.624103, "value1":0.1565943994591959, "value2":5829676556068380878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.259139, "value1":0.08662232425939413, "value2":2137874058817765285, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.495345, "value1":0.7444165320555235, "value2":1512339128303545319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.873798, "value1":0.10364343021646215, "value2":497864294044982023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077492, "value1":0.7825627514339433, "value2":497480348120915725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.887058, "value1":0.6684575895211121, "value2":8461019606631895989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374572, "value1":0.30885911828548585, "value2":6810974248136318973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.381097, "value1":0.062395135046400096, "value2":1505084432772472894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796962, "value1":0.4796896579785329, "value2":5095295117314039911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_6":"k","key_8":"k","key_0":"a", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.625725, "value1":0.5385278812414747, "value2":7715803409892624904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430762, "value1":0.42060637147418184, "value2":7474778452563228226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886443, "value1":0.6738972816294144, "value2":582629852137142515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467512, "value1":0.5597247611878908, "value2":8349056677712047690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301942, "value1":0.5324107668792922, "value2":2601407711369777027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.959260, "value1":0.3409466272260335, "value2":495480232494613355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824942, "value1":0.690980654161434, "value2":73040149987568706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647608, "value1":0.9936863999178829, "value2":2595521887523034584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206529, "value1":0.7042253871626942, "value2":6786989253298041563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654900, "value1":0.04874287974341856, "value2":6080516591012621533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.192783, "value1":0.636728117953383, "value2":7620028626322794241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.757400, "value1":0.3932217938599448, "value2":4695382142084123513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578308, "value1":0.7154241327471093, "value2":8875337832548314386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025767, "value1":0.4896117284329862, "value2":7827833960384552450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695043, "value1":0.4735636518198123, "value2":5776645863491585306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067392, "value1":0.2568532613491557, "value2":4347058259737260198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145851, "value1":0.867279626694445, "value2":7767138234344286903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.491824, "value1":0.43935171533716344, "value2":2117420540525504913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413343, "value1":0.7815374349474034, "value2":7674175325619501370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408821, "value1":0.6468318207569199, "value2":3382854605353241895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864749, "value1":0.5018653748831062, "value2":3163263403613237522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811002, "value1":0.1264068238414509, "value2":8051527093894333669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922550, "value1":0.797889997967527, "value2":4244570533853409024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045039, "value1":0.6136304375492249, "value2":7079529098273717788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914685, "value1":0.6884426743061036, "value2":3608727831529868858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.429556, "value1":0.37532682108769366, "value2":1177251454911069616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779878, "value1":0.5585578516583662, "value2":5769602877819666427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.711503, "value1":0.3057843361151979, "value2":2044462261279390237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142192, "value1":0.0045630377024844515, "value2":4953964644129769622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.542755, "value1":0.236125279127913, "value2":5065525208816588827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578196, "value1":0.8156211595242715, "value2":1914083289002989297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110649, "value1":0.26711648319193754, "value2":2012127151944954874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041387, "value1":0.3776450082432283, "value2":6801275370589593726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_7":"h","key_2":"g","key_5":"i", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808716, "value1":0.04169212753399746, "value2":9054979711039988827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062171, "value1":0.7218616273097948, "value2":740018633881444642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146630, "value1":0.5003843971030013, "value2":1655397867016219461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.309989, "value1":0.11293107623655986, "value2":3732853355411608797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.333206, "value1":0.17591880468368, "value2":8123233873446258834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145427, "value1":0.43545559910718556, "value2":4862315899801656213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.644901, "value1":0.9950957928656415, "value2":4691155465042626084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568884, "value1":0.6163869833398097, "value2":9217755948904745777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930812, "value1":0.3180893474971966, "value2":8238800974641951308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.148564, "value1":0.7643083780125881, "value2":8977913426229482254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369460, "value1":0.37462703037399614, "value2":2532444456287017290, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134410, "value1":0.7407330348790686, "value2":2290478450455253969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.623624, "value1":0.23159512407081487, "value2":39715061290046297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497315, "value1":0.9059175925838082, "value2":9042312906596816530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.591581, "value1":0.9196881162354394, "value2":3577426841534493512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.229214, "value1":0.677998841646216, "value2":4155628140480575006, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_8":"f","key_1":"c","key_6":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419533, "value1":0.14115394562090194, "value2":4482838294646117015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352495, "value1":0.614805207196906, "value2":6313920398652894644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158009, "value1":0.13136879968484017, "value2":1444276749086955965, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_4":"d","key_9":"a","key_0":"c", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394523, "value1":0.390863430258994, "value2":1479817062186057883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084220, "value1":0.8798339682629527, "value2":5374208931949760111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.604039, "value1":0.6734519068343852, "value2":6936119518718549917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.407184, "value1":0.883448918218706, "value2":2783824499389725952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.987554, "value1":0.04159873743578623, "value2":6347778463017159524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015322, "value1":0.9633747370317121, "value2":886249751055268990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903730, "value1":0.2538648955540415, "value2":5257877330086190982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109241, "value1":0.3895615184638186, "value2":2398924749380096990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946726, "value1":0.8408201754901692, "value2":5956645672957756887, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.126734, "value1":0.8247534822394186, "value2":8919709316239049787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419044, "value1":0.20899742823905534, "value2":9208355419350776806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354221, "value1":0.38324817137377604, "value2":7243408993217101630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.620584, "value1":0.25505110416085736, "value2":2000013463174956675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306280, "value1":0.24053747539237172, "value2":104074303789974504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.694994, "value1":0.14347374003976163, "value2":2424092953659426107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399682, "value1":0.5974894049526064, "value2":6226843809112311375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.244602, "value1":0.4472550716634916, "value2":6661706815157471152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154170, "value1":0.15894047878423925, "value2":1910278084951118791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.338581, "value1":0.8000650933715514, "value2":932928130299819996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385782, "value1":0.2346060605687393, "value2":1779773261165382974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.925102, "value1":0.31969179251919677, "value2":2628705870105871940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783196, "value1":0.47412332370344556, "value2":770542511728920904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779626, "value1":0.6092963005254489, "value2":916209608160555146, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194805, "value1":0.5004447430389768, "value2":4252088119108151132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.758680, "value1":0.486921957595336, "value2":989531111026827083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477327, "value1":0.1584089004594205, "value2":8696347531856094472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162006, "value1":0.537289345346911, "value2":2077753746787895762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613845, "value1":0.3033370157767779, "value2":7793623951886359033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.373236, "value1":0.3796012074348545, "value2":6115758866481998084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_9":"b","key_4":"h","key_5":"e", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080535, "value1":0.47198866208030016, "value2":6615627174770526167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.401469, "value1":0.5986326083333331, "value2":2667038551831095428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605284, "value1":0.7390739400172244, "value2":361713221830612946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.543913, "value1":0.568574574470533, "value2":8152708069238595670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262409, "value1":0.058861415000961634, "value2":5954875590777239919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056599, "value1":0.20694947175492623, "value2":3664736464652868067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.557868, "value1":0.04995052697654973, "value2":2073326631524402431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613501, "value1":0.31045156583158495, "value2":8926511362033688194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.047167, "value1":0.7509456855802353, "value2":846763672875326934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362486, "value1":0.30051878680197913, "value2":8317722150065418470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.669592, "value1":0.6148907714481391, "value2":3070072928838789844, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061749, "value1":0.4026807862923137, "value2":2054834671958607537, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_7":"c","key_8":"e","key_1":"h", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077647, "value1":0.02456732222252374, "value2":4163774252593358275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439333, "value1":0.49417199138729123, "value2":8947661323810810369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.592816, "value1":0.25731799708888187, "value2":5220216189538554324, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.344893, "value1":0.0022390888763635323, "value2":6178831795736016910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_5":"i","key_8":"h","key_4":"e", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350091, "value1":0.39159828262162255, "value2":3395049172404740462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417133, "value1":0.19779968591074149, "value2":3982459725160448681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313282, "value1":0.5954778061697951, "value2":2609201495243589975, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416054, "value1":0.6479875646988947, "value2":6379744126737729174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416195, "value1":0.9737347842892922, "value2":3437414765355850053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.351132, "value1":0.5274165691272555, "value2":5569886633354264071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273952, "value1":0.4616579953521536, "value2":4410692243343844144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948309, "value1":0.21495634970202532, "value2":1337568920394438603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.120015, "value1":0.6827231094645871, "value2":4910087454372528211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.220281, "value1":0.5451953602332654, "value2":5295885271016141901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222991, "value1":0.38652141757520153, "value2":611291833871236608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432311, "value1":0.4952506069055437, "value2":6235358954173265246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.305907, "value1":0.7399971630668104, "value2":7853724941674753946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.701437, "value1":0.07448132935373361, "value2":1676692481709109964, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233939, "value1":0.9922833997352541, "value2":8485630674384935881, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.196307, "value1":0.3811609484351306, "value2":6762437704329339708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.598701, "value1":0.7218662020742757, "value2":5746405219362751126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535319, "value1":0.470769974287709, "value2":5160211896753607127, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455357, "value1":0.41061237500976755, "value2":3731616592511902245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682037, "value1":0.02412728786773881, "value2":8461839080730070447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618426, "value1":0.6208202835196076, "value2":3667062185468357188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094952, "value1":0.9317381860791135, "value2":8865888160428380876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295576, "value1":0.034878221959591026, "value2":4553331116235737600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605252, "value1":0.5687954362360221, "value2":4628895813773640059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.167973, "value1":0.8144348491443895, "value2":3884298273747196831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_4":"a","key_1":"k","key_2":"h", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803115, "value1":0.14234455011711838, "value2":7333273581911274613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809534, "value1":0.31251923098668316, "value2":5000940696241059800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.504196, "value1":0.37041029395255115, "value2":7518926478801994928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886399, "value1":0.2510698129904212, "value2":1686947485208902024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322861, "value1":0.6398715658295313, "value2":5260107554586637670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.575248, "value1":0.8821568194363127, "value2":2676120924049029771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.924938, "value1":0.6642600755588765, "value2":9154692137110805453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534144, "value1":0.2220263150896839, "value2":7296067315122241408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680757, "value1":0.36579467004129973, "value2":5336171210655722369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.361476, "value1":0.42049076121425283, "value2":3694543289414649508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.423620, "value1":0.9226817255397154, "value2":5312418306728579961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.398746, "value1":0.4448382205324241, "value2":5199222544329766041, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.492975, "value1":0.6490408315820692, "value2":3620354939805403786, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.732787, "value1":0.26280435119702705, "value2":7186318610805553949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385151, "value1":0.7991171293820531, "value2":1271579952216532684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067778, "value1":0.2653370821341721, "value2":8977665444967590904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941069, "value1":0.7995772580327314, "value2":7272267767125923064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.547209, "value1":0.7653567065104109, "value2":6328078604409399184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649757, "value1":0.49061605503159605, "value2":5991812000082844933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.017745, "value1":0.8857176370602151, "value2":8890483656354164816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478915, "value1":0.49344955581785127, "value2":6898479319738793663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.600610, "value1":0.1145004603393322, "value2":7934326719175266154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752045, "value1":0.7917363851798865, "value2":6655960268552880542, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.198446, "value1":0.8915785759457425, "value2":6424163196395537238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415881, "value1":0.6309493378355412, "value2":623919229201632317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.652344, "value1":0.6344286572245156, "value2":9189348363675107233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774566, "value1":0.042498471702267726, "value2":6073051427209338110, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.918626, "value1":0.4530903546971259, "value2":5667043007795158116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.340057, "value1":0.13032389853235377, "value2":2621616876191250318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363370, "value1":0.25548611447808117, "value2":7869413584380375849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316114, "value1":0.7913570973054609, "value2":714341173131110534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.726210, "value1":0.6340822577426605, "value2":7078070670524096081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809728, "value1":0.31189335942918994, "value2":2593764393145326151, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828156, "value1":0.2513167249865656, "value2":4199156936630630581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074063, "value1":0.6299866633999338, "value2":7375082593010468605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551341, "value1":0.5634049844701874, "value2":553811382543796103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395240, "value1":0.9163098471877911, "value2":2102676899677978041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812285, "value1":0.5658853008127968, "value2":860920050673825541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.699745, "value1":0.6686081106918065, "value2":4484995258456828647, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862811, "value1":0.006750603964471356, "value2":8966268587906792618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649656, "value1":0.33455087075476775, "value2":3638358249838833627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978094, "value1":0.9765011127569251, "value2":5363905042672157366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824326, "value1":0.5852292268881885, "value2":8525521016560267029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972834, "value1":0.2713220476194647, "value2":1730194033904802699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.513347, "value1":0.20240183224033417, "value2":3586323241046235606, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_7":"f","key_9":"b","key_3":"a", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165801, "value1":0.25754092027876374, "value2":3897618512711030520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245034, "value1":0.276430555426299, "value2":7235576154009687664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209920, "value1":0.33218423022426385, "value2":6956800146565179953, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802797, "value1":0.9498551679797749, "value2":4204819498986608868, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.002994, "value1":0.33304452518131644, "value2":299998551436331074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.136076, "value1":0.58569890261748, "value2":8821512503402703105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605724, "value1":0.3779582962124763, "value2":4381826070482821493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.664393, "value1":0.7508643438704989, "value2":9133405472026903918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236332, "value1":0.4456796524317141, "value2":3982698031313790805, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320127, "value1":0.5073225371595068, "value2":2378617268397591284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865511, "value1":0.36097655561074876, "value2":2650611720712784810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777541, "value1":0.27613383935735863, "value2":3133546266043557517, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647086, "value1":0.09971158325173911, "value2":7192486223907690883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_9":"j","key_2":"f","key_5":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.172420, "value1":0.6050664492556429, "value2":3816871624777837698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639560, "value1":0.40402358999541305, "value2":7105433557941999346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.945205, "value1":0.38392410498955126, "value2":1178256579293465746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277675, "value1":0.3660884472168037, "value2":6046887408166693464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219346, "value1":0.5871741254566942, "value2":2616399527910270141, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116739, "value1":0.7713481330592996, "value2":7718152730102741547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.951685, "value1":0.6692969608681443, "value2":6198709686728043657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.880602, "value1":0.3938915734742536, "value2":5600053552458054507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.831337, "value1":0.47065388482990184, "value2":5918647530397871720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865810, "value1":0.9557189505427555, "value2":1533596579704888254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578483, "value1":0.3164962822621824, "value2":4784731554506893709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158191, "value1":0.36054084773482775, "value2":6343389237742355198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761167, "value1":0.9149570761492709, "value2":6140964050711145447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.955044, "value1":0.34237203961558577, "value2":7321019711278117639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137258, "value1":0.2734770926499478, "value2":5312006546786938240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876289, "value1":0.584796985987568, "value2":2146521016660183394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.345388, "value1":0.7891673161299368, "value2":9211813724293662180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.291358, "value1":0.5443723430932246, "value2":1305196521853642686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107992, "value1":0.33317636728859645, "value2":4303995107486054548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.065010, "value1":0.792101797297302, "value2":5624493167148885444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.512622, "value1":0.026747806334027496, "value2":31336873860845918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796113, "value1":0.3713464074931213, "value2":4041884710949108537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.794333, "value1":0.5998279188033826, "value2":2263989663686243457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930716, "value1":0.7150630462147562, "value2":5925307983835648806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.631145, "value1":0.12175047903656237, "value2":2424100894887965716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889832, "value1":0.33399829473514187, "value2":3091557798470852457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.036666, "value1":0.8679594907099448, "value2":6691229273816200596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.853608, "value1":0.5774160801209688, "value2":980449146620020072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.356874, "value1":0.6962074391730027, "value2":2178872746047471215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162362, "value1":0.6573318971529195, "value2":8364158949095374627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110201, "value1":0.41287549892040376, "value2":7444688864434046520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.441462, "value1":0.960708146788197, "value2":7272057802651917640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.020362, "value1":0.14970758198915565, "value2":5476649529178658344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434430, "value1":0.7114147585195696, "value2":6784565480587871276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795723, "value1":0.9222061229277446, "value2":5415396163282444341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.092199, "value1":0.0594949540077623, "value2":733783646287392653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277448, "value1":0.36348546035264123, "value2":2527022975440439707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607130, "value1":0.21203009019347513, "value2":3167160934083348057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424117, "value1":0.42561548619630274, "value2":439532732667459958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365073, "value1":0.8492201961187499, "value2":1130712017318335183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248310, "value1":0.6476029521632828, "value2":5228653953193583715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744619, "value1":0.5713572042259071, "value2":1575714640192742698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.026559, "value1":0.543673726494076, "value2":3134877395946459112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972610, "value1":0.09611280545479016, "value2":4984229756632609213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763906, "value1":0.8517215040951974, "value2":6804317567810698432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105838, "value1":0.8805288612544102, "value2":7653928454276555905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383435, "value1":0.3683543847634455, "value2":2147384927542025505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027377, "value1":0.4869984514514779, "value2":6620303412327477269, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417188, "value1":0.24989866283446016, "value2":7011787964949226310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084529, "value1":0.9844535394576189, "value2":6876673068454117053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032097, "value1":0.6712678103298776, "value2":3320740930339209104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394244, "value1":0.7796841432655915, "value2":4446257418849643255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828297, "value1":0.2648442239286722, "value2":25756618593200917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731687, "value1":0.07632667820503707, "value2":5775744844960229233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.348738, "value1":0.4462284223326741, "value2":7761375931867531240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817431, "value1":0.9846258074003919, "value2":5377182091927671364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603703, "value1":0.1579124346010748, "value2":7829825462713051842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743604, "value1":0.012350717590309025, "value2":3299133246704731924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941914, "value1":0.37406772316207776, "value2":3581449411399106092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.323291, "value1":0.2629524147159168, "value2":2560363092546342475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206164, "value1":0.1605470780394023, "value2":1476326866755963525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376246, "value1":0.8410819090377242, "value2":6697199584394109087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_6":"c","key_0":"d", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.438458, "value1":0.05246539756995679, "value2":2689405601803288451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.807203, "value1":0.730536792123947, "value2":7996980666084265973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.842861, "value1":0.2085319390948787, "value2":750997239048285659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.097539, "value1":0.943951867959508, "value2":8362598506758468383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363032, "value1":0.5870242519277287, "value2":6264967362494680685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212907, "value1":0.4603451527715893, "value2":4171196630516497508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177573, "value1":0.8948006671577682, "value2":5720852449593066754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286908, "value1":0.7373750433446898, "value2":2889582543857330611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.780354, "value1":0.11954278924179859, "value2":206386290438050068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744582, "value1":0.5414546611545269, "value2":1573274748881736123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262127, "value1":0.8114751808169933, "value2":1156734310814584450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.627688, "value1":0.05870166177597653, "value2":8802667963431750791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432936, "value1":0.5733149806402271, "value2":8971566642401975801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252007, "value1":0.6150552921263627, "value2":6524226679068869341, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230350, "value1":0.8727510444388746, "value2":5913218581367894399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.989663, "value1":0.8376111097555229, "value2":2874987014333615633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.435356, "value1":0.7022017770410863, "value2":6778124246365180945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673069, "value1":0.2749380629969126, "value2":1514980079784151790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328789, "value1":0.865778242569301, "value2":384106145681984426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.986292, "value1":0.9273192971072537, "value2":6526516407617182264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.240749, "value1":0.7234109649728769, "value2":6790075567204476237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774780, "value1":0.5871689076780541, "value2":5434961575452625626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.406076, "value1":0.7420142270194664, "value2":4969819996381131142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146123, "value1":0.2540849065850774, "value2":4942775185551508816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.928078, "value1":0.7980654446229606, "value2":2117433885577041648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933008, "value1":0.05348899498806063, "value2":7864335047299894108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655622, "value1":0.7668514308351416, "value2":5108933691323452489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799422, "value1":0.8658511508098131, "value2":1584220631636831527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153780, "value1":0.5100425920946189, "value2":6860613724470124155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.832729, "value1":0.6352815444925497, "value2":6914920278709539881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862121, "value1":0.40978666741053005, "value2":8297372411728279444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.046814, "value1":0.6860769857954689, "value2":7207194751565200226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.106175, "value1":0.9141202615431735, "value2":8093659818506693875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568924, "value1":0.2818708416167068, "value2":4682897128638978687, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743387, "value1":0.19055511730870822, "value2":6005851706828394010, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.131634, "value1":0.786038669470468, "value2":2052265664401672629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864417, "value1":0.6509022777920431, "value2":6405639625996257390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769553, "value1":0.3306779516460208, "value2":63644576329067046, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705994, "value1":0.39211177275150355, "value2":6019747875631249901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.085415, "value1":0.989053918126363, "value2":8753375624734757247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.966642, "value1":0.146948164341443, "value2":7640844254824954510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_9":"k","key_5":"b","key_8":"b", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.548525, "value1":0.7240074879750936, "value2":8956195213201860326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073937, "value1":0.7314322966939647, "value2":254693443211564548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347726, "value1":0.8209138159763902, "value2":6620059103742183058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.378046, "value1":0.5808416882480918, "value2":6139629334504131344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_6":"a","key_1":"d","key_4":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797601, "value1":0.7784388478906132, "value2":354987022131004184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411013, "value1":0.45158233474379805, "value2":6507365193047028596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.152084, "value1":0.34791281429560394, "value2":2152822969747742943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808545, "value1":0.6451074006535574, "value2":4535693696526329424, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735815, "value1":0.5617099301887051, "value2":4545722791278282166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170237, "value1":0.7594425432841077, "value2":2121502996639945525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839462, "value1":0.42088416942799856, "value2":4668152333347445855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977985, "value1":0.9874529650666419, "value2":7279887982544559299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777993, "value1":0.32638782808208466, "value2":7945394269696237457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931446, "value1":0.5421003602914947, "value2":7796969745729042644, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746112, "value1":0.6059100687669938, "value2":8599287190529748569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.950726, "value1":0.8607618900249634, "value2":2432626342683975270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647237, "value1":0.7468406151868747, "value2":7200054131729412480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.460709, "value1":0.4549015015070196, "value2":1628296475817521290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_5":"d","key_0":"b","key_2":"f", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007652, "value1":0.3132830085813196, "value2":1943034905431272670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.103043, "value1":0.029190956951399328, "value2":5218539584338803865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506730, "value1":0.4547431011729936, "value2":3644260335220928016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564064, "value1":0.2217922532972611, "value2":3866244575842378014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.308096, "value1":0.4735939830858083, "value2":6374663223385153398, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415080, "value1":0.9760882060049622, "value2":1970837347852222146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909084, "value1":0.6294752667120725, "value2":9199477369979291397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.969935, "value1":0.10788341374154452, "value2":7002111391558217482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.622182, "value1":0.21938176298467404, "value2":4706131254461339381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.722928, "value1":0.21266609523973992, "value2":5957309419507426661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062556, "value1":0.1304962392242022, "value2":4535950458577175850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056693, "value1":0.6559280448597679, "value2":6798153078557023165, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.275009, "value1":0.8708722343930831, "value2":2992233127991751043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027324, "value1":0.8089539697327325, "value2":6582915861289307476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769519, "value1":0.6866136235615642, "value2":3188730142010270506, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471216, "value1":0.036922660768241786, "value2":69160799499838231, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.289160, "value1":0.9707556984503585, "value2":3226749743394249266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719377, "value1":0.6150053305666618, "value2":8057451250881285379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884188, "value1":0.32970677783871427, "value2":6299179245210453660, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.128583, "value1":0.46621614630474323, "value2":3810272679777334373, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.016304, "value1":0.07070759725845735, "value2":3860287779798372481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556253, "value1":0.3162226930220112, "value2":811964996210240316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.662220, "value1":0.04332778395279835, "value2":1557538607744070859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713709, "value1":0.6625325269051697, "value2":4541028658322459718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297119, "value1":0.15013885945438676, "value2":2149756203418913335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685110, "value1":0.9036182355788681, "value2":4377728436719035164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846262, "value1":0.34678741602681534, "value2":2162478118726349508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410862, "value1":0.11378608528656459, "value2":614293064254453793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.060980, "value1":0.22871224178742908, "value2":4099107042293178536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960058, "value1":0.5820948808821428, "value2":1201783870513585432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.214885, "value1":0.03550184669484978, "value2":2051901639161172531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696549, "value1":0.44298333983599963, "value2":8432193022117819833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368813, "value1":0.6629289022292644, "value2":6617784174875177478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313857, "value1":0.15526487291119703, "value2":95708758682513207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744683, "value1":0.6423949710674116, "value2":3611912809247643043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.786335, "value1":0.589184376420666, "value2":5802753188533704826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_9":"a","key_0":"a","key_5":"b", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930246, "value1":0.20872893267238513, "value2":2224049910905425698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515204, "value1":0.20960234541456155, "value2":7647445186122035759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.800072, "value1":0.20249832362898804, "value2":1073544363493262212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.420652, "value1":0.651081573013809, "value2":355187443325178523, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760554, "value1":0.28915917130829083, "value2":5965361551796616223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.852984, "value1":0.45938574348035566, "value2":2528322122553170113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352738, "value1":0.900978009547613, "value2":3975764535045467018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066294, "value1":0.08305245567845902, "value2":2394249365413597097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654208, "value1":0.08608535610142205, "value2":2036163695122831466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.988436, "value1":0.04790022290628863, "value2":6418792057544120186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884316, "value1":0.5495267159618948, "value2":4438533595454630352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154168, "value1":0.4348957861418331, "value2":8838135202914157189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874389, "value1":0.42584041610286777, "value2":2273157634255144393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431882, "value1":0.06486477270973905, "value2":3345519389285150964, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685508, "value1":0.5499323474722744, "value2":5054596449098633988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.660995, "value1":0.17858690508054326, "value2":8567568075077444285, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271767, "value1":0.8266242576408339, "value2":8856688759394626086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755581, "value1":0.7494066282765741, "value2":4893716534306029427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171573, "value1":0.6765014228252123, "value2":5524113092747902135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269557, "value1":0.24446627014298006, "value2":2739808324008179094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.967939, "value1":0.5539434802495194, "value2":3700007071044991493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.656891, "value1":0.9116275072171186, "value2":7153356187047849001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248386, "value1":0.09058562459659583, "value2":7526211020899452764, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948164, "value1":0.6887494090196015, "value2":4600657675678245799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228675, "value1":0.8494478798473957, "value2":317720425357796205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101265, "value1":0.8190567291465818, "value2":4663927178372352248, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820988, "value1":0.775820207166787, "value2":6038722642101305434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922087, "value1":0.22553959289224842, "value2":779713635671076382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.888330, "value1":0.020416312330820526, "value2":7204412129555533981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.290446, "value1":0.9852816320447644, "value2":4592385145364652482, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.679824, "value1":0.17818186899277277, "value2":252209780074899647, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686778, "value1":0.8339304122124636, "value2":3658007614464097080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257632, "value1":0.614764442927346, "value2":4642146906724062657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482850, "value1":0.18190770107834264, "value2":8954437725061891309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998993, "value1":0.7044002667070637, "value2":431084763192513731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.875898, "value1":0.7812743007979214, "value2":2705148521595588507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084211, "value1":0.8180424469178956, "value2":7499444983471620842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053317, "value1":0.3379873516583276, "value2":5874640652872975301, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_8":"i","key_1":"k","key_2":"b", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746275, "value1":0.11387537552973875, "value2":4778407492588259899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744142, "value1":0.945713466310749, "value2":3761848793498692794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.274112, "value1":0.14967892390238677, "value2":3777191830426872225, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680316, "value1":0.5765506588965501, "value2":3913164426928614622, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.582474, "value1":0.13820731268553305, "value2":1359306396719437577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.572068, "value1":0.9236046727786195, "value2":731327351641596838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.838138, "value1":0.5915185036534663, "value2":4354316161210605047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755088, "value1":0.18889070665371863, "value2":4180101489297988878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362663, "value1":0.4708250834755644, "value2":8678758888644206492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.464079, "value1":0.13171451384673036, "value2":8187780706682520691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839674, "value1":0.16354088623961133, "value2":8461213608449952033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777782, "value1":0.005150900343416017, "value2":9058250336565790520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.520149, "value1":0.6583211185838453, "value2":179671992454725443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354725, "value1":0.26618356833133044, "value2":4502556297298851018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.588275, "value1":0.40556503705171876, "value2":1831243556070013492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.991768, "value1":0.5964588343272011, "value2":7216754835822865383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.829272, "value1":0.8185283045838991, "value2":7054142095046490626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028671, "value1":0.9570557166393036, "value2":5667948933240210255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171455, "value1":0.3984068015975904, "value2":6709012624962450094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112456, "value1":0.8897199020551853, "value2":619757189037583586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.366328, "value1":0.9348857036272434, "value2":4207327317390530443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.787818, "value1":0.06046424642679486, "value2":3800391541062933624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431436, "value1":0.5742437522880309, "value2":3338797017937456724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.130697, "value1":0.8564226707318178, "value2":2563408398646331614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531522, "value1":0.21916844982697795, "value2":6377428321672110838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799330, "value1":0.2117471404567077, "value2":6841854064863394582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415853, "value1":0.7458907419184496, "value2":3858225758706811596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769460, "value1":0.9292446985209156, "value2":4294237114969210404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053422, "value1":0.19156586881852822, "value2":8401149164501095807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.634910, "value1":0.13000039932274055, "value2":4131562742009700028, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703579, "value1":0.6862332441253757, "value2":7454437023717068913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327549, "value1":0.40704854485658054, "value2":4932202555006075846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.314755, "value1":0.5681370276361754, "value2":3506285901156371358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.466686, "value1":0.42167473519309434, "value2":3558723778128133952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160258, "value1":0.4575847781643395, "value2":6023651621402119686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228508, "value1":0.5111004373311687, "value2":8895687587282710909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.111511, "value1":0.09506770522467269, "value2":2655655566098996088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.784900, "value1":0.5435091084418109, "value2":5760921829736802814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210512, "value1":0.25861533775417506, "value2":6914577108338158111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.663100, "value1":0.8688427076237359, "value2":9173486087944898117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066304, "value1":0.7818969735770589, "value2":4913317894269750379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824798, "value1":0.9535971366603737, "value2":8366422359718269509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.090470, "value1":0.7644445079507416, "value2":6086882240277966349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994994, "value1":0.046461773505700306, "value2":1146509415122743778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409080, "value1":0.028382316546416403, "value2":2068020782874554558, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.815596, "value1":0.08796047569888332, "value2":8378736063122737266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683447, "value1":0.24029660357741323, "value2":8797505888002523812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.611937, "value1":0.2489739097238809, "value2":7210056341279458271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347662, "value1":0.7553826852514858, "value2":8429002700088766063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_8":"i","key_0":"e","key_5":"g", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318289, "value1":0.5406931125323163, "value2":1745967906139345488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.900516, "value1":0.7025144762707813, "value2":456260840471493385, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153699, "value1":0.20283897976623247, "value2":7209402260011396171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.813264, "value1":0.23005364054850463, "value2":4244429874559011543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.174420, "value1":0.6527990870752745, "value2":5344843801392876512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142219, "value1":0.1627182050289727, "value2":114175501496781641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.481454, "value1":0.37374721367966557, "value2":1638215468267960931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.523380, "value1":0.48736316265440816, "value2":4581537275255860768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317190, "value1":0.0039195876566355085, "value2":5341281001441055802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619251, "value1":0.5258022392764804, "value2":4434264026251453750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.641615, "value1":0.33039615993236077, "value2":6928858155661176736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961004, "value1":0.5399790911878092, "value2":7046629202789685841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025798, "value1":0.5984426914038641, "value2":6528145482673170360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.901084, "value1":0.06259887743500082, "value2":3236298745887395724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.489095, "value1":0.21376661515997028, "value2":7928954356959433352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655274, "value1":0.5385351363841719, "value2":7272624008760795422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.082369, "value1":0.4014980361714972, "value2":7005111860489403789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519418, "value1":0.19683342336444604, "value2":1435942937723913885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226945, "value1":0.9922423305002035, "value2":867760431487143669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752765, "value1":0.9916422768378336, "value2":3430696359309587884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.078412, "value1":0.6964630260101603, "value2":2102364069319122633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328609, "value1":0.19123451863699675, "value2":8170554763374290069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.470499, "value1":0.03919799070813115, "value2":3366292364134765186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_8":"g","key_9":"d","key_6":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740670, "value1":0.08849655325518448, "value2":1691579140130982717, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.393865, "value1":0.061408201218711954, "value2":174448511358385380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.200654, "value1":0.2836024427807901, "value2":4326318289877991961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970400, "value1":0.7730199010023195, "value2":8574178501642024588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_6":"f","key_1":"f","key_3":"h", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.465169, "value1":0.542725080263732, "value2":9196833379545093375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.563410, "value1":0.7958042531287333, "value2":7393645101828557391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_5":"i","key_8":"k","key_1":"j", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213281, "value1":0.7928629505338592, "value2":1752466947098105566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954474, "value1":0.3032546869220648, "value2":1748556721646671927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.847068, "value1":0.7688745080204612, "value2":5187070635183292150, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251152, "value1":0.5682862581346041, "value2":2363792865069284658, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360666, "value1":0.08726468762579924, "value2":643126609584411238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619231, "value1":0.7434795567208692, "value2":4822313794436821661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603123, "value1":0.10532993201555661, "value2":2544972440464847934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607459, "value1":0.20036031311934685, "value2":1168760905394403690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447800, "value1":0.7195092335832372, "value2":5642489879930119928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.850819, "value1":0.6761002637886042, "value2":833307298139301862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539724, "value1":0.5294892272320822, "value2":2667359803805326763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.208291, "value1":0.8393697010611292, "value2":683397536019101988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550311, "value1":0.9963724696480404, "value2":5468577449184757808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.023114, "value1":0.32353069050613953, "value2":9187169003922087415, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286063, "value1":0.9137524095402076, "value2":6445901279543298486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_9":"k","key_0":"k","key_3":"j", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.204901, "value1":0.4076696093660646, "value2":6677345639081163017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_6":"k","key_0":"b","key_3":"f", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107007, "value1":0.5776778447129313, "value2":8463235586423448216, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271841, "value1":0.6930651904253268, "value2":6787444131360802180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.069717, "value1":0.7690475248125641, "value2":1837461702894857026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368696, "value1":0.5454486705451115, "value2":3041446083114186683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_5":"c","key_3":"g", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395274, "value1":0.24368851515268314, "value2":4392956019271269002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352289, "value1":0.22988542503215345, "value2":6550852329237611667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703725, "value1":0.026480349904897907, "value2":1885153320482283414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.256528, "value1":0.26111767999722507, "value2":6808025827523544315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252909, "value1":0.20711336158884097, "value2":3024039988582180695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_9":"h","key_1":"d","key_8":"d", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.633970, "value1":0.07845906332760666, "value2":6511716740509653564, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605465, "value1":0.5950957840062573, "value2":2806262404335314036, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.081867, "value1":0.08398189819376621, "value2":5728413114044088113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074111, "value1":0.6218602698989528, "value2":1011556855791832544, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675019, "value1":0.11095025598530854, "value2":5207952003333828816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.738057, "value1":0.5410628771237361, "value2":2540645323761255412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317014, "value1":0.5259626957217648, "value2":3505614768801064603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776152, "value1":0.302147585405568, "value2":7518755412283013617, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094787, "value1":0.5814132866660779, "value2":4822671923080781538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.517582, "value1":0.10766355589362117, "value2":8453589798026915731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882088, "value1":0.8040313368583433, "value2":8495228225039478737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803773, "value1":0.22925549140141077, "value2":8938270200397652857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403205, "value1":0.07946210415899989, "value2":7443253750975595381, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211183, "value1":0.27303249192461276, "value2":3758490705819764753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684989, "value1":0.19635657331096867, "value2":3768344870136296803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596986, "value1":0.9569957010370201, "value2":5843413270558977752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796083, "value1":0.4870882675033077, "value2":8040326882339558395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.674664, "value1":0.05017490820454691, "value2":1512965384940722080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682218, "value1":0.05038298218752238, "value2":3666135732590503511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871616, "value1":0.007362621326077148, "value2":3382155619299232500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.479139, "value1":0.48744354938142825, "value2":2536302313422526956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408860, "value1":0.5030243670352939, "value2":3709214658604138842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.079849, "value1":0.4036712273225728, "value2":2906136993668691733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735107, "value1":0.11572752191146224, "value2":9135232014249483896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.579000, "value1":0.832018905121829, "value2":4291485121105439680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297326, "value1":0.3053519839954546, "value2":7762295205550727215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077473, "value1":0.5244918634463549, "value2":3651586678818959294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211495, "value1":0.3162790532742153, "value2":8307153509301118839, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.018730, "value1":0.4578847558537525, "value2":3121380712594178301, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.254109, "value1":0.0022335788974661955, "value2":1113680598327977122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327840, "value1":0.38572838732788006, "value2":2539441632284031099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802473, "value1":0.6047951359580281, "value2":1001558191118170380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299508, "value1":0.05862512584912177, "value2":5514618695776223541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.011661, "value1":0.31240922806678983, "value2":6137561733651595636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728655, "value1":0.1278483939203575, "value2":9035381570106713199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.667349, "value1":0.5613783650549519, "value2":596862476082819858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.253799, "value1":0.021910250550882106, "value2":6533597878636060832, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_9":"k","key_3":"a","key_5":"b", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.288444, "value1":0.30057474196214407, "value2":3169010106041213703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.798940, "value1":0.9681683301198919, "value2":3021363800209965251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.981962, "value1":0.9612929543752854, "value2":9171352851679704895, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149939, "value1":0.09981385175599812, "value2":2281824057904455016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943557, "value1":0.6955975747592801, "value2":2634327756448196264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459593, "value1":0.6446822058280443, "value2":6342926358496707008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.734204, "value1":0.6639025089026711, "value2":2476897145958348918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506327, "value1":0.10053599414522371, "value2":2221394931414407943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410931, "value1":0.8059701840141592, "value2":3838253097677401347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.964398, "value1":0.603279381255061, "value2":5782827613318848456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_3":"f","key_0":"f","key_2":"d", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.278202, "value1":0.7115230775467353, "value2":8371900549612489852, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408547, "value1":0.21860759782537603, "value2":8039563012338877282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.261349, "value1":0.11227782989658946, "value2":7783215654484926349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059288, "value1":0.07895743947800146, "value2":5667050368393356509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.601079, "value1":0.6482240475140798, "value2":4251195673123293243, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.225112, "value1":0.7216063470142905, "value2":148937314023613788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977412, "value1":0.6598247780794319, "value2":5965401930753401354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194225, "value1":0.6141698639690517, "value2":8817766861356540801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.597081, "value1":0.07693962991598453, "value2":5029342104400232103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.335617, "value1":0.5583831809146028, "value2":1406686773640908295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916403, "value1":0.7018505752435668, "value2":664450605422502237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497447, "value1":0.8277122605282176, "value2":1219079943087879547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716326, "value1":0.7509816698301687, "value2":3921665603218422131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716585, "value1":0.2920975208652007, "value2":3756232698934542100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301567, "value1":0.8942632920542656, "value2":2291395595568091863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.599725, "value1":0.5659814539312558, "value2":7320359369690410226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.614192, "value1":0.7911920709201787, "value2":8781166815844312813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.195843, "value1":0.3963219582946417, "value2":4716889546078718843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555996, "value1":0.602486402959913, "value2":4687524095847830496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.917870, "value1":0.15392369008366974, "value2":7045583703847585198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.616039, "value1":0.6919323182152138, "value2":511455339796298853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930107, "value1":0.12610440100742193, "value2":6997765950733061577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194345, "value1":0.7209468362854391, "value2":556025580881809360, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797831, "value1":0.2266151348199214, "value2":2004107184327193155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_7":"i","key_1":"d","key_6":"b", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949681, "value1":0.8884059641939672, "value2":7011478951165849176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213425, "value1":0.9929181204525178, "value2":8631476778841936239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821834, "value1":0.3090901919909019, "value2":5522454281472043559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.181027, "value1":0.06720359916226379, "value2":4456725170730825432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.762490, "value1":0.474581541255306, "value2":7005342387987070026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.759790, "value1":0.5398913013405661, "value2":2663889184009973941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112196, "value1":0.8025326007392437, "value2":883355344681130779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374886, "value1":0.9718695344446878, "value2":1646239120227544644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.729457, "value1":0.9682037233981026, "value2":6151398746046248850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048424, "value1":0.7273914683080029, "value2":8000884246159354431, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.937043, "value1":0.1526646293780592, "value2":266299265780041426, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752328, "value1":0.9655606802627454, "value2":5596664612933771240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666200, "value1":0.10048257972248233, "value2":4289061240642605106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922103, "value1":0.8963130727289086, "value2":8836663020903305631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105202, "value1":0.8314839649252564, "value2":3551543011617183084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886891, "value1":0.66836051888972, "value2":72366475317978094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432056, "value1":0.745064543848614, "value2":2189880642993257536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042716, "value1":0.3956017159748215, "value2":3605211356176936944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.984359, "value1":0.2911243881226883, "value2":7856628067522273320, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295731, "value1":0.4520766897124352, "value2":610878216891620607, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.650753, "value1":0.03928359938969399, "value2":2072987167295070336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211564, "value1":0.7625006233645502, "value2":4401190374987004409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792725, "value1":0.47390601969195995, "value2":4329714295913254034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.444277, "value1":0.5857721636217157, "value2":3580156964556605157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535990, "value1":0.9433719155892893, "value2":5643233427116703402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.688127, "value1":0.7720856573472716, "value2":5631377472070167195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.697966, "value1":0.9263837993798216, "value2":4093734777340860370, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.996439, "value1":0.3910900735196092, "value2":8375564545525062485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872727, "value1":0.8324218421388521, "value2":7152928233109153205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.246110, "value1":0.2551941160691682, "value2":5602869232123697435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931727, "value1":0.5795265646922866, "value2":7403691202626500132, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673141, "value1":0.06896155961831285, "value2":4415828287435091542, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_8":"a","key_9":"h","key_7":"d", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684610, "value1":0.16872221976263158, "value2":505317720854796636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559417, "value1":0.12924994486885147, "value2":4787886921303451969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094359, "value1":0.5164211297919008, "value2":1315713014945297109, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007976, "value1":0.7735634457421625, "value2":8579409036573615608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286892, "value1":0.43624877878480095, "value2":6924175474906299981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.902926, "value1":0.6780220008663965, "value2":4822375571733927097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.486540, "value1":0.23333524037004374, "value2":6314520721238807162, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007170, "value1":0.9857337079596743, "value2":4448133028853613455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.030552, "value1":0.24355749578071725, "value2":8327413108187969392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.379606, "value1":0.4729923914146341, "value2":914941226083135930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835232, "value1":0.9602105867588695, "value2":3810448989361248064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433684, "value1":0.9595084872985944, "value2":455874222103117607, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411653, "value1":0.037665634780994134, "value2":6833101830522402640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_4":"a","key_5":"j","key_2":"e", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713632, "value1":0.5664925972399699, "value2":7794307235376671149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391266, "value1":0.6608939962095463, "value2":5860330325168511608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212303, "value1":0.2089060448996, "value2":5651881484326487830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180078, "value1":0.7876093968011537, "value2":2862206598597477486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.175199, "value1":0.14395244434800714, "value2":2376804168343245120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809965, "value1":0.12003398504952464, "value2":5187708593154594566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.851381, "value1":0.607956049195976, "value2":7288813225458184427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933118, "value1":0.7246239590339857, "value2":1234716149527350625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.939251, "value1":0.34949617152726564, "value2":1847804623713516854, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665651, "value1":0.7331985679295692, "value2":1370393562180863722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.827386, "value1":0.6896205618409067, "value2":1887650763331073033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.637520, "value1":0.13783942741851515, "value2":7482561145284406008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657534, "value1":0.26275939644072077, "value2":6263897972267140208, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740684, "value1":0.9117133919549965, "value2":6149129956357538684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032295, "value1":0.9865289627211871, "value2":5555291464918052387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545212, "value1":0.5648289415381519, "value2":6789070914120574869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.321143, "value1":0.6917810996904441, "value2":4171532082462131046, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.693006, "value1":0.9813132387635279, "value2":6633038595296515364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.895453, "value1":0.7852256444188942, "value2":143518625251882031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165192, "value1":0.9962691534924663, "value2":1006093794801043666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.353290, "value1":0.5341309097782416, "value2":1258165174425707067, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257167, "value1":0.7039937676297155, "value2":1145181626509784790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413647, "value1":0.8270957619940229, "value2":2929580620701623843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817697, "value1":0.5617132216229666, "value2":9160744958657565723, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.117851, "value1":0.20816644440142626, "value2":5357495560720255737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545126, "value1":0.5346084869679877, "value2":7401215357009964293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824766, "value1":0.44946104541199977, "value2":4136655445147421363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.292920, "value1":0.8837887841085235, "value2":7386066379061583540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949095, "value1":0.3445432872175305, "value2":7354448880530731294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705243, "value1":0.4595023646368128, "value2":1229720604223933971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_5":"c","key_6":"a","key_1":"j", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610093, "value1":0.4977300016550588, "value2":6055289293785959372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.710138, "value1":0.8494303412007083, "value2":3284734735098008783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.503055, "value1":0.7306081415151926, "value2":7335475436979640441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194198, "value1":0.4080577435044893, "value2":1817328694183447312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409059, "value1":0.8174198346097175, "value2":1028006350387911539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696870, "value1":0.3114167302109578, "value2":5761149591452007692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483258, "value1":0.6191704897662682, "value2":6062885989254649767, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534924, "value1":0.6117230725465559, "value2":193279975656637243, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639609, "value1":0.47495483196675986, "value2":5738372947955229915, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532964, "value1":0.01280767557324011, "value2":3610414855244726673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.741210, "value1":0.5736887790720044, "value2":9118221200383527859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459701, "value1":0.09043619123087489, "value2":4263857141830808661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.064891, "value1":0.8303048678510601, "value2":1417745368253260336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.193982, "value1":0.5954818624712542, "value2":1866813575466479927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728941, "value1":0.42773413448912906, "value2":5011276511494346486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978643, "value1":0.9720431088123873, "value2":5835438838603942392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376007, "value1":0.5894070491287171, "value2":1536141728264317947, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_8":"g","key_3":"k","key_6":"k", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226693, "value1":0.26901914665508003, "value2":1625748998624229948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.963696, "value1":0.5769249508695262, "value2":9054438071997375931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.418644, "value1":0.766497146060167, "value2":5911142909752098984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564856, "value1":0.11883244261524029, "value2":1743904390192603929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055140, "value1":0.52087787431266, "value2":402388317326273396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165833, "value1":0.8230062512281381, "value2":3737986001509812244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.537380, "value1":0.3719641116241002, "value2":7620785809655703628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059138, "value1":0.3359364898323379, "value2":2258599254367397283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532103, "value1":0.07168825170242699, "value2":8419082906568750462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_7":"f","key_1":"a","key_6":"c", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286202, "value1":0.8607616531331588, "value2":5301488987302943339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477954, "value1":0.7462715469038064, "value2":8695685405375237694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_2":"e","key_9":"h","key_1":"e", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.377516, "value1":0.1251480000451169, "value2":5693123483571141635, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799701, "value1":0.377835354984664, "value2":7635656423964308029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947251, "value1":0.7193521006934637, "value2":3495998112139060094, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_9":"c","key_3":"g","key_8":"h", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.630158, "value1":0.27928250023508844, "value2":8779396817967517012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886093, "value1":0.6334013505397469, "value2":5098375435747464833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.942245, "value1":0.3390011070632797, "value2":1737381722392040741, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_7":"k","key_4":"f","key_6":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.009762, "value1":0.2515798026660567, "value2":2465310044177783178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.896699, "value1":0.584030830613032, "value2":5977614913522628306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820378, "value1":0.7811580321417821, "value2":1709536028386531684, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653683, "value1":0.7005991085562125, "value2":7295655368561133786, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522361, "value1":0.7296163959268779, "value2":5916302487026661508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.127496, "value1":0.24876472584091772, "value2":8637191483262110011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.091531, "value1":0.9069922237355105, "value2":3364698751790785415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137633, "value1":0.9475238029065713, "value2":6717128691639834645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550582, "value1":0.1420769771765079, "value2":1644264737205720610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059092, "value1":0.38378121050522995, "value2":4775737880122110580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236099, "value1":0.8215350110894001, "value2":1784075564711617189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_8":"b","key_0":"b","key_5":"e", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.636343, "value1":0.4453072911799391, "value2":3315454218709810887, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799575, "value1":0.17197278191202653, "value2":5903905346180673218, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.584133, "value1":0.4235206584790561, "value2":1691406943949519298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015622, "value1":0.34252290670938007, "value2":5505899012400247300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.021661, "value1":0.11232931832514273, "value2":6468715321661687664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.593330, "value1":0.8735006589361365, "value2":831831146444791684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273986, "value1":0.25953904800502825, "value2":5320773233674225264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357026, "value1":0.11876193765893824, "value2":8703386441096799470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903715, "value1":0.0317054368098292, "value2":2551886074115870652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.905359, "value1":0.9780528237638186, "value2":3462176768443711504, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.272907, "value1":0.5077414966128044, "value2":5351891928699763264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610476, "value1":0.5253243304839273, "value2":7118905075072261164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684508, "value1":0.3558646139061106, "value2":5157018975048237545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748383, "value1":0.7510255655970447, "value2":4180265211771950319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.801038, "value1":0.2649030050080282, "value2":509475464752529120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657950, "value1":0.2544231826829063, "value2":4768343360725290987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149149, "value1":0.24278734421658493, "value2":2310223808628977449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.558331, "value1":0.3866811766896264, "value2":3120828999446479419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.076776, "value1":0.8191629913767823, "value2":531265332159947696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881067, "value1":0.16630603149873838, "value2":8244214645789653431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385842, "value1":0.7521248304618617, "value2":4350867045350111094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914596, "value1":0.12306660567858013, "value2":5899167982504356408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.159683, "value1":0.24808395371130693, "value2":3321017992196518857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.058378, "value1":0.4115012836332614, "value2":5551372136464658518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_6":"g","key_0":"a","key_2":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819794, "value1":0.9548946315713374, "value2":2661379806705881657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.293628, "value1":0.30579616916280866, "value2":2771189581765698489, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.904159, "value1":0.33983467932998995, "value2":3076540259059103682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946959, "value1":0.14422457278803882, "value2":619548712042120909, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383295, "value1":0.23130372874617672, "value2":6179693520548172628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482434, "value1":0.8720643520387502, "value2":6593209094509926896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613596, "value1":0.9574764532675973, "value2":8706916487148773835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551130, "value1":0.9901941419895447, "value2":5559435899121181473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954673, "value1":0.7022122624865335, "value2":6926608561209244320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781834, "value1":0.1129731019718482, "value2":3356276176028256450, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_4":"f","key_9":"e","key_0":"b", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746610, "value1":0.5148787087784333, "value2":2343142454537691934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409647, "value1":0.3828843970677094, "value2":4346661084777988652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.427543, "value1":0.6220594320308941, "value2":653819026690948218, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_9":"e","key_1":"d", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467056, "value1":0.23591956688182053, "value2":5924417417665977435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245070, "value1":0.32465613049106207, "value2":8745282484552741587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719344, "value1":0.6149579128373653, "value2":7721284234183306230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095318, "value1":0.7783448314627689, "value2":7496490910707147022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696761, "value1":0.23959350451136963, "value2":7667698021180616606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170976, "value1":0.2107634553446009, "value2":1067019224393043111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177955, "value1":0.15559883601716398, "value2":570151235008619291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.334876, "value1":0.05250568880425774, "value2":313709473500400875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.645622, "value1":0.5874495193049836, "value2":4998753194711581798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403716, "value1":0.681377437500311, "value2":6083653541111639303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.511374, "value1":0.3173656336054262, "value2":3280525464397494846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.218153, "value1":0.6649425163171951, "value2":984695368325234886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_5":"g","key_6":"j","key_4":"a", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424332, "value1":0.13915015029096536, "value2":177058255857848790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882802, "value1":0.9604830357087591, "value2":3462432728417263080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_9":"f","key_0":"a","key_5":"d", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933587, "value1":0.05218287981155221, "value2":9126870231879692354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426076, "value1":0.7165203610998703, "value2":4392854854028036091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.302645, "value1":0.34553849779237905, "value2":2907279110591723700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269877, "value1":0.8711177705575849, "value2":8366161311313542515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.995800, "value1":0.6906708270307872, "value2":2179499360020827214, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433217, "value1":0.612549221250184, "value2":3062667501076675896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550193, "value1":0.8112010714019721, "value2":3511775054758615153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947917, "value1":0.4309103807271251, "value2":5927887337207634099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874180, "value1":0.8528260534422161, "value2":6693253358548032140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.714464, "value1":0.7660112509524851, "value2":4667923922359952270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.088061, "value1":0.2662708805339307, "value2":2956847029720911185, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960166, "value1":0.6603171866146907, "value2":4062360360972662646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.771991, "value1":0.06535928533875911, "value2":3275575695469134507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.907738, "value1":0.4729789504578976, "value2":6665898542309531788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_6":"c","key_9":"e","key_5":"f", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.446527, "value1":0.054360638009933175, "value2":8447197342747029062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871013, "value1":0.9325815658063547, "value2":3295816637987069156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322468, "value1":0.7896334184172146, "value2":2062716273940484317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145073, "value1":0.15908044288327283, "value2":31202777072956548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.450280, "value1":0.20707514852594208, "value2":3916360901630366923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.834638, "value1":0.6993008706531769, "value2":5937180712489291188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116654, "value1":0.23637601643034672, "value2":4626933419743360623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_5":"h","key_9":"b","key_3":"a", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.265735, "value1":0.37825344249371357, "value2":1259576346841819002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683748, "value1":0.8969615470937745, "value2":3878280364099631128, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219470, "value1":0.2735521439606725, "value2":2004029025316087157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476407, "value1":0.3060298444596037, "value2":2752192683000499763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973916, "value1":0.6236548692215601, "value2":1218902262886276264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777866, "value1":0.4605467848114755, "value2":1483419366258255115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045114, "value1":0.6713933571025698, "value2":7389349750224761078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.570321, "value1":0.6477885368549638, "value2":952734344221236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821043, "value1":0.6987507461356852, "value2":7961454653125032378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.586904, "value1":0.6114733442241732, "value2":3313794499633786815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705481, "value1":0.05021439091843144, "value2":1692908847970065889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497544, "value1":0.7911397038857368, "value2":7192760038857907146, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.932189, "value1":0.9148929522533059, "value2":9058585624863979355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769986, "value1":0.08834899410804685, "value2":1734900992084257549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695397, "value1":0.9679316547671841, "value2":7641677535001923086, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084900, "value1":0.7534718430023655, "value2":5595569809726413737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.562250, "value1":0.19691330948497934, "value2":5079334807276275801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.843384, "value1":0.13037385236459426, "value2":1781212766913480310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.102533, "value1":0.8945673902737089, "value2":4413715911002881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350432, "value1":0.9789410844857529, "value2":8061203484333522616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857648, "value1":0.17405749140176002, "value2":9069075917121269047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559262, "value1":0.2097671258596821, "value2":613585525667388168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.712886, "value1":0.7017574550139256, "value2":807895944720896464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.936984, "value1":0.8849159113766228, "value2":6360061394425352910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545107, "value1":0.7492436378397768, "value2":1930556756703562417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.681593, "value1":0.044080654493611975, "value2":4672445330328029132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388330, "value1":0.4856178458787539, "value2":6153421365310799825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.879197, "value1":0.16996161009540478, "value2":260583885651182711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.770981, "value1":0.7283458398178451, "value2":6187029331538785539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101721, "value1":0.5814924092013966, "value2":3246021123474652609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665903, "value1":0.2698824852319885, "value2":1593984595070302417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.014945, "value1":0.41108393803888965, "value2":849125531814643716, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455276, "value1":0.875114673636132, "value2":5155846223745306265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.044985, "value1":0.9997830302808556, "value2":2517744318699317082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236657, "value1":0.10301472150509888, "value2":3916175947622962913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.595845, "value1":0.681705831836613, "value2":4919968063926505445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909129, "value1":0.10658700130911442, "value2":9017518189462648176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994913, "value1":0.5742718119176038, "value2":2926652004213539521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350277, "value1":0.15279738754509417, "value2":5229335911692726266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.022399, "value1":0.3330863087425972, "value2":4953311412687090451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268930, "value1":0.7193122390898454, "value2":4256538916269794836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.983431, "value1":0.0510995090351011, "value2":1768238660724776680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672570, "value1":0.255109482234144, "value2":5911656502600207006, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478340, "value1":0.7629467314333804, "value2":2248461028173940598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954691, "value1":0.780819131964203, "value2":4988575109325316188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.854956, "value1":0.888056149862958, "value2":8089385147769653445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028845, "value1":0.6170371731466705, "value2":4427914831964658823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639652, "value1":0.8230090544258113, "value2":6386454055832759048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.810891, "value1":0.8133492204735011, "value2":6725307713283048525, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761452, "value1":0.9287463566978789, "value2":1984332858107282238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318552, "value1":0.12867977830402738, "value2":4258557511687812956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316280, "value1":0.5022437385431537, "value2":995773241737628619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954122, "value1":0.5240772572203263, "value2":6037304727626529988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177671, "value1":0.76376619698564, "value2":2077146903364533171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731650, "value1":0.7327573768170443, "value2":3115299633570203304, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.050578, "value1":0.7589539042984136, "value2":8472839195693038957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.606514, "value1":0.9364993124842877, "value2":5612839525396767856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369368, "value1":0.048054238834199556, "value2":5282267936079713586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_9":"i","key_8":"d", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619618, "value1":0.3228482622950357, "value2":8295983975137448078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783482, "value1":0.20802671279789772, "value2":3029896725457460111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_7":"j","key_8":"d","key_5":"k", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332994, "value1":0.9302305981560293, "value2":1560834164036895150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320192, "value1":0.22351800921367507, "value2":367222968940142358, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881266, "value1":0.08414682717481595, "value2":4156359981061160049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.708929, "value1":0.5859616740437592, "value2":4218658472873761504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.974600, "value1":0.18236565991254491, "value2":7158153525091699583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760005, "value1":0.2499852109211892, "value2":3103407843415565026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994395, "value1":0.6346134433025367, "value2":8895656809884955572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482016, "value1":0.09096777794371665, "value2":6086112719790576068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442570, "value1":0.8266959930861106, "value2":2275327626493666884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416700, "value1":0.36518175488705273, "value2":5793835688591222111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.488107, "value1":0.8111992750792233, "value2":3432067066553462639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.296157, "value1":0.748182736526456, "value2":5908492840734386304, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655322, "value1":0.77012733378921, "value2":2328983933411827460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299380, "value1":0.8617587030482643, "value2":4249851149905507488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.569550, "value1":0.8799767655587674, "value2":114484019272599760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.737420, "value1":0.660133828675834, "value2":47092260178100634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961538, "value1":0.5805975021285331, "value2":3098519932913684476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197731, "value1":0.7355297940768138, "value2":5347385484676578771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653672, "value1":0.9639932888139914, "value2":4154150822821006863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209735, "value1":0.25867188901232296, "value2":7573003393945357593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.243923, "value1":0.34698853560785725, "value2":4046809595104442791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055125, "value1":0.9118136199691569, "value2":6132859986145115351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.129146, "value1":0.737958001899574, "value2":3639000020359385593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439824, "value1":0.7612995291197047, "value2":9120669622233440416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222157, "value1":0.2453738047252645, "value2":5040757825498916672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916336, "value1":0.7380620101187605, "value2":4223451018219910420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365543, "value1":0.035239810434389264, "value2":7558213648465294836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.860762, "value1":0.4102850563256095, "value2":1429493420542448489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388872, "value1":0.5076851349389522, "value2":5984142931688899418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.324767, "value1":0.32798077886519317, "value2":7010471358073816774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605843, "value1":0.9054516940429083, "value2":5196636487460887513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653438, "value1":0.9546349180436571, "value2":1964265110886490351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576178, "value1":0.9072096436167739, "value2":4650061523788539671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.304613, "value1":0.07126080366571305, "value2":2739761425628315393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399413, "value1":0.9339282528280148, "value2":2738842204765000811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943959, "value1":0.08784682744168541, "value2":7785212390244201422, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_7":"g","key_8":"h","key_1":"j", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.155440, "value1":0.06596281114416563, "value2":5614935012679054029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_4":"d","key_2":"f", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.585232, "value1":0.38735104883356297, "value2":5357846878220935711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.231739, "value1":0.7284804092280743, "value2":1167665308911410443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555892, "value1":0.5915854017921878, "value2":1567147347614329809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.855303, "value1":0.09176207020704319, "value2":3808984845296778232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.938710, "value1":0.4819531123010938, "value2":4697805519732493178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476902, "value1":0.22082616880251751, "value2":2129509252592428060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744743, "value1":0.33257150382979367, "value2":7648865469799769588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306532, "value1":0.04481922646422544, "value2":5772762433855832524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364825, "value1":0.3587026951251621, "value2":5278057080002614026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_4":"c","key_3":"j", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.484995, "value1":0.39478883780380347, "value2":3136752353967205980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.359118, "value1":0.08414989767840077, "value2":1942845542819450866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095178, "value1":0.16106032134150491, "value2":2585752892335863488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268896, "value1":0.613902523132248, "value2":687319106737333189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716029, "value1":0.4960997765937663, "value2":2263474829486803965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748556, "value1":0.30380613001187684, "value2":569759032424044386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_3":"c","key_5":"b","key_1":"c", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819011, "value1":0.9766133829878352, "value2":4041486474077430360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_4":"i","key_8":"j","key_3":"i", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109307, "value1":0.7831795468260019, "value2":6028647505036494731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.179692, "value1":0.36834450950873376, "value2":5143099232657374567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408811, "value1":0.37776607558956526, "value2":6856480231263656565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.006253, "value1":0.5270930124398272, "value2":1411653061914296136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776912, "value1":0.4551811310760191, "value2":1536148212309064628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447453, "value1":0.8174960750892899, "value2":4331073111578365688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696722, "value1":0.2434463944121451, "value2":3006348927793308525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.747552, "value1":0.46652870663413926, "value2":8487775958608976630, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618043, "value1":0.5667967701695303, "value2":445864738400581228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162568, "value1":0.6628301293329222, "value2":8197909955604355013, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.861425, "value1":0.3485000903134516, "value2":8816338849324478138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.806516, "value1":0.4953223448201175, "value2":7711648028791565560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970539, "value1":0.2083396416590356, "value2":5815736098137722214, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108983, "value1":0.42614208367663026, "value2":5403929578330914973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095108, "value1":0.4675226715836234, "value2":4103322823365161263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.279008, "value1":0.5189838609534998, "value2":5583248834742899795, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080151, "value1":0.04025953736590447, "value2":1709967085975216083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960603, "value1":0.48558007294067956, "value2":147087377780565755, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522699, "value1":0.8996419465199031, "value2":8383086671138869204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812906, "value1":0.42145746065428685, "value2":3807545219581503501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973610, "value1":0.9578107981118065, "value2":4941505183486571577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972491, "value1":0.7644375474791567, "value2":5826757011862504756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048527, "value1":0.2281920875592009, "value2":4718173727183162630, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515627, "value1":0.3746899214010938, "value2":5127946555951269169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.346004, "value1":0.5308363801498025, "value2":1141799226947758562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_7":"d","key_3":"k","key_5":"c", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391384, "value1":0.5665502681985652, "value2":5138410786658734048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857410, "value1":0.1451852744317182, "value2":6603434642077187944, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.414061, "value1":0.265028330167258, "value2":8861513360159934632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653333, "value1":0.28437623266560746, "value2":5007130114350069495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.123846, "value1":0.29659698191393213, "value2":8540905938700351347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477802, "value1":0.6141130483574686, "value2":4145514544028354410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531351, "value1":0.3974291747311908, "value2":7891249160701823099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197949, "value1":0.9490013407410278, "value2":8218574898456721251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411857, "value1":0.8458181580037749, "value2":3063835093954499296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.877510, "value1":0.828501312919876, "value2":5054318282608028643, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665063, "value1":0.5885292293078199, "value2":7952781418541268636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041521, "value1":0.9878113457449454, "value2":7450867973872472368, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073061, "value1":0.939649958460976, "value2":7875175737198226822, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_5":"k","key_9":"d","key_4":"i", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781888, "value1":0.5031750456506159, "value2":7116386269513023773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160017, "value1":0.23130907810117704, "value2":2438724480357189127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835359, "value1":0.4415459106308202, "value2":8077656610946018544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251312, "value1":0.10148089002006272, "value2":1896400908090076616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.632483, "value1":0.7757884759608441, "value2":4168124167987668028, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.772638, "value1":0.14296917471420942, "value2":5892426599914094550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426554, "value1":0.4152913794309927, "value2":3491478566670263783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672520, "value1":0.5730477814369939, "value2":7734177692857714726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025475, "value1":0.8422765595744146, "value2":7729206509934976065, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384117, "value1":0.370521152586741, "value2":5964592513649221172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666556, "value1":0.5944594406776452, "value2":6664609919274167291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424484, "value1":0.6898557357450742, "value2":7620342586495006501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313298, "value1":0.7769705418438511, "value2":5904168920374633694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556524, "value1":0.7197157815883978, "value2":1785992040018522529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360907, "value1":0.5368763819006307, "value2":1221035619431068900, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576297, "value1":0.2976581980063287, "value2":6069726578064867551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.425007, "value1":0.05760812531545847, "value2":3218040486200104849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442385, "value1":0.7140138290502642, "value2":5982710133241839409, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364950, "value1":0.7133219094796532, "value2":2621884575274392083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211537, "value1":0.9662389413186122, "value2":6837048628615994370, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.161776, "value1":0.9162528731861198, "value2":98225928051258831, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811493, "value1":0.5668121931771126, "value2":4899385784647025957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822383, "value1":0.016988952685027904, "value2":7343988970036816255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865454, "value1":0.48779466867997384, "value2":3996429126028951858, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751902, "value1":0.517057218253837, "value2":4621244619940358234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.525579, "value1":0.7270638229553396, "value2":4556294068287353316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822049, "value1":0.8487037862810487, "value2":8759327482669918657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483767, "value1":0.795243973695518, "value2":5137269638112922032, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042738, "value1":0.7385935337493647, "value2":1091563145939490509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.521929, "value1":0.10598411546030329, "value2":1018022188446431653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792757, "value1":0.4495950673323956, "value2":1856520078086460155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695775, "value1":0.8122734246000456, "value2":2520061166650080735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105722, "value1":0.9820835252391706, "value2":7455782937968387088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820568, "value1":0.22530315350938107, "value2":3276960573086645000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.140320, "value1":0.5664165038340241, "value2":7444501896936670459, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.501828, "value1":0.6731262889242471, "value2":1425113709817927232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930172, "value1":0.48625040372483846, "value2":5327468289350467201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.706858, "value1":0.07914368053314678, "value2":1800402440657604521, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134725, "value1":0.4678852928796053, "value2":2325913710025246340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876578, "value1":0.9424035369996965, "value2":8055626132574707503, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108826, "value1":0.29002692913973105, "value2":774202804060462158, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.031892, "value1":0.9251224968475468, "value2":6954662396929356191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539689, "value1":0.24245379006948578, "value2":460788187782305876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.858085, "value1":0.02056945040393724, "value2":4590009984548317252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.202540, "value1":0.37546125893661825, "value2":6399435061817101457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384494, "value1":0.010928897877400335, "value2":2783102565309398205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_1":"f","key_9":"g","key_0":"k", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061578, "value1":0.259570974455371, "value2":8569411528487848312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_8":"h","key_9":"e","key_2":"h", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027239, "value1":0.09519589010929196, "value2":6611488629400003141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210858, "value1":0.6912816460992459, "value2":7725980859343417825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.825726, "value1":0.48050045655332785, "value2":6843673580007675724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430241, "value1":0.8940246106870715, "value2":7204593499636831454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_5":"d","key_9":"f","key_1":"h", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332710, "value1":0.05615399939144247, "value2":4573382128185893311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374418, "value1":0.5152317418659397, "value2":7908425193534726276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519252, "value1":0.2299372870124918, "value2":6913359063794068428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.405529, "value1":0.49571648923268996, "value2":1675726536905469427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_8":"i","key_2":"j", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605031, "value1":0.09917638686306547, "value2":3890367351649717445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194280, "value1":0.5756913899097403, "value2":6546196099308660172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763643, "value1":0.049961431289849915, "value2":2156338695320389229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755741, "value1":0.4853393193247385, "value2":5380709479032006182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434015, "value1":0.6730243170500183, "value2":4072172223941247851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.188499, "value1":0.8036991963254426, "value2":2042780350133201142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.788778, "value1":0.6590319721348468, "value2":6545712378323399560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431899, "value1":0.10111044541197162, "value2":3033625496942345343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846633, "value1":0.4624813135357754, "value2":5961232902625455903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649911, "value1":0.42335939078880974, "value2":9206873087714105578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532557, "value1":0.8374513257646443, "value2":4063679098902368460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.993435, "value1":0.026158890681344544, "value2":8561365511354827010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808220, "value1":0.6771449672369052, "value2":2392568978099066809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_2":"h","key_9":"g","key_0":"b", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357037, "value1":0.7840791560680648, "value2":1293280054604430799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686467, "value1":0.009361168431443715, "value2":1158105653011230048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.971166, "value1":0.5749373357687312, "value2":5056280036307903399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.544825, "value1":0.6259231852657728, "value2":1572064409805832355, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_9":"c","key_5":"i", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889816, "value1":0.6691815472587526, "value2":5159786074932669495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751169, "value1":0.5117344773320296, "value2":2709445168494389044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.358316, "value1":0.5060937970199147, "value2":4487218678538317276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.577717, "value1":0.8145066136721544, "value2":3597604992290980548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.990084, "value1":0.031203549542102884, "value2":6033709675731250939, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613856, "value1":0.5443798055071071, "value2":1023967738912955990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795434, "value1":0.2935457393386216, "value2":1392092134182260244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233100, "value1":0.7087138744541079, "value2":6901046651435105011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.215643, "value1":0.301381153698089, "value2":5724617386824483093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.536387, "value1":0.050992650300612716, "value2":1926309233688447762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_9":"c","key_3":"e", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.661031, "value1":0.5259350399353443, "value2":3757516621940142781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.883217, "value1":0.9897587697648693, "value2":166453122026887271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482631, "value1":0.7903296358928328, "value2":6917975765021057279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301142, "value1":0.03214802664084894, "value2":1209824808667363268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618630, "value1":0.34174203368906997, "value2":687273686120110296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872422, "value1":0.7312694201106051, "value2":1954246364933112681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675094, "value1":0.02386805295420225, "value2":2582799143372120508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.397537, "value1":0.7375439234748727, "value2":3761195316124557282, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998621, "value1":0.44001824362826686, "value2":2298817484668315029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471460, "value1":0.5040752524973563, "value2":7440124910918057262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303559, "value1":0.34760281561681783, "value2":3856193801537489239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id":-6361865120410491981, "__mgmt_id":-6361865120410491981} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477238, "value1":0.10358452410298057, "value2":3433455763442399890, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_5":"g","key_6":"a","key_4":"g", "__series_id":-7417794931361433675, "__mgmt_id":-7417794931361433675} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974082, "value1":0.9936842430072798, "value2":7647779573135010712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id":5649892766065403969, "__mgmt_id":5649892766065403969} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.679212, "value1":0.33597414933520936, "value2":7871330604163867562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id":2491409761442743947, "__mgmt_id":2491409761442743947} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.420730, "value1":0.4119033435452023, "value2":7398827678901559124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.546764, "value1":0.0476713858704466, "value2":112659874965942961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id":-8265343809411007481, "__mgmt_id":-8265343809411007481} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730784, "value1":0.2868023177018098, "value2":7913444002207179200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id":-2350915848502491710, "__mgmt_id":-2350915848502491710} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862440, "value1":0.5312714100525405, "value2":6175515648240114931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id":2613217386916107323, "__mgmt_id":2613217386916107323} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769163, "value1":0.9602531490116414, "value2":5345461780217485343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id":-3082451392512806683, "__mgmt_id":-3082451392512806683} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855124, "value1":0.553411156209368, "value2":4062088865979243439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id":4593224894946540012, "__mgmt_id":4593224894946540012} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.210418, "value1":0.8711908622408605, "value2":8107060141858943082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id":-4771670780972268233, "__mgmt_id":-4771670780972268233} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436733, "value1":0.7746479416891845, "value2":3021240871572817126, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id":-3867686789927146028, "__mgmt_id":-3867686789927146028} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.692065, "value1":0.594498398968098, "value2":988266694346712016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id":-7557088021808002998, "__mgmt_id":-7557088021808002998} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.441926, "value1":0.8776750344471822, "value2":6190746113032965957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id":-1674287583760082371, "__mgmt_id":-1674287583760082371} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013512, "value1":0.14091811490255743, "value2":4960449076483043020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id":606159230414197914, "__mgmt_id":606159230414197914} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548236, "value1":0.32376195419487824, "value2":4889372139161861463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id":1993954834508334303, "__mgmt_id":1993954834508334303} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.828595, "value1":0.6520731819283484, "value2":508152344585097791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id":4180237166358025978, "__mgmt_id":4180237166358025978} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759392, "value1":0.11332264235829892, "value2":8149117049560006848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id":6877367775299575478, "__mgmt_id":6877367775299575478} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901588, "value1":0.1665732927247099, "value2":1166517989622232910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id":1066752686003512442, "__mgmt_id":1066752686003512442} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035913, "value1":0.07988414283718974, "value2":7894355898795371390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id":1070579679626379987, "__mgmt_id":1070579679626379987} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.583982, "value1":0.25757663417932825, "value2":6007061793954380382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id":-1894427622426263214, "__mgmt_id":-1894427622426263214} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636209, "value1":0.38383005316111324, "value2":4618851528086053316, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id":1193720041638406067, "__mgmt_id":1193720041638406067} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672395, "value1":0.002191687957779578, "value2":1207882083575373188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_5":"c","key_7":"f","key_0":"f", "__series_id":-3676185689090961166, "__mgmt_id":-3676185689090961166} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462535, "value1":0.3192805953827191, "value2":365124979437384544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id":1957099006661774855, "__mgmt_id":1957099006661774855} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508318, "value1":0.6798889100739353, "value2":255815407107212152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id":-3818704758089256053, "__mgmt_id":-3818704758089256053} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905933, "value1":0.882876839434679, "value2":1335222091221270902, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id":399363825354316617, "__mgmt_id":399363825354316617} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.721902, "value1":0.3457876641727297, "value2":7448197074268866329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id":5494931827757869027, "__mgmt_id":5494931827757869027} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.837057, "value1":0.03599562374649498, "value2":1792244431983896309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id":-8183722192123266908, "__mgmt_id":-8183722192123266908} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056476, "value1":0.22903239483804314, "value2":5638633766102759583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_7":"j","key_2":"e", "__series_id":-2041989208784968202, "__mgmt_id":-2041989208784968202} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.457531, "value1":0.05016525338775747, "value2":2976936137139451754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id":-6691559199598613340, "__mgmt_id":-6691559199598613340} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617621, "value1":0.16600638886403743, "value2":3903308221116403976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id":-3560743409529210195, "__mgmt_id":-3560743409529210195} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415798, "value1":0.5206207545088288, "value2":7426460172854544634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id":1507879893505985029, "__mgmt_id":1507879893505985029} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.470076, "value1":0.02758169975135837, "value2":3279193513719457677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id":143205045117664527, "__mgmt_id":143205045117664527} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.847049, "value1":0.818759672974311, "value2":5946222659493106047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id":1382081309323095369, "__mgmt_id":1382081309323095369} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367937, "value1":0.4074258090017152, "value2":6858775210252472573, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id":-1504596091368927264, "__mgmt_id":-1504596091368927264} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973712, "value1":0.13887537149474657, "value2":2800343207358775074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id":6028509310571555531, "__mgmt_id":6028509310571555531} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403585, "value1":0.2978164609006992, "value2":6234312472150253679, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id":1849668056084388119, "__mgmt_id":1849668056084388119} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904490, "value1":0.7280336451114688, "value2":2026079147622689137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id":-311983220320231940, "__mgmt_id":-311983220320231940} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.541237, "value1":0.1368259441713387, "value2":6424894922861577254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id":-2882676588454577349, "__mgmt_id":-2882676588454577349} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.783065, "value1":0.5704544299311838, "value2":4180282487041693175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id":-3229117909436309362, "__mgmt_id":-3229117909436309362} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.890800, "value1":0.7757728483786852, "value2":8132981856885227577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id":-8814450514247281081, "__mgmt_id":-8814450514247281081} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901025, "value1":0.22908718180210688, "value2":4747104448031391939, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id":-59315983402160733, "__mgmt_id":-59315983402160733} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180212, "value1":0.5476402281995001, "value2":8223100966387528196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id":-2101950891145961065, "__mgmt_id":-2101950891145961065} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626159, "value1":0.7996650121856338, "value2":64003761107808118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_9":"g","key_3":"k","key_4":"f", "__series_id":-8681749725467241001, "__mgmt_id":-8681749725467241001} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545550, "value1":0.1740742638651991, "value2":5824563041193046282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id":-781004633247834938, "__mgmt_id":-781004633247834938} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026911, "value1":0.9915609719154322, "value2":3839917984332114189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id":6425939792925870980, "__mgmt_id":6425939792925870980} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970605, "value1":0.049651840615534626, "value2":3663818871674092196, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id":8203812288919961648, "__mgmt_id":8203812288919961648} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.795861, "value1":0.6046374060545386, "value2":6467693625848837489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id":6111785442276013538, "__mgmt_id":6111785442276013538} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741291, "value1":0.4689122935380642, "value2":8701125505618349182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_6":"h","key_2":"k","key_4":"e", "__series_id":-8269425104286464293, "__mgmt_id":-8269425104286464293} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.877913, "value1":0.596765963129067, "value2":8094571743010403516, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id":7532055644348176501, "__mgmt_id":7532055644348176501} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.790394, "value1":0.06093826578205693, "value2":733010313976904838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_9":"b","key_1":"e", "__series_id":8866600531285973006, "__mgmt_id":8866600531285973006} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603208, "value1":0.8364659181351286, "value2":1088146971265159206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id":6579635354321950989, "__mgmt_id":6579635354321950989} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823621, "value1":0.47775163306986124, "value2":7344279199603749996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id":-8314831493892247699, "__mgmt_id":-8314831493892247699} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171666, "value1":0.02782471041039711, "value2":1659046247294927834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id":3850395059791465036, "__mgmt_id":3850395059791465036} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.899246, "value1":0.23664326650032677, "value2":764233009249486091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id":7996759019555320515, "__mgmt_id":7996759019555320515} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658885, "value1":0.13203154011115373, "value2":5571262153472749697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_4":"a","key_7":"k","key_1":"j", "__series_id":8816773697446392595, "__mgmt_id":8816773697446392595} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.711786, "value1":0.6314799048378197, "value2":8955079539774261393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id":-3451026078807038448, "__mgmt_id":-3451026078807038448} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.358690, "value1":0.4626190815088963, "value2":5349880367436474865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id":-5636974685330669635, "__mgmt_id":-5636974685330669635} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436669, "value1":0.7604788504004301, "value2":2830736413446294733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id":7762095970378497348, "__mgmt_id":7762095970378497348} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283630, "value1":0.11435547311453685, "value2":5764151651661369748, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id":-7984852869056523373, "__mgmt_id":-7984852869056523373} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550602, "value1":0.9394300496620045, "value2":3385128630739961936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id":-910590231097844596, "__mgmt_id":-910590231097844596} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004479, "value1":0.08103496493647538, "value2":1096104398248692524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id":-471143189299142315, "__mgmt_id":-471143189299142315} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272179, "value1":0.007258870887977933, "value2":2565647835692463774, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id":-6570289835833793858, "__mgmt_id":-6570289835833793858} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.207218, "value1":0.8139650845703748, "value2":905077995346686402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_3":"g","key_9":"k","key_0":"g", "__series_id":1154074168030394148, "__mgmt_id":1154074168030394148} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217842, "value1":0.48125835395143424, "value2":6789722720604854599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id":-1497332666716611286, "__mgmt_id":-1497332666716611286} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.155093, "value1":0.7189778774843847, "value2":6292044296017930854, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id":-1714854460223739254, "__mgmt_id":-1714854460223739254} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723501, "value1":0.3703438091023322, "value2":3955334986917106318, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_9":"f","key_3":"j","key_5":"i", "__series_id":2282711603191250138, "__mgmt_id":2282711603191250138} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216869, "value1":0.12053628422917362, "value2":636444633907082482, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id":6573048433538125738, "__mgmt_id":6573048433538125738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.568591, "value1":0.23735656022841886, "value2":4679225743620568386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id":6163909700883376820, "__mgmt_id":6163909700883376820} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.792289, "value1":0.8769118352474913, "value2":8705903369099981004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id":-9084206948870776280, "__mgmt_id":-9084206948870776280} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.377367, "value1":0.9028525772471804, "value2":8394233771809162273, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_3":"d","key_6":"c","key_0":"d", "__series_id":-5863629355677328674, "__mgmt_id":-5863629355677328674} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511370, "value1":0.06603888812647277, "value2":1587071294143636433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id":571477606045741529, "__mgmt_id":571477606045741529} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060860, "value1":0.3291125358805135, "value2":7312301032222805401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id":4534444186575327872, "__mgmt_id":4534444186575327872} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816308, "value1":0.6388914782369252, "value2":1128303197769336870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id":6974900966400270870, "__mgmt_id":6974900966400270870} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.068805, "value1":0.319277617656504, "value2":6681946261878539245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id":-3699381386157183052, "__mgmt_id":-3699381386157183052} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.805301, "value1":0.009280749666995351, "value2":544578594920738627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id":-640392872965375373, "__mgmt_id":-640392872965375373} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493659, "value1":0.6717590147189912, "value2":2097325929268646339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id":467337617461513179, "__mgmt_id":467337617461513179} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.253857, "value1":0.2354585485783823, "value2":5182832084934966477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id":-186107375404205023, "__mgmt_id":-186107375404205023} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454134, "value1":0.7059786460183258, "value2":8734869026681606253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id":3255744113557958985, "__mgmt_id":3255744113557958985} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769480, "value1":0.16571806998598893, "value2":1451771174019239273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id":-5608070008742618972, "__mgmt_id":-5608070008742618972} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.043302, "value1":0.0805122859716286, "value2":8615098369512434691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id":-5630411415645077903, "__mgmt_id":-5630411415645077903} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493432, "value1":0.37562446579716846, "value2":447590635610718249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id":1861220525601045919, "__mgmt_id":1861220525601045919} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.340933, "value1":0.476702778910786, "value2":2962733058848207480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id":8215289218858840673, "__mgmt_id":8215289218858840673} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.351270, "value1":0.04893100925940487, "value2":3216639061913163460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id":5453110602370885979, "__mgmt_id":5453110602370885979} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819062, "value1":0.43774247002290106, "value2":6810500585509381104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id":3212540495939612883, "__mgmt_id":3212540495939612883} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.789988, "value1":0.13044652854726224, "value2":6447819044361282673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id":7697276239589840540, "__mgmt_id":7697276239589840540} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.137973, "value1":0.9280703615023543, "value2":2854967575973809701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id":8069055566639478930, "__mgmt_id":8069055566639478930} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359816, "value1":0.6661196608352099, "value2":7585154254761447569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id":-1514690877472430993, "__mgmt_id":-1514690877472430993} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.374877, "value1":0.2719104405916207, "value2":1097487272974828210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id":2509711739945206486, "__mgmt_id":2509711739945206486} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766465, "value1":0.8656426220974218, "value2":2886501368125017740, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id":850770171572844857, "__mgmt_id":850770171572844857} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267362, "value1":0.594401465949902, "value2":1715419869280476267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id":-6714122630243923351, "__mgmt_id":-6714122630243923351} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.689698, "value1":0.09904556283985433, "value2":361858677456228074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id":1425425266702944455, "__mgmt_id":1425425266702944455} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244000, "value1":0.4739214533638735, "value2":6722075717885814318, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id":-1511448512568506759, "__mgmt_id":-1511448512568506759} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957522, "value1":0.4793380026682715, "value2":8094832594089347058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id":8031847456333582984, "__mgmt_id":8031847456333582984} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578816, "value1":0.32497702056237604, "value2":8938729328568281395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_6":"h","key_1":"f","key_5":"e", "__series_id":-5978406926652878392, "__mgmt_id":-5978406926652878392} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999599, "value1":0.703313156867648, "value2":5755903281901537381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id":5179748796340991382, "__mgmt_id":5179748796340991382} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829223, "value1":0.9601134377422823, "value2":4062518676536818057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id":-1943556293666606448, "__mgmt_id":-1943556293666606448} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.145200, "value1":0.7216961923243395, "value2":9217282407920967916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id":-2041801578517922928, "__mgmt_id":-2041801578517922928} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820803, "value1":0.3093137747462916, "value2":7468155061834676810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id":6273045326923481571, "__mgmt_id":6273045326923481571} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682889, "value1":0.04485861766843415, "value2":4270651219964958841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id":1613127414612560725, "__mgmt_id":1613127414612560725} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549256, "value1":0.7924629761166294, "value2":5462651594092502924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id":-1984980466248998163, "__mgmt_id":-1984980466248998163} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515774, "value1":0.1753099089980214, "value2":4245273970497284960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id":-3125920023802002767, "__mgmt_id":-3125920023802002767} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280472, "value1":0.19807313433421816, "value2":6932983129449398217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id":6282741732940207638, "__mgmt_id":6282741732940207638} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.183908, "value1":0.8803355871960044, "value2":3795185008893112660, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id":8373535740233056743, "__mgmt_id":8373535740233056743} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.250576, "value1":0.8766744958241532, "value2":8316217539045644702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id":-3343911352267775498, "__mgmt_id":-3343911352267775498} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811589, "value1":0.7673136679205708, "value2":1656000207332411296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id":-3921123017101809079, "__mgmt_id":-3921123017101809079} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617752, "value1":0.33750626542384204, "value2":2818737959409200195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id":-1426521396101200607, "__mgmt_id":-1426521396101200607} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.402465, "value1":0.5975891290632979, "value2":8117852232376404604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id":7638108789977515061, "__mgmt_id":7638108789977515061} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.779395, "value1":0.6126614135133973, "value2":1972761209604651335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id":9194958382530241609, "__mgmt_id":9194958382530241609} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171014, "value1":0.3147679970132471, "value2":3872394099215939892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id":4793341946183464793, "__mgmt_id":4793341946183464793} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.602514, "value1":0.9979964274456362, "value2":8280861801559743509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id":9018975067096005230, "__mgmt_id":9018975067096005230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.870286, "value1":0.1346288239954595, "value2":2167488486464723184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id":8844840765227221040, "__mgmt_id":8844840765227221040} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.902696, "value1":0.53464205049288, "value2":8019370943564861440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id":597549186953823831, "__mgmt_id":597549186953823831} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050828, "value1":0.7038559591596831, "value2":6864265904757939564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id":-3268807719566530897, "__mgmt_id":-3268807719566530897} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.465826, "value1":0.7686595552419602, "value2":1927886873036321447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id":-7441759397162295151, "__mgmt_id":-7441759397162295151} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149509, "value1":0.9153544198103084, "value2":7814774915075726621, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_9":"f","key_0":"k","key_1":"k", "__series_id":-1794846826413575242, "__mgmt_id":-1794846826413575242} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.485222, "value1":0.9188620757546552, "value2":5192187368195488529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id":-6864550317982743146, "__mgmt_id":-6864550317982743146} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520165, "value1":0.1037499454739713, "value2":2861179515410191072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id":-8442342819784056585, "__mgmt_id":-8442342819784056585} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894795, "value1":0.6670064141722539, "value2":8570373623610344103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id":-362456897096662535, "__mgmt_id":-362456897096662535} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858375, "value1":0.9690125335533832, "value2":6598305524856472910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id":9126429541977437814, "__mgmt_id":9126429541977437814} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639161, "value1":0.07029962179108588, "value2":4198012464202851461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id":5675848291676949887, "__mgmt_id":5675848291676949887} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.422856, "value1":0.243592288273798, "value2":7060563137853890653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id":-8033450871750625569, "__mgmt_id":-8033450871750625569} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356376, "value1":0.12506395613411664, "value2":3055799774165215720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id":2604548282526698701, "__mgmt_id":2604548282526698701} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296059, "value1":0.7972570100142576, "value2":2611996617533897408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id":4117833149263619100, "__mgmt_id":4117833149263619100} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.461343, "value1":0.024502184861611945, "value2":1123093231220631621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id":9137075327044920409, "__mgmt_id":9137075327044920409} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529133, "value1":0.12083989402126777, "value2":7219868115171352433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id":6357857786217316014, "__mgmt_id":6357857786217316014} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110681, "value1":0.8049351220640796, "value2":4305443631669757518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id":6767355780455098874, "__mgmt_id":6767355780455098874} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.981566, "value1":0.19931635115078358, "value2":4376781646759017505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id":4247426367176705919, "__mgmt_id":4247426367176705919} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631835, "value1":0.4428488032269089, "value2":2962840158667509272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id":2924902692178299433, "__mgmt_id":2924902692178299433} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596817, "value1":0.01388179851872705, "value2":5526059113621970146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id":1154434336808537704, "__mgmt_id":1154434336808537704} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988044, "value1":0.08966434896908937, "value2":845094337456207269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id":-7451864092156701946, "__mgmt_id":-7451864092156701946} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032366, "value1":0.4632294657773622, "value2":2896178627081634846, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_9":"i","key_0":"i","key_3":"a", "__series_id":-2069560104892378999, "__mgmt_id":-2069560104892378999} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.195349, "value1":0.3544490738754114, "value2":5485050023862879453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id":-8474357488465320783, "__mgmt_id":-8474357488465320783} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.488827, "value1":0.05815655955779658, "value2":5446494658153199948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id":1284856134736405307, "__mgmt_id":1284856134736405307} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723246, "value1":0.6834006388190103, "value2":4274252704044712919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id":-3182385159433660007, "__mgmt_id":-3182385159433660007} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141449, "value1":0.3967110267075465, "value2":2849522299224594762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id":-7985943178384355012, "__mgmt_id":-7985943178384355012} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.712863, "value1":0.2643526520952433, "value2":6055035031034459665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id":5765688178051837788, "__mgmt_id":5765688178051837788} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974542, "value1":0.07826746989883603, "value2":8013938940173093770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id":8590391987934982290, "__mgmt_id":8590391987934982290} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476496, "value1":0.08282820679282239, "value2":8740534065670450787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id":-558308991247212939, "__mgmt_id":-558308991247212939} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.265568, "value1":0.7141009753996177, "value2":8015024495118709084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id":-6238834606581445812, "__mgmt_id":-6238834606581445812} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163000, "value1":0.9787912408769939, "value2":1407010919457389562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id":2031701996903259529, "__mgmt_id":2031701996903259529} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560997, "value1":0.22142486556739396, "value2":2439658579639630159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id":-8634907421753499502, "__mgmt_id":-8634907421753499502} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325355, "value1":0.2775049821475937, "value2":2346830518660166745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id":-3116728592375192395, "__mgmt_id":-3116728592375192395} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491491, "value1":0.3135314864967855, "value2":1129771089143019455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id":28452354456102768, "__mgmt_id":28452354456102768} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.459317, "value1":0.2812010467625179, "value2":7830625013744386941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id":6364458956549614348, "__mgmt_id":6364458956549614348} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.684694, "value1":0.4935589110227349, "value2":2540823529765104488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id":-2332013085895889868, "__mgmt_id":-2332013085895889868} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.237886, "value1":0.07014579422912577, "value2":7022680572815993337, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id":5563967542164640109, "__mgmt_id":5563967542164640109} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.414046, "value1":0.06496797245021041, "value2":2711277679569543532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799639, "value1":0.8106870375099371, "value2":1007449970411192794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id":9223070499161054046, "__mgmt_id":9223070499161054046} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.264752, "value1":0.0960343886163872, "value2":3312684645465619564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id":-1724818728110458595, "__mgmt_id":-1724818728110458595} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035469, "value1":0.0008360965935936516, "value2":5539611781321403864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id":8270758825059813284, "__mgmt_id":8270758825059813284} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.306352, "value1":0.07245144250837937, "value2":3686038376310116060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id":-590968405571452953, "__mgmt_id":-590968405571452953} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.048629, "value1":0.3482697350949442, "value2":2011496763457379089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id":-1784521893174042304, "__mgmt_id":-1784521893174042304} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978673, "value1":0.9982728323868918, "value2":702860377347588565, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id":-2817668530773309530, "__mgmt_id":-2817668530773309530} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987564, "value1":0.21515438859453273, "value2":2583082073184523816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id":7423366130131613786, "__mgmt_id":7423366130131613786} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.842602, "value1":0.3444834642501587, "value2":4908475207072135082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id":3207093433415838288, "__mgmt_id":3207093433415838288} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966769, "value1":0.7858094998520652, "value2":6077992589658011123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id":3452730991306891257, "__mgmt_id":3452730991306891257} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.165450, "value1":0.6383111602434741, "value2":7226409376141032655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id":-6643065536824436920, "__mgmt_id":-6643065536824436920} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.575923, "value1":0.8562665414193692, "value2":2173652645149705334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id":-8405131525831889671, "__mgmt_id":-8405131525831889671} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010707, "value1":0.5955696347147411, "value2":5814682382951778842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id":2961372573105506275, "__mgmt_id":2961372573105506275} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271135, "value1":0.1229263700863702, "value2":6350918748538162755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id":-8351460038839747494, "__mgmt_id":-8351460038839747494} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143782, "value1":0.975371469047403, "value2":8279280670157614673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id":-4057754142371582237, "__mgmt_id":-4057754142371582237} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536001, "value1":0.2818838543932868, "value2":5548964524114515649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id":-1798332360868592711, "__mgmt_id":-1798332360868592711} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.274203, "value1":0.525661345793623, "value2":7632566212944122955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id":-7847383196004021619, "__mgmt_id":-7847383196004021619} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394070, "value1":0.7913688238824117, "value2":2991067692464176886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id":1804397642211293298, "__mgmt_id":1804397642211293298} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243187, "value1":0.9989819654882474, "value2":7814314371764134982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id":3764804326138865786, "__mgmt_id":3764804326138865786} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490869, "value1":0.30039262696070124, "value2":7927229014613646504, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_6":"i","key_0":"f","key_5":"i", "__series_id":-2684979140479497919, "__mgmt_id":-2684979140479497919} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.850916, "value1":0.8481086988896461, "value2":3502125731621593449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id":-5840655664568080155, "__mgmt_id":-5840655664568080155} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113680, "value1":0.32930706694057965, "value2":6445034393695608409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id":5570741952766792787, "__mgmt_id":5570741952766792787} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192297, "value1":0.3973423811708676, "value2":8167271156640822516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id":6454348724083965233, "__mgmt_id":6454348724083965233} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646981, "value1":0.6421917320177064, "value2":2441775949474583008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id":7915149619701574573, "__mgmt_id":7915149619701574573} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.482724, "value1":0.3152543309412001, "value2":8846147135578854122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id":5504457377504091819, "__mgmt_id":5504457377504091819} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777161, "value1":0.26611205622992573, "value2":4707069095348547526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id":-1599150549254325081, "__mgmt_id":-1599150549254325081} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529669, "value1":0.25143936257925226, "value2":9113227763618672958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id":4231140258106081014, "__mgmt_id":4231140258106081014} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708875, "value1":0.10515643642729285, "value2":7582828897771165151, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id":-317199018426739656, "__mgmt_id":-317199018426739656} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041864, "value1":0.7204022074713844, "value2":4505351249313209540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id":3354108409869678272, "__mgmt_id":3354108409869678272} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.517118, "value1":0.2778141202519239, "value2":7041089660646534661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id":-4244341193881875964, "__mgmt_id":-4244341193881875964} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255695, "value1":0.5252045968012302, "value2":1609597628350217466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id":4564325618898740878, "__mgmt_id":4564325618898740878} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.001450, "value1":0.15419422555422022, "value2":6306687575854728640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id":-5304717018001475026, "__mgmt_id":-5304717018001475026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.230403, "value1":0.7229181740004053, "value2":1306679799234664681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id":87816280540931590, "__mgmt_id":87816280540931590} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035841, "value1":0.6895973487797646, "value2":2559143873675894370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id":-404532909115380097, "__mgmt_id":-404532909115380097} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.449904, "value1":0.7368166767257398, "value2":7052097343090100521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id":7971366120583252671, "__mgmt_id":7971366120583252671} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139346, "value1":0.807172748525886, "value2":4308240895194146306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id":-4147830406690806858, "__mgmt_id":-4147830406690806858} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977592, "value1":0.7560595694496225, "value2":3367349773561100464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id":-105708042353818101, "__mgmt_id":-105708042353818101} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.831129, "value1":0.43661553653957313, "value2":1514573148689105923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id":-3492803355001090052, "__mgmt_id":-3492803355001090052} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804508, "value1":0.7956852333376209, "value2":8820887809742814806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id":-5856257105266260668, "__mgmt_id":-5856257105266260668} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858280, "value1":0.8493727617212654, "value2":8684654170930913793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id":-7501418065343643474, "__mgmt_id":-7501418065343643474} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640988, "value1":0.2223414127619492, "value2":7250214226981174458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id":540058938037018937, "__mgmt_id":540058938037018937} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153838, "value1":0.9864893077863718, "value2":1366599113373589549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id":-5031137265295521924, "__mgmt_id":-5031137265295521924} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144317, "value1":0.034179582463187676, "value2":4811674883286124820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_4":"g","key_7":"h","key_1":"k", "__series_id":6384311434791564160, "__mgmt_id":6384311434791564160} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.670072, "value1":0.3811265659697653, "value2":4090252794464650298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id":-2628080776869037402, "__mgmt_id":-2628080776869037402} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486186, "value1":0.9084897972797951, "value2":2292875585752622652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id":-6809092537603749426, "__mgmt_id":-6809092537603749426} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724703, "value1":0.04582892017411178, "value2":5556549536478579022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id":358820623790332114, "__mgmt_id":358820623790332114} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012322, "value1":0.6305358760062275, "value2":1288975268472701530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_6":"a","key_8":"f","key_0":"f", "__series_id":-2847970172981397249, "__mgmt_id":-2847970172981397249} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.218849, "value1":0.8926861107470235, "value2":7051558642534889749, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id":2557920836494908533, "__mgmt_id":2557920836494908533} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.577242, "value1":0.96150058991849, "value2":5205402300881087071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id":7929286596360127546, "__mgmt_id":7929286596360127546} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525938, "value1":0.010643579624627687, "value2":9179800346032814438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id":7118429874597622369, "__mgmt_id":7118429874597622369} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393624, "value1":0.5282994953423363, "value2":6629726373755372175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id":-7936570873523680981, "__mgmt_id":-7936570873523680981} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697100, "value1":0.328819156896584, "value2":7890032700245067940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id":3639352674740645135, "__mgmt_id":3639352674740645135} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.911016, "value1":0.23451404605523937, "value2":1559532623894065416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id":8823283681255314421, "__mgmt_id":8823283681255314421} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128593, "value1":0.958219222491235, "value2":5238496228333042095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id":-4500545634898797152, "__mgmt_id":-4500545634898797152} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704394, "value1":0.5513948920447201, "value2":9062299692444161349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id":1531212333300811705, "__mgmt_id":1531212333300811705} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255075, "value1":0.35898456496590414, "value2":5087989108776558277, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id":-4469691104781668428, "__mgmt_id":-4469691104781668428} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788201, "value1":0.7466905262312481, "value2":479894845243154201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id":-2014901588395566836, "__mgmt_id":-2014901588395566836} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565236, "value1":0.15707319443827472, "value2":93010754190809493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id":6517439227519034897, "__mgmt_id":6517439227519034897} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462851, "value1":0.1393959893683005, "value2":1383438862980713878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id":2739631612785937741, "__mgmt_id":2739631612785937741} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.587928, "value1":0.9257613742750218, "value2":461452136435423636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id":-4056544719737454862, "__mgmt_id":-4056544719737454862} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.204910, "value1":0.45361150342145223, "value2":8114896084179399798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id":-3440838895860869905, "__mgmt_id":-3440838895860869905} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970393, "value1":0.8262379198959926, "value2":1410864297783479414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_1":"e","key_3":"g","key_0":"k", "__series_id":-4893960988253975402, "__mgmt_id":-4893960988253975402} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571697, "value1":0.05523474900350812, "value2":6912470816433257487, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id":278539141476854582, "__mgmt_id":278539141476854582} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900965, "value1":0.3994455375010903, "value2":5910800813383852824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id":-2634149506137991219, "__mgmt_id":-2634149506137991219} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533983, "value1":0.8439578978089138, "value2":6520871065252687028, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id":-6152235210239582771, "__mgmt_id":-6152235210239582771} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732970, "value1":0.4096847894313233, "value2":4961560961953820016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id":-9078008273642591081, "__mgmt_id":-9078008273642591081} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.512990, "value1":0.9925070463343235, "value2":4031040144184448394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_9":"j","key_3":"b","key_5":"a", "__series_id":3789075411738053288, "__mgmt_id":3789075411738053288} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.875107, "value1":0.8529856732344743, "value2":8171410486624791720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id":2418661337577043035, "__mgmt_id":2418661337577043035} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.662542, "value1":0.7969197657043129, "value2":4707105172837538302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id":-8769274023320427631, "__mgmt_id":-8769274023320427631} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.132562, "value1":0.5095481874741737, "value2":8469586257322312509, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id":7110746347371624233, "__mgmt_id":7110746347371624233} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171675, "value1":0.7140266335782444, "value2":8998618655550288053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_9":"h","key_1":"h","key_4":"a", "__series_id":-9044049296600485351, "__mgmt_id":-9044049296600485351} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829529, "value1":0.8556707112048024, "value2":84360751498479327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id":-2702487361565044629, "__mgmt_id":-2702487361565044629} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.383509, "value1":0.6267421890569228, "value2":3548620263321255935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_5":"k","key_7":"h","key_1":"a", "__series_id":-4467063443052925540, "__mgmt_id":-4467063443052925540} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725412, "value1":0.6792289410958199, "value2":8743030074795737682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id":-4360566614594027009, "__mgmt_id":-4360566614594027009} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217960, "value1":0.06523873388277363, "value2":4237433473740985929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id":-7365143648791602426, "__mgmt_id":-7365143648791602426} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.052828, "value1":0.39596522726411565, "value2":5268673916985408378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id":-5771429335930317158, "__mgmt_id":-5771429335930317158} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243583, "value1":0.24855400987123744, "value2":7145331513033105803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id":-5063514376085494737, "__mgmt_id":-5063514376085494737} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.643757, "value1":0.4621121441928861, "value2":4607779943623171423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id":-384155371210942527, "__mgmt_id":-384155371210942527} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777085, "value1":0.9713754435345697, "value2":3872286492266737542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id":9053870091904767989, "__mgmt_id":9053870091904767989} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741256, "value1":0.33772530129863026, "value2":2720260557973633476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id":-6003259494256310900, "__mgmt_id":-6003259494256310900} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040901, "value1":0.5366678442503715, "value2":5043767454211598671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id":921644124074663399, "__mgmt_id":921644124074663399} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.566517, "value1":0.3199753043754913, "value2":1787736737657465265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id":-7491755130759432770, "__mgmt_id":-7491755130759432770} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871703, "value1":0.40150570806322944, "value2":7426113550614212915, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id":163960823954443450, "__mgmt_id":163960823954443450} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.063390, "value1":0.8093881999536079, "value2":6062591517984503571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id":-1595629483288736026, "__mgmt_id":-1595629483288736026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630695, "value1":0.5987756911109672, "value2":4564890959914284940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id":-3765777172219922311, "__mgmt_id":-3765777172219922311} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089316, "value1":0.9756718556017356, "value2":4422554680156925050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id":1085094834466986289, "__mgmt_id":1085094834466986289} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433679, "value1":0.6504367128725771, "value2":7179863028410040853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id":7762506373054977320, "__mgmt_id":7762506373054977320} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418340, "value1":0.5167928405286337, "value2":796797443554380087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id":-1219396340318367916, "__mgmt_id":-1219396340318367916} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148063, "value1":0.2982285613686749, "value2":3455828331199929793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id":4286369265264583743, "__mgmt_id":4286369265264583743} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.562273, "value1":0.1744043124258891, "value2":7056453619952557866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id":7176509012015840215, "__mgmt_id":7176509012015840215} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554680, "value1":0.9985428971035419, "value2":3134607000357482375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id":3478232514132647486, "__mgmt_id":3478232514132647486} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.498692, "value1":0.34351042666983833, "value2":9219919937586266788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id":3089314848340334430, "__mgmt_id":3089314848340334430} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983651, "value1":0.20667520837153053, "value2":8261113453188452260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id":2147688371268242959, "__mgmt_id":2147688371268242959} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900233, "value1":0.49430245377625404, "value2":1887699130768202184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id":-9171186733886989406, "__mgmt_id":-9171186733886989406} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.269041, "value1":0.24848195612656065, "value2":27297122896884729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id":-5350966201698202835, "__mgmt_id":-5350966201698202835} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.716521, "value1":0.703324321123364, "value2":5138993531610984864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id":235119588830124008, "__mgmt_id":235119588830124008} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.331287, "value1":0.6585119514499647, "value2":7805239120031598554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id":6120835208953874959, "__mgmt_id":6120835208953874959} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.898740, "value1":0.7897434254535518, "value2":8577413153173117857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id":-3573749945377143552, "__mgmt_id":-3573749945377143552} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366667, "value1":0.4615000106211479, "value2":4478388423093675456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id":-4086421139340971808, "__mgmt_id":-4086421139340971808} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060808, "value1":0.3611416577570406, "value2":1763611736817673526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id":-7974731736398137339, "__mgmt_id":-7974731736398137339} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987608, "value1":0.6511729153448047, "value2":7895118250604250511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id":-5418919362836041490, "__mgmt_id":-5418919362836041490} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.920802, "value1":0.8966340709602193, "value2":1619385988446426949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id":-2260859392228394720, "__mgmt_id":-2260859392228394720} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759468, "value1":0.9772199770736059, "value2":486666067155730811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id":9030574374130186147, "__mgmt_id":9030574374130186147} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697569, "value1":0.2215623962484913, "value2":998498599369301173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id":712723991923371456, "__mgmt_id":712723991923371456} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550984, "value1":0.23501917809435333, "value2":203018852406902455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id":-153500247406501062, "__mgmt_id":-153500247406501062} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865253, "value1":0.24390812385740993, "value2":2606627451901201346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id":15499690335888556, "__mgmt_id":15499690335888556} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050134, "value1":0.6480270130439445, "value2":6247279314207653751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id":5372354603718802516, "__mgmt_id":5372354603718802516} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614221, "value1":0.7050465104295126, "value2":3377246595104905199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id":2200789686389610660, "__mgmt_id":2200789686389610660} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166533, "value1":0.3877660847452995, "value2":7536153185490054501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id":-4182605193058408022, "__mgmt_id":-4182605193058408022} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644807, "value1":0.7190092729952982, "value2":5813482232408839023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id":-2588047046090071899, "__mgmt_id":-2588047046090071899} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739661, "value1":0.7257627180887302, "value2":3214351184373245408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id":-610758200249505967, "__mgmt_id":-610758200249505967} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552516, "value1":0.4939513402753451, "value2":465350226895561432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id":4102647630186908830, "__mgmt_id":4102647630186908830} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503358, "value1":0.14929982411048387, "value2":2610344885697195594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id":-1094261238021429217, "__mgmt_id":-1094261238021429217} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.765487, "value1":0.45537690586006585, "value2":4288868163982148480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id":-8125072424306725518, "__mgmt_id":-8125072424306725518} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.838240, "value1":0.7605183341936481, "value2":7774940282079152902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id":7114248156060763562, "__mgmt_id":7114248156060763562} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646978, "value1":0.988755613370451, "value2":4713140451733570872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id":-581163781821289738, "__mgmt_id":-581163781821289738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746410, "value1":0.8174040579458727, "value2":7395122770602669759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id":6056165502075962127, "__mgmt_id":6056165502075962127} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823505, "value1":0.758453176804426, "value2":7518087779323040688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id":-5164253557477824984, "__mgmt_id":-5164253557477824984} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020233, "value1":0.29899993635819666, "value2":5085917800846866623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id":-3886540797918122503, "__mgmt_id":-3886540797918122503} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561926, "value1":0.8862149028157266, "value2":3614075635724672575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_2":"i","key_6":"f","key_0":"j", "__series_id":-1628388129285480657, "__mgmt_id":-1628388129285480657} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.437110, "value1":0.32845862992503655, "value2":2977427326936422575, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id":-2367882984291197965, "__mgmt_id":-2367882984291197965} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706546, "value1":0.15047181179887134, "value2":2307845458700050829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id":6636444104004800874, "__mgmt_id":6636444104004800874} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.619350, "value1":0.1117344044076149, "value2":4988357694886705006, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id":4540625073764757770, "__mgmt_id":4540625073764757770} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.808461, "value1":0.715214325361221, "value2":7172177171113874947, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id":-8073602608240199567, "__mgmt_id":-8073602608240199567} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154987, "value1":0.3436009437038216, "value2":35294647035701645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id":3782764195228049883, "__mgmt_id":3782764195228049883} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.293185, "value1":0.1982820741163276, "value2":4721034174581841393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id":-7196643947589422854, "__mgmt_id":-7196643947589422854} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.177501, "value1":0.24102187665043326, "value2":5183952097737049563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id":-2802937496335850283, "__mgmt_id":-2802937496335850283} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434009, "value1":0.4074440055222358, "value2":2740366314909969021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id":-7868070420964471300, "__mgmt_id":-7868070420964471300} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.962158, "value1":0.6945642879060417, "value2":8488893912024069218, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_3":"b","key_7":"e","key_0":"b", "__series_id":-107956853517303026, "__mgmt_id":-107956853517303026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037976, "value1":0.05977827983958813, "value2":8714018046983615411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id":8743004723181703205, "__mgmt_id":8743004723181703205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491879, "value1":0.5875339799333666, "value2":3507531774420107282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id":-6080411562657950991, "__mgmt_id":-6080411562657950991} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708852, "value1":0.941564068848663, "value2":7441568589430319196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id":-5085436153869299952, "__mgmt_id":-5085436153869299952} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665427, "value1":0.15650940720497003, "value2":2725018303601038315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id":-3652832040735163825, "__mgmt_id":-3652832040735163825} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728790, "value1":0.16909603056430217, "value2":7913664106090409634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id":-4827582452648415077, "__mgmt_id":-4827582452648415077} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748451, "value1":0.9309585511638759, "value2":6423846277129286464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id":-8331465659376192273, "__mgmt_id":-8331465659376192273} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701551, "value1":0.2886585284950581, "value2":1577558950874330311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id":-7479610835286291676, "__mgmt_id":-7479610835286291676} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356768, "value1":0.42594704925771354, "value2":3150589955894640969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id":8712448124612225936, "__mgmt_id":8712448124612225936} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056847, "value1":0.8050681337140542, "value2":3504760629437797890, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_9":"k","key_4":"b","key_7":"e", "__series_id":-6611753493518302504, "__mgmt_id":-6611753493518302504} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.797591, "value1":0.9381497767913094, "value2":8069233160965701213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id":7452684393613271105, "__mgmt_id":7452684393613271105} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.653188, "value1":0.21115616320062328, "value2":2091006864800018643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id":8312888208372477794, "__mgmt_id":8312888208372477794} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489289, "value1":0.6249865113250967, "value2":3636072468848176197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id":-3794951196896029722, "__mgmt_id":-3794951196896029722} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.426149, "value1":0.7525465264386705, "value2":2554715790387615481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id":2203039791051080232, "__mgmt_id":2203039791051080232} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629922, "value1":0.244780100518832, "value2":2525567513505106874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id":-3485863643022996311, "__mgmt_id":-3485863643022996311} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.652167, "value1":0.44363110594266053, "value2":2074305704102402281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id":-5266012011623334172, "__mgmt_id":-5266012011623334172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111575, "value1":0.7142448949298847, "value2":2599619943612027614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id":-3153928638315052797, "__mgmt_id":-3153928638315052797} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.720934, "value1":0.46314177190410183, "value2":5473312180675819943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_8":"d","key_9":"c","key_4":"d", "__series_id":-6076759433305586941, "__mgmt_id":-6076759433305586941} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.542023, "value1":0.17074635857117051, "value2":576994586783200158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id":8834769746480377365, "__mgmt_id":8834769746480377365} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862606, "value1":0.849600128173387, "value2":4044011490168844428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id":-8329489313689560671, "__mgmt_id":-8329489313689560671} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.581818, "value1":0.4717710471390932, "value2":2021524622009984217, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id":-9030241219395401074, "__mgmt_id":-9030241219395401074} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.634138, "value1":0.3045108371861026, "value2":769588972141174181, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id":5111506451882024399, "__mgmt_id":5111506451882024399} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554282, "value1":0.9061849342045651, "value2":5246859609213477778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id":-6099394299024364632, "__mgmt_id":-6099394299024364632} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.275353, "value1":0.12209379665466558, "value2":3903264969659125855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_8":"f","key_9":"i","key_5":"d", "__series_id":-2889220439504706291, "__mgmt_id":-2889220439504706291} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931382, "value1":0.38822648110972485, "value2":4321311553224557998, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id":-77561723918691242, "__mgmt_id":-77561723918691242} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.703867, "value1":0.6637597201672076, "value2":5683430536850722079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id":-1773617826702713581, "__mgmt_id":-1773617826702713581} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958497, "value1":0.011335239730124937, "value2":2491874770397540377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id":5942386536226088787, "__mgmt_id":5942386536226088787} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569325, "value1":0.8472934144463635, "value2":6850134665353105473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id":1447592414457730764, "__mgmt_id":1447592414457730764} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564173, "value1":0.28056996036761495, "value2":1661710999114510072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id":4924210048711090822, "__mgmt_id":4924210048711090822} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.046264, "value1":0.7988719698656381, "value2":80936298017494610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id":1859691237979991845, "__mgmt_id":1859691237979991845} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.086050, "value1":0.7269357269059649, "value2":3703081511929708827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id":6844297030045792921, "__mgmt_id":6844297030045792921} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.691310, "value1":0.968367674569065, "value2":8685309070712113009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id":-126935382287702459, "__mgmt_id":-126935382287702459} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.734580, "value1":0.16065685872806734, "value2":8341255006788851479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id":-8355447172459145172, "__mgmt_id":-8355447172459145172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110471, "value1":0.2857648075220212, "value2":7895547866112316696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id":8424755961445166330, "__mgmt_id":8424755961445166330} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.594791, "value1":0.6011580461407761, "value2":8667655502460513802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id":8975641431294639588, "__mgmt_id":8975641431294639588} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.074613, "value1":0.7518175733891237, "value2":6808912328433269494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id":-5148903865376086541, "__mgmt_id":-5148903865376086541} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827510, "value1":0.4635989764886008, "value2":8535571864109924880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id":1983990905315571262, "__mgmt_id":1983990905315571262} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020243, "value1":0.9625575743602495, "value2":3255772979573003294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id":7756220639980209274, "__mgmt_id":7756220639980209274} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508001, "value1":0.9246513785407203, "value2":6789019308045406067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_3":"e","key_2":"b", "__series_id":290238844948171466, "__mgmt_id":290238844948171466} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.175770, "value1":0.6231844141584031, "value2":5604032128383909386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id":2212165241620182378, "__mgmt_id":2212165241620182378} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338187, "value1":0.7085089535016765, "value2":7296894563489025180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id":-5641349172597598498, "__mgmt_id":-5641349172597598498} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924177, "value1":0.9072759724228442, "value2":2056681116966727911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id":4136822437169743683, "__mgmt_id":4136822437169743683} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474372, "value1":0.09374734312421011, "value2":8184327150731414451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id":5588418121070213379, "__mgmt_id":5588418121070213379} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970300, "value1":0.9441105421259884, "value2":6689514820107172641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_4":"f","key_9":"f","key_0":"e", "__series_id":5487859603492872728, "__mgmt_id":5487859603492872728} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035348, "value1":0.14347791413195674, "value2":2423771496925044659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id":-7510162853285205458, "__mgmt_id":-7510162853285205458} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.807051, "value1":0.1533684521364213, "value2":8351770123785179919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id":8960166785941818502, "__mgmt_id":8960166785941818502} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814282, "value1":0.48165153274600025, "value2":2348575893675627866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id":1191951441001778119, "__mgmt_id":1191951441001778119} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438370, "value1":0.4065426353146516, "value2":5566206066472585427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id":-7226896218622797912, "__mgmt_id":-7226896218622797912} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.321909, "value1":0.057863583124824335, "value2":2888660799506273189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id":-5283488199064223774, "__mgmt_id":-5283488199064223774} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.191383, "value1":0.5857228209038977, "value2":343682983842502232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id":-2097373818072827151, "__mgmt_id":-2097373818072827151} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.030645, "value1":0.054007147421613756, "value2":7408821068820948342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id":1668656312027665848, "__mgmt_id":1668656312027665848} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.022320, "value1":0.5082171149433462, "value2":708896583672613900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id":-3365185338503951486, "__mgmt_id":-3365185338503951486} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359147, "value1":0.10928795884420436, "value2":5509040341747881439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id":-1972359892487806876, "__mgmt_id":-1972359892487806876} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245107, "value1":0.08990020039433881, "value2":4415685661085482000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id":-5888979106861644322, "__mgmt_id":-5888979106861644322} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.685276, "value1":0.9327230470765273, "value2":7932626821430255186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id":4350337870315148248, "__mgmt_id":4350337870315148248} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160518, "value1":0.9697823127450904, "value2":9176665002489756811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851145, "value1":0.3835892105704814, "value2":5122340871731266104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id":407580274955802052, "__mgmt_id":407580274955802052} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390684, "value1":0.8638095835380812, "value2":2196309567306303445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id":5055564084756188165, "__mgmt_id":5055564084756188165} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197750, "value1":0.5321904528249127, "value2":7059427737086849843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id":-4657725461050027132, "__mgmt_id":-4657725461050027132} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.606374, "value1":0.1712913717696049, "value2":6671507460576496016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id":1853720382505804304, "__mgmt_id":1853720382505804304} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814189, "value1":0.21162810005419255, "value2":6089699189069603789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id":-4222087669256355081, "__mgmt_id":-4222087669256355081} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565029, "value1":0.651039299686122, "value2":2488735588911638142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id":9141431587741375337, "__mgmt_id":9141431587741375337} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310043, "value1":0.3341800696393593, "value2":1886591498798226069, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id":3652241825851980931, "__mgmt_id":3652241825851980931} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646773, "value1":0.587523277238801, "value2":3984875805092838666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id":8334871318878782618, "__mgmt_id":8334871318878782618} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487672, "value1":0.4085632878496485, "value2":4006763171436697564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id":-3854993404396318547, "__mgmt_id":-3854993404396318547} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180539, "value1":0.614724172390471, "value2":3474361726777387017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id":3475233701191086871, "__mgmt_id":3475233701191086871} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413423, "value1":0.8015313942257714, "value2":3699130733479555941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id":-3248778423373175907, "__mgmt_id":-3248778423373175907} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375412, "value1":0.6194400337512462, "value2":89560215401334488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id":-5544260272497947688, "__mgmt_id":-5544260272497947688} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739736, "value1":0.01565923183582112, "value2":2040823465506133480, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id":6759359888504379937, "__mgmt_id":6759359888504379937} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164236, "value1":0.05609752284300993, "value2":459144572970603788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_5":"a","key_8":"g","key_3":"b", "__series_id":2983615759963648182, "__mgmt_id":2983615759963648182} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857605, "value1":0.3918518578057805, "value2":2660076743726482223, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id":-5636891213483281862, "__mgmt_id":-5636891213483281862} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182324, "value1":0.19737269797519194, "value2":4090312209234045823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id":5337681060194140560, "__mgmt_id":5337681060194140560} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413638, "value1":0.23128603835680528, "value2":2432247666638343859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id":-8711805126627568168, "__mgmt_id":-8711805126627568168} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029664, "value1":0.7948810691175306, "value2":1090049786919167701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id":-3947171592563903621, "__mgmt_id":-3947171592563903621} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953471, "value1":0.7020043042253941, "value2":4064796540960008955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id":-6388370063319424844, "__mgmt_id":-6388370063319424844} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925306, "value1":0.8273248639036314, "value2":7383533353614933778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id":-2115245116045217035, "__mgmt_id":-2115245116045217035} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.353560, "value1":0.7154582581173291, "value2":7672433526090470392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id":6934366355856667102, "__mgmt_id":6934366355856667102} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.440133, "value1":0.8443212771497766, "value2":4871633339564000823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id":-620632640643666803, "__mgmt_id":-620632640643666803} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079309, "value1":0.9841787259191465, "value2":5158435517537975833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id":-2179974092013090529, "__mgmt_id":-2179974092013090529} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687146, "value1":0.547850782348478, "value2":107890618456621161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id":-296190224456934318, "__mgmt_id":-296190224456934318} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013887, "value1":0.9829126596506367, "value2":2398801344797864345, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id":8742164021866916569, "__mgmt_id":8742164021866916569} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142865, "value1":0.17685569327146947, "value2":2003576222991068603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id":4071060278153716716, "__mgmt_id":4071060278153716716} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503089, "value1":0.2089707954081848, "value2":5329263674364077685, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id":-9118286530436017049, "__mgmt_id":-9118286530436017049} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055628, "value1":0.9427569700789281, "value2":8376349363379609336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id":731731928735478358, "__mgmt_id":731731928735478358} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663522, "value1":0.5447970553906242, "value2":7034169322760923664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id":7612998775307430333, "__mgmt_id":7612998775307430333} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569193, "value1":0.4748210869235274, "value2":5009934188959179459, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id":1213816084730454228, "__mgmt_id":1213816084730454228} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.964851, "value1":0.1480433642451733, "value2":5780759776802899813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id":-3958867498699920655, "__mgmt_id":-3958867498699920655} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.674266, "value1":0.01926887357642196, "value2":5539462413096535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id":-8505220651241159003, "__mgmt_id":-8505220651241159003} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397018, "value1":0.3343175727423989, "value2":8363367362679630761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_8":"c","key_2":"e","key_7":"k", "__series_id":-2954096273482449380, "__mgmt_id":-2954096273482449380} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665761, "value1":0.9909509158985629, "value2":598831903509406706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id":8139057134364652777, "__mgmt_id":8139057134364652777} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225685, "value1":0.9692539977538356, "value2":6753560478082272468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id":-5686626809685373979, "__mgmt_id":-5686626809685373979} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857888, "value1":0.3089073345023022, "value2":469238271712070571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id":3877400967867596203, "__mgmt_id":3877400967867596203} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296144, "value1":0.8953425439665634, "value2":5808544938153527415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id":-6807168092950711042, "__mgmt_id":-6807168092950711042} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641028, "value1":0.26770927578250153, "value2":4804718717106479393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id":5574887167499930380, "__mgmt_id":5574887167499930380} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865055, "value1":0.21277992187222125, "value2":7652322888419294762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id":771413411855505413, "__mgmt_id":771413411855505413} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.147216, "value1":0.4361812114988848, "value2":5453489060842625813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id":-6271489171177472422, "__mgmt_id":-6271489171177472422} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931652, "value1":0.9338772117034952, "value2":7926554101044226499, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id":-4846483967205784738, "__mgmt_id":-4846483967205784738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451647, "value1":0.49187863740688065, "value2":626417169671954650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id":-8691105454096685296, "__mgmt_id":-8691105454096685296} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552836, "value1":0.08117641553234622, "value2":6479037192065714806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id":8557606287926271463, "__mgmt_id":8557606287926271463} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644667, "value1":0.8782871355247011, "value2":7618066153253340565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id":-1732152959738129259, "__mgmt_id":-1732152959738129259} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486533, "value1":0.34858818937194813, "value2":8450118240549444684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id":-7668104505878687309, "__mgmt_id":-7668104505878687309} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080371, "value1":0.1720187423008763, "value2":7683320479544120606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id":5740007109533794595, "__mgmt_id":5740007109533794595} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527916, "value1":0.09000309934383972, "value2":5784629514237788639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id":-6226140292610819318, "__mgmt_id":-6226140292610819318} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682327, "value1":0.9194439212395367, "value2":4269189023261883692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id":2424948940195949107, "__mgmt_id":2424948940195949107} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.961106, "value1":0.4405062734385544, "value2":1799260749356167755, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id":1112883101154142655, "__mgmt_id":1112883101154142655} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640959, "value1":0.04934157301568955, "value2":4101326969656042416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id":3159384072740264381, "__mgmt_id":3159384072740264381} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827544, "value1":0.6093692966542871, "value2":5408051744545184279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id":-2354035420748708685, "__mgmt_id":-2354035420748708685} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.916148, "value1":0.37486434605250213, "value2":8005097965237384093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id":608500456554677241, "__mgmt_id":608500456554677241} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477104, "value1":0.7823899457952176, "value2":2425125801220437344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id":1099531734875113482, "__mgmt_id":1099531734875113482} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631094, "value1":0.38182215652170554, "value2":6885679048992226816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id":2312983566483226074, "__mgmt_id":2312983566483226074} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242126, "value1":0.897478447775271, "value2":5819274494025981203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_4":"g","key_8":"b","key_2":"c", "__series_id":-381978251896117922, "__mgmt_id":-381978251896117922} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134175, "value1":0.8364024700095963, "value2":2669419474556636937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id":7921693770137371726, "__mgmt_id":7921693770137371726} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.785456, "value1":0.47900969825766393, "value2":427990087950056586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id":-501490243711306648, "__mgmt_id":-501490243711306648} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.702759, "value1":0.4509768057551493, "value2":6899018348994248896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id":8692326850526983033, "__mgmt_id":8692326850526983033} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.844804, "value1":0.4506687041971474, "value2":2704465019491437568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id":8901305062217396154, "__mgmt_id":8901305062217396154} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841097, "value1":0.08259262088200636, "value2":3254311166592288697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id":9008787801683632143, "__mgmt_id":9008787801683632143} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.579894, "value1":0.634370403808335, "value2":8239103463468673989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id":-69191581506590721, "__mgmt_id":-69191581506590721} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.671756, "value1":0.3978266397618443, "value2":2011463990612690211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id":8615557143580011860, "__mgmt_id":8615557143580011860} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639712, "value1":0.5843126119887666, "value2":8514657269434627024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id":6714523309571458463, "__mgmt_id":6714523309571458463} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.090535, "value1":0.47166906237349404, "value2":2986627777164864344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id":-2436038626322037217, "__mgmt_id":-2436038626322037217} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079733, "value1":0.49607189078610653, "value2":7935174201590062011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id":-9153846671465256928, "__mgmt_id":-9153846671465256928} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484442, "value1":0.10087228739324149, "value2":5758223217302730640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id":-5512922667133642205, "__mgmt_id":-5512922667133642205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.881297, "value1":0.743802063178793, "value2":7666746241262123009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id":-2023745873911087429, "__mgmt_id":-2023745873911087429} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189244, "value1":0.8471771982112483, "value2":8717600770763450338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id":5253653569099281848, "__mgmt_id":5253653569099281848} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958729, "value1":0.24046081337833752, "value2":2429080107883352722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id":2768830248655599965, "__mgmt_id":2768830248655599965} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904544, "value1":0.5967867401359257, "value2":5915623307556355066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id":5713557924465070249, "__mgmt_id":5713557924465070249} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.016356, "value1":0.2664341494092446, "value2":6082384923182522956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id":3213205110735329894, "__mgmt_id":3213205110735329894} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906737, "value1":0.8646356549293164, "value2":6093331013624350146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id":75456790631548363, "__mgmt_id":75456790631548363} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.650026, "value1":0.17820070872685148, "value2":2506978860590163255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id":7710697653197251649, "__mgmt_id":7710697653197251649} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.380897, "value1":0.559051486368018, "value2":959051280528283072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id":6985690126792518272, "__mgmt_id":6985690126792518272} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.645954, "value1":0.44113348983757733, "value2":3640682441395107897, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id":-2773568618767656687, "__mgmt_id":-2773568618767656687} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.705006, "value1":0.7603563234560595, "value2":185556188647790850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id":7861793687162385908, "__mgmt_id":7861793687162385908} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.946294, "value1":0.6649942159160177, "value2":8061815176820463265, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id":45276947053002039, "__mgmt_id":45276947053002039} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.007871, "value1":0.472486274082402, "value2":8110170220979486982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id":2924521682348973507, "__mgmt_id":2924521682348973507} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.817193, "value1":0.01062083501470273, "value2":8652411971395108438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id":-2737327345688605951, "__mgmt_id":-2737327345688605951} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.203058, "value1":0.2545105965864827, "value2":3692372485982923495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id":6562077812082021205, "__mgmt_id":6562077812082021205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724358, "value1":0.49614729928160556, "value2":794673815448737267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id":-2272216867643398248, "__mgmt_id":-2272216867643398248} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029363, "value1":0.8128873282396352, "value2":5575910506626889012, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_3":"b","key_6":"b","key_1":"b", "__series_id":5144841073600140923, "__mgmt_id":5144841073600140923} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.388594, "value1":0.37922666609989497, "value2":5857319265944344227, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id":4158991472976645693, "__mgmt_id":4158991472976645693} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111761, "value1":0.07280095456744823, "value2":7190870932028618354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id":6965159951419158515, "__mgmt_id":6965159951419158515} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556559, "value1":0.011144718316818122, "value2":5637958779343170502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id":-3222371660815337538, "__mgmt_id":-3222371660815337538} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900852, "value1":0.2436048429102491, "value2":3178328441413468775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id":-1567565762361025496, "__mgmt_id":-1567565762361025496} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746269, "value1":0.44159443384096786, "value2":1064726188435771697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id":-1238077055518429859, "__mgmt_id":-1238077055518429859} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089137, "value1":0.5802300524767475, "value2":1179475447484216893, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id":3771919008646036855, "__mgmt_id":3771919008646036855} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045149, "value1":0.8412834417368987, "value2":2843566571645769718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id":-1309345613761574603, "__mgmt_id":-1309345613761574603} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032148, "value1":0.9106893921082526, "value2":2375027858901255873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id":1682100904241849578, "__mgmt_id":1682100904241849578} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180832, "value1":0.8856494080629095, "value2":2259505020949178568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id":-8447273919193029542, "__mgmt_id":-8447273919193029542} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571194, "value1":0.2087472777529694, "value2":3787711915274839208, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id":7000894493225784097, "__mgmt_id":7000894493225784097} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113569, "value1":0.6253289147788459, "value2":2290822587909135561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id":-7658939295344412745, "__mgmt_id":-7658939295344412745} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708878, "value1":0.3051231954056551, "value2":536614295748659218, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id":-3816976869357525258, "__mgmt_id":-3816976869357525258} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397834, "value1":0.004032765819560939, "value2":2143090210108317054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id":-1506749977011375670, "__mgmt_id":-1506749977011375670} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.279982, "value1":0.6698352738967139, "value2":8265876412027179825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id":-5166232873100135825, "__mgmt_id":-5166232873100135825} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.117595, "value1":0.36898116171165335, "value2":7917937119372643007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id":6996402696140969043, "__mgmt_id":6996402696140969043} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490128, "value1":0.7119026496334794, "value2":4217226709077888871, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id":-3373007512765482806, "__mgmt_id":-3373007512765482806} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.337373, "value1":0.24700927278867024, "value2":8633933010561287594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_4":"c","key_9":"j","key_1":"j", "__series_id":135101285191202942, "__mgmt_id":135101285191202942} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393403, "value1":0.3187858750570018, "value2":7155530342572901954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id":-7303512666916957767, "__mgmt_id":-7303512666916957767} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787897, "value1":0.731243787879042, "value2":3162978630167698275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id":5515312804905743253, "__mgmt_id":5515312804905743253} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394636, "value1":0.15744716640466827, "value2":6804804340000012886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id":-4447699968119962519, "__mgmt_id":-4447699968119962519} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589125, "value1":0.9206991440936233, "value2":3240974047558242833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id":-3084091086636934702, "__mgmt_id":-3084091086636934702} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122199, "value1":0.9027140609185327, "value2":4793574504420000662, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id":2885514682724414397, "__mgmt_id":2885514682724414397} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739368, "value1":0.0296490298815606, "value2":7450276243203379932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id":-5450738729835841525, "__mgmt_id":-5450738729835841525} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839046, "value1":0.885358696687765, "value2":2580404634255699486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id":-4333434873637454832, "__mgmt_id":-4333434873637454832} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432366, "value1":0.6379661863831838, "value2":4996786498907777190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129751, "value1":0.5622916100360914, "value2":8341777138936467511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id":6925990235282897427, "__mgmt_id":6925990235282897427} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202469, "value1":0.24574613351572672, "value2":7146206121672605367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id":-5525623792640186251, "__mgmt_id":-5525623792640186251} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.832142, "value1":0.5178470928944944, "value2":4133801221430187756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id":-662733069139439850, "__mgmt_id":-662733069139439850} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080311, "value1":0.486175008187209, "value2":5039371713095786937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id":-2265846297529327958, "__mgmt_id":-2265846297529327958} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723875, "value1":0.552001350718302, "value2":7101322585254858336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id":-2289697776939494418, "__mgmt_id":-2289697776939494418} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434143, "value1":0.6525700542145968, "value2":2081065923746015157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id":7170156078773313786, "__mgmt_id":7170156078773313786} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.410329, "value1":0.15418313782350743, "value2":2121964379701341165, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id":-1249302972235794145, "__mgmt_id":-1249302972235794145} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423463, "value1":0.984744888870281, "value2":7148485292923472325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id":-2140177391110748196, "__mgmt_id":-2140177391110748196} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658931, "value1":0.9086990564903296, "value2":4792038160363904869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id":6087272256412147288, "__mgmt_id":6087272256412147288} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.891060, "value1":0.6990632896148243, "value2":8537384774472959390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id":1653825528745674615, "__mgmt_id":1653825528745674615} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154122, "value1":0.10983855840529697, "value2":3710113544658117053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id":-6800341232275211831, "__mgmt_id":-6800341232275211831} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.543143, "value1":0.3045282199429521, "value2":6184622652371484263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id":9133274435304278352, "__mgmt_id":9133274435304278352} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174103, "value1":0.1622180223685842, "value2":1046120514820502820, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id":-8792288547333335206, "__mgmt_id":-8792288547333335206} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393987, "value1":0.12110279784132964, "value2":5830532384578071588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id":-5438437909542518597, "__mgmt_id":-5438437909542518597} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907289, "value1":0.9509521630571898, "value2":4933206466498891083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id":-2968545362379784314, "__mgmt_id":-2968545362379784314} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987070, "value1":0.5242278192300958, "value2":7374919878503376521, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id":2472948582313971389, "__mgmt_id":2472948582313971389} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641701, "value1":0.3760479049716257, "value2":4296425671360614419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id":-7988236866042391249, "__mgmt_id":-7988236866042391249} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.460082, "value1":0.7715932085121491, "value2":4216486260610422297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id":8355473217236408944, "__mgmt_id":8355473217236408944} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.308698, "value1":0.6733032497636853, "value2":8314310095556752952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id":8293129132311830565, "__mgmt_id":8293129132311830565} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356337, "value1":0.9180371571261539, "value2":7442497884658250592, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id":-4752720542644485754, "__mgmt_id":-4752720542644485754} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407046, "value1":0.8826200541831951, "value2":6272206334208867752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id":-4674174992473955574, "__mgmt_id":-4674174992473955574} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263547, "value1":0.5493018603625203, "value2":5332141292811821371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id":-3194670603174080572, "__mgmt_id":-3194670603174080572} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.690903, "value1":0.6510082736025081, "value2":2282336418703477702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id":6308978262160939848, "__mgmt_id":6308978262160939848} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599129, "value1":0.21529129751322154, "value2":5758745763053757888, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id":1579468215752806140, "__mgmt_id":1579468215752806140} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.294727, "value1":0.17537508292575937, "value2":1562987372702839556, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id":-7402209760770487886, "__mgmt_id":-7402209760770487886} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905165, "value1":0.5372817599107224, "value2":5788710138851546437, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id":-6170144038960690855, "__mgmt_id":-6170144038960690855} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714618, "value1":0.5147887871360798, "value2":2734992135513585397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id":-5087344553152355661, "__mgmt_id":-5087344553152355661} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.845879, "value1":0.03659856815217442, "value2":717088561981717268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id":5180580127014211139, "__mgmt_id":5180580127014211139} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.172178, "value1":0.5465356547159791, "value2":6019732615424773946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id":7455696895619819425, "__mgmt_id":7455696895619819425} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263201, "value1":0.7472793901983326, "value2":6735352007287694901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_4":"i","key_8":"a","key_2":"d", "__series_id":-588642916571085263, "__mgmt_id":-588642916571085263} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418982, "value1":0.029853021112705932, "value2":7962917578850181628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id":8224240495996749932, "__mgmt_id":8224240495996749932} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.866946, "value1":0.10874512532228968, "value2":4523576734362004451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id":4454815443328996164, "__mgmt_id":4454815443328996164} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556178, "value1":0.3725090999387309, "value2":8591394434164967395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id":6592318257860924169, "__mgmt_id":6592318257860924169} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827178, "value1":0.564605338181153, "value2":1565791376780985293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_9":"d","key_5":"b","key_8":"j", "__series_id":-4340249040349917919, "__mgmt_id":-4340249040349917919} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.427240, "value1":0.5937818761029834, "value2":725738161862814755, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id":-7157525334911084152, "__mgmt_id":-7157525334911084152} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153851, "value1":0.6812022447305227, "value2":2627803719401441193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id":-501644526880268050, "__mgmt_id":-501644526880268050} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982516, "value1":0.5197201488713766, "value2":5245967664424169262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id":-5128277451134372125, "__mgmt_id":-5128277451134372125} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.922654, "value1":0.8797142069534722, "value2":2538441830233727435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id":-5102999602639548770, "__mgmt_id":-5102999602639548770} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036732, "value1":0.8882312994752156, "value2":2860925793915692434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id":341574877665132696, "__mgmt_id":341574877665132696} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.848915, "value1":0.5650391916990911, "value2":8021099293155564160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id":7373492341893200542, "__mgmt_id":7373492341893200542} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189564, "value1":0.5152547802767392, "value2":1569067174314315770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id":1901060728526527618, "__mgmt_id":1901060728526527618} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.802112, "value1":0.9536154419324024, "value2":6565733445920939612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id":4352772564110544322, "__mgmt_id":4352772564110544322} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983922, "value1":0.4632188030903474, "value2":3429972812413807249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id":1055170066665701026, "__mgmt_id":1055170066665701026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477177, "value1":0.333324845660299, "value2":4203450320031165809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id":8421383249890408754, "__mgmt_id":8421383249890408754} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097938, "value1":0.9171714255895188, "value2":2925569760522108797, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id":6086829639929716526, "__mgmt_id":6086829639929716526} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.551984, "value1":0.23531625096597034, "value2":8336676757418711666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id":1075978440552535992, "__mgmt_id":1075978440552535992} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.774461, "value1":0.26483886652245114, "value2":1317358716847210463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id":8178311935413748067, "__mgmt_id":8178311935413748067} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354793, "value1":0.850445247905183, "value2":3123908181356206943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id":-2493248663566896078, "__mgmt_id":-2493248663566896078} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.409594, "value1":0.7075083567078372, "value2":7315484295756660732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id":1438560060133493844, "__mgmt_id":1438560060133493844} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407267, "value1":0.6525751933510557, "value2":5480727731388217494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id":-6171460502415307816, "__mgmt_id":-6171460502415307816} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.572642, "value1":0.5899578957097297, "value2":5245828360681206097, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id":1395566022379593096, "__mgmt_id":1395566022379593096} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.731119, "value1":0.6986058457616005, "value2":4059549341372221062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id":5547056308479578094, "__mgmt_id":5547056308479578094} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.627725, "value1":0.5600488168022775, "value2":2564410170230629806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id":3327209389577407524, "__mgmt_id":3327209389577407524} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050520, "value1":0.8920464631997315, "value2":9043850999622300098, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id":2544607124526528771, "__mgmt_id":2544607124526528771} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.500683, "value1":0.004883654125166552, "value2":8338418914599161440, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id":5779250518202732840, "__mgmt_id":5779250518202732840} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970722, "value1":0.01152412783847713, "value2":4977793912981679064, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id":-2839670665154701142, "__mgmt_id":-2839670665154701142} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.860222, "value1":0.796875300130385, "value2":126152372728632215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id":-1266368252316689299, "__mgmt_id":-1266368252316689299} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629467, "value1":0.658754492873807, "value2":2189248151613602825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id":7466907812445173243, "__mgmt_id":7466907812445173243} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.098126, "value1":0.3838755732008427, "value2":8710340059607681652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id":5689680869548667384, "__mgmt_id":5689680869548667384} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812187, "value1":0.7527055571509035, "value2":4522503161830090202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id":-3733803121350110594, "__mgmt_id":-3733803121350110594} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303237, "value1":0.4170647409134182, "value2":1359570910802075414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id":8712710324521618111, "__mgmt_id":8712710324521618111} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794250, "value1":0.581379594095589, "value2":6842995986442641402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id":6109391882474182075, "__mgmt_id":6109391882474182075} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247703, "value1":0.7617287060418184, "value2":8955434883327208405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id":-7178736237433154402, "__mgmt_id":-7178736237433154402} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.815768, "value1":0.5618395871174356, "value2":6646583475463132751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_2":"i","key_5":"b","key_0":"d", "__series_id":4499962670615869243, "__mgmt_id":4499962670615869243} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.248823, "value1":0.1954800995898102, "value2":1363474452253194433, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id":-1065871006724225368, "__mgmt_id":-1065871006724225368} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.531845, "value1":0.018792519031917622, "value2":2287281788624224406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id":-659431953737416175, "__mgmt_id":-659431953737416175} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966028, "value1":0.08281088225013297, "value2":1000202598795297845, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id":8189907994910807144, "__mgmt_id":8189907994910807144} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.935610, "value1":0.3880941622449169, "value2":3231927844022227730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id":-5767774656667519977, "__mgmt_id":-5767774656667519977} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.268000, "value1":0.37669283923759284, "value2":1820767283408305591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id":-7385360351933713516, "__mgmt_id":-7385360351933713516} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.229141, "value1":0.6757864654636687, "value2":850049194951914043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id":5118302165653502709, "__mgmt_id":5118302165653502709} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.889660, "value1":0.14317979698283573, "value2":4562037574783866851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id":-3913420601372985247, "__mgmt_id":-3913420601372985247} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836795, "value1":0.7422552273700601, "value2":1164639099163690154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id":5489942196046137682, "__mgmt_id":5489942196046137682} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.311980, "value1":0.6163196714525997, "value2":5427592556050607568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id":3213709674737485646, "__mgmt_id":3213709674737485646} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584578, "value1":0.38212773421861185, "value2":1414397003551807223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id":3354432852796179518, "__mgmt_id":3354432852796179518} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065172, "value1":0.5125177091208547, "value2":2833074963007555228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id":3033113919287788353, "__mgmt_id":3033113919287788353} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564271, "value1":0.5135791893267198, "value2":3074560175179976918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id":7586011388498218156, "__mgmt_id":7586011388498218156} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894567, "value1":0.8429539964906779, "value2":3097821505618288757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id":-6286863743200497075, "__mgmt_id":-6286863743200497075} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.733587, "value1":0.9459948105676399, "value2":3812465080486684733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id":-2440202381808951160, "__mgmt_id":-2440202381808951160} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.053881, "value1":0.6018467371683773, "value2":5589588667277003729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id":-5305523743817782515, "__mgmt_id":-5305523743817782515} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599521, "value1":0.5951991237753406, "value2":4485462521384926953, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id":1669966689336778997, "__mgmt_id":1669966689336778997} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354099, "value1":0.15573795731250434, "value2":8080731261812523696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id":-5859891958230088706, "__mgmt_id":-5859891958230088706} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969724, "value1":0.21276604577267808, "value2":8483165896229109105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id":-8614030474790995564, "__mgmt_id":-8614030474790995564} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263088, "value1":0.8705529546951637, "value2":1868660470516930885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id":-5335613431145046141, "__mgmt_id":-5335613431145046141} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.212304, "value1":0.28383507730046376, "value2":3678145555670791199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id":7716255806460584549, "__mgmt_id":7716255806460584549} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965823, "value1":0.23336598309503237, "value2":1387918772122086546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id":536692980690746972, "__mgmt_id":536692980690746972} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536800, "value1":0.6576906678976834, "value2":6988018932226580717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id":-2729388522952487322, "__mgmt_id":-2729388522952487322} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.884180, "value1":0.9109732193854059, "value2":2950864149550098491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id":-2477031115887723911, "__mgmt_id":-2477031115887723911} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448663, "value1":0.37035460819883087, "value2":6933572641209900634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id":2278565491101561575, "__mgmt_id":2278565491101561575} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965580, "value1":0.9403308475710539, "value2":5776171572361995550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id":1973129972136064399, "__mgmt_id":1973129972136064399} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.349421, "value1":0.19847751056585392, "value2":3418305989652553355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id":-6372545903488947811, "__mgmt_id":-6372545903488947811} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.469447, "value1":0.13572479625794354, "value2":4687020001075041801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id":-5454642247960722939, "__mgmt_id":-5454642247960722939} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393705, "value1":0.5764820211787193, "value2":296609812894901072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id":4278955237678039512, "__mgmt_id":4278955237678039512} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862549, "value1":0.3023964248018198, "value2":7566431551834336763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id":-959503629477387471, "__mgmt_id":-959503629477387471} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415447, "value1":0.07086082925570908, "value2":6961625726898421777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id":-1742054426754355322, "__mgmt_id":-1742054426754355322} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495354, "value1":0.5778296524778606, "value2":2860533724845138756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id":-8666413906498636151, "__mgmt_id":-8666413906498636151} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.510139, "value1":0.9905369504866598, "value2":7400245650080116394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id":3424389104673400266, "__mgmt_id":3424389104673400266} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129326, "value1":0.3322893702950888, "value2":273820673011485924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id":-58410619722955524, "__mgmt_id":-58410619722955524} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.793922, "value1":0.5795215776373468, "value2":7565118610448753571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id":-6221064276046606416, "__mgmt_id":-6221064276046606416} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114305, "value1":0.4965968551084071, "value2":3076343444745895229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id":5954441612890880180, "__mgmt_id":5954441612890880180} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539515, "value1":0.7424425979048557, "value2":3570552565123324839, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_8":"b","key_9":"b","key_0":"b", "__series_id":118846903220307185, "__mgmt_id":118846903220307185} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714983, "value1":0.4711244304607692, "value2":5134628196697329789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id":4553344308252282896, "__mgmt_id":4553344308252282896} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182520, "value1":0.11684538125954054, "value2":4642866567682796714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id":-4492587331751010254, "__mgmt_id":-4492587331751010254} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701117, "value1":0.46816724142010296, "value2":8359014347252866849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id":6888405107466727721, "__mgmt_id":6888405107466727721} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135967, "value1":0.6405455196141, "value2":6186468831622665956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id":4146396063903913458, "__mgmt_id":4146396063903913458} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.424454, "value1":0.5144728145894553, "value2":1160618526301999790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id":-8996471880146267514, "__mgmt_id":-8996471880146267514} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476344, "value1":0.8109846569048221, "value2":1049565304185759462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id":7974823066874456629, "__mgmt_id":7974823066874456629} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571700, "value1":0.5504531727903846, "value2":133387170146495310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id":-7222273993022799913, "__mgmt_id":-7222273993022799913} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324604, "value1":0.16411596790106633, "value2":4350527762007764055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_9":"g","key_1":"e","key_5":"g", "__series_id":8195321219257482319, "__mgmt_id":8195321219257482319} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272811, "value1":0.06793234743561845, "value2":3491914082935836765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id":-7823955719877279009, "__mgmt_id":-7823955719877279009} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496778, "value1":0.2139795243463974, "value2":9108500124467919738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id":-7533557651239603121, "__mgmt_id":-7533557651239603121} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855922, "value1":0.8034347088304469, "value2":1547578413909373466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.516953, "value1":0.46095603855138756, "value2":5334901216429463877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id":4836771707913559561, "__mgmt_id":4836771707913559561} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.941610, "value1":0.9546476007247299, "value2":1561805534534904078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id":-1936761894710132790, "__mgmt_id":-1936761894710132790} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157907, "value1":0.9790803369585773, "value2":7490355100154440772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id":773876279162078760, "__mgmt_id":773876279162078760} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.667071, "value1":0.23582538943103912, "value2":6462516448359960159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_3":"d","key_9":"f","key_2":"b", "__series_id":1091864146610553369, "__mgmt_id":1091864146610553369} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.447846, "value1":0.08703603490291748, "value2":6137779903742056640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id":892072526354293416, "__mgmt_id":892072526354293416} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.201754, "value1":0.703586761350304, "value2":5895955356127986300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id":-1093921118198888503, "__mgmt_id":-1093921118198888503} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.214135, "value1":0.28977641931297293, "value2":2670270804816119209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id":-211929146564550067, "__mgmt_id":-211929146564550067} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171493, "value1":0.09189861336902101, "value2":3835094797774579940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id":1873826160144342211, "__mgmt_id":1873826160144342211} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.507098, "value1":0.8469316808229229, "value2":4394084036001197157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_2":"j","key_4":"d","key_1":"f", "__series_id":2746195208096574288, "__mgmt_id":2746195208096574288} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.567706, "value1":0.4315878020065951, "value2":3026063068553888025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_9":"f","key_3":"i","key_5":"c", "__series_id":588182638119919303, "__mgmt_id":588182638119919303} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.369862, "value1":0.25873173058422444, "value2":2694491932093728275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id":3248509495393935407, "__mgmt_id":3248509495393935407} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393535, "value1":0.2941316364368656, "value2":3384752395363156914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id":9158941661696733777, "__mgmt_id":9158941661696733777} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354927, "value1":0.13513861393265603, "value2":8481431502383347556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id":1735221365148676523, "__mgmt_id":1735221365148676523} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.468119, "value1":0.5679212228606271, "value2":2713405380262941377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id":-7133474471903036847, "__mgmt_id":-7133474471903036847} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163006, "value1":0.4227063637930681, "value2":6321042013913967311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id":479037977924539995, "__mgmt_id":479037977924539995} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.656131, "value1":0.5736248371417144, "value2":2472629476600653129, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id":-4396946655183034762, "__mgmt_id":-4396946655183034762} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511635, "value1":0.7728428788355186, "value2":7276523252477639153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id":7316972110622220521, "__mgmt_id":7316972110622220521} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080037, "value1":0.9949587251210614, "value2":1148201472548354758, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id":-5290103708506341880, "__mgmt_id":-5290103708506341880} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725064, "value1":0.05030431554866517, "value2":4579601693593819193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id":1480107244684990778, "__mgmt_id":1480107244684990778} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.342713, "value1":0.6797941637549237, "value2":2282393859292368923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id":-7177607339064466056, "__mgmt_id":-7177607339064466056} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012227, "value1":0.25217885012179175, "value2":2820047039844714200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id":-5844764843762034587, "__mgmt_id":-5844764843762034587} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.121409, "value1":0.08353369182495488, "value2":6648793005050735517, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id":-6473745524789598047, "__mgmt_id":-6473745524789598047} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.944843, "value1":0.7935143781883287, "value2":1141746305838051122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_7":"b","key_9":"b","key_1":"k", "__series_id":1150094511583599589, "__mgmt_id":1150094511583599589} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216784, "value1":0.687358900553268, "value2":5413083464946109711, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id":-7604649366390654955, "__mgmt_id":-7604649366390654955} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.744704, "value1":0.26762670212162987, "value2":4557617436876999047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id":8408649499406926918, "__mgmt_id":8408649499406926918} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.105367, "value1":0.7098778066529884, "value2":2505389065681570054, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_3":"k","key_2":"f", "__series_id":-1770598027909394560, "__mgmt_id":-1770598027909394560} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164496, "value1":0.375746421410976, "value2":874151975168015904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id":1743799415683896668, "__mgmt_id":1743799415683896668} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648692, "value1":0.26898807878557085, "value2":8174372151408578377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id":-1515679063946229924, "__mgmt_id":-1515679063946229924} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160502, "value1":0.8995428170685682, "value2":3027568952484727683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id":5401788690197380769, "__mgmt_id":5401788690197380769} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149122, "value1":0.0907853096527507, "value2":4677811628653537246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id":2418823558047176150, "__mgmt_id":2418823558047176150} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296780, "value1":0.3995862207774202, "value2":3598801053628990992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id":7326416396107376619, "__mgmt_id":7326416396107376619} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.464060, "value1":0.9713793366221031, "value2":4865123999270835737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id":-7152958117457316606, "__mgmt_id":-7152958117457316606} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288334, "value1":0.6512177032146146, "value2":1792245924965210603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id":279512434362576559, "__mgmt_id":279512434362576559} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.869907, "value1":0.007513793036623767, "value2":7995353402339020473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_8":"i","key_1":"i","key_5":"e", "__series_id":5960810744832862505, "__mgmt_id":5960810744832862505} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527381, "value1":0.5920004855495694, "value2":1386169961861528891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id":4012996769448248803, "__mgmt_id":4012996769448248803} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.928125, "value1":0.7418903709677874, "value2":5060751330275548305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id":-3053901563656550752, "__mgmt_id":-3053901563656550752} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131800, "value1":0.9351703924333601, "value2":827562310794270736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_8":"g","key_0":"b","key_2":"g", "__series_id":69745064320595303, "__mgmt_id":69745064320595303} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367161, "value1":0.9314616031794654, "value2":8545717112597189310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id":-4390751437491974261, "__mgmt_id":-4390751437491974261} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704807, "value1":0.7062626357522712, "value2":1882734585108600742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id":-1861794853839231064, "__mgmt_id":-1861794853839231064} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270485, "value1":0.942646876486616, "value2":4665280302852454779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id":-212718667757998205, "__mgmt_id":-212718667757998205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.521647, "value1":0.7127769947534454, "value2":7002036018179718342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id":-573574065846351162, "__mgmt_id":-573574065846351162} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.011614, "value1":0.2150115012001749, "value2":2368106908969391947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id":3087654112644698738, "__mgmt_id":3087654112644698738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160161, "value1":0.5544338078106065, "value2":6068552071959300958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id":-8037889140845157220, "__mgmt_id":-8037889140845157220} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545350, "value1":0.5865458699587862, "value2":1122097202466314910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id":9079902230263459230, "__mgmt_id":9079902230263459230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.948757, "value1":0.9353310253197936, "value2":9083948862293457994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id":8051016939451460244, "__mgmt_id":8051016939451460244} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.003970, "value1":0.13057333820161732, "value2":4887307186667923394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id":3503362438270569084, "__mgmt_id":3503362438270569084} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811016, "value1":0.0597992472783255, "value2":7198325554706277943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id":7050945119323337512, "__mgmt_id":7050945119323337512} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687715, "value1":0.029450254550187717, "value2":7347375517848816816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id":3553226356531159335, "__mgmt_id":3553226356531159335} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892901, "value1":0.5081883055870532, "value2":3289845879859033841, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id":-9155171941742690136, "__mgmt_id":-9155171941742690136} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006320, "value1":0.35956731353734483, "value2":6174658822769262086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id":1068182841660501054, "__mgmt_id":1068182841660501054} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.088057, "value1":0.5893474112602632, "value2":2264301146979603153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id":-6900891616086871180, "__mgmt_id":-6900891616086871180} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366406, "value1":0.09531376481437555, "value2":8460504532640850958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id":2476461678528820161, "__mgmt_id":2476461678528820161} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.742879, "value1":0.21104814940031102, "value2":8239563761654323619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id":-3049533378195154637, "__mgmt_id":-3049533378195154637} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014364, "value1":0.27433866046434513, "value2":1123249940897490575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id":2279437384480879496, "__mgmt_id":2279437384480879496} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442350, "value1":0.5779165272183289, "value2":7076967122005445282, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_6":"e","key_2":"c","key_3":"k", "__series_id":-4803521805997940763, "__mgmt_id":-4803521805997940763} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.363242, "value1":0.7223710776768617, "value2":4947532477450255339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id":-3252709821186360534, "__mgmt_id":-3252709821186360534} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794231, "value1":0.9595692700611697, "value2":7086029131344930712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id":-3485520211684722157, "__mgmt_id":-3485520211684722157} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.696096, "value1":0.2902282025796308, "value2":2826023125040762860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id":4549675943707384462, "__mgmt_id":4549675943707384462} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366795, "value1":0.8338186933189231, "value2":7469909471986494672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id":-2171418575767265313, "__mgmt_id":-2171418575767265313} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.642061, "value1":0.025310231945638022, "value2":5571346287819468860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id":-9129308575221288808, "__mgmt_id":-9129308575221288808} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673449, "value1":0.9385869454324781, "value2":1722013371242991173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id":5101814561270847054, "__mgmt_id":5101814561270847054} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375120, "value1":0.09609963357456859, "value2":5591579449982830960, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id":8885896453973036591, "__mgmt_id":8885896453973036591} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391308, "value1":0.5308398826498623, "value2":2744980541531859716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id":1198284593719567346, "__mgmt_id":1198284593719567346} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658264, "value1":0.08305585159828374, "value2":3868051792086152227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id":-7940551193719921457, "__mgmt_id":-7940551193719921457} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.540598, "value1":0.9729141879466757, "value2":2886891615332464723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id":8092412906218116635, "__mgmt_id":8092412906218116635} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.077420, "value1":0.756799108978319, "value2":3709648144634270824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id":4176822743017399294, "__mgmt_id":4176822743017399294} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.762914, "value1":0.7697479099606849, "value2":3475167187400734782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_5":"f","key_6":"g","key_0":"c", "__series_id":478848127830000932, "__mgmt_id":478848127830000932} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907718, "value1":0.44276263620911915, "value2":8007229889762742191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id":8101237851747291018, "__mgmt_id":8101237851747291018} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.677735, "value1":0.7114610811603074, "value2":420499484804274745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id":-2776938222678525934, "__mgmt_id":-2776938222678525934} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097065, "value1":0.9614980550005483, "value2":401687894704730769, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id":-4653576099777458808, "__mgmt_id":-4653576099777458808} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107273, "value1":0.7588043489806351, "value2":5258772379410637950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id":-1382504029843268487, "__mgmt_id":-1382504029843268487} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907641, "value1":0.16046492238486323, "value2":5755077471601284208, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_7":"f","key_9":"b","key_2":"j", "__series_id":8725412239193059199, "__mgmt_id":8725412239193059199} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407605, "value1":0.5733175508809538, "value2":2858689191661453591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id":4735366876457776088, "__mgmt_id":4735366876457776088} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412001, "value1":0.6163004444656547, "value2":3938161718625625501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id":-2247559342224111326, "__mgmt_id":-2247559342224111326} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174438, "value1":0.7741718765670191, "value2":8151281186076102637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id":5375659145983011030, "__mgmt_id":5375659145983011030} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433895, "value1":0.6190720762613371, "value2":4753613664907841729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id":-5274555278497901604, "__mgmt_id":-5274555278497901604} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.754894, "value1":0.6861936508507025, "value2":4982660479879807579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id":5847739109028520326, "__mgmt_id":5847739109028520326} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135440, "value1":0.8484346139801648, "value2":4916410761830427120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id":4275286754468204823, "__mgmt_id":4275286754468204823} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901526, "value1":0.7029579471688302, "value2":7091187814115860335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id":-1771117005440832452, "__mgmt_id":-1771117005440832452} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969298, "value1":0.9971487141151381, "value2":4678178370126159004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id":9167379236828970956, "__mgmt_id":9167379236828970956} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.302462, "value1":0.5451450337065363, "value2":4647862155949132966, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id":3813234962899481725, "__mgmt_id":3813234962899481725} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174504, "value1":0.8364678736703143, "value2":4662336943284880044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id":-7333165572846986991, "__mgmt_id":-7333165572846986991} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724429, "value1":0.7605319213856574, "value2":5775369825299529065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id":-6728759103923072294, "__mgmt_id":-6728759103923072294} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196992, "value1":0.485221307178674, "value2":8161727305469743810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id":-1482359697266060568, "__mgmt_id":-1482359697266060568} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673298, "value1":0.12884478288346873, "value2":3433606731713363675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id":1728706682623658501, "__mgmt_id":1728706682623658501} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.008175, "value1":0.6854875474834051, "value2":7177608068823530117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_8":"b","key_1":"j","key_5":"j", "__series_id":300660524563048433, "__mgmt_id":300660524563048433} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.152906, "value1":0.937989165728203, "value2":2388297772896467042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id":-9079998154472042434, "__mgmt_id":-9079998154472042434} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354518, "value1":0.5251867146486633, "value2":2890885067302311396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id":-1667478984335911846, "__mgmt_id":-1667478984335911846} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.872640, "value1":0.7546063425064781, "value2":8461706694506205274, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_8":"f","key_3":"h","key_4":"j", "__series_id":-7580463150974888537, "__mgmt_id":-7580463150974888537} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403113, "value1":0.7711761108505557, "value2":8776285961045082566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id":-2535477892348838006, "__mgmt_id":-2535477892348838006} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.713712, "value1":0.3686652392717231, "value2":3561820735837135599, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id":-4203576198063720420, "__mgmt_id":-4203576198063720420} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.115073, "value1":0.21496854825964173, "value2":1666306538045744009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id":8723049886767350264, "__mgmt_id":8723049886767350264} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.127362, "value1":0.0348593634697097, "value2":3277092601539134615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id":-3836128937730354935, "__mgmt_id":-3836128937730354935} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.042887, "value1":0.9391085748107487, "value2":3037900421018347064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id":960597803634566310, "__mgmt_id":960597803634566310} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957170, "value1":0.2742109117364031, "value2":7538881467268645993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id":1621069224688140711, "__mgmt_id":1621069224688140711} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.284083, "value1":0.6552095129564431, "value2":8324596282478983070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id":-6270497400672793233, "__mgmt_id":-6270497400672793233} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139776, "value1":0.32060963733799414, "value2":4387891090637798442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id":8806806789681853791, "__mgmt_id":8806806789681853791} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.446847, "value1":0.7514268853453937, "value2":1042496829752491158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id":-5236981910974541822, "__mgmt_id":-5236981910974541822} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.429327, "value1":0.7027895753887083, "value2":4302894157835600757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id":7563784551155215837, "__mgmt_id":7563784551155215837} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496272, "value1":0.2459555820489922, "value2":4743527397302686896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id":-43719334813854435, "__mgmt_id":-43719334813854435} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599745, "value1":0.3085143386003063, "value2":8510884550594616394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id":3514458506748617193, "__mgmt_id":3514458506748617193} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233545, "value1":0.4081732430516885, "value2":4780887317944713264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id":4505204653313518164, "__mgmt_id":4505204653313518164} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.396869, "value1":0.3834820243913789, "value2":2867737400343386587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id":-232361702857049638, "__mgmt_id":-232361702857049638} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.000965, "value1":0.7645272489414043, "value2":8292218247137907442, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id":5720057473891543633, "__mgmt_id":5720057473891543633} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390310, "value1":0.9922190988066435, "value2":6520801004207332848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id":1284561962336905172, "__mgmt_id":1284561962336905172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.408746, "value1":0.11967416242419873, "value2":1289404280671317844, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id":8363757466721457609, "__mgmt_id":8363757466721457609} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.513154, "value1":0.43853228030860203, "value2":3816263914269322442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id":8721730122668119771, "__mgmt_id":8721730122668119771} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.425564, "value1":0.9721541652281699, "value2":1199225910826614784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id":-1988713756160061772, "__mgmt_id":-1988713756160061772} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.896976, "value1":0.5841175850919846, "value2":6380661791453585128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id":6964189443680817110, "__mgmt_id":6964189443680817110} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.385017, "value1":0.3642021068005677, "value2":7460310647019127617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id":-2570923717357710934, "__mgmt_id":-2570923717357710934} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.861063, "value1":0.18830600982566692, "value2":6167598179189147828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id":5508198311483266554, "__mgmt_id":5508198311483266554} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.942353, "value1":0.4661402168565476, "value2":8018642185562026949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id":6489846711316186670, "__mgmt_id":6489846711316186670} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.750347, "value1":0.3217686952651615, "value2":830941682724984711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id":-8142533296407611789, "__mgmt_id":-8142533296407611789} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.926420, "value1":0.0451768298368594, "value2":3858422103614938711, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id":-4185260234803399992, "__mgmt_id":-4185260234803399992} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451255, "value1":0.012250475428480134, "value2":468924453611892408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id":4321156897846092336, "__mgmt_id":4321156897846092336} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708013, "value1":0.5360411826543746, "value2":7399971881816481719, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id":-3779813639604971340, "__mgmt_id":-3779813639604971340} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.661279, "value1":0.5535815660872848, "value2":4401269288089200510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id":-924271420436849046, "__mgmt_id":-924271420436849046} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.710860, "value1":0.6074402471823324, "value2":7073880640711856002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id":-6286400394770701784, "__mgmt_id":-6286400394770701784} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245087, "value1":0.7738521153135302, "value2":5302937747460405991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id":4370944294749235713, "__mgmt_id":4370944294749235713} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724822, "value1":0.06719994230964553, "value2":1984993846520154478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id":6563059919034350414, "__mgmt_id":6563059919034350414} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841255, "value1":0.7199212526961583, "value2":3580548125031735289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id":-2565464395137357469, "__mgmt_id":-2565464395137357469} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130289, "value1":0.14169983508615547, "value2":7980800511434824689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_6":"i","key_8":"d","key_5":"k", "__series_id":7709323868101614601, "__mgmt_id":7709323868101614601} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.215185, "value1":0.05456621441388223, "value2":3313875019669857059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id":1006029062071662005, "__mgmt_id":1006029062071662005} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730585, "value1":0.1938148933748891, "value2":2133259010192691032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id":-8935610045063183738, "__mgmt_id":-8935610045063183738} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603048, "value1":0.27374938601691284, "value2":6401354620294279338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id":8769961756187444147, "__mgmt_id":8769961756187444147} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148949, "value1":0.5947828484485513, "value2":8147515290794947255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id":5226949060967918762, "__mgmt_id":5226949060967918762} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.610152, "value1":0.4085609201069522, "value2":9002523177156729302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id":463887977784803555, "__mgmt_id":463887977784803555} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.607808, "value1":0.9266705539974761, "value2":4953852228607795114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id":-4239837266494773189, "__mgmt_id":-4239837266494773189} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197811, "value1":0.7637421466978226, "value2":7727001922124797631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id":5973456801068675752, "__mgmt_id":5973456801068675752} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672455, "value1":0.2140320622344972, "value2":8504587633123200194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id":-7318834342806204719, "__mgmt_id":-7318834342806204719} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781600, "value1":0.18861407062595717, "value2":2041192855707568703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_4":"i","key_5":"e","key_0":"e", "__series_id":4358356389076501559, "__mgmt_id":4358356389076501559} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648320, "value1":0.9980339865005591, "value2":6740770170971470746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id":4592924626643072689, "__mgmt_id":4592924626643072689} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481193, "value1":0.5203406695297639, "value2":6425306083284764527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id":7094885463570430101, "__mgmt_id":7094885463570430101} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954444, "value1":0.5949050635321682, "value2":2690790583476351571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id":6146792295053586949, "__mgmt_id":6146792295053586949} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971678, "value1":0.3007304152123297, "value2":4562717443975915242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id":-2103117075427447025, "__mgmt_id":-2103117075427447025} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026021, "value1":0.27575882598297025, "value2":978861683515042668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id":-2256474065290328854, "__mgmt_id":-2256474065290328854} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.840073, "value1":0.15154871188435406, "value2":7561212279807822666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id":5011990492559181611, "__mgmt_id":5011990492559181611} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393063, "value1":0.5006737130316841, "value2":835200053720218776, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id":107727703190771119, "__mgmt_id":107727703190771119} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589637, "value1":0.29002140408035565, "value2":5208166554872726624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id":648754459071852131, "__mgmt_id":648754459071852131} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525315, "value1":0.7084919237025886, "value2":4888712288784480920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id":4070672682488929531, "__mgmt_id":4070672682488929531} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026489, "value1":0.9459638977891762, "value2":6799511668473867834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id":4263193799253961991, "__mgmt_id":4263193799253961991} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356460, "value1":0.8482487483215856, "value2":4902836930964609800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id":174857968371242321, "__mgmt_id":174857968371242321} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871117, "value1":0.35037006883754834, "value2":6550768382747712274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id":-7804957703108199728, "__mgmt_id":-7804957703108199728} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.062992, "value1":0.3329675649141359, "value2":8563631384946292163, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id":4819173149125300851, "__mgmt_id":4819173149125300851} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.751101, "value1":0.9811341531679872, "value2":621345745999214417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id":-519381987822359762, "__mgmt_id":-519381987822359762} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.535295, "value1":0.922840944798324, "value2":586063777073635225, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id":-3336015353703936523, "__mgmt_id":-3336015353703936523} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.258523, "value1":0.7865906481221376, "value2":1141638202189461929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id":-205391342820866960, "__mgmt_id":-205391342820866960} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.10833711533264617, "value2":5739011556983431812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id":-2006641466119522129, "__mgmt_id":-2006641466119522129} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407251, "value1":0.4266309513517482, "value2":4708724256538796193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_6":"h","key_2":"e","key_4":"d", "__series_id":9026717280601479145, "__mgmt_id":9026717280601479145} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931023, "value1":0.6949078276162761, "value2":6914579995867144592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id":-8286050823772945646, "__mgmt_id":-8286050823772945646} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.882968, "value1":0.27938573922269233, "value2":2134628658651324357, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id":-2956385702922923027, "__mgmt_id":-2956385702922923027} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599828, "value1":0.31593636777024753, "value2":2026576607314660982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id":9222865247780531255, "__mgmt_id":9222865247780531255} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166118, "value1":0.30221394298164866, "value2":7042763501275182896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id":-4059907291499193197, "__mgmt_id":-4059907291499193197} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114195, "value1":0.16993347985800208, "value2":356239511173356550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id":-1270585470594030161, "__mgmt_id":-1270585470594030161} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.676940, "value1":0.8142744662164688, "value2":1615963097558644501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id":-7518756723037496675, "__mgmt_id":-7518756723037496675} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525949, "value1":0.9287689609270591, "value2":8144174091016595622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id":-6000742232332707948, "__mgmt_id":-6000742232332707948} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852923, "value1":0.6682421979821775, "value2":1660916359233146706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id":4486095316201482072, "__mgmt_id":4486095316201482072} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714742, "value1":0.6940562364820247, "value2":8279633113622088778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id":-4604912175677665552, "__mgmt_id":-4604912175677665552} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.069728, "value1":0.023054188899827854, "value2":7432273276574908949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id":6934979284740660675, "__mgmt_id":6934979284740660675} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397060, "value1":0.6029753734289958, "value2":5859678381429923175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id":-7255426429556711853, "__mgmt_id":-7255426429556711853} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788868, "value1":0.4080940744104717, "value2":2775616148615876742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id":2945323205155494561, "__mgmt_id":2945323205155494561} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.497778, "value1":0.0933297131976096, "value2":2710674316640074263, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id":5757871568571719806, "__mgmt_id":5757871568571719806} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.251386, "value1":0.5835060543515784, "value2":1694819851220806994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id":4674231085769924073, "__mgmt_id":4674231085769924073} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144280, "value1":0.7762012742074322, "value2":6551654579563921110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id":-6413509023368384468, "__mgmt_id":-6413509023368384468} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778177, "value1":0.8529872059895475, "value2":4843965544345174856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_7":"c","key_6":"k", "__series_id":4513696354870901992, "__mgmt_id":4513696354870901992} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495383, "value1":0.8821308188431486, "value2":7181474101563747015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id":1385361195095616515, "__mgmt_id":1385361195095616515} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421122, "value1":0.04021723515347381, "value2":8758814764030728391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id":4929797706112254955, "__mgmt_id":4929797706112254955} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055252, "value1":0.5894200751471586, "value2":3745371455267104958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id":1992091351558433546, "__mgmt_id":1992091351558433546} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343067, "value1":0.2157486959489704, "value2":3077439566978720910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id":-5772222342437624470, "__mgmt_id":-5772222342437624470} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489306, "value1":0.4542119549735926, "value2":5846254702440735178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id":6445180479932093175, "__mgmt_id":6445180479932093175} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233627, "value1":0.2946889464915457, "value2":2465181057584653557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id":-5864279158098419036, "__mgmt_id":-5864279158098419036} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.523076, "value1":0.579630183513931, "value2":4633358909899504280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id":-7458229987167107139, "__mgmt_id":-7458229987167107139} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.317577, "value1":0.49633251300853154, "value2":7243885459219565500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id":4346321792630722699, "__mgmt_id":4346321792630722699} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142016, "value1":0.41458124596836643, "value2":87263970443989795, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id":-5354114461131055911, "__mgmt_id":-5354114461131055911} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014459, "value1":0.3889314073670658, "value2":6642542110052864260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id":-2698660337334151799, "__mgmt_id":-2698660337334151799} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.249899, "value1":0.5981564529561014, "value2":4725189280029367657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id":-7342214837885234023, "__mgmt_id":-7342214837885234023} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.087078, "value1":0.48311123248467286, "value2":7915582091365577059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id":-2988860553018354593, "__mgmt_id":-2988860553018354593} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.681336, "value1":0.3569069050336655, "value2":7513851204641167627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id":9196263055514333027, "__mgmt_id":9196263055514333027} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266570, "value1":0.6724875939535222, "value2":4615308903415712327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id":2242518671299929316, "__mgmt_id":2242518671299929316} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474729, "value1":0.3876423090885996, "value2":319900338975917805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id":-1694708898393520778, "__mgmt_id":-1694708898393520778} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.328471, "value1":0.061053039369872907, "value2":9085903408798527191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id":-4903270819894625628, "__mgmt_id":-4903270819894625628} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010016, "value1":0.7472091713600674, "value2":8976037153524824381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id":-870546761351951316, "__mgmt_id":-870546761351951316} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412206, "value1":0.7796477061197584, "value2":6842085213034340664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id":-322672579238854109, "__mgmt_id":-322672579238854109} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288276, "value1":0.26992787569132787, "value2":1806846857113271486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id":6392136191524438462, "__mgmt_id":6392136191524438462} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.501201, "value1":0.781398815935773, "value2":4251812051340782659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id":-2261204010007076042, "__mgmt_id":-2261204010007076042} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270221, "value1":0.051986823543989535, "value2":4993776637838984787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id":-5219068177349312395, "__mgmt_id":-5219068177349312395} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.986643, "value1":0.7021309080168643, "value2":703842817955022555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id":-274426148915748697, "__mgmt_id":-274426148915748697} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.453898, "value1":0.4599169268098947, "value2":7963554429950594113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id":-5109918428716040627, "__mgmt_id":-5109918428716040627} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202118, "value1":0.94451324780601, "value2":7780362140982384262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id":5199655525790884573, "__mgmt_id":5199655525790884573} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.801389, "value1":0.7867483731478175, "value2":7192213512007631986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id":8135807143640598550, "__mgmt_id":8135807143640598550} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.228749, "value1":0.4182819773434862, "value2":3027632485243175966, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id":-3767462971529671998, "__mgmt_id":-3767462971529671998} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.184080, "value1":0.9567901938339193, "value2":5313717148514876661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id":-7292257139050233576, "__mgmt_id":-7292257139050233576} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.707330, "value1":0.826196713820751, "value2":8078395604816162581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id":4040887025298036951, "__mgmt_id":4040887025298036951} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130983, "value1":0.31932740597678144, "value2":8325619009408095997, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_6":"j","key_7":"j","key_5":"j", "__series_id":-8575913527796938168, "__mgmt_id":-8575913527796938168} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477577, "value1":0.8037112800341328, "value2":4641052188020943712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id":242486419215872474, "__mgmt_id":242486419215872474} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.835784, "value1":0.981119547136869, "value2":5830864768532244942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id":6502989350228972750, "__mgmt_id":6502989350228972750} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.951325, "value1":0.9017984159519533, "value2":7172321278356171509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id":5180505041240508376, "__mgmt_id":5180505041240508376} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949416, "value1":0.9735936936041008, "value2":2397210877296618753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id":-4268724067754570381, "__mgmt_id":-4268724067754570381} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.227491, "value1":0.056278467666299646, "value2":3619861078542842990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id":-8558923509815976303, "__mgmt_id":-8558923509815976303} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614325, "value1":0.9338179733958348, "value2":1856588427373456649, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id":-7411561607080856762, "__mgmt_id":-7411561607080856762} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.167780, "value1":0.6264999203061654, "value2":1887390160426943194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id":446092245233888064, "__mgmt_id":446092245233888064} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421682, "value1":0.3034767704367279, "value2":1117719742198046331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id":-5792100618530611790, "__mgmt_id":-5792100618530611790} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131869, "value1":0.7226359930389739, "value2":6009067442340606148, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id":-1414104685122802275, "__mgmt_id":-1414104685122802275} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.576451, "value1":0.4077319058968275, "value2":5262129379842658951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id":6008408224877168924, "__mgmt_id":6008408224877168924} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487400, "value1":0.49878540971656044, "value2":378267425259112843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id":-8137066097479334829, "__mgmt_id":-8137066097479334829} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417401, "value1":0.7695532802557139, "value2":6353721086899765325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id":-3785821557868987000, "__mgmt_id":-3785821557868987000} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.597343, "value1":0.9334570369057625, "value2":8106460523980946423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id":-1328640254547822589, "__mgmt_id":-1328640254547822589} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947755, "value1":0.599677921450711, "value2":1518293404194401856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id":-6495179908941869036, "__mgmt_id":-6495179908941869036} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533432, "value1":0.4134611723516883, "value2":4789921078575497056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id":9138369831619846992, "__mgmt_id":9138369831619846992} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271038, "value1":0.7693055378733823, "value2":2557684070989427251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id":2446733477236612230, "__mgmt_id":2446733477236612230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.209928, "value1":0.5501502486470365, "value2":6523836811607831530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id":-6200607058083360641, "__mgmt_id":-6200607058083360641} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.514479, "value1":0.742552198141705, "value2":5448311945921729044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id":4358771423920240400, "__mgmt_id":4358771423920240400} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097323, "value1":0.342716151747015, "value2":5333351730397550620, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id":5138476994793802870, "__mgmt_id":5138476994793802870} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036400, "value1":0.9364465322438232, "value2":3515155010826544091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id":-4249457985570074994, "__mgmt_id":-4249457985570074994} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.161709, "value1":0.4801914817306215, "value2":4167056653387877843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id":6935428815854533176, "__mgmt_id":6935428815854533176} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128367, "value1":0.42861309279829746, "value2":6367124429829008192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id":-5504675752837572439, "__mgmt_id":-5504675752837572439} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.104233, "value1":0.5604661779053219, "value2":8908744031459681115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id":1985563680930509217, "__mgmt_id":1985563680930509217} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814224, "value1":0.8505352643130756, "value2":3611537515395296790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id":-4328016376789558277, "__mgmt_id":-4328016376789558277} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.257817, "value1":0.9534356262275627, "value2":6358735114691060805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_3":"i","key_5":"j","key_2":"e", "__series_id":9179346307913990022, "__mgmt_id":9179346307913990022} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590971, "value1":0.8532839122834444, "value2":4683754442672481085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_9":"f","key_7":"b", "__series_id":-5432215965398723066, "__mgmt_id":-5432215965398723066} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110492, "value1":0.563891691575366, "value2":4631985245890497320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id":-136250900044526365, "__mgmt_id":-136250900044526365} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953652, "value1":0.20700443841383587, "value2":1925175802270628257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id":4875237959865447451, "__mgmt_id":4875237959865447451} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012273, "value1":0.5960529408710756, "value2":6611884946690949474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id":1497740143071633371, "__mgmt_id":1497740143071633371} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107781, "value1":0.5528403788424425, "value2":2332669698469083563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id":6481359940216246477, "__mgmt_id":6481359940216246477} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289974, "value1":0.4420135407607212, "value2":3690892585669795676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id":3152074120145678841, "__mgmt_id":3152074120145678841} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.193764, "value1":0.5159643736419933, "value2":6218735166584063092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id":3438830744121252320, "__mgmt_id":3438830744121252320} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.386095, "value1":0.777297131904327, "value2":636830504959980122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id":-3553038228034037944, "__mgmt_id":-3553038228034037944} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977928, "value1":0.01676315692753733, "value2":4060533084263704194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id":-988677622257336616, "__mgmt_id":-988677622257336616} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.066993, "value1":0.001865435996499948, "value2":4739969679441502931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id":-6528423083423872684, "__mgmt_id":-6528423083423872684} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.910616, "value1":0.057836079222670464, "value2":5276870038639917478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id":-8717617664237859896, "__mgmt_id":-8717617664237859896} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971825, "value1":0.18153964718314727, "value2":2807508624852997267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id":6851652104392799931, "__mgmt_id":6851652104392799931} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.782626, "value1":0.41404427259634824, "value2":6643422113997639050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id":-3260329422376183072, "__mgmt_id":-3260329422376183072} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892943, "value1":0.5830967595104455, "value2":499806168567753593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_9":"b","key_7":"b","key_8":"e", "__series_id":5207812472034615754, "__mgmt_id":5207812472034615754} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423402, "value1":0.9103545774481498, "value2":911868702803131336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id":-6610823537308815374, "__mgmt_id":-6610823537308815374} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006069, "value1":0.7748758341609289, "value2":6539224747903805609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id":9158185736224292675, "__mgmt_id":9158185736224292675} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143311, "value1":0.3741630815676959, "value2":375500557355639144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id":-7560317748452054958, "__mgmt_id":-7560317748452054958} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267012, "value1":0.06234671061027635, "value2":5595306219658860658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id":999269332951330346, "__mgmt_id":999269332951330346} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548719, "value1":0.7992113362076428, "value2":7214856807865350002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id":732108570561027442, "__mgmt_id":732108570561027442} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.919970, "value1":0.973745629473067, "value2":4169123266267227326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id":-8959368755671248306, "__mgmt_id":-8959368755671248306} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.071680, "value1":0.08778281877161515, "value2":7668684104220358875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id":489626333821893626, "__mgmt_id":489626333821893626} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.222513, "value1":0.4425030874471083, "value2":5614432156299415760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id":-5982588030568173473, "__mgmt_id":-5982588030568173473} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.031556, "value1":0.6971180191067721, "value2":6128893201277772027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id":-2565229845181866432, "__mgmt_id":-2565229845181866432} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839957, "value1":0.03151274581462018, "value2":8360563655204493448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id":4084306027943312742, "__mgmt_id":4084306027943312742} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924623, "value1":0.5790628849747803, "value2":6190166511034220622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id":-5467297294721627584, "__mgmt_id":-5467297294721627584} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.647381, "value1":0.18478113887851288, "value2":2971279323948578904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id":3522844273960320717, "__mgmt_id":3522844273960320717} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.818067, "value1":0.36190113937740354, "value2":1714448127957842853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id":4737984741355726102, "__mgmt_id":4737984741355726102} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.715974, "value1":0.23843892303460118, "value2":2439937704889068215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id":-1605771282004434260, "__mgmt_id":-1605771282004434260} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192672, "value1":0.6080907905788115, "value2":976012584384444918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id":6086398502324277057, "__mgmt_id":6086398502324277057} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157561, "value1":0.6894676876528992, "value2":7135259052981755980, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id":-5140571792209393544, "__mgmt_id":-5140571792209393544} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134773, "value1":0.09678670230466922, "value2":8180183833096777286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id":-9156936360867657099, "__mgmt_id":-9156936360867657099} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.616057, "value1":0.8520049469696344, "value2":2950033021270268791, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id":3968087823541282551, "__mgmt_id":3968087823541282551} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255317, "value1":0.7477685612049465, "value2":179441442795111864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id":1018474149027954404, "__mgmt_id":1018474149027954404} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906502, "value1":0.8256982980631076, "value2":4367074024600682775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_4":"d","key_6":"e","key_3":"j", "__series_id":2939099060116576031, "__mgmt_id":2939099060116576031} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.116793, "value1":0.8891425230837835, "value2":1719946264236570624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id":6511586288648013387, "__mgmt_id":6511586288648013387} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324866, "value1":0.5386590615308351, "value2":8011171897264590756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id":7054595087473761218, "__mgmt_id":7054595087473761218} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.352403, "value1":0.7153316706866707, "value2":4379546221964794474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id":-1949065403756251767, "__mgmt_id":-1949065403756251767} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836874, "value1":0.14443484962198383, "value2":1482104610634689373, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_9":"b","key_5":"c","key_8":"a", "__series_id":-6012620175802667643, "__mgmt_id":-6012620175802667643} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403735, "value1":0.8924124150178864, "value2":9156970764441343023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id":-8852023680179592672, "__mgmt_id":-8852023680179592672} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141142, "value1":0.434266569307909, "value2":4978801873133265222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id":-5796750192728400794, "__mgmt_id":-5796750192728400794} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.234292, "value1":0.7164352385365066, "value2":8038096300931135389, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id":-5308657687944985141, "__mgmt_id":-5308657687944985141} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973287, "value1":0.6600392290118359, "value2":2638933878817864633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id":-6764254286344273733, "__mgmt_id":-6764254286344273733} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728882, "value1":0.476991229582368, "value2":6510855349407866452, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id":2941893197325067644, "__mgmt_id":2941893197325067644} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787371, "value1":0.7330600795491425, "value2":6570294519251889382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id":-1019576594847263341, "__mgmt_id":-1019576594847263341} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903661, "value1":0.5944117402324142, "value2":6524268081747181442, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id":-8832669958039901063, "__mgmt_id":-8832669958039901063} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.659983, "value1":0.26592399871964695, "value2":1212972365053449050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id":8212472004484910391, "__mgmt_id":8212472004484910391} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723204, "value1":0.31589384939280846, "value2":2041435063818693781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id":-587356467785105936, "__mgmt_id":-587356467785105936} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701487, "value1":0.5974401671491981, "value2":625551200860992484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id":3248867546045330458, "__mgmt_id":3248867546045330458} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.893866, "value1":0.38643143057076507, "value2":6815713871810191176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id":5999814608189423062, "__mgmt_id":5999814608189423062} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561678, "value1":0.4057036812724311, "value2":4919580315129580518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id":3265694429872982019, "__mgmt_id":3265694429872982019} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004391, "value1":0.4801368737089002, "value2":7547608457928538573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id":4395099723947130617, "__mgmt_id":4395099723947130617} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.923757, "value1":0.7739736243035474, "value2":6832671633674661606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id":-391108988132571423, "__mgmt_id":-391108988132571423} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819284, "value1":0.7572793995611461, "value2":68290908865890434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id":-3512838032523373226, "__mgmt_id":-3512838032523373226} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.524308, "value1":0.44777317605800043, "value2":4272485517844220240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id":-2019541078929997410, "__mgmt_id":-2019541078929997410} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804762, "value1":0.20222965946470728, "value2":2164767398150388054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id":-5919712371960744559, "__mgmt_id":-5919712371960744559} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060732, "value1":0.060291087703427844, "value2":8373529279962935628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id":4237925648967851541, "__mgmt_id":4237925648967851541} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325849, "value1":0.2785168531780935, "value2":8895250356037639155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id":-770834305924250230, "__mgmt_id":-770834305924250230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.378439, "value1":0.43808690997464833, "value2":7750734163790876517, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id":6485264920164065997, "__mgmt_id":6485264920164065997} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.341809, "value1":0.21201598129686366, "value2":5439024219774015639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id":2073654399592069134, "__mgmt_id":2073654399592069134} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.027383, "value1":0.519384455440142, "value2":2830105304694072087, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id":2882295507697788229, "__mgmt_id":2882295507697788229} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578370, "value1":0.4103102602537048, "value2":3491530677431875174, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id":-7220759306975072706, "__mgmt_id":-7220759306975072706} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.588914, "value1":0.6688147984910581, "value2":7129525479125432993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id":7896555977684467219, "__mgmt_id":7896555977684467219} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781407, "value1":0.7254015801455519, "value2":2135519716242173777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id":6393502484828816639, "__mgmt_id":6393502484828816639} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852755, "value1":0.6592390034228756, "value2":5378675889994915559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id":-1817488686594165705, "__mgmt_id":-1817488686594165705} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.996821, "value1":0.5491613669334785, "value2":3858296464786926451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id":-5891578966361414907, "__mgmt_id":-5891578966361414907} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.364027, "value1":0.6056949341817615, "value2":2956076236179631886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id":-5759786475451562505, "__mgmt_id":-5759786475451562505} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.605233, "value1":0.2433469771258824, "value2":8502972839119365612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id":2288700931631591137, "__mgmt_id":2288700931631591137} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.092828, "value1":0.29453399849858286, "value2":5192582279143186491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id":-5779278934174818645, "__mgmt_id":-5779278934174818645} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.179799, "value1":0.19121582692459024, "value2":9103159579935481171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id":-1876395674046103849, "__mgmt_id":-1876395674046103849} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.054748, "value1":0.7255577017707996, "value2":8476394128272498701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id":-7012885648223233057, "__mgmt_id":-7012885648223233057} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.959235, "value1":0.5345272240165994, "value2":8621793756396494146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id":5804665130094856991, "__mgmt_id":5804665130094856991} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.735109, "value1":0.7362204691534435, "value2":942332422397834734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id":-7344671437631279298, "__mgmt_id":-7344671437631279298} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778088, "value1":0.8485308761635748, "value2":8355286652172671018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id":-6621083798343228179, "__mgmt_id":-6621083798343228179} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026529, "value1":0.058403258262061046, "value2":3881865567315099972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id":-1607750710100122463, "__mgmt_id":-1607750710100122463} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.405396, "value1":0.78515016074467, "value2":3124806945169635921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id":-8161171256724117796, "__mgmt_id":-8161171256724117796} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325780, "value1":0.8484383369933163, "value2":5639577297430380059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id":5946130633726276979, "__mgmt_id":5946130633726276979} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.125526, "value1":0.9437452957692816, "value2":1600583134494091930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id":-8946617436119621199, "__mgmt_id":-8946617436119621199} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040638, "value1":0.8412837985326597, "value2":1891897645252751874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id":8845218157639111760, "__mgmt_id":8845218157639111760} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225539, "value1":0.7607755326390468, "value2":4494740517836964298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id":2951654281967391321, "__mgmt_id":2951654281967391321} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417989, "value1":0.8351179906534547, "value2":6640751019261660918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id":8257293525391999222, "__mgmt_id":8257293525391999222} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.930391, "value1":0.7797566384384467, "value2":5344540873017186839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id":-3884817009449312047, "__mgmt_id":-3884817009449312047} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130784, "value1":0.9081520679550602, "value2":2673655563019876645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id":2341952400790155478, "__mgmt_id":2341952400790155478} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065288, "value1":0.37878147094860193, "value2":40939122026803562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id":6875804869664046307, "__mgmt_id":6875804869664046307} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.023954, "value1":0.7421706325870447, "value2":3936838989809781841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id":-593944646664066983, "__mgmt_id":-593944646664066983} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639378, "value1":0.04809332390342072, "value2":138564553177528595, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id":-3088435001982835586, "__mgmt_id":-3088435001982835586} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.968036, "value1":0.4229941922357839, "value2":3252691734036720501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_6":"k","key_9":"j","key_4":"e", "__series_id":3654818614521571007, "__mgmt_id":3654818614521571007} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925601, "value1":0.5862382194484029, "value2":8809171247244862983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id":-8643978816140336726, "__mgmt_id":-8643978816140336726} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.615163, "value1":0.41538444706998723, "value2":7806072407872661917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id":1969862486326467304, "__mgmt_id":1969862486326467304} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432657, "value1":0.2054174608330251, "value2":523721991362366859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id":9047687757698452618, "__mgmt_id":9047687757698452618} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584157, "value1":0.38115353295991117, "value2":5695252506949414616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id":1895373427612509316, "__mgmt_id":1895373427612509316} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969537, "value1":0.31115941644555445, "value2":3813252754578465682, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id":3557796764686593576, "__mgmt_id":3557796764686593576} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.755713, "value1":0.6078558755538082, "value2":4745658402471707078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id":7934961687481133421, "__mgmt_id":7934961687481133421} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.299770, "value1":0.39494561763395397, "value2":7454472376009363943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id":7028579199313468984, "__mgmt_id":7028579199313468984} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549446, "value1":0.9276674467108077, "value2":7450051253648573468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_4":"c","key_6":"c","key_3":"c", "__series_id":-5487303349427062046, "__mgmt_id":-5487303349427062046} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.339276, "value1":0.16417640235809516, "value2":376449946500069421, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_7":"j","key_4":"j","key_6":"c", "__series_id":6074226575508628248, "__mgmt_id":6074226575508628248} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.044679, "value1":0.022401336029968563, "value2":7584563374980649526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id":-7977482960971959221, "__mgmt_id":-7977482960971959221} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978082, "value1":0.634805138120068, "value2":2509593268797839298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id":5541777429936042172, "__mgmt_id":5541777429936042172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020832, "value1":0.2523976489068679, "value2":5330781529709694724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id":-2446119420668167024, "__mgmt_id":-2446119420668167024} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.874340, "value1":0.5447943369952332, "value2":753638192956179932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id":-6264079928411176214, "__mgmt_id":-6264079928411176214} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.532731, "value1":0.3415364500896806, "value2":4617908501249248943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id":5259931197981260673, "__mgmt_id":5259931197981260673} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.888667, "value1":0.03025871907918381, "value2":389467035709739331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id":6649881170970133250, "__mgmt_id":6649881170970133250} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630653, "value1":0.3815158272162314, "value2":1365697000438851836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id":-3642784698751404809, "__mgmt_id":-3642784698751404809} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947344, "value1":0.018503516339994433, "value2":6964047457370572921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id":-8290345855141384414, "__mgmt_id":-8290345855141384414} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.635335, "value1":0.7735503313212675, "value2":7863214250826862536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id":4177659989436222538, "__mgmt_id":4177659989436222538} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.323808, "value1":0.9277969228346735, "value2":7104799245414939756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id":5224088293720667434, "__mgmt_id":5224088293720667434} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527571, "value1":0.19552679967510578, "value2":2150832405165980506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id":5463684432285000465, "__mgmt_id":5463684432285000465} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.076891, "value1":0.45024886897327476, "value2":7461894110864047862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id":8013027916162000847, "__mgmt_id":8013027916162000847} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433797, "value1":0.03730319472343055, "value2":8041320080812623904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id":-3273450908649443185, "__mgmt_id":-3273450908649443185} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745634, "value1":0.6889162665973937, "value2":7455824590286680395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id":5397678799753812971, "__mgmt_id":5397678799753812971} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310508, "value1":0.4992657909321023, "value2":5571385578668134720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id":-2152602439220104890, "__mgmt_id":-2152602439220104890} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343053, "value1":0.21251046051646363, "value2":6470523853548900348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id":9073385261045683729, "__mgmt_id":9073385261045683729} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903927, "value1":0.9720310949900105, "value2":7380705280316406577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id":7510078098956056860, "__mgmt_id":7510078098956056860} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.200927, "value1":0.42664784751514406, "value2":5730113255528003277, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id":3753170390494216769, "__mgmt_id":3753170390494216769} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.563845, "value1":0.8055339411145567, "value2":7403476714487686904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id":5554465061930605789, "__mgmt_id":5554465061930605789} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.329254, "value1":0.8021973639133464, "value2":3080745955617166362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id":-2011474303426561839, "__mgmt_id":-2011474303426561839} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.879352, "value1":0.4277546544111605, "value2":1478811657161573091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_3":"a","key_9":"h","key_0":"i", "__series_id":-7410343999162991368, "__mgmt_id":-7410343999162991368} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442846, "value1":0.006346529763383217, "value2":5543697338259725446, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id":-3993102668143572443, "__mgmt_id":-3993102668143572443} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.094781, "value1":0.40728088001797547, "value2":6613387129832293245, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id":6443612413446657716, "__mgmt_id":6443612413446657716} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.975150, "value1":0.3661351435962595, "value2":830819320558121686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id":3230940730205655533, "__mgmt_id":3230940730205655533} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245141, "value1":0.02569157141735858, "value2":2603499185616374516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id":4336773822987251236, "__mgmt_id":4336773822987251236} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949515, "value1":0.3643114110362523, "value2":832829152625075749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_6":"k","key_2":"f","key_4":"i", "__series_id":-3917471179921730230, "__mgmt_id":-3917471179921730230} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741852, "value1":0.28794172682449687, "value2":453442634675086185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id":-8475829672726930205, "__mgmt_id":-8475829672726930205} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.738181, "value1":0.6087693219046798, "value2":7047508567290984350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id":-2835267016325291387, "__mgmt_id":-2835267016325291387} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.492510, "value1":0.5382269802655012, "value2":2689541564459522343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_3":"d","key_7":"i","key_2":"i", "__series_id":2755156865704901559, "__mgmt_id":2755156865704901559} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988702, "value1":0.0728816330863758, "value2":5556532985784164765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id":-3684675206016205580, "__mgmt_id":-3684675206016205580} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.655057, "value1":0.7734197867503856, "value2":5244006855203800595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id":-5902123974422169223, "__mgmt_id":-5902123974422169223} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520001, "value1":0.4376795320695461, "value2":1470774311764721388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id":6986853351187564285, "__mgmt_id":6986853351187564285} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164473, "value1":0.6572062696131487, "value2":1949710197196336093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id":-8229195773865957634, "__mgmt_id":-8229195773865957634} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.190159, "value1":0.9525604598487357, "value2":5478966602860717950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id":-5149502230124017537, "__mgmt_id":-5149502230124017537} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829372, "value1":0.6923157559412111, "value2":5023489111088251295, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id":-4421406399737878814, "__mgmt_id":-4421406399737878814} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391391, "value1":0.6340217327197775, "value2":8000248111604879168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id":-389408712295111002, "__mgmt_id":-389408712295111002} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883302, "value1":0.8070620002939071, "value2":653446263691796732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id":6306506282210639638, "__mgmt_id":6306506282210639638} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745426, "value1":0.45672801954182946, "value2":6893619651580083925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id":6312106658869392745, "__mgmt_id":6312106658869392745} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982512, "value1":0.5060051247468136, "value2":7419982333827237040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id":5247126023882251421, "__mgmt_id":5247126023882251421} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827729, "value1":0.15692262245568137, "value2":3804375123078218665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id":2218975377452306524, "__mgmt_id":2218975377452306524} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244618, "value1":0.6655062510702947, "value2":1607640639377335930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id":5113417579211753098, "__mgmt_id":5113417579211753098} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.945643, "value1":0.3793044572114196, "value2":8429298239346478761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id":-4936732084670057584, "__mgmt_id":-4936732084670057584} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626649, "value1":0.22068614209697351, "value2":8899816776188911022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id":-5163554538407216849, "__mgmt_id":-5163554538407216849} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045981, "value1":0.34313373362766186, "value2":1556434949234629007, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id":8148489552467310439, "__mgmt_id":8148489552467310439} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.381111, "value1":0.4930245395823471, "value2":3486502897378084847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id":-3944050620113208172, "__mgmt_id":-3944050620113208172} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596053, "value1":0.23068160326995202, "value2":1094706285002463668, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id":-4730499102096054036, "__mgmt_id":-4730499102096054036} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454071, "value1":0.4016206555788349, "value2":7766949472696777624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id":-2283650812855623557, "__mgmt_id":-2283650812855623557} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122082, "value1":0.12814082114630934, "value2":6860595913103836152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id":8383629273121902238, "__mgmt_id":8383629273121902238} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267563, "value1":0.915043661363235, "value2":7117774485449523197, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id":8192386169222148683, "__mgmt_id":8192386169222148683} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883324, "value1":0.48609131055046695, "value2":168158041910482971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id":-6271717978981701453, "__mgmt_id":-6271717978981701453} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.798512, "value1":0.9160093333280128, "value2":5957101416277255647, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_6":"c","key_9":"a","key_0":"k", "__series_id":-6816225054378181173, "__mgmt_id":-6816225054378181173} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.956973, "value1":0.08410692933360829, "value2":3670111299929079308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id":-2352461444413661306, "__mgmt_id":-2352461444413661306} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799611, "value1":0.006983554187354543, "value2":560914434528784616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id":-3385512603376303519, "__mgmt_id":-3385512603376303519} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539372, "value1":0.1527069613506525, "value2":5396674816694339651, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id":-6851409186770428825, "__mgmt_id":-6851409186770428825} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.538750, "value1":0.21348159925068225, "value2":4409731296615206554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id":1189528832259185027, "__mgmt_id":1189528832259185027} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.854320, "value1":0.921941847389472, "value2":4199458567691684987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id":-4493020240885746936, "__mgmt_id":-4493020240885746936} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451148, "value1":0.5527973855585238, "value2":5090625521103681352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id":8661813703504300442, "__mgmt_id":8661813703504300442} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.585722, "value1":0.08217397453744285, "value2":5891174882356923897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_8":"k","key_9":"g","key_2":"c", "__series_id":-8613423020305213465, "__mgmt_id":-8613423020305213465} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280675, "value1":0.17159417706244065, "value2":1155748852941674950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id":-5237232801702246555, "__mgmt_id":-5237232801702246555} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673546, "value1":0.9064641866629127, "value2":5826431744512876414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id":-2788154845987041856, "__mgmt_id":-2788154845987041856} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748734, "value1":0.7184282675490825, "value2":465504921239629350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id":-9060373555969354210, "__mgmt_id":-9060373555969354210} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448890, "value1":0.21947167931430242, "value2":97944850538911159, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id":5845352045669007793, "__mgmt_id":5845352045669007793} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.034158, "value1":0.6471204512621925, "value2":6508843210808028097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id":4773367547171091406, "__mgmt_id":4773367547171091406} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495253, "value1":0.7273452290068572, "value2":2698451345131648958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id":2681296371108068246, "__mgmt_id":2681296371108068246} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.362458, "value1":0.6109728168729301, "value2":8601029350565666815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id":-1620750445123524491, "__mgmt_id":-1620750445123524491} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263467, "value1":0.5089131780165215, "value2":7125458167541328714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id":1541348361758017758, "__mgmt_id":1541348361758017758} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373946, "value1":0.4318385725650386, "value2":2714678475704530055, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id":7246491393905121980, "__mgmt_id":7246491393905121980} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714267, "value1":0.9575832102733836, "value2":6392945512202004757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id":-3841678152688368567, "__mgmt_id":-3841678152688368567} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448342, "value1":0.22459353529927165, "value2":3260297440141888471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id":-3322655866165129051, "__mgmt_id":-3322655866165129051} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636333, "value1":0.5831279192445404, "value2":4643961286576988019, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id":-8920885921549084638, "__mgmt_id":-8920885921549084638} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.544083, "value1":0.5292455018046274, "value2":2947716990317042035, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id":-2133694680914957018, "__mgmt_id":-2133694680914957018} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.834093, "value1":0.03492084160563978, "value2":317252294366234894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id":6445232353634246963, "__mgmt_id":6445232353634246963} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266819, "value1":0.3363934052516993, "value2":7398703592096323308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id":-8058668717700101835, "__mgmt_id":-8058668717700101835} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006504, "value1":0.7543858008870775, "value2":1775651744055633759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id":8609030237689560074, "__mgmt_id":8609030237689560074} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114763, "value1":0.5343311798406293, "value2":3440557676505536535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id":126931402558459640, "__mgmt_id":126931402558459640} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338231, "value1":0.46269924741830853, "value2":2414529329564577894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id":7103485997179993993, "__mgmt_id":7103485997179993993} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.613570, "value1":0.5208853551038555, "value2":1476820686457140000, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id":2618446898923694438, "__mgmt_id":2618446898923694438} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954830, "value1":0.41362302765893333, "value2":5510842372132552038, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id":5745116829380616979, "__mgmt_id":5745116829380616979} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.009033, "value1":0.39228566619392624, "value2":4427054644662596059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id":-8638927876938940545, "__mgmt_id":-8638927876938940545} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182689, "value1":0.6549868089884497, "value2":4035004692027824929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id":509974957040433669, "__mgmt_id":509974957040433669} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037729, "value1":0.7486872272646202, "value2":4847504507990849152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id":1971862127581653694, "__mgmt_id":1971862127581653694} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.025112, "value1":0.34907292952301494, "value2":6027470898165077200, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id":-1804379138188887773, "__mgmt_id":-1804379138188887773} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438904, "value1":0.6318755547819239, "value2":7111135668532014691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id":-2151408237051366225, "__mgmt_id":-2151408237051366225} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812145, "value1":0.3675412312936663, "value2":5692745323903444622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id":6639686090018856376, "__mgmt_id":6639686090018856376} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.445020, "value1":0.3643115537301237, "value2":1216854539448528229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id":-3178230256095965210, "__mgmt_id":-3178230256095965210} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149138, "value1":0.5909969143971484, "value2":7044155605397820069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id":2114766337578869876, "__mgmt_id":2114766337578869876} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841137, "value1":0.5031802947436413, "value2":4756712241095891395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id":8896905802866397624, "__mgmt_id":8896905802866397624} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766360, "value1":0.42396402951055434, "value2":5375665261877407507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id":7871636101908571807, "__mgmt_id":7871636101908571807} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663263, "value1":0.21109789310310256, "value2":7791325581861318665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id":5714461649218477173, "__mgmt_id":5714461649218477173} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.623457, "value1":0.10997466706442727, "value2":4022405196702857232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id":-2237621028132536337, "__mgmt_id":-2237621028132536337} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687642, "value1":0.5890580006230834, "value2":4798397386267850627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_2":"b","key_0":"i", "__series_id":227860907906626026, "__mgmt_id":227860907906626026} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041534, "value1":0.7259891148801754, "value2":7530272514921895889, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id":-8040910452447775473, "__mgmt_id":-8040910452447775473} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.8243889839817137, "value2":2465746906164654846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id":606884971259349760, "__mgmt_id":606884971259349760} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.282429, "value1":0.6519583818560468, "value2":6092402468810251742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id":-5478061519059366611, "__mgmt_id":-5478061519059366611} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816484, "value1":0.18010854645368016, "value2":8191892113724148654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id":-124525408228495085, "__mgmt_id":-124525408228495085} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851506, "value1":0.5899554257608193, "value2":4617440579117228711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id":-7770945742068148908, "__mgmt_id":-7770945742068148908} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.382630, "value1":0.5412707148385829, "value2":3548532358946210519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id":980402580434312102, "__mgmt_id":980402580434312102} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.473179, "value1":0.6228095400098033, "value2":5572845251799335628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id":-4104284881803245064, "__mgmt_id":-4104284881803245064} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266666, "value1":0.5868701622069099, "value2":6991916315373855609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_9":"d","key_3":"d","key_7":"f", "__series_id":-8580448352974816854, "__mgmt_id":-8580448352974816854} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.952001, "value1":0.6189910604042005, "value2":6928874617046934238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id":5093385098667663207, "__mgmt_id":5093385098667663207} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107532, "value1":0.28068061790544596, "value2":7672880301382817972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id":-7638387450661547132, "__mgmt_id":-7638387450661547132} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.688840, "value1":0.6035662636797529, "value2":5891970996329552554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id":3013184525747786739, "__mgmt_id":3013184525747786739} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283889, "value1":0.9151251791608558, "value2":2952612587882160016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id":-3208973475909870211, "__mgmt_id":-3208973475909870211} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629939, "value1":0.5930273861647344, "value2":7387132779647308781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id":-5881314218080136649, "__mgmt_id":-5881314218080136649} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418325, "value1":0.9507448442764749, "value2":167010402424657095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_9":"f","key_1":"h","key_5":"d", "__series_id":-1869366054771332234, "__mgmt_id":-1869366054771332234} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373327, "value1":0.6218852126400313, "value2":8371256546452205404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id":853455729293892, "__mgmt_id":853455729293892} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515481, "value1":0.0854866845303678, "value2":6987828829468502321, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id":-1880586229668866123, "__mgmt_id":-1880586229668866123} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820817, "value1":0.698882043745769, "value2":8235587889784697376, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id":-4611408819127013586, "__mgmt_id":-4611408819127013586} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.365626, "value1":0.4531560215251981, "value2":4722270617283077560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id":8076734271383159573, "__mgmt_id":8076734271383159573} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484276, "value1":0.6659326817404172, "value2":8721021931100680532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id":660960526644120465, "__mgmt_id":660960526644120465} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.528708, "value1":0.03487880924213162, "value2":6041728758379789938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id":7454847514743186106, "__mgmt_id":7454847514743186106} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590377, "value1":0.34239911489608105, "value2":7295038165138773264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id":-4087977803898185094, "__mgmt_id":-4087977803898185094} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481162, "value1":0.8346535177821227, "value2":4835378003160835691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id":7365105254973640807, "__mgmt_id":7365105254973640807} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.664721, "value1":0.9670197543347829, "value2":7077761062907993448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id":3179964060483693959, "__mgmt_id":3179964060483693959} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242914, "value1":0.7732131934363016, "value2":6013582492758534765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id":4943495658983356925, "__mgmt_id":4943495658983356925} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.185221, "value1":0.10448347281111905, "value2":4626218721873500711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id":5476702965391324178, "__mgmt_id":5476702965391324178} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.126356, "value1":0.6396318028070207, "value2":7845764314900334021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id":-7659960989015255316, "__mgmt_id":-7659960989015255316} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673777, "value1":0.7918565155988029, "value2":7808431007480445571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id":1217326367461456504, "__mgmt_id":1217326367461456504} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706238, "value1":0.49404656958089044, "value2":8867399074074176905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id":3600556749683819879, "__mgmt_id":3600556749683819879} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216638, "value1":0.2610211982854062, "value2":5851177857276860153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id":6176476763516588600, "__mgmt_id":6176476763516588600} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247760, "value1":0.720377370743505, "value2":284915355108242656, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id":5570056772565202366, "__mgmt_id":5570056772565202366} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484098, "value1":0.20488798005491424, "value2":8924950640215344491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id":8119351734948487376, "__mgmt_id":8119351734948487376} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.345583, "value1":0.26657891861913446, "value2":1138178968371346598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id":6742140614929127838, "__mgmt_id":6742140614929127838} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.315052, "value1":0.7472751608991773, "value2":5324337837076333029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id":-1525457297647045442, "__mgmt_id":-1525457297647045442} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247789, "value1":0.35415032152422526, "value2":3640906326280302592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id":-3761616453811965441, "__mgmt_id":-3761616453811965441} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787688, "value1":0.6162646188121896, "value2":8883014095767101323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id":-1779822276119301297, "__mgmt_id":-1779822276119301297} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477237, "value1":0.04016123924628, "value2":9093318721099578903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id":8284198859322921936, "__mgmt_id":8284198859322921936} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560140, "value1":0.30318020971980014, "value2":2686556711005804654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id":-4372668294353943556, "__mgmt_id":-4372668294353943556} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.002924, "value1":0.5022585667052755, "value2":3145515076422963133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id":-7954753156742574681, "__mgmt_id":-7954753156742574681} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732614, "value1":0.3545238139152862, "value2":1059232874175794196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id":-7563389249691286962, "__mgmt_id":-7563389249691286962} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.332613, "value1":0.32230993437491456, "value2":2724091454501972237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id":-1843525876921281439, "__mgmt_id":-1843525876921281439} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012206, "value1":0.931677320101252, "value2":5370261932149351692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id":1826472716586726705, "__mgmt_id":1826472716586726705} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196655, "value1":0.8224901135235602, "value2":9208484728253262649, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id":-6562641159861820695, "__mgmt_id":-6562641159861820695} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.326865, "value1":0.5922171068495821, "value2":7224402625385076257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id":2665602518452103625, "__mgmt_id":2665602518452103625} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289542, "value1":0.9512116686992638, "value2":7523857296190024264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id":-481551255976768557, "__mgmt_id":-481551255976768557} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.170535, "value1":0.5872728280225836, "value2":6413447856713598093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id":-636081195137433630, "__mgmt_id":-636081195137433630} -{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434673, "value1":0.18109149352889556, "value2":1218373397895276791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id":-6832493598228589478, "__mgmt_id":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.860439, "value1":0.30809143156043617, "value2":8296859231789889769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697105, "value1":0.07464704753781262, "value2":3626726022956376302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499281, "value1":0.2598747988733079, "value2":1472024199460334836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389559, "value1":0.9035230182410444, "value2":5735152112214129185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259346, "value1":0.6088348511469825, "value2":2118449310673513905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_6":"k","key_7":"e","key_3":"f", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265140, "value1":0.41315907782222017, "value2":6967606456828959585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273243, "value1":0.3670231186776885, "value2":3094287292036347916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.697145, "value1":0.4712183861515544, "value2":233543654937300193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.570507, "value1":0.11967792858372435, "value2":1467009477255473639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421572, "value1":0.25352243267763397, "value2":5433840270731074230, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423291, "value1":0.6724498851050211, "value2":652727500980374465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072653, "value1":0.4078722430828867, "value2":6479204450678692857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.576237, "value1":0.03506340983128008, "value2":2409142807432889102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928873, "value1":0.9749671198417696, "value2":806127466997750314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.121356, "value1":0.3710118368988569, "value2":2961209029330787894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.527082, "value1":0.427919527676362, "value2":4168209888386865559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971852, "value1":0.4558638608790214, "value2":7271190379082523197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758405, "value1":0.642984416191772, "value2":635944986021178390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.191870, "value1":0.4028374291432409, "value2":4925274338231144549, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.177741, "value1":0.112412024356175, "value2":1799812085131373089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.562889, "value1":0.9463319128677472, "value2":8604221624040619668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.145145, "value1":0.6344301595600267, "value2":5934019334637844764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512765, "value1":0.12698238608047413, "value2":3179145500529665851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.947023, "value1":0.9362933587076374, "value2":6552003615608175727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479188, "value1":0.8368835012768996, "value2":1048667356432629681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.886895, "value1":0.147063588220359, "value2":2554258082886382046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495557, "value1":0.4897837285320758, "value2":3386687451644953805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267510, "value1":0.7137029646630675, "value2":4723657657521756601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662037, "value1":0.39308757655823606, "value2":5160142240464547787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.001360, "value1":0.2816582812804884, "value2":6076665375380746353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.501969, "value1":0.4236689406205709, "value2":725918256085862567, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.956392, "value1":0.4674704766085337, "value2":4478503621053198593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442861, "value1":0.15012494378573155, "value2":6132679974455498265, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394657, "value1":0.8655032404449484, "value2":2033181385746149676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.574670, "value1":0.8578324023084722, "value2":9030512731073145161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844866, "value1":0.401882448563829, "value2":6461587433040802039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.677301, "value1":0.17574043431028194, "value2":13200916467489149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643407, "value1":0.09478762453322248, "value2":9040728601657069686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.461584, "value1":0.2313038745196538, "value2":8542291140491717734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480373, "value1":0.33926139409918454, "value2":1470854458100424231, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.387467, "value1":0.9027179517679272, "value2":2152676801390448497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.081729, "value1":0.5930629975633843, "value2":3441892662554339501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.523164, "value1":0.27807377593445504, "value2":8726571622906981746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643880, "value1":0.8856942439946653, "value2":6532657219501338712, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.233730, "value1":0.12117132337083063, "value2":5262200042073716921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113242, "value1":0.15754750121773875, "value2":3512597140842356375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.592540, "value1":0.8501957441234611, "value2":1120617963379990831, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.754300, "value1":0.2516365689787487, "value2":1009529273537079157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.738781, "value1":0.9115797570978498, "value2":654040880693718238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_5":"j","key_2":"a","key_3":"b", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730945, "value1":0.567968215036453, "value2":3597506204077270120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.940173, "value1":0.9712119929757523, "value2":5983635236065602039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326670, "value1":0.07295490390000714, "value2":6022184335907772623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.571799, "value1":0.5813154467308365, "value2":9191368645337346592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.547202, "value1":0.12691857025568046, "value2":3820781159866636536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097732, "value1":0.6481650315571341, "value2":6773475876286839908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.184127, "value1":0.5736971101036478, "value2":7284609149743260027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.925677, "value1":0.3771049289873205, "value2":1181794630742224031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.157660, "value1":0.5233716379944181, "value2":4014079834827187874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.413654, "value1":0.21255841055237895, "value2":4422822169208563976, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_9":"a","key_0":"c","key_4":"d", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282429, "value1":0.5134755380331291, "value2":2875914720894216239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510799, "value1":0.4811695837424015, "value2":2131765002626290228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855059, "value1":0.1803859944439004, "value2":4143870194342227353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841548, "value1":0.8921162909498762, "value2":2995639098842929032, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904367, "value1":0.6827455696983789, "value2":4486024070965990492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903308, "value1":0.5275068815139481, "value2":4431489892412543662, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369269, "value1":0.3088848570530923, "value2":2711948390039375929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.246425, "value1":0.22469514297321577, "value2":3311794715258074199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.627147, "value1":0.5085084020761419, "value2":7449844520407226749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757065, "value1":0.7340080407469496, "value2":1399142812283896331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404001, "value1":0.6927872548099725, "value2":4611922647039691390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.548980, "value1":0.9504303368975154, "value2":784663243740669037, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088930, "value1":0.5112126366657939, "value2":1061303860942908201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808273, "value1":0.07227725639854525, "value2":705291637800185980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.195134, "value1":0.9915791690400784, "value2":1380206536675804157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588271, "value1":0.9410252962292832, "value2":2976150472594021014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830440, "value1":0.5990964794781901, "value2":6208539757039787288, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_9":"a","key_1":"g","key_6":"b", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.988764, "value1":0.12421784563285446, "value2":600116094525731026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793643, "value1":0.013993638720735705, "value2":249013181149620349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_5":"j","key_1":"d", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537335, "value1":0.9710360883032237, "value2":3990188937402642001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.783911, "value1":0.5202417888338783, "value2":9207852713084863631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_4":"b","key_6":"f","key_2":"k", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000369, "value1":0.4757654945131251, "value2":617662898409566699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720049, "value1":0.11616063522827627, "value2":1134111314294224704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850537, "value1":0.856287241545387, "value2":3915015074142268096, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117449, "value1":0.23938708551093754, "value2":6182563875693526929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.720636, "value1":0.6929840624902686, "value2":7184719611552077354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890463, "value1":0.39990668829147297, "value2":5236439635142539127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598510, "value1":0.794840854439141, "value2":7790247245768394828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_2":"e","key_7":"j","key_1":"c", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.103618, "value1":0.837788909134762, "value2":1746463551678400384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.890891, "value1":0.701693361054133, "value2":750546117784279688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243694, "value1":0.04252945224095913, "value2":7022047191936829657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698720, "value1":0.8318935119459053, "value2":2403728383571794535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.362462, "value1":0.5425805070018154, "value2":7722798226519636439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447582, "value1":0.6901117678727204, "value2":1225566659535077888, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884009, "value1":0.9848633762091424, "value2":1508601246687431755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.895176, "value1":0.1370326687436836, "value2":7673037864112111033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.521633, "value1":0.14853540424497083, "value2":6875647668629772843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388849, "value1":0.856681841687182, "value2":289008565390108021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.747443, "value1":0.735786478790032, "value2":7309545501096360932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528790, "value1":0.010059663849976508, "value2":3366632872144662566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.586068, "value1":0.8316103940598538, "value2":6216090017784726937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154422, "value1":0.6449454723419513, "value2":5588370277273470937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.476987, "value1":0.688075774177057, "value2":3232344568636281729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.176248, "value1":0.8630722552735859, "value2":7054623290814757387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.553861, "value1":0.6275284095417549, "value2":8413581575795957099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482699, "value1":0.30152766631318734, "value2":2728742455813076667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228157, "value1":0.09912287337858451, "value2":1368124455579266783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.363726, "value1":0.2556344439654489, "value2":8083466247113334458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598631, "value1":0.00654705967457182, "value2":678136325524679028, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188097, "value1":0.33693220746708563, "value2":5349169986765993063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.591197, "value1":0.3158077067804786, "value2":8408038257495371649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.723537, "value1":0.8935982855510853, "value2":6781320910854884909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.714830, "value1":0.8483752317059017, "value2":4114947449469032042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.510153, "value1":0.8569271663104114, "value2":2163617609666100779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.556403, "value1":0.46846499185707535, "value2":4543935722642344774, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115027, "value1":0.13242237154578534, "value2":4001391755557856082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700402, "value1":0.31919253163578853, "value2":9168039730828764814, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473591, "value1":0.914966327309789, "value2":7574079944969934140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657212, "value1":0.7123671399711553, "value2":5845893001941155972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389666, "value1":0.46892477810031413, "value2":3773448585226589360, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365529, "value1":0.10038955567825718, "value2":7677417085782135970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.181780, "value1":0.3807188874415628, "value2":213730861941923164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632990, "value1":0.6352610882615881, "value2":7695215886438259449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.801703, "value1":0.5435516239407236, "value2":1197124892718412788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943146, "value1":0.6459645738185148, "value2":257718117926443886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047331, "value1":0.2599891672558978, "value2":1859555081827581052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.329003, "value1":0.7536967883211322, "value2":8944365307464980699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750531, "value1":0.8871777984871712, "value2":4584326435451561707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820378, "value1":0.6275908409813838, "value2":2808244151477882254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.664852, "value1":0.971622999773127, "value2":3565786429109692783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502157, "value1":0.19081539446623427, "value2":1479393056820182163, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396316, "value1":0.5123967312660013, "value2":8943627767884961054, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934631, "value1":0.19466544110858217, "value2":3956006751549658946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206549, "value1":0.017226775576345523, "value2":7020366329672792405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118256, "value1":0.5188472954068934, "value2":8138153858856845378, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584343, "value1":0.14448963894972008, "value2":4790553180218846981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.877457, "value1":0.6880296047171478, "value2":1215785358113912003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409285, "value1":0.05411761237341633, "value2":4709001633389005280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364854, "value1":0.8998190824029622, "value2":2511578657661549153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.973548, "value1":0.8069354470874932, "value2":2595652771451514899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.073905, "value1":0.2517811560702055, "value2":6140023292659546232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.262163, "value1":0.5630653619265122, "value2":3018188000144695876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641275, "value1":0.6900145697913901, "value2":5216098013140123128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392564, "value1":0.018717055152633055, "value2":8077083634713789634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932220, "value1":0.8388125579510224, "value2":2685186121830778256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.206042, "value1":0.49376606624615665, "value2":5850726108082863170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215184, "value1":0.356531340492791, "value2":4642776908341780996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.558758, "value1":0.575409651955595, "value2":2207683625879983306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953416, "value1":0.9073618953615663, "value2":5651749773058007750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.648335, "value1":0.46358450476088653, "value2":1290620516019440255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.487910, "value1":0.012547280333283235, "value2":4919259233823782968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.378926, "value1":0.7085830860476057, "value2":2171649428344764858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_8":"j","key_3":"b","key_4":"k", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802501, "value1":0.31728201505130527, "value2":5419079224591774860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.656795, "value1":0.24875018847328167, "value2":485829881375178402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938831, "value1":0.08769582144908596, "value2":6951540802412020838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231956, "value1":0.7863053106916462, "value2":1030265256144215264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.631246, "value1":0.46917511334330375, "value2":3774063736313175623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305077, "value1":0.834556547518894, "value2":636203159633468096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.808070, "value1":0.5245784354956413, "value2":5259003801435693215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.641827, "value1":0.7755759714857714, "value2":3467599768461222251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.306769, "value1":0.47958408429359173, "value2":5365936635633468542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294040, "value1":0.5158994633208911, "value2":7429034992856357163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.961943, "value1":0.7435043392340425, "value2":8500250963468273773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935561, "value1":0.5801624578827164, "value2":8754050195411098007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444846, "value1":0.030970547839451835, "value2":8387858736748608077, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358418, "value1":0.5401368375372775, "value2":7107146838213157067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.546704, "value1":0.46330699877542014, "value2":8769213962266908815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.270889, "value1":0.46793759900681503, "value2":6364231395353410490, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549392, "value1":0.3430929972372329, "value2":4966477192488000261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428697, "value1":0.7243138275027047, "value2":2836857643122176656, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993399, "value1":0.40084653235121936, "value2":6794009333725376121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.506725, "value1":0.07262914457616802, "value2":3459356114730929347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202924, "value1":0.07381206265687722, "value2":2649194281139606297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.411238, "value1":0.9684314554036981, "value2":3705584143921682379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_5":"h","key_6":"k","key_4":"c", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358603, "value1":0.534711919539449, "value2":4138168909279457429, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989681, "value1":0.19207319251633845, "value2":1593591737448226138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.052714, "value1":0.27756026777792103, "value2":6377557735209128817, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614036, "value1":0.018601161874125947, "value2":151552447676967905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.305181, "value1":0.031288435638661224, "value2":3937949086874523036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715232, "value1":0.6397334976781403, "value2":7614941574913315366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705549, "value1":0.6996034412212425, "value2":6200772172174019130, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.302276, "value1":0.5239925847096263, "value2":1057710529698426724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200111, "value1":0.2567985665426193, "value2":4529602646404884783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_2":"g","key_8":"f","key_1":"a", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.727436, "value1":0.43685005741153693, "value2":3304580792456252079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.351586, "value1":0.3164739225094333, "value2":3897998541684002875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.665954, "value1":0.5906227199646251, "value2":6746215572356057637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361921, "value1":0.3364933451115498, "value2":761972198573021391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.889779, "value1":0.36794570756599093, "value2":2090465055639313788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.699184, "value1":0.5483254879513706, "value2":7720905658039555449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.388162, "value1":0.5287855210410441, "value2":7195732574645022274, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.154496, "value1":0.6641476496489667, "value2":9137939751947784172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327473, "value1":0.5518373975194654, "value2":3971910670147599886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063454, "value1":0.43820340286329656, "value2":5405166066848368254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.781841, "value1":0.031101766659472223, "value2":4189381224837071716, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171755, "value1":0.46498479978119844, "value2":4931395791650935239, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.559617, "value1":0.8752912652134259, "value2":8586908304160851241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.323852, "value1":0.35821728605182146, "value2":5163275989952284916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030550, "value1":0.8225486647874358, "value2":2481647473431927932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.712533, "value1":0.368964069168292, "value2":8062115249497297834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298843, "value1":0.30262512534550123, "value2":5999722310995378983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778244, "value1":0.3716152000047593, "value2":6978129792906951032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.144352, "value1":0.42812039567105226, "value2":6591666290600795514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616112, "value1":0.5501694032291233, "value2":5953070646927834342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843613, "value1":0.7957093053513518, "value2":779447045934204434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.021516, "value1":0.3651263217338076, "value2":1101615937937937836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.707183, "value1":0.9474415697823648, "value2":3163832075607398652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807066, "value1":0.328067579350897, "value2":1404186285892115543, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327059, "value1":0.4531450671817094, "value2":3186214527240993583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.629874, "value1":0.555595498018525, "value2":7512625623217629070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203064, "value1":0.24518269440439186, "value2":7974348531674940058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865398, "value1":0.7125714321236861, "value2":1185618794516530658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.098668, "value1":0.6050958429061423, "value2":7737776979675298527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659514, "value1":0.8901996322311656, "value2":7752156196236457080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_6":"h","key_9":"d","key_5":"g", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650309, "value1":0.7869508663362883, "value2":6010716207202506080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.779127, "value1":0.6366846028665012, "value2":7797452629031711950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.281652, "value1":0.9118975722819298, "value2":6103834325448620626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868834, "value1":0.18117722257021981, "value2":6098726761586315616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.865255, "value1":0.5168875940701739, "value2":7616170573520868343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000667, "value1":0.8879098452429443, "value2":2716291319460579102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938952, "value1":0.27931352835868933, "value2":2351410106131072829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524371, "value1":0.5375950258397566, "value2":7234763394924546015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718980, "value1":0.7918266548241415, "value2":1156714261082582562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532951, "value1":0.4063578415664877, "value2":2401980297367889095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780599, "value1":0.15947463153475608, "value2":6812388220933080605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078621, "value1":0.6070073793267291, "value2":1012316510571499763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.404621, "value1":0.9613908610430107, "value2":6888867545087821595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.130043, "value1":0.26321351304547996, "value2":4446851417687808950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602508, "value1":0.38864606344069585, "value2":2797998328851090673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244218, "value1":0.5263437247014857, "value2":884821963010111580, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415267, "value1":0.3515494200462383, "value2":3356463166369801537, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303765, "value1":0.5270577861362743, "value2":2138283533246685033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.254002, "value1":0.303619701206094, "value2":7629281847530948891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.588498, "value1":0.5481446854335168, "value2":7641330013258723790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.620575, "value1":0.6420150180085478, "value2":7965173856733830205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243174, "value1":0.400523011031962, "value2":4931302022232098402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_9":"c","key_4":"d","key_8":"d", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.104291, "value1":0.4501470045022247, "value2":6419830717700946282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605242, "value1":0.5489672412234975, "value2":25272414308366080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705962, "value1":0.2696978321817019, "value2":7266661553560831399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.204594, "value1":0.050667752810758936, "value2":4939383554296719335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.694791, "value1":0.8451677349011486, "value2":1862202581082766068, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178458, "value1":0.47634419936555994, "value2":3993368962808231698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879184, "value1":0.05969390345398635, "value2":5274917008847651276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854470, "value1":0.8497662902937875, "value2":3035062077693412013, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.397422, "value1":0.08213093074253511, "value2":5967759139282299209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.584032, "value1":0.08273000380656788, "value2":2735658568798834919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822339, "value1":0.581737952515166, "value2":2604615641240906232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.308039, "value1":0.34242828701171635, "value2":3092061640651599674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.913723, "value1":0.005597195694116873, "value2":2620704762797681315, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_3":"b","key_8":"c","key_1":"f", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618957, "value1":0.6413425640197833, "value2":5287503767189177120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454987, "value1":0.32512559877918956, "value2":2506424876460932044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447066, "value1":0.20850990987663237, "value2":7977478851800531911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.683345, "value1":0.838649315370365, "value2":4999282712485449007, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935535, "value1":0.4089194818619663, "value2":722917113975878160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439326, "value1":0.8812716905619982, "value2":6658838831662104981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069099, "value1":0.15024751394280958, "value2":1328305014122533446, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_7":"f","key_3":"a","key_5":"e", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.473312, "value1":0.9280459649583414, "value2":3438360323166932806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.215711, "value1":0.8566516621694267, "value2":1663749581830575806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312134, "value1":0.566681231011358, "value2":6911167899525793851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862090, "value1":0.5364584771183928, "value2":7729755763890728353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456397, "value1":0.7096399907515605, "value2":5531147306124199678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.587843, "value1":0.8985325214047336, "value2":4030305760543278105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710467, "value1":0.5099257391896752, "value2":1179017586099298270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.188204, "value1":0.8723809422062396, "value2":5093258983701866401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.681846, "value1":0.23032248565211208, "value2":3357194399728955391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_2":"j","key_4":"c","key_1":"i", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.614220, "value1":0.9249072784237294, "value2":4254302260697988757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.028000, "value1":0.2236758406407766, "value2":5499863050282572666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923027, "value1":0.4209047854675161, "value2":4430077989918325993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203873, "value1":0.8404470170534011, "value2":532376833912389989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_2":"f","key_6":"b","key_1":"d", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.541847, "value1":0.24084743744285772, "value2":2314457094095648600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.932124, "value1":0.7743913731102018, "value2":7842905515694643998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.986636, "value1":0.18488491173653657, "value2":1233277417792576747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_6":"f","key_0":"j","key_2":"i", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.491189, "value1":0.5404267488261955, "value2":5193038540757554939, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.449064, "value1":0.9915502184839048, "value2":3914891481887672833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203773, "value1":0.11544040430435766, "value2":8720914144754355340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.637865, "value1":0.5397959742632489, "value2":5016935922539098494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.164476, "value1":0.40957356383804516, "value2":8101990321123394900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212208, "value1":0.31170589199654203, "value2":8172096864950686883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427578, "value1":0.9500075990560398, "value2":4448206698192420471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375888, "value1":0.08043137384857349, "value2":5428459057669693793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.253419, "value1":0.30426981771564987, "value2":3701960397375147641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.064111, "value1":0.6613366419136151, "value2":9214004723321204161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.618634, "value1":0.28919063506417325, "value2":2549339895850480456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.581311, "value1":0.5011828194867549, "value2":2394702011956086199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.619872, "value1":0.3579296101205142, "value2":3114339742928723546, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_6":"f","key_0":"c", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.928952, "value1":0.7843473952520837, "value2":4007100053019753408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.678699460362967, "value2":1449401258062210262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.791597, "value1":0.5225036343492986, "value2":7759663471842896645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794616, "value1":0.3395042657376233, "value2":1662502503940889595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.466754, "value1":0.9041538990976105, "value2":535273624649157699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693734, "value1":0.009440601901378784, "value2":2148867264362048197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841713, "value1":0.059732250656449104, "value2":6677903169635737819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.746032, "value1":0.16006431951166478, "value2":5022444125278904149, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.158292, "value1":0.8619758377742992, "value2":3793514584183151927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364688, "value1":0.7050682785197309, "value2":8826236482901828470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_9":"f","key_2":"e","key_7":"e", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.273970, "value1":0.40265473643498934, "value2":8303926135399734280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030530, "value1":0.7286415975665108, "value2":160039363895729267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_5":"j","key_3":"g", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266449, "value1":0.033547786099323434, "value2":7620487585623102750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025977, "value1":0.17793807237193235, "value2":8744525786084307091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.806031, "value1":0.5052650450730324, "value2":4060064708931174932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035732, "value1":0.9743453198182447, "value2":4099915557434576704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.646092, "value1":0.21956311768718226, "value2":8589888788620337480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078229, "value1":0.4079449893422866, "value2":6173402600986499343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666881, "value1":0.9863097557928853, "value2":8279527785132450943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.741108, "value1":0.31651275509392585, "value2":1547895388613496809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866620, "value1":0.6883437972608208, "value2":8802986125706842330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.718942, "value1":0.8847629636653906, "value2":800711854768452001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.375188, "value1":0.9532398566953418, "value2":2394341312296044798, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624088, "value1":0.6934451210618476, "value2":8474880456434193538, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680602, "value1":0.2209467711169388, "value2":8621094656072865370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.522533, "value1":0.5439997834907557, "value2":8198708215843245045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439975, "value1":0.8101860664171935, "value2":8929518033480490042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.602144, "value1":0.17084337908018365, "value2":6000683736434217499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594415, "value1":0.3959186882607224, "value2":1454959879954934650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917433, "value1":0.849867816493911, "value2":8615713829914363729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439894, "value1":0.7578484758847723, "value2":3887712481108512291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.965697, "value1":0.8251003697840577, "value2":836422853680285494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259251, "value1":0.4596388195214864, "value2":1597474370288095053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512513, "value1":0.783528381108141, "value2":8045517537006666296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_4":"i","key_6":"g","key_3":"c", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.452258, "value1":0.4477592871421156, "value2":1459300159451076858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735408, "value1":0.5067192602284456, "value2":7587131980242978873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.108601, "value1":0.3076343047473968, "value2":5066182910326911353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419278, "value1":0.12962316194187043, "value2":8864033357034639760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336677, "value1":0.24448588819210298, "value2":4567827434979480913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131541, "value1":0.02538083388681256, "value2":6449819714583284688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235312, "value1":0.7537710944451972, "value2":3840033782203811299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713440, "value1":0.6660557019192793, "value2":6317879942694845725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250100, "value1":0.2226451425042414, "value2":6756354166360377412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.545039, "value1":0.7203310314493744, "value2":6637075765032095898, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846538, "value1":0.5133638544522705, "value2":6146673076285104795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.099134, "value1":0.26713136173061386, "value2":5047631075889612988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.999599, "value1":0.19120338269199497, "value2":3009559225203387009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.456369, "value1":0.6496125050532283, "value2":8358687825156497010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.882684, "value1":0.4767076839685737, "value2":2058131066392704343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.326891, "value1":0.0072166654150041835, "value2":3877874561619455924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.753206, "value1":0.9891987223468547, "value2":2500770621881849387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.505150, "value1":0.33485751726586077, "value2":3075226382158740800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_6":"k","key_3":"h", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.337847, "value1":0.7587725777413157, "value2":4030061892629350122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.061555, "value1":0.5715183894256888, "value2":1380477982282562120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102225, "value1":0.18128048387136908, "value2":1107305884228635610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.663571, "value1":0.2082433172088735, "value2":2331488443653185612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400352, "value1":0.36344640764280667, "value2":4156784209596494480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567282, "value1":0.6853752986060114, "value2":1738184775840508143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580625, "value1":0.21556187277712435, "value2":1716362597563830669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.068750, "value1":0.5617303119904301, "value2":215978150703712746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948465, "value1":0.4015798445122638, "value2":4918723285934260509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818268, "value1":0.4923649301244579, "value2":2640505566674328479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344093, "value1":0.24900402040054898, "value2":3729697701643357874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096571, "value1":0.84402994415599, "value2":5393515911153006286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880117, "value1":0.7901487854013907, "value2":2074107282217049907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923332, "value1":0.6838866899470619, "value2":1947909482655854334, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.432183, "value1":0.35953677139878504, "value2":3191972662401496477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251030, "value1":0.9314169082560206, "value2":2792551904040732916, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.315849, "value1":0.9521659578315848, "value2":278261838735054052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.790453, "value1":0.9152891069156645, "value2":1668356628128321724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.260477, "value1":0.2126605025888137, "value2":6877799862560969541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.213669, "value1":0.023418304769384207, "value2":4056268380917249508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.647354, "value1":0.1608657659472805, "value2":7997008030753632127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.364473, "value1":0.2924502770902306, "value2":2291771922128424424, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.399561, "value1":0.6211538305973102, "value2":6848308885300872697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135252, "value1":0.6565821688014496, "value2":4331814359581194434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.365542, "value1":0.9118204132722055, "value2":2417655238291803790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758482, "value1":0.17112508145844177, "value2":1991118257939966294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.032543, "value1":0.08125554957598005, "value2":6803892048654362563, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.520072, "value1":0.4275632442269591, "value2":2250122493336577938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.900940, "value1":0.14035051591842881, "value2":3790466116523762354, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794030, "value1":0.6000173402567552, "value2":1696544400981072969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.760746, "value1":0.10139199442682838, "value2":2726344565529290448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687331, "value1":0.770768299751441, "value2":8765361349894188719, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035711, "value1":0.48562696854381604, "value2":95585559412343711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739784, "value1":0.5825982451536238, "value2":2923240563522303723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.454300, "value1":0.4006888000990195, "value2":2468975659398879205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.884626, "value1":0.8026957862944495, "value2":3605855909002945057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395577, "value1":0.28352828399222396, "value2":7602348181385211161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.250646, "value1":0.2316228525268274, "value2":29961024893090406, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.421024, "value1":0.7990468794285002, "value2":2264750479744979673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380413, "value1":0.677589393211687, "value2":1235396941989449609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_4":"f","key_1":"j", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680912, "value1":0.25625503996995874, "value2":2396418552817135385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.624528, "value1":0.5836651399092099, "value2":8089438271359233374, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.578517, "value1":0.7858613429590018, "value2":3691936139324370664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_7":"h","key_0":"b","key_2":"b", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.824035, "value1":0.3471915666356168, "value2":7523600740099466883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.069777, "value1":0.6471590850539041, "value2":545376892289254611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390523, "value1":0.8681181997676135, "value2":5603080382596898622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.590994, "value1":0.6390330221088167, "value2":6898933669096943734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222958, "value1":0.7361676398339201, "value2":7798894502817396812, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.313614, "value1":0.9956617360294369, "value2":8114343594051219927, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.471497, "value1":0.2028990027253043, "value2":8672046519486478878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299387, "value1":0.90821231206369, "value2":7139029616253188226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_7":"c","key_9":"g","key_2":"j", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822834, "value1":0.7462012785016429, "value2":2497140784779218477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.958689, "value1":0.6493610280581492, "value2":7375518635495424751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755680, "value1":0.3173769236575059, "value2":1038885732982663879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.906259, "value1":0.7803156579997583, "value2":1671762930600330124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320586, "value1":0.047928946477745184, "value2":2364817790122909266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.237873, "value1":0.5919698389786022, "value2":2456361237893496498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_9":"c","key_2":"j","key_4":"d", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.183324, "value1":0.25157377599570596, "value2":3302857039571325572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427134, "value1":0.9570675751476113, "value2":241686111761765133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.353599, "value1":0.19102611162010216, "value2":7635824178108933655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933576, "value1":0.5497618436271264, "value2":178483151772454986, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.888923, "value1":0.6996148110998511, "value2":7261028738310365771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414681, "value1":0.45596913295291375, "value2":8486467978432271730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354962, "value1":0.6882059355313528, "value2":4219371503140919650, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149786, "value1":0.048507930319514725, "value2":3361043402215780354, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.320396, "value1":0.48978604231367256, "value2":8391350623429868941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.189903, "value1":0.49045511147932325, "value2":4568889448781542889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.531173, "value1":0.015565420626107091, "value2":1532700275127700107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.911622, "value1":0.9315116645649003, "value2":845964128004551720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.447558, "value1":0.6852687200103666, "value2":1997580526601582597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.389703, "value1":0.7679907754096043, "value2":8650394236218808766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221750, "value1":0.8409665471906171, "value2":8068148192803213421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.679200, "value1":0.5889377996015627, "value2":6603283664639278646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.658212, "value1":0.20335938525562192, "value2":6814120282733950138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_8":"c","key_4":"d","key_7":"g", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.917850, "value1":0.07209516605018491, "value2":3465054040297242627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.463048, "value1":0.8121431937413348, "value2":7711657955494481167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280683, "value1":0.9792224347770722, "value2":6637142200642308638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938341, "value1":0.6616785812007566, "value2":4430772053877820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.778057, "value1":0.21301644528770988, "value2":5550954213897210258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.435560, "value1":0.25143911919093503, "value2":8602791844656832235, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980296, "value1":0.4483217784998845, "value2":8061950324304064559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.845189, "value1":0.23223688199088727, "value2":2019205569808444797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.030298, "value1":0.009150371898544056, "value2":2179401507551835981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_9":"e","key_4":"j","key_7":"k", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.439657, "value1":0.5569344336018628, "value2":3896413214679997959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037278, "value1":0.0752646633945245, "value2":2986036073412577705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005641, "value1":0.470809080359862, "value2":7672259019023724818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169543, "value1":0.6908284999086921, "value2":6236799599986001574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.168435, "value1":0.1656190822947285, "value2":2938449310867540367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901700, "value1":0.6243405512256929, "value2":2234990933462726180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.282060, "value1":0.14169460807404005, "value2":820280722029319872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.868372, "value1":0.8809198418653182, "value2":7786878634034667094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700989, "value1":0.028109982788470232, "value2":1201575202891520734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.304734, "value1":0.8385104418360362, "value2":6703304420616815596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.901530, "value1":0.8011320750502264, "value2":7539936763030777880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.862233, "value1":0.47963788532278256, "value2":3960941336665870754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.511307, "value1":0.1832580868434161, "value2":7188980001326450399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700199, "value1":0.6958679254162045, "value2":2878046054907323632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.299563, "value1":0.31054488059495927, "value2":8075467755079023785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383092, "value1":0.290554447303189, "value2":6694033035935161393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.696053, "value1":0.16066281765668294, "value2":3961201674767505870, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.392128, "value1":0.5736001777474582, "value2":168331305812241403, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_9":"h","key_3":"j","key_5":"a", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752731, "value1":0.4825725339245043, "value2":7800665573914476062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293547, "value1":0.12126767569308682, "value2":1612512565132344690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_8":"a","key_0":"b","key_3":"d", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874204, "value1":0.7014202076829791, "value2":476647849984184674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.132714, "value1":0.5122323816164429, "value2":646780496771729886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358264, "value1":0.8238092496263985, "value2":9061776775816645023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.358876, "value1":0.15679289393208987, "value2":8163846432367416312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921374, "value1":0.8339787140810333, "value2":3649804286907707134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395687, "value1":0.875997280853855, "value2":9212367459114589124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.459642, "value1":0.7528040980492385, "value2":4150547104368392230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782674, "value1":0.7121212215364499, "value2":6531310437181949577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.229939, "value1":0.32814366772056375, "value2":6374161097327904245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594347, "value1":0.6163078141302316, "value2":1538685760517154018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.009874, "value1":0.36102842691855586, "value2":1131193318052688252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.606094, "value1":0.7074086026345966, "value2":7018588286889757726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.777708, "value1":0.9795566345359867, "value2":7358687237436509918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_4":"a","key_2":"i","key_3":"k", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.963990, "value1":0.9872552891680104, "value2":6171290753066239119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772158, "value1":0.4830977012506835, "value2":6211566808050254036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.952768, "value1":0.43798623227092814, "value2":3513514828808633618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566018, "value1":0.7713759201698291, "value2":5320413786417127410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.563469, "value1":0.6599121888373632, "value2":1404388254364767252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786109, "value1":0.9075953036601083, "value2":8494075902382221765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812207, "value1":0.5042296550006388, "value2":2362200937900291190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093099, "value1":0.3386496883604208, "value2":7071312823587957868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.381119, "value1":0.031714519630475106, "value2":2849328345417435639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.815827, "value1":0.14737457839891155, "value2":199255383994378266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050041, "value1":0.5503190199883978, "value2":7356958752349024534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.472083, "value1":0.12465887530301682, "value2":6392360821928490144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713056, "value1":0.12067504962982098, "value2":4673070422231359809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.034034, "value1":0.25231429703394476, "value2":5489447259456982836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.502818, "value1":0.4156967719918407, "value2":4612443309724781532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.427071, "value1":0.24921718574451088, "value2":7646229328406586652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643652, "value1":0.07902787804812644, "value2":34188484750195348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.488097, "value1":0.7204444807131233, "value2":4412925367441450672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.180195, "value1":0.3056776312063749, "value2":7687653658691030932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.873382, "value1":0.6827451433784463, "value2":1751876103177277887, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529999, "value1":0.39651784939201806, "value2":1747188854631655971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.971674, "value1":0.8695844176281972, "value2":2878591399245613156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702173, "value1":0.9786793419693177, "value2":4388723377141224465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.844886, "value1":0.690355517305138, "value2":6283981589400857877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.298362, "value1":0.6666814629745373, "value2":6187669811817639042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.266804, "value1":0.7091063369563013, "value2":4664946045775147969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400948, "value1":0.8630878707275965, "value2":7287257182404940937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_3":"f","key_6":"k","key_0":"b", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.135133, "value1":0.6122563921627874, "value2":2139209888725866722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.370672, "value1":0.01650778323939298, "value2":3489547069770540399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.279256, "value1":0.7619200195662909, "value2":5915401631721082002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170496, "value1":0.01005373661086586, "value2":1482576688135431000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.499284, "value1":0.36435815211219275, "value2":1326048436273736829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185099, "value1":0.9391170841095738, "value2":8924546898546218998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.772826, "value1":0.7574432732083424, "value2":4488845007322620734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_6":"h","key_5":"f", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169744, "value1":0.5739689347042248, "value2":741420931729893242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.998044, "value1":0.8965817632976586, "value2":6239062200473429860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594662, "value1":0.6301979977894112, "value2":2054875786295626784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755433, "value1":0.06261106040596412, "value2":5455645386351916850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.682584, "value1":0.03167274357401835, "value2":310883965859994658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.072455, "value1":0.9148162667555992, "value2":382836579698407350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.725171, "value1":0.3647728943926306, "value2":2421892347307790025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798022, "value1":0.22510760744817873, "value2":2442091377551743699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688933, "value1":0.07828818881955478, "value2":6735636563745227652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.649962, "value1":0.9679958182791478, "value2":2607996809419409341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_8":"d","key_9":"h","key_1":"d", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.382199, "value1":0.14064207963814382, "value2":1623159006724541520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.293647, "value1":0.919909760697381, "value2":58019425907871235, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.715683, "value1":0.40584027014114893, "value2":7884356555342436928, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.496280, "value1":0.6458580424511011, "value2":1385234809290785609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.037411, "value1":0.5356364996267606, "value2":4485181030850732806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.092167, "value1":0.08471879259284998, "value2":1047090205307301338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.203791, "value1":0.9626197309889007, "value2":458130535579789756, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_5":"b","key_0":"d","key_2":"i", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.395018, "value1":0.7684680824595476, "value2":787781374740198173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408536, "value1":0.147741569501137, "value2":3967017244497727033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.693578, "value1":0.011896444619318294, "value2":4942230313540034093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.780418, "value1":0.8750655230631833, "value2":8170450018971591774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812004, "value1":0.011539724469496575, "value2":1314191368195668298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.090381, "value1":0.870578773268634, "value2":5701169500016143920, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512721, "value1":0.9238819426049193, "value2":5759090244464429103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.938235, "value1":0.5991043580028484, "value2":7404773139120927140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.751490, "value1":0.1795090853269717, "value2":6360470713237738478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.267071, "value1":0.625707646354563, "value2":6040902311152523365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.002110, "value1":0.08011219998212876, "value2":5850595301611238867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257610, "value1":0.8187562858163207, "value2":8061987864802598079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.446622, "value1":0.7315172185892279, "value2":2428635533374338014, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.730403, "value1":0.7209673234694193, "value2":8416924883569369777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.814732, "value1":0.8815237283497432, "value2":1852270652578106849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.318594, "value1":0.610086630044871, "value2":4524314099691048466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.852591, "value1":0.9383715352044596, "value2":453270928675336465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.989585, "value1":0.8337913033606761, "value2":6440744488245743508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.666424, "value1":0.9182270518817748, "value2":6618322333494361668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.010788, "value1":0.6687390259133706, "value2":1979105606239962267, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_9":"a","key_0":"c","key_6":"j", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752852, "value1":0.4586423067764851, "value2":3289626005749005579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.005307, "value1":0.4780784157090063, "value2":2142574953538675780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752795, "value1":0.7908677214518265, "value2":5326267086837385392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.202377, "value1":0.001998326079086547, "value2":2359915256333417411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216786, "value1":0.03377151611828389, "value2":5536807322346137933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.243069, "value1":0.48298272047333335, "value2":6984955197084383580, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199895, "value1":0.6668507451944505, "value2":1160751712622009388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.605559, "value1":0.014501073348709297, "value2":2976544013604671560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046312, "value1":0.45283346882557124, "value2":1106313893608419094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.841418, "value1":0.9439193659449161, "value2":1585819181629654484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.816329, "value1":0.8314432076191595, "value2":671388396305183300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.672039, "value1":0.19428152515853786, "value2":4678503251263728412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.652936, "value1":0.05999646842980615, "value2":6549432288618958292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.212442, "value1":0.6061066332871502, "value2":6044859751282824016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.200351, "value1":0.9858548441265623, "value2":6095603519503403992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.047144, "value1":0.8823748802173365, "value2":2346527148275584633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272669, "value1":0.49903571079222264, "value2":7606961266627055929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.244016, "value1":0.3505008339279857, "value2":9165454668662797150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_6":"e","key_0":"a","key_4":"f", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595436, "value1":0.1769134432972796, "value2":4183298646463696655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_7":"e","key_6":"f", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.832887, "value1":0.05781334979440461, "value2":2976605700126921242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.417486, "value1":0.03855593830020744, "value2":6499287743834404350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.850592, "value1":0.8819483685824858, "value2":3365942495366951256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.904339, "value1":0.4096492831256871, "value2":7870603688751891889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849740, "value1":0.5985549217817763, "value2":5972692186135936344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080962, "value1":0.6508975263449264, "value2":620195756611450814, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.172604, "value1":0.36758717062599394, "value2":3093546059237948200, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.611964, "value1":0.09212647541898711, "value2":600064894791793802, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.136731, "value1":0.3518781010346145, "value2":4172668838781409561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.221150, "value1":0.7696791785774878, "value2":4882972115027784491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118097, "value1":0.3822037091545315, "value2":5547505099823892976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.209184, "value1":0.3119528694472212, "value2":8126435007719933970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.708227, "value1":0.8414289260599137, "value2":1887259462865725736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127263, "value1":0.417919506626244, "value2":3560935891426387222, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.500871, "value1":0.16110435350355187, "value2":2428171488019731436, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.848583, "value1":0.4961159478803282, "value2":5456304085389531044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.468311, "value1":0.9289954682212651, "value2":4285742227173179225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.160825, "value1":0.59740490878922, "value2":9140238758107732393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_4":"d","key_1":"f","key_2":"j", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423818, "value1":0.2325925821959117, "value2":7321787707684172397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_7":"k","key_5":"i","key_6":"g", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.689012, "value1":0.5753370946686959, "value2":5125074341961139213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941993, "value1":0.009508850201920292, "value2":2477284877979206282, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344874, "value1":0.012483207365877868, "value2":4898429158465049577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625728, "value1":0.9281594076368007, "value2":771428325581269474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701217, "value1":0.8504319614540372, "value2":3030161014659152421, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.288648, "value1":0.7225335235603462, "value2":4184194222447924377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.735765, "value1":0.7338027307218647, "value2":8053319255149398119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853312, "value1":0.31226366748735784, "value2":48160047989763460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289371, "value1":0.5473984505709404, "value2":2530747852470429293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825683, "value1":0.2907108545003152, "value2":3061253498575467483, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.532231, "value1":0.5859058541023265, "value2":4975638816250812874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512440, "value1":0.511793469522306, "value2":5419688896766823090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709116, "value1":0.0431590704274129, "value2":565903028384253279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.709410, "value1":0.45910825737421945, "value2":425558835807211279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.970542, "value1":0.1219905172153899, "value2":24331626097709718, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.043601, "value1":0.49284843085319935, "value2":7672646678928068810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.739151, "value1":0.4246916831482226, "value2":4312260845153215448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489280, "value1":0.8582555742505251, "value2":3110601136234274420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.935791, "value1":0.4664631517475295, "value2":4539773222290649379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.654384, "value1":0.10389342147634097, "value2":5731036292461383708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193783, "value1":0.5459140576966053, "value2":6006070322533917783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.409382, "value1":0.472091611755319, "value2":4277330658526282078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.423822, "value1":0.4751601960954781, "value2":2652715507916534737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.660521, "value1":0.38225985421870495, "value2":9011573276516869930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975891, "value1":0.5238616298909866, "value2":472916590695431743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843227, "value1":0.6679925899402425, "value2":906046342124593465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829312, "value1":0.16298135421846682, "value2":1838361763539535864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.480674, "value1":0.259388407981778, "value2":6440770726430652573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690879, "value1":0.0735354737983211, "value2":9181104589648553044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126782, "value1":0.4486397942700849, "value2":7462094819783188022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948901, "value1":0.9959329498625173, "value2":5479458805805092746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.127088, "value1":0.9575385572151783, "value2":2275563983258531267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_7":"b","key_0":"k","key_1":"e", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.259685, "value1":0.3372857504409974, "value2":6098187454917492671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.336884, "value1":0.24553452230487985, "value2":8241989909296706238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_3":"a","key_9":"e","key_1":"g", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.169076, "value1":0.17392408086361133, "value2":7622066852267723117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977238, "value1":0.5433155948072138, "value2":6898148363520327391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.278455, "value1":0.6005033834895888, "value2":8001662694808432224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.993722, "value1":0.9597867721155696, "value2":1299805439881399559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.530999, "value1":0.2209888826520892, "value2":4066778144109849949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.782378, "value1":0.7523792160016672, "value2":5556510651542494721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.219804, "value1":0.1717195689538366, "value2":7048403972015561437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.946856, "value1":0.5985084131744225, "value2":4885828087902675075, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759489, "value1":0.27172565847552677, "value2":7664668067438271399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.687774, "value1":0.42250850460802286, "value2":7088707511295378411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560049, "value1":0.2616853019800064, "value2":2950184265013984605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097585, "value1":0.2684028422029654, "value2":675122184817942746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205813, "value1":0.44731824636014533, "value2":1428863070750239410, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.543640, "value1":0.544977219392789, "value2":2157919960028175094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.566856, "value1":0.8008181036581771, "value2":7402387212023409312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.765221, "value1":0.8596387394697663, "value2":1232636922510789742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.386997, "value1":0.10171781350903462, "value2":2933191729226405023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.674282, "value1":0.05504930364585503, "value2":724010655267136473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.820827, "value1":0.1512145830665697, "value2":6612187124412654634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.537712, "value1":0.6818284933003707, "value2":833546983771380703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.020134, "value1":0.7293182995525076, "value2":2126763229656176078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.919742, "value1":0.47707314485124014, "value2":8035884626808128837, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_5":"d","key_7":"j","key_1":"k", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.941486, "value1":0.13322259126479358, "value2":2182305210138540441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.089091, "value1":0.6999475934755898, "value2":5942262186593070970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.922220, "value1":0.01137462679977003, "value2":4337636877649564119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.846010, "value1":0.10326336153396976, "value2":8556623018979218867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_9":"b","key_2":"j","key_7":"f", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361361, "value1":0.9603555432714893, "value2":7276258796427191775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.303375, "value1":0.9198009606494528, "value2":3654960055261327174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354372, "value1":0.9154736221447883, "value2":6754725607301294498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836769, "value1":0.17500751807732254, "value2":5225830062891206724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.514481, "value1":0.712612132062798, "value2":5405795909104498480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.857662, "value1":0.4771648896771537, "value2":9197955420056459031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218504, "value1":0.43388279897052595, "value2":4876293374213376800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.854318, "value1":0.29673697161398044, "value2":2557767286012732771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.621120, "value1":0.07462842813102197, "value2":5246869953360521284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.482880, "value1":0.4261787117422624, "value2":7122235599178548659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880280, "value1":0.38443671400954443, "value2":5167341510277945257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.849192, "value1":0.5161558417921461, "value2":4578699148249247151, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_8":"g","key_9":"d","key_1":"i", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826648, "value1":0.873997512950574, "value2":7985090519609883665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_3":"i","key_7":"j","key_2":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.199619, "value1":0.8346974218021976, "value2":8292656933621980920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.959402, "value1":0.6444072126847691, "value2":4320840087483118841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.117065, "value1":0.9272062952128088, "value2":4910297155014939874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.551716, "value1":0.13475804603313432, "value2":8222696695222433380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.643768, "value1":0.8311693472811903, "value2":7965400634034447655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.484767, "value1":0.5064114690168263, "value2":5755735513825830154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377345, "value1":0.1504036301883094, "value2":2988203951245132390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991676, "value1":0.972772556086247, "value2":1470890803841584803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.361353, "value1":0.6540839592057779, "value2":985314732060254627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585584, "value1":0.7709261757498476, "value2":2565995410162492585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.704212, "value1":0.3118041755703027, "value2":6201623885191421475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.518173, "value1":0.9814365163985729, "value2":8388580742116222397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.187366, "value1":0.6732011290892043, "value2":5408814647988876882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.296035, "value1":0.5028422803953023, "value2":7236554080452628944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.768145, "value1":0.8394583660519598, "value2":5325173567069752762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549064, "value1":0.12911593374108088, "value2":8305911548302819241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_9":"h","key_1":"e","key_2":"e", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758468, "value1":0.0009372001034959489, "value2":5525854549400493996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.525822, "value1":0.9778487221852099, "value2":2741333418274183362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.396148, "value1":0.9074357631670523, "value2":8809499083604789974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.414129, "value1":0.8132493023109502, "value2":4942425104780689529, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.398880, "value1":0.39814811004109957, "value2":6997600198610651439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.667851, "value1":0.7812827159518755, "value2":111949400321343617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.065047, "value1":0.5608722991855313, "value2":128957686943303349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.494481, "value1":0.17816439204883128, "value2":8141342025149658982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.515335, "value1":0.9138017166447161, "value2":4317898835838327168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.713454, "value1":0.43899440111826654, "value2":143487884063527116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.185777, "value1":0.5460630190657308, "value2":3180840112779032107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450264, "value1":0.9313181115462055, "value2":8328870888335802008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_5":"k","key_6":"e","key_1":"d", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.567912, "value1":0.6046426149595721, "value2":3206194822374359841, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.896555, "value1":0.6026393505223315, "value2":4823045987798108744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.710017, "value1":0.5880756405721221, "value2":4714155846612246340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.265002, "value1":0.824988737224255, "value2":8146626468221994382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029140, "value1":0.9928569418325714, "value2":1115616498765210900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.994788, "value1":0.6191499887981317, "value2":1491651902318479820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310266, "value1":0.6540537390689438, "value2":8869916203173244502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.786610, "value1":0.5759332726822003, "value2":7232889220252179533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036697, "value1":0.31309276046690226, "value2":4197456548662367361, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.684262, "value1":0.39030765212524027, "value2":8590977643860948235, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.295632, "value1":0.4747190839423866, "value2":2819114746608115923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.613481, "value1":0.02792670630263574, "value2":4669805487355644126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.462933, "value1":0.7475159252861773, "value2":2383279322222812533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179395, "value1":0.19102052131729996, "value2":1856517930791738382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991666, "value1":0.3546374487300389, "value2":6354333338863606534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.568174, "value1":0.4332276746738033, "value2":8645759887904070759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.321899, "value1":0.2992200951220634, "value2":3116962887997170979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.390308, "value1":0.02747272568305014, "value2":926243844801181597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.805520, "value1":0.5096841522295752, "value2":7414378226567615667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230674, "value1":0.03960399937089453, "value2":1710851993634315848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.425150, "value1":0.4167707239792382, "value2":5244023674899540768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.879335, "value1":0.4867774224489378, "value2":553896137255768087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.322266, "value1":0.021415180040758975, "value2":6576886716131511876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.875807, "value1":0.22236040526017317, "value2":8247944924957719281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.596786, "value1":0.5382707900600514, "value2":6504957777564642511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853544, "value1":0.6591705273657046, "value2":6314593273328284879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.792273, "value1":0.8726534877662263, "value2":8304088035724056780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_8":"a","key_3":"i","key_5":"h", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.050549, "value1":0.962790936441039, "value2":4045656520780865827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493970, "value1":0.39340114596435655, "value2":5294670971014916935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.251929, "value1":0.5462996764305353, "value2":5897296033220091874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.115726, "value1":0.8575474248088841, "value2":2146148594913460918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.787443, "value1":0.8185914401069716, "value2":6891346883169646879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.982163, "value1":0.8404209142239473, "value2":2046890513913645569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.529940, "value1":0.15216580562912718, "value2":235763707139823565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.036141, "value1":0.21797894391861414, "value2":2037043718914992589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489425, "value1":0.578085671307559, "value2":1968373430108550733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.377536, "value1":0.5230650685409198, "value2":1948377520185967051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.644803, "value1":0.09118740593062584, "value2":3209698888069603256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.149640, "value1":0.2149828010685946, "value2":324803347557649497, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.788006, "value1":0.35151805877228876, "value2":5736730574020425869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.528326, "value1":0.12892234054630594, "value2":40722644873633005, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.517453, "value1":0.12096018419440699, "value2":8591818996293676301, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.580879, "value1":0.18567989692889683, "value2":2424108579790648867, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.418474, "value1":0.10109548853357979, "value2":6805501278544557691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192558, "value1":0.5389855440079544, "value2":2468356112053849635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_6":"h","key_9":"d","key_5":"e", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.700652, "value1":0.8426222872981881, "value2":5586434719045643994, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836161, "value1":0.6261463649204317, "value2":7998455739612119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.410987, "value1":0.19322483253914569, "value2":7602154106800870019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920472, "value1":0.07895320792744571, "value2":3621399116338923589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_8":"a","key_9":"g","key_1":"c", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058885, "value1":0.038506638786952846, "value2":8443519094492233492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.357953, "value1":0.49873265104684444, "value2":6689514258543734175, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.444144, "value1":0.7257783571311162, "value2":2132381976912359864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.003034, "value1":0.21025100795379004, "value2":4437597220292182437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.794651, "value1":0.0987214108191527, "value2":4856891238525466286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.101167, "value1":0.5530900889416458, "value2":5869924593453820746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813524, "value1":0.7032391307910569, "value2":2962680547459287448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.102641, "value1":0.05280577735508823, "value2":6197989994765662479, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.042807, "value1":0.46787236774210145, "value2":6987091087352317601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023412, "value1":0.4117202092054757, "value2":5734736277249168119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.582529, "value1":0.14733901739015626, "value2":2008347713878225513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.843555, "value1":0.45648522904759514, "value2":8748231721848476899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.609920, "value1":0.5332106841490003, "value2":3789122902542055079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_9":"d","key_0":"h","key_1":"b", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.004873, "value1":0.9250068088247592, "value2":4060601419917043509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.598410, "value1":0.20935422794575423, "value2":3458524447228408749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.327673, "value1":0.6622534351078718, "value2":1975526610989314262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.701834, "value1":0.018453031281240855, "value2":7208203361477507972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.544888, "value1":0.022192838541852087, "value2":420807956823837942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_8":"e","key_2":"i", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.333657, "value1":0.6116791297905804, "value2":6780782056382077493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.224370, "value1":0.58595743970275, "value2":2273745727787058306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_5":"h","key_0":"f","key_1":"a", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.874813, "value1":0.4160482798834957, "value2":5130896603060858656, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_8":"k","key_4":"i", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800557, "value1":0.6485257630686503, "value2":8645194213531066428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_6":"b","key_2":"g","key_3":"e", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.798578, "value1":0.3562881109226897, "value2":1936635714706401113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222829, "value1":0.18800753190841593, "value2":3868467029118261102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.394596, "value1":0.9715849939407708, "value2":2892398559190644533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.088141, "value1":0.19677712178571452, "value2":1290002108232651066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046604, "value1":0.7961651460552367, "value2":6142675575156594148, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.755275, "value1":0.26382736011465824, "value2":4790349544446362608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.300482, "value1":0.6458439650096045, "value2":5076464481756841749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.294901, "value1":0.3937755673758644, "value2":4837753175838197566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717849, "value1":0.8465941273216573, "value2":4538488428913350745, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018027, "value1":0.7099513420503992, "value2":6589927070191288064, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686148, "value1":0.31341443389871104, "value2":4542163293900247497, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554562, "value1":0.6021518819649633, "value2":657935532838913237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.549050, "value1":0.9913051319754902, "value2":9133655827436281017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.564021, "value1":0.3933144754852601, "value2":8691343872756358848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.555563, "value1":0.12216755310514293, "value2":8181517965767093469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991822, "value1":0.9046701031504584, "value2":7169764745600183537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536610, "value1":0.6245723927615995, "value2":5410550698568344254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.640481, "value1":0.7953791600529995, "value2":8896491255145975602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.983650, "value1":0.39162313489970674, "value2":5865250699451794074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.748358, "value1":0.3770029381067136, "value2":7671767291617145847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632308, "value1":0.9672641688276767, "value2":3043416827366033954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.829101, "value1":0.63693380465596, "value2":3688916592463553468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.271187, "value1":0.7822523952945989, "value2":7538651950678502209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.757851, "value1":0.4993335289484615, "value2":8433879559771522559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.022503, "value1":0.5685717542133112, "value2":5232283843486040657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.046586, "value1":0.6991953129630171, "value2":767301512539468704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.170088, "value1":0.9099698273235788, "value2":6375281583140134087, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.819909, "value1":0.9395198166293325, "value2":9208855013717569494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.078675, "value1":0.35105170816316494, "value2":7441300028994205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.977562, "value1":0.27747207540421476, "value2":2610474267165011454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.577027, "value1":0.4084658901530552, "value2":7348713799224820014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705040, "value1":0.13989635719875274, "value2":8770545555481284876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.821460, "value1":0.5135499070770553, "value2":1707896495831766919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369139, "value1":0.07309187404043328, "value2":5262455623289086241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536485, "value1":0.3546193597483191, "value2":5532328840062432345, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.632890, "value1":0.87913605477188, "value2":4206324763882989783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.352893, "value1":0.22467668553540646, "value2":8020391351501404610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.058110, "value1":0.08476564402932071, "value2":2402316898103266567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.312920, "value1":0.14715981410041812, "value2":515215788552749041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335941, "value1":0.5607746607254104, "value2":4029056354863603408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493922, "value1":0.5228229815154212, "value2":5917395680452547526, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.721451, "value1":0.8180794492453682, "value2":1670905717947422538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.347574, "value1":0.7935601761833552, "value2":6084751528698562987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.722657, "value1":0.4075231573944958, "value2":7748408088453297294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.451863, "value1":0.3657057400324225, "value2":7302526748126423806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.495468, "value1":0.10049524830706323, "value2":7515132691589300738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.193570, "value1":0.5847911145927462, "value2":4656707981929867290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.866643, "value1":0.9003805975682342, "value2":3827225598428319501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.535740, "value1":0.5365482035917958, "value2":3695801733453249426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.197799, "value1":0.7228560209917014, "value2":5697296448849523746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.536756, "value1":0.2916665533141402, "value2":8616484406626615233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.231668, "value1":0.5491980830875538, "value2":5401247188111922849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836905, "value1":0.6668911346325062, "value2":5956397501314084543, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.060078, "value1":0.41433534941101835, "value2":259739043603601632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.933921, "value1":0.15068665469322526, "value2":4615815953640952732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.147080, "value1":0.17748824888193676, "value2":838395034976203435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.662077, "value1":0.7882764714348716, "value2":4457961488907810285, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.023154, "value1":0.24361462379316554, "value2":7839025824541823553, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.826915, "value1":0.569504859021508, "value2":6251982817444532308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.585595, "value1":0.8340020457470065, "value2":2938024413765410558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146190, "value1":0.9581727102338353, "value2":4413141265483838870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.245440, "value1":0.6417183759653234, "value2":5364021531028647444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.493343, "value1":0.4938700840392069, "value2":3091997225543832896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.807379, "value1":0.4690524091058998, "value2":3061093742738985330, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.208752, "value1":0.8639618600039569, "value2":3187072739982271504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.011216, "value1":0.18367276511432976, "value2":2360627840986057827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_8":"d","key_6":"i", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.440539, "value1":0.6977700817568234, "value2":3581694346818534680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.489990, "value1":0.309872070478555, "value2":7457465639523954616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.113130, "value1":0.6897957784827682, "value2":90019976280058795, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.604031, "value1":0.797243453747431, "value2":1234756804789352079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.881184, "value1":0.28155440991949554, "value2":4070028978929644169, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.367426, "value1":0.016167903292426008, "value2":4863636678376461059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.769132, "value1":0.967909876972054, "value2":9131005500727231455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.905490, "value1":0.6194024275206055, "value2":210353074026913884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.915138, "value1":0.6781979759382127, "value2":6372020977221093753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.093709, "value1":0.6187727010105798, "value2":312866041452100946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758312, "value1":0.8273322264897901, "value2":7512565884920091651, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.284383, "value1":0.39739645167776044, "value2":8773985992873355471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.216803, "value1":0.8653155039130678, "value2":8937229562905505386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.770263, "value1":0.9293790766191896, "value2":4560937013728510181, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096191, "value1":0.22323546625261562, "value2":8010469458396680860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.984434, "value1":0.9454942818827864, "value2":660593493740706018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.222194, "value1":0.02420800916517378, "value2":5602553742039024765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752900, "value1":0.3672193480709247, "value2":6716506478528347816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.967703, "value1":0.2944363320939628, "value2":7783913193752729609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.600560, "value1":0.1440348243160143, "value2":9110640288768831559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400353, "value1":0.2233756240085349, "value2":2151981935371948568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.269597, "value1":0.9641297731318997, "value2":8169933506866910786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.486730, "value1":0.1006988853266946, "value2":3116477845424070312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953298, "value1":0.4756644907363518, "value2":2464865227545117596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.705992, "value1":0.8173093209310923, "value2":8219739418749111053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.851844, "value1":0.28729444422491424, "value2":191979366247033806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.616037, "value1":0.08828307120467993, "value2":1006916952645293364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.007265, "value1":0.43891779173017603, "value2":4642029911324023153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408095, "value1":0.9633558663387003, "value2":2764041604557827140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.825923, "value1":0.12824733189500837, "value2":7952369571604255477, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.853643, "value1":0.8488628783599163, "value2":3110877858489702411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686891, "value1":0.5538740126823626, "value2":7618257543247002890, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.408459, "value1":0.2427343835864084, "value2":1637171870696137413, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.635245, "value1":0.42768890444095087, "value2":4013428109133869754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.793205, "value1":0.23512439386454564, "value2":5799487535459689842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.920141, "value1":0.9415332414772868, "value2":1520706400052294853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.930133, "value1":0.47818008921632604, "value2":4567447002026438924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.599550, "value1":0.9293362893760267, "value2":1195481394235809552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.903387, "value1":0.11210859774886604, "value2":3335652489364502188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_4":"h","key_0":"f", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.698033, "value1":0.3148016773642106, "value2":4939787017916634516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.813706, "value1":0.3325590668831616, "value2":3252287249574729793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_2":"d","key_6":"b","key_1":"i", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.048837, "value1":0.053013772957975554, "value2":4819883401481045104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.310950, "value1":0.41839244392518726, "value2":8733225969530074129, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.062225, "value1":0.858129811136947, "value2":7961256500615532109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428880, "value1":0.2502872252911392, "value2":7064927828432353118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_6":"c","key_0":"c","key_3":"b", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.560002, "value1":0.11724155985130476, "value2":3115965127965032390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657940, "value1":0.5473444163031204, "value2":8727302484888278411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_9":"d","key_0":"k","key_5":"g", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.554826, "value1":0.376684927630788, "value2":2585922874708871992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079791, "value1":0.9982500603963644, "value2":7391513557121794219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_8":"e","key_1":"h","key_4":"k", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.174380, "value1":0.13350980294424206, "value2":9115933928609206333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.690448, "value1":0.8022085172421284, "value2":5503579136981394270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.178959, "value1":0.5153762777175099, "value2":4750624720468516504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880368, "value1":0.3826318223193023, "value2":3994162284701586149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.594211, "value1":0.0032667252772956594, "value2":211951125332337865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.080419, "value1":0.3371270895888467, "value2":970359359029436118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.354035, "value1":0.6830560620508377, "value2":351440888758257733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.686006, "value1":0.15832224781392848, "value2":7973904723234304705, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.688086, "value1":0.14780349920952468, "value2":2027463870413046281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.407959, "value1":0.04437078472668187, "value2":1566333717176214387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_6":"d","key_3":"i","key_5":"a", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.943264, "value1":0.6495072511389037, "value2":5853361075489349645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400773, "value1":0.2676114612948212, "value2":6666522398737719346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.228255, "value1":0.29124903807175784, "value2":128611338987653451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.702835, "value1":0.39306652609769904, "value2":4553551121428133233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346580, "value1":0.9157209666515556, "value2":4177607716262676740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.000990, "value1":0.3155726084083431, "value2":7473596942971857359, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_8":"d","key_1":"b","key_4":"b", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380807, "value1":0.5730835803736063, "value2":6788452783424900655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.018359, "value1":0.11046900836774676, "value2":61241953976306769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.097897, "value1":0.6529098432421965, "value2":9123131154054179256, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.118936, "value1":0.312336727346538, "value2":7294156215339952794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.256520, "value1":0.817765585396917, "value2":4276314621602341409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.239028, "value1":0.27988068073239764, "value2":8048493989378214976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.359463, "value1":0.33204054027814056, "value2":4286372625461580954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.625120, "value1":0.8053374337323631, "value2":1700890355040268131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.519453, "value1":0.2764225180730322, "value2":3111696426903897415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.019685, "value1":0.5511793734194003, "value2":3991968620227079942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.040695, "value1":0.6967938051735891, "value2":8919927137208366483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.750443, "value1":0.9319592750217813, "value2":7402732360820495311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.497932, "value1":0.6545667018250457, "value2":7616613351817353333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.773399, "value1":0.5580363664821956, "value2":314222158348617253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.869323, "value1":0.7418088662526952, "value2":8657709877874943583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.812016, "value1":0.4480192862830272, "value2":8470890763908250393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079326, "value1":0.3872756599905638, "value2":4349661970334556580, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.186686, "value1":0.8815811840060216, "value2":938888750792010987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.235117, "value1":0.8673030373216688, "value2":4951632884410168399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.335964, "value1":0.6073261439232968, "value2":8008517076279782152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063573, "value1":0.12394184604832796, "value2":1362755555847887698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.685355, "value1":0.2361653248224142, "value2":2210259628223947558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.029806, "value1":0.3517864573309168, "value2":6754043647766872898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.593026, "value1":0.38937941917051955, "value2":1945974862147158833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.595711, "value1":0.9836966964700219, "value2":4159340747167974849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.143026, "value1":0.5945202518873367, "value2":3646754986945085167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733493, "value1":0.18903944802082667, "value2":8837255893708962102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.120646, "value1":0.4261710458329441, "value2":6946544380857137617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.163665, "value1":0.24245216879890383, "value2":555667735507028882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.934782, "value1":0.31976487170355755, "value2":6301272802564976177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.975795, "value1":0.2007851651150272, "value2":866319078855783895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630757, "value1":0.2788228743238552, "value2":2540783851122363232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.661317, "value1":0.5218066875354199, "value2":8869741999431102494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.980967, "value1":0.7624092409640837, "value2":628203903335846730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.380622, "value1":0.4837921884626896, "value2":3628530218991586661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.248953, "value1":0.02097807162306582, "value2":4598542592315882026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.384743, "value1":0.7608954854738191, "value2":1886663813765371215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.950140, "value1":0.2550158514673341, "value2":4065397223823067237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.516999, "value1":0.6292445006161491, "value2":4125156478992786356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.733841, "value1":0.37206344867268365, "value2":3082112298619357468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086119, "value1":0.5738504787109479, "value2":5573707094695082413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.800475, "value1":0.7767745931253813, "value2":6702226199646136395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.175876, "value1":0.8346328055447434, "value2":1066680083349750545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.752359, "value1":0.7257723037237084, "value2":7568317247183343047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.165816, "value1":0.19937439179339156, "value2":2068495750767911677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.597980, "value1":0.6555890535602861, "value2":2883065227864053972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.138569, "value1":0.22532513024599726, "value2":8297154458719010882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.668715, "value1":0.11645984529741488, "value2":5555518408390232617, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.063265, "value1":0.001988016117254865, "value2":8483308279719955566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086644, "value1":0.4644931461203292, "value2":6366519140385299368, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.140571, "value1":0.5416650487590775, "value2":1665979086436164947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.045805, "value1":0.9888897436719353, "value2":7697021265197143502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.272912, "value1":0.2088625700639475, "value2":8643238667192348189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_9":"k","key_3":"e","key_7":"b", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289278, "value1":0.005139072821858179, "value2":1003021960883868455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.242063, "value1":0.32725526486053264, "value2":2911166424297597473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.393077, "value1":0.6314586962746932, "value2":2631691711252794732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.274319, "value1":0.9496413905130506, "value2":3462918152034782288, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.659954, "value1":0.9955163986622194, "value2":2213038727032863641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.657842, "value1":0.11877952023160425, "value2":2754577958762836530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.146197, "value1":0.5152391159350134, "value2":4396760573611697803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.759786, "value1":0.051119901574673096, "value2":965796598734594927, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.891935, "value1":0.6695409149263635, "value2":7204593893035944941, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.096828, "value1":0.7785299263381299, "value2":843269019498255824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.822107, "value1":0.06887356745350987, "value2":3890683236062212546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.818726, "value1":0.3529788280342954, "value2":2023962428787621327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.356164, "value1":0.8626419946204176, "value2":3137373073213134707, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.855692, "value1":0.3727202998903748, "value2":1868074746512330224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.899737, "value1":0.49932818019000214, "value2":5247723900283585273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.205758, "value1":0.752048623202744, "value2":405978535011434243, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.492517, "value1":0.7587591048194695, "value2":570855823822973980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025167, "value1":0.9352906688766658, "value2":409025014085718364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.524374, "value1":0.456131184246001, "value2":6492711389883342442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.173025, "value1":0.7216031002138218, "value2":1900883846963453532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.991241, "value1":0.9435537729789849, "value2":6744125432328938384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.126845, "value1":0.9483471232225058, "value2":2429204263257823531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.079973, "value1":0.14998274238182085, "value2":1609358502486987778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.257883, "value1":0.09750371692825593, "value2":5050315729103715843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.074081, "value1":0.21018141055409362, "value2":9169596995481234183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.218738, "value1":0.9047134659722221, "value2":9140209877004134552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.745794, "value1":0.39056909591189853, "value2":4170147818303980952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179295, "value1":0.9580414905588824, "value2":4615545745316982257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.442990, "value1":0.018901929544229163, "value2":2360761388936503714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.422188, "value1":0.5272513199907374, "value2":3200950480296318673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.192831, "value1":0.33577000839194515, "value2":8465993285574620045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.227691, "value1":0.814891459750041, "value2":5020817184863163049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_7":"f","key_1":"d","key_2":"h", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.171624, "value1":0.32281185598307915, "value2":8148752929610663940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_2":"c","key_7":"h","key_1":"b", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.490347, "value1":0.4777481785589013, "value2":2903978983753469512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217872, "value1":0.6839140142887666, "value2":3130769003103218800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.992440, "value1":0.1702388495618268, "value2":6704155277469548917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.834388, "value1":0.855009067710764, "value2":8656769438285137265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.230809, "value1":0.6026285375258597, "value2":1622052925759175826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.344218, "value1":0.6418025631389815, "value2":8245500230948114788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086815, "value1":0.450124296857116, "value2":4866053118697224871, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.802304, "value1":0.9289196028774398, "value2":6637963067322995548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_6":"j","key_7":"d","key_1":"e", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.840991, "value1":0.6695124623580792, "value2":5437276365728147032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.833585, "value1":0.22730504686256575, "value2":1520963367111518952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.744045, "value1":0.8193501328748979, "value2":279541774649644194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_9":"g","key_0":"b","key_2":"h", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.736126, "value1":0.6177732267307673, "value2":738077807298530261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.131891, "value1":0.46856485152824584, "value2":1245296884531108950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.419961, "value1":0.9237029205265445, "value2":1223547652782219884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.512633, "value1":0.3170684617445516, "value2":3777467556351824325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.630355, "value1":0.021704073637768814, "value2":8674907845809369167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.309670, "value1":0.16245539540033113, "value2":5206956291412580457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.348146, "value1":0.7658591727792013, "value2":8113096525411729139, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.346099, "value1":0.09833103805087655, "value2":3329067421410580481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086337, "value1":0.8850305283927763, "value2":8626010840258936486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.479373, "value1":0.20404187483062786, "value2":2412557827504873850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.921982, "value1":0.7871473016496364, "value2":1874563226125050237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.415457, "value1":0.9305937676701617, "value2":4246181816757480088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.035701, "value1":0.11812212256088726, "value2":7168381109733048473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.225156, "value1":0.9869870812720318, "value2":137813480312065053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.836380, "value1":0.860500054517146, "value2":6804060104865284962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.289062, "value1":0.385288303480441, "value2":3917001550422656910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_7":"g","key_1":"k","key_2":"a", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885780, "value1":0.1315685249986314, "value2":2596575576892432777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.948363, "value1":0.36071231455630776, "value2":628986020889092104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.369788, "value1":0.40617296244272144, "value2":1817522335798570734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.086188, "value1":0.6439285363835136, "value2":4867721675008841698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.995659, "value1":0.2954211415738649, "value2":316663728175376955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.129393, "value1":0.050215829131005675, "value2":6231361900716566583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.400907, "value1":0.39112211803964236, "value2":5808809619414218326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_7":"f","key_0":"d","key_4":"d", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.717436, "value1":0.943850779238098, "value2":1447465700474277383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.217700, "value1":0.1707128166287673, "value2":4689212155367810810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_9":"e","key_0":"b","key_7":"c", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.908215, "value1":0.13836337191309858, "value2":3286491733476954470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.383068, "value1":0.3171639131993519, "value2":5768120780013343252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.758654, "value1":0.8248294031178783, "value2":1259633164417337834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.885742, "value1":0.7054910487633926, "value2":965825539457944695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.450755, "value1":0.5160791051802902, "value2":980291621965651603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.280407, "value1":0.20023478153485316, "value2":216774102527727538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.953286, "value1":0.17708932991625895, "value2":7861520056013490110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.428154, "value1":0.3840775072053381, "value2":5128389599856507984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.650144, "value1":0.5475742595996844, "value2":37238921657180760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.880799, "value1":0.27347132520371564, "value2":7086278310918665364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.179258, "value1":0.09483479166767275, "value2":3722096865477015794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.025175, "value1":0.24738190446733826, "value2":2326403401160366476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.680444, "value1":0.6441894024482266, "value2":2704266700740984761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.830593, "value1":0.6314118537062262, "value2":8997005770669681872, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_9":"h","key_3":"c","key_5":"b", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177647, "value1":0.3203303212434556, "value2":8965923512665629393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:00+08:00", "value":0.923489, "value1":0.29642566667944875, "value2":1203313530737226131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.811446, "value1":0.5561455146353509, "value2":570880137152142782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544802, "value1":0.9164364466036977, "value2":3723655635493793654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.542566, "value1":0.7689437007202478, "value2":3938961297360550598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819459, "value1":0.4225560622422357, "value2":2765920289473813577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971131, "value1":0.7588578513174423, "value2":554701584189491661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638701, "value1":0.9383542993382983, "value2":2675775339679034911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658985, "value1":0.8385957143790167, "value2":9118144992996965702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449591, "value1":0.6354046681746691, "value2":4796404340287643460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.851526, "value1":0.7827632730352198, "value2":4383742830241177102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929626, "value1":0.7374713580232243, "value2":6076523832966313810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.289625, "value1":0.6865021261714863, "value2":2978049869639889816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394738, "value1":0.6344060546968167, "value2":293482016535092319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.487799, "value1":0.01884442354627017, "value2":6314666407653758310, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.189751, "value1":0.5131352606856101, "value2":5370657937415537626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.532641, "value1":0.7500243297537671, "value2":5814407549612083184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413938, "value1":0.35879356972502013, "value2":7060341260629410136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464627, "value1":0.5028910273480031, "value2":4767839332233680092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.840321, "value1":0.13633703551235915, "value2":2307803453888302997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177282, "value1":0.5212414108183161, "value2":7948818597964566740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394656, "value1":0.22032364884959718, "value2":8010953658147063588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_4":"b","key_2":"j", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.193561, "value1":0.6737972615452449, "value2":7205681986933877897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919748, "value1":0.5508022412545106, "value2":1935155841798114792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.549841, "value1":0.6412542942797744, "value2":5725068566993480801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755602, "value1":0.7570856701300407, "value2":2582449631680230244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569314, "value1":0.9682571063393881, "value2":3213998604408735177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220995, "value1":0.4682404519517274, "value2":1296712240822053674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030188, "value1":0.13463362539315563, "value2":6167313691161388680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955102, "value1":0.12893740940580098, "value2":937550317375879979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912054, "value1":0.23584661351992364, "value2":8270059203922837294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248552, "value1":0.873739706342713, "value2":5483897643292875706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.668856, "value1":0.06480463060086489, "value2":3429839268398826587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628990, "value1":0.06856495811246685, "value2":2353082280387987376, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.364427, "value1":0.8257731797802594, "value2":8162649044337833619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488610, "value1":0.47476344691483213, "value2":7563742944450884141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.659600, "value1":0.22983785061318257, "value2":5566259052586123346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715473, "value1":0.5778415124343756, "value2":379000163266106555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265567, "value1":0.856852436111815, "value2":96718295321822248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647902, "value1":0.9996700403145788, "value2":6837057660421261848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.866334, "value1":0.8219801167193398, "value2":8280931000915145951, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144409, "value1":0.604589772741639, "value2":4205152374038619548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192185, "value1":0.27115827101755235, "value2":5770338217899591097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300761, "value1":0.009332804989256857, "value2":4018403627826051427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.312951, "value1":0.45431720098201106, "value2":2937253947933810175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_4":"d","key_0":"j","key_1":"c", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647625, "value1":0.3268497558683786, "value2":5914220923444612314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878656, "value1":0.9409966965511731, "value2":1289273008938800365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057134, "value1":0.3301792558402197, "value2":3318812578583185116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.439955, "value1":0.2652202727470005, "value2":6508684150448276421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827995, "value1":0.5808204202626653, "value2":2858312188918612316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863057, "value1":0.013094412054074, "value2":8217074167029879656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936866, "value1":0.17076998229838808, "value2":1405785613574281058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.188817, "value1":0.4102266284907626, "value2":9200292054736167191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919291, "value1":0.849823159364745, "value2":8783211669019586620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.684825, "value1":0.5612249810486886, "value2":2459193643509253683, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.132931, "value1":0.06165133682856612, "value2":3724912889225979746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939721, "value1":0.7022523416408233, "value2":3787551523413319921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423514, "value1":0.16397574471331047, "value2":2217544936229011402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556341, "value1":0.3363631636072514, "value2":8296724122152500212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.797045, "value1":0.9775102685569385, "value2":8717038963718857217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.037101, "value1":0.5188119627862039, "value2":8253271773680540946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.472369, "value1":0.723269226446299, "value2":1830427992517853225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350765, "value1":0.45427831592086265, "value2":2459543486390463113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.172879, "value1":0.9405309891223543, "value2":2118657389939971315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902535, "value1":0.42952097429562314, "value2":3142662533255542276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.863286, "value1":0.06876792355358291, "value2":1443459731022840202, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.137921, "value1":0.6964987593861792, "value2":7256675183245953924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.477178, "value1":0.3581558179516239, "value2":6351662172251250145, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286204, "value1":0.148299090562339, "value2":1443868893374130520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286734, "value1":0.45842257280884463, "value2":1988366284391476937, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787180, "value1":0.5028191040745897, "value2":4148707399092925318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.586935, "value1":0.0518139414168187, "value2":8586911735374624592, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_9":"k","key_0":"g","key_3":"g", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021854, "value1":0.8978228143933898, "value2":1413164249310483673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_6":"b","key_4":"k", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318004, "value1":0.5329901050180244, "value2":700502281930980696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381834, "value1":0.37533803419078043, "value2":1751644850942508358, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781575, "value1":0.7741607009730129, "value2":7455242385880444942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.901091, "value1":0.9760509636456571, "value2":1834882752759920223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805005, "value1":0.18640631542045302, "value2":3539140922926641426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734757, "value1":0.14536274157280238, "value2":6893258320350162281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.340380, "value1":0.2042782161421506, "value2":2006565586318459391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.271132, "value1":0.008492488378786304, "value2":6129670920743570493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947658, "value1":0.35165969319536833, "value2":4951697844001184553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.735049, "value1":0.4770090728759761, "value2":2452955784370225782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337551, "value1":0.7050663230338188, "value2":4249578695875539573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.633134, "value1":0.6874967232601639, "value2":8449525705234658507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412613, "value1":0.8000775398308927, "value2":5399334563984433319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.520666, "value1":0.3288020539774752, "value2":1684836256027900746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063862, "value1":0.4985615503389622, "value2":5250749533704010894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.776154, "value1":0.039036691522280384, "value2":8622652029902283303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.115396, "value1":0.2649092470624877, "value2":678667215803198769, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.728254, "value1":0.3531001293726284, "value2":2004607802946409884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242732, "value1":0.9820165334533849, "value2":7506111952193927451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969092, "value1":0.45874224514029954, "value2":6402112089715656350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.528740, "value1":0.8320354986171156, "value2":8412309334445466976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.907053, "value1":0.18136550029947487, "value2":6212227004856794281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630067, "value1":0.8986694902601496, "value2":9213857752862359137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133736, "value1":0.8699930415468984, "value2":4081508478635244211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.198697, "value1":0.017191657622434766, "value2":1486415534842367537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.690493, "value1":0.3223491896226482, "value2":7372352323440542564, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.404617, "value1":0.6240663182478166, "value2":4691037631120171189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120685, "value1":0.9919062682339983, "value2":8460618157376864427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.688005, "value1":0.3953177302348419, "value2":5961626802588462897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093767, "value1":0.23061987147968935, "value2":2892986034831352810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.543984, "value1":0.16501271900248737, "value2":317259514400114949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842969, "value1":0.25446808569640833, "value2":6002379582827585880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269131, "value1":0.8842302265963506, "value2":2140205221312415384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972033, "value1":0.9857791825041854, "value2":8003941482296671818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508672, "value1":0.5492264125933453, "value2":4184449934578666153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.812228, "value1":0.3691868332062656, "value2":940690450246364953, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033340, "value1":0.4347207480857335, "value2":7925423134984867831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921934, "value1":0.7008140011057534, "value2":3950204529429726081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_8":"e","key_3":"i", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.159043, "value1":0.4015464920656329, "value2":6141367601228839763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_8":"h","key_4":"e","key_5":"i", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207886, "value1":0.923323917554864, "value2":2538142530573084913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560000, "value1":0.988479108310322, "value2":3590995919192964810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324806, "value1":0.14623600559543842, "value2":7983994507018983485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.278078, "value1":0.47875025775481017, "value2":9135606324353342316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_9":"e","key_5":"i", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978376, "value1":0.646562029258319, "value2":6969654958747462453, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.038811, "value1":0.27278148584909157, "value2":3766949452289754200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174153, "value1":0.456204859878279, "value2":5412969439184865040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152394, "value1":0.6304591834176713, "value2":7508490770671251540, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079217, "value1":0.7093512830025359, "value2":8041107853739821078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736508, "value1":0.9456361684627089, "value2":2695850503301130596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.689579, "value1":0.8717416155878271, "value2":914033909063513507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_7":"d","key_2":"d","key_3":"e", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716602, "value1":0.6181082501584888, "value2":7525886338688800823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485128, "value1":0.805498923411989, "value2":3515190992320449261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_2":"c","key_0":"k","key_1":"a", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761534, "value1":0.3987574394917293, "value2":565378419404884224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234797, "value1":0.32687033897000584, "value2":4678721697918141366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_9":"k","key_1":"g","key_8":"i", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099258, "value1":0.5165730450377122, "value2":7897877163898991964, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.855254, "value1":0.6113796894495002, "value2":5439835292581168290, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098218, "value1":0.6331475060559588, "value2":7978662530169854255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.392907, "value1":0.7806228764363937, "value2":5349308434124377933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174842, "value1":0.7994951238912167, "value2":213268455628772714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.654871, "value1":0.23130216238112586, "value2":6353589500271587044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255121, "value1":0.029192800621726373, "value2":2049991280590108935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.454836, "value1":0.6345686250008671, "value2":8305250329173845884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.196946, "value1":0.9679432223428265, "value2":4005113690213723694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.514102, "value1":0.9460847081604973, "value2":2275533825263312731, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.267365, "value1":0.5628401308203138, "value2":2718632724301463408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.702558, "value1":0.3615743138551236, "value2":5265258619209069561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.157371, "value1":0.9914192491349261, "value2":7107333633697743515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367362, "value1":0.30046228876641146, "value2":8731213027442541680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502940, "value1":0.33029188840622603, "value2":7282340644676767522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773440, "value1":0.7398342089000881, "value2":6064862964524030137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370133, "value1":0.5464083248908671, "value2":8595565634194089280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.819624, "value1":0.22563478104006193, "value2":6291072404703346422, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.822444, "value1":0.6431602385348965, "value2":3571751115605471571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154889, "value1":0.661415650101091, "value2":2897716721956096259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.706177, "value1":0.9541112843149568, "value2":4382405639585131280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.892582, "value1":0.273637580698836, "value2":1569537455975017743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.912404, "value1":0.02958676932622315, "value2":864163789414463895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164999, "value1":0.8749507184007825, "value2":9170076213645968437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.609248, "value1":0.6230590575325244, "value2":1182671592625390639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402824, "value1":0.8555668782169765, "value2":3430464294959504849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.905017, "value1":0.6152490194289306, "value2":3258848281050130827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_8":"j","key_9":"d","key_5":"i", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.669648, "value1":0.6859907392101928, "value2":2219730699877135652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435602, "value1":0.12441938240097333, "value2":3663118450934184743, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736951, "value1":0.9806346124707411, "value2":6495060662787801507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502755, "value1":0.6962847488848957, "value2":7479724851955507611, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.618421, "value1":0.31789340504585667, "value2":8738918799232514086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827476, "value1":0.7869780281846211, "value2":5474883946012414730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490095, "value1":0.6962736110006601, "value2":6409713241665137545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985050, "value1":0.2615773286934893, "value2":6009116516814474343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502430, "value1":0.19305751383330208, "value2":1654152222563323901, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_3":"e","key_8":"b","key_2":"g", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349955, "value1":0.7772270603571774, "value2":73068209272270733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781804, "value1":0.6130002187106758, "value2":5094562490162409190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.158107, "value1":0.12199479281505482, "value2":887695971080523742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.575029, "value1":0.9090539881797902, "value2":3726306097696429975, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.197382, "value1":0.010351058305108105, "value2":3390177787518625882, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959617, "value1":0.10545857069830916, "value2":8523151888175341811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789446, "value1":0.7606780493785521, "value2":2782849609854347711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384471, "value1":0.3297433712153455, "value2":8732496713018613730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.440642, "value1":0.05991406624275138, "value2":8221843418889420393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.485145, "value1":0.4324948321911064, "value2":1223798989569651011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930239, "value1":0.797530519094042, "value2":3933281842105360464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_7":"j","key_0":"h","key_4":"c", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.921694, "value1":0.3827550255479697, "value2":1163470615431943983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780691, "value1":0.8316468698353685, "value2":3969005124884722997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563279, "value1":0.5313651121371132, "value2":1275907147042029939, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099445, "value1":0.8066245132140291, "value2":1404833370385176869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.897823, "value1":0.395354453907472, "value2":4125787827950753058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562742, "value1":0.0008610845550682808, "value2":2683131454862577523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981050, "value1":0.6811899501971241, "value2":2963479936998599518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.090076, "value1":0.7872489837661779, "value2":5920042185451264661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904987, "value1":0.582385162743904, "value2":3052394546288639081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.807620, "value1":0.8276864738323398, "value2":1770112128995343558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.313153, "value1":0.7000557306246208, "value2":8804754539464742313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648178, "value1":0.6035738426070452, "value2":274869451073352526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382503, "value1":0.7570596198853545, "value2":8494223821423922555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981825, "value1":0.4954131435773338, "value2":5546139769442373925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.896817, "value1":0.3057668662697838, "value2":2146747890118623175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_5":"k","key_1":"b", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857258, "value1":0.7535991261099572, "value2":7241562330594412223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.954553, "value1":0.6272550393596944, "value2":7059824369010485707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611570, "value1":0.896770550906892, "value2":4157172203390286832, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.200060, "value1":0.6395440920752251, "value2":2313959820318356273, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_9":"k","key_2":"g","key_3":"j", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.305817, "value1":0.7944717065357103, "value2":9079049510197634026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771177, "value1":0.5686457722223018, "value2":5926649416008895009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.194460, "value1":0.2030568091950128, "value2":5211606615226569619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_2":"c","key_6":"b","key_1":"b", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541777, "value1":0.38229879472765593, "value2":8330651996424197311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566620, "value1":0.24307591832233774, "value2":2839998819511281148, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376323, "value1":0.5739727137071835, "value2":5399793491142182010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480111, "value1":0.7895749118902282, "value2":7450043826116159674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453645, "value1":0.4875344798695633, "value2":6926040231429456296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.734559, "value1":0.3865838466523982, "value2":9001676782517223243, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.171659, "value1":0.33364493236506704, "value2":5739131266488499620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.966196, "value1":0.1807801172943221, "value2":7703455497589255747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087000, "value1":0.14745151659831435, "value2":3780273793092000978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537557, "value1":0.6679609579098379, "value2":7524235382942600940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624433, "value1":0.43524838478159544, "value2":8707151351953730491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.764878, "value1":0.9857854697655962, "value2":2628899540411422018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.087516, "value1":0.9235134986127471, "value2":1244667248757759893, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.092707, "value1":0.8123903328089516, "value2":1510741329638496293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212617, "value1":0.7803051371802152, "value2":1782941121051923216, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110139, "value1":0.9520250869731262, "value2":8515109734542519287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217411, "value1":0.569745410096317, "value2":7158729472403289306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578681, "value1":0.4421036889589163, "value2":5172660091174561754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.917747, "value1":0.9646587109518411, "value2":4932202474624997807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490742, "value1":0.18375289991112742, "value2":5134730937919546380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698691, "value1":0.8462874020691947, "value2":6789099466073417633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987429, "value1":0.47456618667904216, "value2":7333730409059985654, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.552031, "value1":0.8731675557704086, "value2":2383267067188016407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.024510, "value1":0.7398931410894473, "value2":4688375563960023614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.885940, "value1":0.5267822661948509, "value2":3106942003445818164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.128634, "value1":0.17304999217293912, "value2":4202058405826758012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919245, "value1":0.5246293149604686, "value2":7962001396490479168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.784582, "value1":0.9051007718281217, "value2":6345441367106704431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.631365, "value1":0.18200897376922306, "value2":2743187358723124435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632586, "value1":0.49203338164778787, "value2":310459975834593918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.986910, "value1":0.21502125960121787, "value2":9213029499233048926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.903530, "value1":0.20307359325640195, "value2":3425839773997981556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495175, "value1":0.42906594953086846, "value2":1910613263092063181, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560265, "value1":0.27498766656590357, "value2":8893579102721900369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.298348, "value1":0.05982261910728163, "value2":8036644582013512610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.624831, "value1":0.8394666399726175, "value2":1345154507982673038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.123368, "value1":0.13834539586421774, "value2":6342191610731262762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.086336, "value1":0.17439021756766837, "value2":6128400320706208802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421248, "value1":0.39843282468447905, "value2":8680280397492385017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139454, "value1":0.456434331043375, "value2":4836743646420773621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351219, "value1":0.777135306661456, "value2":3903934605632263710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.414090, "value1":0.8143501039400677, "value2":4413351639615300728, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_7":"f","key_0":"d","key_5":"h", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231721, "value1":0.3138412091160347, "value2":9168517044165068794, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_1":"h","key_0":"d", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.230702, "value1":0.8804508000643604, "value2":512236854100511456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.708957, "value1":0.47311614120900003, "value2":7136526398161878544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.475928, "value1":0.32108039730398563, "value2":7292530251105851041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.643383, "value1":0.9689860631255371, "value2":1734391035548707204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.792203, "value1":0.8844015947765959, "value2":6318884852252048224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.323993, "value1":0.18316994233066625, "value2":3785806626345479252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.055153, "value1":0.9975090536646855, "value2":9101914663697333394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762055, "value1":0.14538112505308043, "value2":6589674161435979608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457017, "value1":0.9313588309515796, "value2":2386410717081263989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118975, "value1":0.2538090940895061, "value2":2359952847484695171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887012, "value1":0.26127955373376677, "value2":7323797183712585078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465777, "value1":0.39956363244564946, "value2":4290799816610955241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.919817, "value1":0.24654070770474018, "value2":6141631506827241212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.878721, "value1":0.7298060681766352, "value2":7628091662411871299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308905, "value1":0.8062336262819177, "value2":3004783841504994012, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630086, "value1":0.9740479758667072, "value2":7021198218943014146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661157, "value1":0.0030467411917742306, "value2":8705979164600447258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865574, "value1":0.36919272421680055, "value2":4523649159933689100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.185605, "value1":0.5443213859941823, "value2":3228781454351122262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865038, "value1":0.09385639771871399, "value2":7227272424843868171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798023, "value1":0.8684611657481685, "value2":2233953571760268466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730804, "value1":0.08710486562835452, "value2":6236391362613657084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.202824, "value1":0.4792114852099944, "value2":445497349923776863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173553, "value1":0.039235946095809125, "value2":7869971288440786845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908688, "value1":0.5052600498486294, "value2":4519674082380595002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341267, "value1":0.007052586435709472, "value2":3150628489852334883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881706, "value1":0.2885539517449333, "value2":1797762712648874995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216914, "value1":0.4200170713130975, "value2":8711469515389655710, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.219622, "value1":0.8598302806140433, "value2":3863311400727592540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.656021, "value1":0.6463537908056233, "value2":1050074674903654190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641122, "value1":0.40560841010225995, "value2":742946648471490108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_9":"d","key_6":"j","key_8":"c", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.755478, "value1":0.42687719134929036, "value2":8589479839598188558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_8":"d","key_0":"a","key_1":"d", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.848432, "value1":0.9962729980353733, "value2":780000159187900550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.195275, "value1":0.6773988604333073, "value2":5522000829662160867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787345, "value1":0.6486468864516003, "value2":4138825123686123358, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.553619, "value1":0.7566207604613129, "value2":5341523136140526945, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982202, "value1":0.8174248836774913, "value2":4875324336295746206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317829, "value1":0.25574317763537074, "value2":3850513628767917465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366612, "value1":0.634692324496315, "value2":8508176811169559100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.931082, "value1":0.9070818575294899, "value2":2941126198839449331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419466, "value1":0.6035953996525765, "value2":5226492465996637298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207446, "value1":0.7148273386681443, "value2":6830161919863899962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339115, "value1":0.9680080839353958, "value2":3346579129519946511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329518, "value1":0.5052033589082856, "value2":8326166807060477796, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_2":"c","key_0":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314104, "value1":0.23744990369573346, "value2":6679180757317175227, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177954, "value1":0.6916907655144761, "value2":6598624424043999988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220478, "value1":0.9419151786449748, "value2":6868454728302182507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.962386, "value1":0.29200153388880246, "value2":3531740125944089227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.522912, "value1":0.4384617611060238, "value2":7012028395173793023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.701530, "value1":0.008494038169472137, "value2":4096870035163582532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019574, "value1":0.540551382306373, "value2":7575443997806932703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_8":"d","key_0":"e", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558742, "value1":0.30242558718747614, "value2":1312291174476606343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.597198, "value1":0.9990604033963557, "value2":5377437181323103863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321750, "value1":0.8571357709325697, "value2":3223326577646383979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976585, "value1":0.794736424631542, "value2":7297990135456560387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053373, "value1":0.9010563829072253, "value2":5702408670366972568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.500145, "value1":0.16016689363963474, "value2":7655127244946192653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.347755, "value1":0.19254449629837572, "value2":1438411533080776332, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.047080, "value1":0.9596252631833848, "value2":4943940353426671568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.737278, "value1":0.6304724450380977, "value2":5075506433971212474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.842472, "value1":0.9682019821931608, "value2":7537248780629599850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210066, "value1":0.016073835132125416, "value2":1148941512571603438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019003, "value1":0.15376503679496498, "value2":6744619422647456093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.972351, "value1":0.6522824510722263, "value2":3635978187235320184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464404, "value1":0.1987173098042302, "value2":2325620189736714816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505822, "value1":0.8969967220271704, "value2":2772428523245878847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.858017, "value1":0.42532104055952885, "value2":7057664582771584406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.827767, "value1":0.11843519151662998, "value2":4642257980991996610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.457807, "value1":0.6680540521232995, "value2":1413093760337990232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307237, "value1":0.2483558233491924, "value2":1657535478313961463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187495, "value1":0.8787274497214357, "value2":1012440122187350073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_5":"h","key_6":"k","key_2":"b", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.935106, "value1":0.6622345557285527, "value2":873018521033719372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212483, "value1":0.03664769373990424, "value2":8662944034974337497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.279577, "value1":0.06940570148672788, "value2":3397490112016095138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.679928, "value1":0.94467398885254, "value2":8140288354763339321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348611, "value1":0.09177297933527545, "value2":2243053697955925595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.033795, "value1":0.004266699548950342, "value2":8869596053230724066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.716957, "value1":0.952311087977132, "value2":3488323066281405120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275244, "value1":0.9153132837467353, "value2":6017024334903831134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523199, "value1":0.8894820330529598, "value2":2495751532034294665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.548536, "value1":0.36452752239967207, "value2":4087473006792877554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_9":"g","key_2":"b", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452175, "value1":0.03794808741352215, "value2":8378297363369384496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156999, "value1":0.3422567916396813, "value2":3950482866588863620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.508543, "value1":0.5441836551476523, "value2":5324891351187729379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397072, "value1":0.8991563074648984, "value2":484650303993845367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074791, "value1":0.08635154390096551, "value2":7636185572736407362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214033, "value1":0.529318647161814, "value2":3428935700531030330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023178, "value1":0.9863231971076605, "value2":952801547411635885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.655562, "value1":0.1342499141610202, "value2":6341506872735720793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834066, "value1":0.5801423696428913, "value2":2989261336292506730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.107225, "value1":0.08595942695174727, "value2":7162816956772874074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.303930, "value1":0.5732092807009482, "value2":6606617148160680700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.915970, "value1":0.10263255284529503, "value2":8480457956761770794, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356355, "value1":0.6141228451134201, "value2":7156945200312274543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645757, "value1":0.7597072830851564, "value2":2605675246955821391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_9":"e","key_1":"f","key_4":"b", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760044, "value1":0.048059905194606296, "value2":3102725962018871003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949450, "value1":0.803383211503761, "value2":5157179277981661443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124187, "value1":0.16836795463840862, "value2":5627024937490827799, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621202, "value1":0.11873453496853606, "value2":971627245321019634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.762026, "value1":0.5957950719970895, "value2":1706809321238862597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391493, "value1":0.7101377073031195, "value2":6966030075201405586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.168903, "value1":0.22863352792292077, "value2":1427332489767053365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.698944, "value1":0.5031422927319381, "value2":2536670790706398066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.985917, "value1":0.7010791398723792, "value2":2424373004486759234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.611987, "value1":0.023052053167488012, "value2":1572028785346213886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.084732, "value1":0.8225285464369793, "value2":7247308476502074358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476584, "value1":0.6320619568566006, "value2":3013845267574452926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.358865, "value1":0.9597368730404013, "value2":1895755548879639735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559280, "value1":0.7939551495955407, "value2":9185811549617457738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338153, "value1":0.6433609372521104, "value2":5456961662990895069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.120489, "value1":0.02598857402797232, "value2":2690067127374441413, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.026856, "value1":0.3509430162667547, "value2":5912250260618117609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.911457, "value1":0.5658091707632997, "value2":3970272443516058585, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.563432, "value1":0.24659033075994388, "value2":728046743750740729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.596212, "value1":0.5615056517168183, "value2":7033619460114819908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051460, "value1":0.5771358250664421, "value2":8203484790496543671, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.824484, "value1":0.05697931696566816, "value2":6635808805223685263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627182, "value1":0.9130566878289693, "value2":8110244852760276756, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_3":"e","key_9":"e","key_2":"d", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.207192, "value1":0.7274854390257628, "value2":5772335372245409001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799146, "value1":0.5204858317687999, "value2":128757778781441708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014826, "value1":0.7097519460607616, "value2":9006806179184131719, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.648719, "value1":0.01042589454449313, "value2":1630317652722526040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_6":"j","key_9":"k","key_3":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.665998, "value1":0.6184281613297867, "value2":3187261205020385384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.501321, "value1":0.23807711629991066, "value2":1285207877035639838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184165, "value1":0.7331865941823509, "value2":8439731025981805138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_9":"f","key_0":"a","key_5":"j", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801569, "value1":0.9860558447882487, "value2":6774600722253198808, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.384223, "value1":0.22369926816026842, "value2":5826656674460169842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.980495, "value1":0.6229185444454209, "value2":4872634527752777210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.380826, "value1":0.7806130098804899, "value2":384791734560367497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163193, "value1":0.5398716860823138, "value2":4387950486971806786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.823934, "value1":0.5810656874809516, "value2":450490831163611562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.608002, "value1":0.7930931659235, "value2":7050949938710758790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363629, "value1":0.5705690869328316, "value2":4003836970877260330, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_6":"e","key_7":"a","key_5":"i", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.997480, "value1":0.2946419071937674, "value2":5707538095602523121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.835470, "value1":0.023853067484955352, "value2":5038338303194879356, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.720798, "value1":0.48474572095528634, "value2":682322783035637435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.573983, "value1":0.3514813766672496, "value2":2903766540993385647, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.730487, "value1":0.8217554814463921, "value2":2948047805979067123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016747, "value1":0.12213673109770266, "value2":3888549493731213502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.081414, "value1":0.18665321280658956, "value2":6792269259144278363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.300754, "value1":0.9731729557371976, "value2":7090692680090167231, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.337262, "value1":0.4023504772804693, "value2":4873228147801762854, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990413, "value1":0.6497908067795414, "value2":3504328384430870621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.276664, "value1":0.785362017819063, "value2":1481672280882279109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.357896, "value1":0.03050056454630596, "value2":6649840382760115378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.095552, "value1":0.31901448906359003, "value2":7956338378631849376, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.518227, "value1":0.922032137621878, "value2":785067472984268171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094243, "value1":0.6170957228718421, "value2":6792110304537802959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782822, "value1":0.20221744914264872, "value2":3178187263071942184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.539464, "value1":0.9703158465709303, "value2":1060145839109092280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.773211, "value1":0.9517836987483416, "value2":7463838442968425219, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.695794, "value1":0.34514076166527163, "value2":1111746250317888633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.074230, "value1":0.19136080585905593, "value2":4573587278465589025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_6":"c","key_7":"b","key_0":"j", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.982043, "value1":0.2232136951539441, "value2":7688263110504645334, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939724, "value1":0.3301646825680086, "value2":5317348954333303134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.275048, "value1":0.1567976387714283, "value2":8628385701502049400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.251237, "value1":0.601867452525591, "value2":4258975777323734306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051509, "value1":0.87447596502846, "value2":390569829027012116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453014, "value1":0.7517900109611292, "value2":5319112069730300764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.859148, "value1":0.03968223769223721, "value2":6092044766196787249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564895, "value1":0.8282334480284984, "value2":8177460795118738605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.213064, "value1":0.37245338326241034, "value2":5313011500830153125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.234759, "value1":0.6391009807318464, "value2":3598159575817066989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922540, "value1":0.8881740732484977, "value2":4573970560902118758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247842, "value1":0.6867454879676517, "value2":2359279288069431907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350225, "value1":0.6867879732162172, "value2":37306184729818872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881388, "value1":0.4406197726220826, "value2":724124821973247241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.950564, "value1":0.1322036312488787, "value2":4198819767892784951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.666395, "value1":0.2251980228042216, "value2":606006971675717615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.344497, "value1":0.029652052432054064, "value2":5846707393638380751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.703773, "value1":0.6376539079560296, "value2":6681127841717987695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295458, "value1":0.5526352586149329, "value2":7043209016069060231, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.435502, "value1":0.09713433968048614, "value2":1019638678991819627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598132, "value1":0.963189125504533, "value2":6458016500274290636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627640, "value1":0.4641061727729553, "value2":8689246494106761338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254760, "value1":0.8326486356175677, "value2":4407797931839765907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.805732, "value1":0.6423132619728287, "value2":6833967417402080897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.326823, "value1":0.9067956612338547, "value2":2841002873709302821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.088104, "value1":0.06965395874499884, "value2":1789284717431249823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.948712, "value1":0.9884448336440479, "value2":1290858986155893057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511709, "value1":0.8105556502461225, "value2":3470144572708895882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163730, "value1":0.8658645130208062, "value2":6466367002189635673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260153, "value1":0.5010858345134921, "value2":555883752624156654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.274796, "value1":0.8267418905611275, "value2":7590347778036203754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.338150, "value1":0.7906313079639156, "value2":7837780129396014797, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938845, "value1":0.03371523590847688, "value2":2642591228031199330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.254822, "value1":0.42059549793106427, "value2":4614016824012787995, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003989, "value1":0.05975709544316542, "value2":1382695454770978636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.577151, "value1":0.8937534168973357, "value2":6521815205949311706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.253675, "value1":0.40184282199800475, "value2":8286880530272716169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205638, "value1":0.7633247932732301, "value2":6913337973948968095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.939093, "value1":0.03322965819693288, "value2":2797917596305577577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.248192, "value1":0.5307465178536894, "value2":6134584955455781673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_9":"j","key_1":"j","key_4":"c", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.593487, "value1":0.05419262497736568, "value2":9190929989995856492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.182515, "value1":0.035610068777040586, "value2":6469776877211677470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160124, "value1":0.662164487805106, "value2":8768604308735587138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317998, "value1":0.9757873511779303, "value2":1199801679672428467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.922339, "value1":0.9554985330074205, "value2":8286943158249961765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023973, "value1":0.01669695080382125, "value2":4408497776354893523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.399088, "value1":0.37811424022789003, "value2":499643435878747492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.974576, "value1":0.6392617923906977, "value2":1832499551397243524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423426, "value1":0.1431493881970496, "value2":4016805275891833987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.220477, "value1":0.006541995474356055, "value2":5608461794122610089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014455, "value1":0.20051860544075079, "value2":4298831902740539404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.771702, "value1":0.9711913949733121, "value2":4548136488165596312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837484, "value1":0.3199900890395432, "value2":7619719494433917806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632963, "value1":0.8179930116909135, "value2":42611344634328983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739321, "value1":0.8571068088303897, "value2":1929086780074260377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169110, "value1":0.2241172770388991, "value2":3015343857994873164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.558214, "value1":0.5245963976733786, "value2":4636974583954671385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551826, "value1":0.8529460688457188, "value2":7676686801578586657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.317389, "value1":0.34597315939279244, "value2":8526544610592186971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131556, "value1":0.9023875402465826, "value2":3147425851236763117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330907, "value1":0.7187683220932978, "value2":6304935909987923644, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419527, "value1":0.3464275048663091, "value2":800017209533647436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325070, "value1":0.32409707654758807, "value2":3093719017131899877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.094166, "value1":0.6576755239099957, "value2":4375841020494708801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.184361, "value1":0.454784333846635, "value2":8511384704032602153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424067, "value1":0.7746556148202395, "value2":8409657959538616305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.282726, "value1":0.19274194173178774, "value2":4484284290811930974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.044679, "value1":0.17881938606509734, "value2":2991918820431220404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622917, "value1":0.5058188298556251, "value2":2680090310035550156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.686600, "value1":0.019991692028205214, "value2":8990563930001611676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.977493, "value1":0.8045660581528546, "value2":3965120458727174879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.910280, "value1":0.15532301386230277, "value2":2751777764504449883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.981678, "value1":0.6634934084628059, "value2":3338856320416894701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_5":"a","key_0":"g","key_3":"e", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099077, "value1":0.4991967563162609, "value2":7696501412266603819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_3":"c","key_0":"a","key_1":"j", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.169242, "value1":0.7573800949559079, "value2":4682651688122452104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.854077, "value1":0.38315712266426233, "value2":7277684317066282125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017721, "value1":0.43030519598600153, "value2":8176728734487897933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.449823, "value1":0.0905672633924478, "value2":8038615141368276704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505199, "value1":0.29347749204963386, "value2":5939807103722962422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.429412, "value1":0.3821527042310373, "value2":7616377560194630586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.645364, "value1":0.31800894756504583, "value2":5324276910842211792, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789951, "value1":0.12346800778619027, "value2":7282935021388766519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.005934, "value1":0.4652775738745803, "value2":5150985627139247681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770208, "value1":0.8318069314007921, "value2":8693296121103057557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.036795, "value1":0.31504722243353955, "value2":5332439897432100886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.557676, "value1":0.04547498827953966, "value2":1747115516814035342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.295720, "value1":0.25765308010583876, "value2":7172964053161542723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.862280, "value1":0.6690305275979546, "value2":8530225967231964009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.464755, "value1":0.015096554191777058, "value2":2158727717387461667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902724, "value1":0.3729818170388733, "value2":2937474665692773587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.150211, "value1":0.9010548939518067, "value2":8689173712211659171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502794, "value1":0.637718327451074, "value2":1058416603563820737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756334, "value1":0.9093486580620052, "value2":3914139843013624351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109763, "value1":0.3482544181233465, "value2":3984218999735374107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_7":"g","key_2":"f","key_3":"f", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.007357, "value1":0.530193826214922, "value2":7784257234281286192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.133042, "value1":0.14994561696634548, "value2":6913972686763727632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.499117, "value1":0.9693770819840257, "value2":292923979595115495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.256986, "value1":0.8560085418753359, "value2":1633179891830212156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.348017, "value1":0.4525195870224779, "value2":1498113059165586202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.389164, "value1":0.4747861877903318, "value2":5663253941505962378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.712219, "value1":0.23770447169372633, "value2":7127281555547067699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.799547, "value1":0.4771589732318728, "value2":2601851094636806560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.761432, "value1":0.9902875537117704, "value2":2441735233721216793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.594915, "value1":0.9327820854039164, "value2":7435962671657247487, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.018250, "value1":0.35410376862709275, "value2":6088069648662464618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.640810, "value1":0.6282310495408681, "value2":4206520774588356560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.929377, "value1":0.30194565379846455, "value2":371174649294622523, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.817275, "value1":0.4745483085572834, "value2":2810603774900787473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.046061, "value1":0.9540450542751275, "value2":6567731530813672613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.976385, "value1":0.2747180574854192, "value2":7052984344810070260, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833030, "value1":0.8111270330849011, "value2":2406611793520084033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146154, "value1":0.019440584474240373, "value2":1431804086688415352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.541319, "value1":0.42910271299796143, "value2":5008131544913964869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.478974, "value1":0.3992297014608565, "value2":6638251650936294767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.591553, "value1":0.9631516171967569, "value2":6328755963396316225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.836675, "value1":0.8925642028256048, "value2":2169296529295363291, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_6":"g","key_3":"e","key_5":"g", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240233, "value1":0.5601423681829663, "value2":2372802153601657174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926558, "value1":0.8669054217402387, "value2":8750941328640682000, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.785340, "value1":0.9294643708923529, "value2":8967392703697598518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321372, "value1":0.6020812110006013, "value2":1942102877471692500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.578919, "value1":0.17388396608421544, "value2":5054354296681821565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830121, "value1":0.06504953435269434, "value2":7812868363877570134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.110821, "value1":0.12907416408116543, "value2":1398896999021006979, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318471, "value1":0.6665917244504806, "value2":5816507337427212221, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881196, "value1":0.35509697330525436, "value2":3799008703902351704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503152, "value1":0.686811705668727, "value2":1281918055642507847, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.393488, "value1":0.8980510009425112, "value2":3157487117727981952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993113, "value1":0.8339917765115039, "value2":1528182230124691520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441452, "value1":0.5723022066521427, "value2":8611586563060872543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_7":"a","key_8":"c","key_5":"i", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.902474, "value1":0.21553093452664723, "value2":1591141288904852990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.937761, "value1":0.5975557556565823, "value2":1778079885483684894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.509879, "value1":0.8107703383515009, "value2":7338854303608988296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321562, "value1":0.7839743525651934, "value2":407341572079351661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_5":"k","key_0":"g","key_3":"g", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.083126, "value1":0.8340311472033722, "value2":414058498508156341, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.266082, "value1":0.7848792912531005, "value2":1487975780946804498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.001354, "value1":0.042658133657431034, "value2":2686477010080712855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984348, "value1":0.12772415816911367, "value2":1744454398245616628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.545130, "value1":0.012749865379431171, "value2":3614758343709591135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.211075, "value1":0.26931470974880833, "value2":5335597373851226907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517993, "value1":0.8277823010534222, "value2":441076351209513172, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.783905, "value1":0.6446774753163863, "value2":2444284131966329588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140755, "value1":0.7307559658816861, "value2":1750453360454837174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.243464, "value1":0.5611645445748624, "value2":5031122288522829688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.247597, "value1":0.26059857605419207, "value2":6822156384989813472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.053905, "value1":0.6298269719196294, "value2":6661136217666523640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.058995, "value1":0.7306532383370615, "value2":5311315770055711299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.030527, "value1":0.08351380687475202, "value2":7155821886727737087, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.498371, "value1":0.2731396400770269, "value2":89197923125259866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397662, "value1":0.0870595280214176, "value2":4020978501682465140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173834, "value1":0.7945264755888277, "value2":3882559205765407193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497055, "value1":0.09364593584875654, "value2":891682366412177732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065765, "value1":0.835147994300049, "value2":2505979555724543511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.765901, "value1":0.992206355507561, "value2":5788507149763623428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_6":"a","key_2":"k","key_3":"h", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.622912, "value1":0.44608293916585673, "value2":946502417712586156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.366924, "value1":0.9938167708003688, "value2":7200121887993962206, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.072114, "value1":0.001073896039030629, "value2":8685581835197772410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265774, "value1":0.30130760063683054, "value2":8411046196525683207, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.749589, "value1":0.9239863104020335, "value2":4219975890726145199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190061, "value1":0.9734172533312344, "value2":8137079207203857103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.519945, "value1":0.07187194342466732, "value2":7825744617647803155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.649495, "value1":0.6408236940193898, "value2":3048798030528052662, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320456, "value1":0.5081060875649162, "value2":719352320345764599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476698, "value1":0.052307893890373255, "value2":2742427547083463898, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_8":"d","key_2":"i", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571531, "value1":0.4132003718642955, "value2":6484766831533191693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682702, "value1":0.44751287991255284, "value2":4055042383084807325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_4":"e","key_6":"b","key_3":"i", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.890718, "value1":0.33741266801359987, "value2":6934783766842267527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.452747, "value1":0.3663537726308516, "value2":3312655545498739922, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199996, "value1":0.15068464159094366, "value2":5665646131255744951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.983795, "value1":0.8775221552243728, "value2":9069927522091992178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.583294, "value1":0.838152847228304, "value2":1035319401212201491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.212587, "value1":0.8012210929096193, "value2":5585326063344558523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.641779, "value1":0.9247211843697717, "value2":2117637859665911236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.406293, "value1":0.6198872109290668, "value2":5911640262763562610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.277897, "value1":0.027423430536495416, "value2":7628618445567261117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.832579, "value1":0.5331161695907243, "value2":8305462638082561375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.205268, "value1":0.018199222166683245, "value2":475925606235710366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.615075, "value1":0.9915416739226851, "value2":2204646069144435635, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_6":"d","key_7":"e","key_1":"e", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.424612, "value1":0.9757938651409276, "value2":6266511548869428739, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.286078, "value1":0.0846398426026624, "value2":3081995387268873440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_9":"g","key_4":"f", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.787171, "value1":0.8365446036859473, "value2":5979460348923075463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381115, "value1":0.9123835513620147, "value2":3398482109219075054, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.114157, "value1":0.804291931301114, "value2":6012424958858906101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134911, "value1":0.00736667824808292, "value2":2400000632738793301, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.164780, "value1":0.21287762980212552, "value2":8645521771535465924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710538, "value1":0.7411138160092244, "value2":8685307998868002992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585644, "value1":0.08786909591629084, "value2":7343395510792995060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125543, "value1":0.7332366768127239, "value2":8370868046930361421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.932834, "value1":0.9015644945676353, "value2":5236314268804910193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093732, "value1":0.497896904599941, "value2":6397325408101186539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486123, "value1":0.5097658314843312, "value2":1337466807316653828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.904889, "value1":0.6556792189077274, "value2":4516335409033295305, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.387498, "value1":0.9910872699779568, "value2":1557997601588004148, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710931, "value1":0.5132845499506883, "value2":3319795795293053140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_9":"k","key_0":"k","key_4":"e", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.515215, "value1":0.049432502275940204, "value2":2300709145422274852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821813, "value1":0.6825139883510115, "value2":8194491426131939536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.865373, "value1":0.7044541486613806, "value2":3703555177588303304, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837580, "value1":0.8572176121238669, "value2":2344100903459752932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314910, "value1":0.4347186053772807, "value2":7676829676819086499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.946948, "value1":0.7950628854157453, "value2":2267010392465955325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015107, "value1":0.8624602552899475, "value2":9148330689311945183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_1":"b","key_6":"f","key_0":"a", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488041, "value1":0.4618418780870866, "value2":8123711349904092581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.710726, "value1":0.4226915062812881, "value2":5491521860827839646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841455, "value1":0.9825903973183027, "value2":3077499782633025057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.592689, "value1":0.9193972918242918, "value2":929238328644383650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993922, "value1":0.14695440679147787, "value2":8882303717901413025, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_4":"b","key_8":"b","key_0":"a", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731320, "value1":0.9934902241842611, "value2":1321453009633438522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604377, "value1":0.24174531205655925, "value2":5870091064000157680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.628164, "value1":0.8064136026821553, "value2":8786888366338651141, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.484350, "value1":0.5079394885547678, "value2":6920245211342331930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.978330, "value1":0.9354707910737625, "value2":7581764658078201917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.632366, "value1":0.23595321295755245, "value2":1511878176205605753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142194, "value1":0.6306719216610874, "value2":2431345756658073693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.397195, "value1":0.7323027634538346, "value2":8521159495965572956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.996298, "value1":0.8484427529278864, "value2":7246857551445151219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.328211, "value1":0.8531255802404514, "value2":4506921974707672140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.308884, "value1":0.5324739320542478, "value2":6505796324920546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_4":"g","key_9":"i","key_1":"e", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.721813, "value1":0.2030510736245931, "value2":493933611483820748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.325138, "value1":0.6318807488600625, "value2":3427677815511198978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.717950, "value1":0.13850468444662895, "value2":3714125763364851426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.984959, "value1":0.2313158857459622, "value2":6179178857252455230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767352, "value1":0.04737365094297706, "value2":7312736110130720225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.517326, "value1":0.38308887661557994, "value2":4397262624851606256, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537458, "value1":0.8630190277228629, "value2":1405251953788249605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.589024, "value1":0.6853812314292282, "value2":6568591835038822877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013121, "value1":0.9022971605517441, "value2":5391410131187089227, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585652, "value1":0.027029007730938633, "value2":6270487841856618503, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.069148, "value1":0.5452016240009335, "value2":7230673973301658030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870318, "value1":0.8893099847439015, "value2":2072311642337421510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.525589, "value1":0.9489296395713415, "value2":1934395693866121179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.411127, "value1":0.5543622037481519, "value2":869973100557162310, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870905, "value1":0.1429227422154167, "value2":1580405259981052384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214057, "value1":0.007127408341584659, "value2":5021161294907689526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324636, "value1":0.2886203159980936, "value2":3536603792281429135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.566190, "value1":0.2435195285039059, "value2":7314231811000308948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.302446, "value1":0.3969373764014023, "value2":861549335043188440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265270, "value1":0.8392596283575425, "value2":229012885919848214, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306041, "value1":0.6755326135952276, "value2":687305007623331029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.920842, "value1":0.14122582181388352, "value2":8912506495776009399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970414, "value1":0.32375225588910955, "value2":3874667148173594242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.098047, "value1":0.5080753238940934, "value2":259459341225408666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814708, "value1":0.5851579994151463, "value2":4279032101744191170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.857725, "value1":0.8750883468056723, "value2":8160658677842295703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288649, "value1":0.3318827172603761, "value2":8934632128032959148, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.651119, "value1":0.31525481325301247, "value2":8244510685212193822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.065672, "value1":0.08637516713839134, "value2":707728667641140855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620357, "value1":0.8983386747310453, "value2":3756826533006983142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707292, "value1":0.4495030238963459, "value2":5996025393915497140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.443961, "value1":0.930457120039426, "value2":6519425791992110608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.109167, "value1":0.09168467762421384, "value2":2057706949013098262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.481603, "value1":0.3529978146205129, "value2":6544227882618708766, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.770746, "value1":0.6378677748113686, "value2":1182963631296444077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967875, "value1":0.8756904550723122, "value2":516292034493240162, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.324624, "value1":0.03689035176673176, "value2":6723940798795470811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.556389, "value1":0.7611742354824225, "value2":6343475929372871725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538576, "value1":0.1188977043392838, "value2":1679649620772949796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_6":"k","key_7":"a","key_1":"j", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.531730, "value1":0.2793539457328025, "value2":8309265375086957584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488778, "value1":0.8806255322454197, "value2":2637346612964090286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.174493, "value1":0.4058829790367517, "value2":4908420813809791761, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.875676, "value1":0.8084081237917962, "value2":1370185119560523418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214353, "value1":0.22148397653369978, "value2":3829075843579764833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.402832, "value1":0.9983967808941802, "value2":373850521427271723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.574399, "value1":0.1403913922108672, "value2":1025325851242540472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_4":"j","key_2":"i", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.538957, "value1":0.5637672549308663, "value2":180146514999582015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043198, "value1":0.6621042931314224, "value2":4672271958823003044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.455391, "value1":0.37463255563131875, "value2":7050585404363795282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.118353, "value1":0.7019235890065948, "value2":6561897677595087821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.562842, "value1":0.4216296582302376, "value2":4671864964861835857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.613979, "value1":0.9101669225413574, "value2":5476069179621211969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.079079, "value1":0.6285830126630634, "value2":1800090753316916399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.990168, "value1":0.4228502254109105, "value2":4769618752656230984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.630399, "value1":0.0593879368852638, "value2":8425926359679957872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.381514, "value1":0.3170447182462336, "value2":134775839032388004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.306533, "value1":0.09290124329204048, "value2":3878589204829724343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.363988, "value1":0.7494592976903763, "value2":588119535129085693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.726401, "value1":0.2843848594033847, "value2":797824336547840313, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.425192, "value1":0.6747059449207641, "value2":6618945540166900762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.486703, "value1":0.9234732601693532, "value2":7284867992059891857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.162651, "value1":0.4998767239554866, "value2":3758664262196916666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.638502, "value1":0.14222618431131662, "value2":7183266992692325102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.814797, "value1":0.04744514500578937, "value2":8807075724873574935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617563, "value1":0.7113712918981732, "value2":7369990382580999103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.391047, "value1":0.4667345592294561, "value2":2593217151341486604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.423210, "value1":0.2993388901152061, "value2":7736063295280636221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.233156, "value1":0.2669621811468438, "value2":9163725732269873210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.780343, "value1":0.4836321272043857, "value2":6239122865745239252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270794, "value1":0.8124904828437979, "value2":5664361933553425625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_2":"k","key_5":"k","key_0":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.567390, "value1":0.1509148834766825, "value2":769811157679024909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502511, "value1":0.3496329905819851, "value2":8064435486484275503, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.995318, "value1":0.07944294849213016, "value2":7290799161307742730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.160027, "value1":0.9323372972664894, "value2":8694402707307208350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.265068, "value1":0.7804375599856387, "value2":2823069570137165166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_3":"b","key_6":"k","key_0":"f", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852790, "value1":0.04269382129150226, "value2":4247785862022220763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.813337, "value1":0.3041270110008107, "value2":1162341534698170174, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_6":"a","key_9":"c","key_1":"k", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.821104, "value1":0.05863251524747595, "value2":5517633377182805397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.801029, "value1":0.7039209651517294, "value2":5777192245942090986, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.292328, "value1":0.0789099713548181, "value2":4341141001609491414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.650140, "value1":0.6547556555322551, "value2":488100188330068974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.881952, "value1":0.6384170556890851, "value2":7393134503131526080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.019410, "value1":0.4521619097561246, "value2":2730473646049580719, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_7":"b","key_8":"c","key_2":"b", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413658, "value1":0.35836104075629194, "value2":5913006016396086189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.388542, "value1":0.347328329631466, "value2":3415671885617145536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.908514, "value1":0.682932258250745, "value2":1535326911373174401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.453424, "value1":0.011430118976046925, "value2":6615057602694952188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.661892, "value1":0.04490375141966574, "value2":3586691570128338490, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.598897, "value1":0.9840071764206562, "value2":6756637664735447989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.614741, "value1":0.45336506256132186, "value2":7160462685010833845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959437, "value1":0.33491802794663117, "value2":4263644093884006866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394710, "value1":0.09508520954191016, "value2":8086669194950309707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.400389, "value1":0.748914928293926, "value2":7561318401530983897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_2":"a","key_7":"c","key_0":"c", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551520, "value1":0.29033239272123923, "value2":219973352444233301, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.433012, "value1":0.8026759332563937, "value2":1035833541375117050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.097876, "value1":0.3123782080487283, "value2":97482352559342166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.971923, "value1":0.09350074090215915, "value2":6603209689656348295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385293, "value1":0.025071498526727982, "value2":7821611042801827653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.582177, "value1":0.17895122462677535, "value2":4851272893829158392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715577, "value1":0.12145793831580125, "value2":6773868244129033160, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_9":"d","key_5":"e","key_6":"h", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.245667, "value1":0.5083106874300092, "value2":5342790051223313569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.191947, "value1":0.7213188287503117, "value2":8184500849109232134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.527410, "value1":0.2848785581313827, "value2":8059297953437098619, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.023841, "value1":0.08963330011439892, "value2":8744048608508050406, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.861975, "value1":0.9569428255590289, "value2":5863440324678791040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_4":"h","key_6":"i","key_0":"j", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.020139, "value1":0.5328792585389814, "value2":6851298079925452135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383316, "value1":0.6736870667139099, "value2":1034982274545531879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.490723, "value1":0.7069141316227696, "value2":1238860616186948989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774619, "value1":0.15017350186839917, "value2":4145739713631541991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.724625, "value1":0.41136382513830944, "value2":2470375246542099625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.014004, "value1":0.45322459474776666, "value2":3424033124770728498, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.370213, "value1":0.7936480778398273, "value2":7651653355869440262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.930663, "value1":0.28375382304907193, "value2":4573094024293120951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.108477, "value1":0.2514498975622512, "value2":6565807048098733104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.318641, "value1":0.09067602148053568, "value2":2118726105929309608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.341849, "value1":0.5146408700742566, "value2":5544935195040202056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.016279, "value1":0.2887329883061575, "value2":4016937139552222061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696532, "value1":0.35345697419681527, "value2":7198819764217760607, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356409, "value1":0.6101596793669094, "value2":2338966817998610022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882128, "value1":0.8757983585066079, "value2":1264113150578508254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.214730, "value1":0.04033669277556926, "value2":2311999714610864651, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.125040, "value1":0.8510776359004306, "value2":9203972476999185975, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.617466, "value1":0.4880211576700948, "value2":7876488006268555933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.580824, "value1":0.738663404243217, "value2":6159559973328325759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.894051, "value1":0.0394639970488642, "value2":477392450960926806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112539, "value1":0.11952574252776728, "value2":2511563277039044978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.918404, "value1":0.30833608301592924, "value2":8563798643952041377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385588, "value1":0.17478857683593713, "value2":6552271330519870932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.139243, "value1":0.42492669927426513, "value2":7691897138631144930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.715207, "value1":0.8647651137464607, "value2":8497862595581854195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.760625, "value1":0.24627316716260067, "value2":1409311771773591601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.880305, "value1":0.7511958117841795, "value2":8663780143091433962, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.121986, "value1":0.503001087851081, "value2":86404765376501910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.378220, "value1":0.7154879418930685, "value2":4699613906263091859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367776, "value1":0.3357733171654874, "value2":2109430818036060968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051853, "value1":0.46132527877007656, "value2":1864815246368296251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.503984, "value1":0.5450705451757059, "value2":2199582755716092199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.270031, "value1":0.15857947720543178, "value2":5111440735129620852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376563, "value1":0.08077432861716789, "value2":7964637453463198864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.017382, "value1":0.4697079883099492, "value2":6862028548685305626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.938781, "value1":0.3698227408392578, "value2":1642534448554432249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.869838, "value1":0.4366942978360045, "value2":4007380928114996020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.309900, "value1":0.05509289000926218, "value2":8881026026944444737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321699, "value1":0.5508642725465765, "value2":1065357963175430930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.242134, "value1":0.05194281440569991, "value2":2955861771024227742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394641, "value1":0.6802778418350491, "value2":7829903676952590571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.011752, "value1":0.2679251298930048, "value2":46390372055260426, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.156821, "value1":0.8759623901682069, "value2":6663991521677358276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621515, "value1":0.8461818248462066, "value2":4793482631115954367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.906690, "value1":0.4454231810933155, "value2":1968123317376605778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.833508, "value1":0.5580322939662696, "value2":542437654648170086, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.864441, "value1":0.13709821675852493, "value2":3775094265545831122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349796, "value1":0.5463888647509524, "value2":3843833923457334958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.210241, "value1":0.13053751777334896, "value2":2547919115148938351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.534237, "value1":0.13104275846768723, "value2":2757438080188257397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.479346, "value1":0.6936882347927468, "value2":3593937018182399578, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.013374, "value1":0.647079383488511, "value2":6027310083597491507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353768, "value1":0.5713686637635252, "value2":3396180021694291926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704388, "value1":0.028813882790100154, "value2":6631675215659864441, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.350375, "value1":0.09828506398906811, "value2":262432636815220332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130984, "value1":0.05874581283090899, "value2":5455068607213223389, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.269711, "value1":0.16897643510231944, "value2":2188619802248362391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.798909, "value1":0.49628972013165096, "value2":8871333460189078193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_8":"k","key_4":"f", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314472, "value1":0.003776179814725882, "value2":5465069878443166467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.841977, "value1":0.016019153138836187, "value2":5090055543864076033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.431741, "value1":0.4503254906693805, "value2":3437452976536184232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.021144, "value1":0.7157770425720729, "value2":1627465028436054242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.492677, "value1":0.170722769795088, "value2":5333195898301269961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.572483, "value1":0.07518223050679318, "value2":749837753058622914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.070547, "value1":0.7819934575507779, "value2":1257841836293236196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.834097, "value1":0.7836504674507244, "value2":7822820024554629611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.725415, "value1":0.6205675290840149, "value2":7546425291277331498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.756846, "value1":0.4577877981753711, "value2":521428696388054794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551591, "value1":0.4656287234062602, "value2":3666151433771478256, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.367967, "value1":0.3303667849726779, "value2":1336037347437447384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.124888, "value1":0.9362913257587775, "value2":8565731764409689287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.356964, "value1":0.763103278688178, "value2":7384734667573077593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.336557, "value1":0.18917586865284508, "value2":3164250330171533065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_4":"e","key_1":"c", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.947920, "value1":0.7279862820897369, "value2":5585150661666129475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.320214, "value1":0.02069659588289354, "value2":4096509130796828102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.339884, "value1":0.3585015534256289, "value2":1745908303614221947, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.736733, "value1":0.6300320540961489, "value2":4976479107859231308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843896, "value1":0.8618710521424175, "value2":409303658417169135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.351779, "value1":0.31536023315738654, "value2":7469097912657923197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.146297, "value1":0.47961745828082863, "value2":120290133475149079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.299654, "value1":0.953430875530432, "value2":1804061300635059106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.653880, "value1":0.6130300298494126, "value2":3600820249700194803, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.241886, "value1":0.4261506936601424, "value2":6410123065530378279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.719862, "value1":0.5557094068088962, "value2":6619604443134241152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.192985, "value1":0.4372446351220488, "value2":5419318851099904514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_5":"i","key_8":"b","key_0":"j", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.062254, "value1":0.35078106654250335, "value2":6337545142680530695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_8":"i","key_9":"d","key_1":"f", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.826832, "value1":0.03911586835937395, "value2":8510739605077299393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.466577, "value1":0.8386295972027331, "value2":3712956826274262169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.774816, "value1":0.2874652343932132, "value2":1661160295946815788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497396, "value1":0.37671291600810797, "value2":7809177916397546184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.061112, "value1":0.6756138122485794, "value2":3821150629117787840, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.711530, "value1":0.9726549946550717, "value2":4361793701491206585, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.419723, "value1":0.8422780123510716, "value2":8387473970870580835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740430, "value1":0.3687691277865257, "value2":49347396348935758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544648, "value1":0.34026181496926355, "value2":3168616117094841931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.031693, "value1":0.35157464321590376, "value2":6956034409157435335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.682310, "value1":0.4968537230062955, "value2":1715909753935447521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.571570, "value1":0.7444971868018181, "value2":773947913317871760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.412817, "value1":0.17790194335965473, "value2":453971816672635527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.154029, "value1":0.695415820029077, "value2":4412491024371312379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.187573, "value1":0.025946190955721947, "value2":8268616070900349780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297634, "value1":0.39036160577230816, "value2":9117936615987168462, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.882305, "value1":0.6498141826508745, "value2":1021392582780518091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.288985, "value1":0.6094023317572287, "value2":5522861781820654768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.516532, "value1":0.4913308899910666, "value2":2877526282810678571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.663856, "value1":0.6052287079408516, "value2":5244892044841001586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.177812, "value1":0.8518899507803316, "value2":4611087097036165557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_5":"f","key_7":"d","key_3":"c", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.789804, "value1":0.9253854154062459, "value2":4746855558380871130, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.163177, "value1":0.054868844952025875, "value2":4671327502125519320, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753554, "value1":0.0480631824565462, "value2":7078677034582099346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537888, "value1":0.3970428718849328, "value2":8317481519599523582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.887867, "value1":0.9625491057630078, "value2":851265403943296561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.739174, "value1":0.3876147534715181, "value2":5378120884849587893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263149, "value1":0.6578316627430515, "value2":199930804061246062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.926006, "value1":0.9018811245039401, "value2":8736925807617681839, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.559532, "value1":0.017637548071686292, "value2":2677656350646305112, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.940793, "value1":0.36532273109204866, "value2":7505680103024683972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416010, "value1":0.7581313246147069, "value2":7561931289424116286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130651, "value1":0.31761722951334315, "value2":7760092958155535983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.330390, "value1":0.3514672157667692, "value2":1963737467345840913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.731094, "value1":0.5668209441148129, "value2":6834927574629705768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.003464, "value1":0.44994944703983886, "value2":1268238147301227399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.488880, "value1":0.15568001025824393, "value2":3802105015027414434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.099752, "value1":0.49578700566450296, "value2":6660233803101117286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.131762, "value1":0.8279103415624635, "value2":8189053738835694283, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.818080, "value1":0.7683869146409037, "value2":4901733131698699140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.502240, "value1":0.2102196895388282, "value2":8419378719359168551, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.130382, "value1":0.00017527290277822845, "value2":7209832064279367432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697720, "value1":0.977742512542472, "value2":5886772442159349223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.152164, "value1":0.5452125862262094, "value2":2301031936204454118, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_9":"i","key_0":"f", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.307306, "value1":0.27827706723478657, "value2":8114605959354081308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.687815, "value1":0.407696242982513, "value2":8251434258695514324, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.957403, "value1":0.9248822561074296, "value2":5435740408050442827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.297405, "value1":0.735268728902142, "value2":6544542317901773513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505182, "value1":0.6434041992526011, "value2":6723613820733288827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953853, "value1":0.08239495901880306, "value2":7797622216144272838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.816223, "value1":0.026034555587018983, "value2":8247070392223609361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781228, "value1":0.009350455430296063, "value2":6344110503212444401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.993318, "value1":0.7246275363827985, "value2":1117977296074531052, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.346481, "value1":0.04745532854484892, "value2":8841440982349072666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.240566, "value1":0.6244142217945766, "value2":4416869640928141403, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.627039, "value1":0.8121528941309705, "value2":2364077225142023978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.605014, "value1":0.041614775390040444, "value2":5587679092002855017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_2":"f","key_4":"g","key_1":"d", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.675250, "value1":0.22265983684713628, "value2":852855496341453985, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.626117, "value1":0.13978101794521924, "value2":4898929345737475246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.272053, "value1":0.8981952631418685, "value2":4155867998384044766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.216853, "value1":0.7930982218994412, "value2":1990843980684870249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.231757, "value1":0.11381414833270587, "value2":5629821232198202573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.967616, "value1":0.9325867021628451, "value2":6458012454076604601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.105465, "value1":0.20026280910311015, "value2":1519145667363005058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.959715, "value1":0.8692424311015335, "value2":3441946116265084781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.916752, "value1":0.09650654250745302, "value2":7435925512873628832, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314127, "value1":0.5339606052113064, "value2":8918601923652673265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.955659, "value1":0.21237559939985842, "value2":6700000271323408810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.462571, "value1":0.8031758318689627, "value2":4200089724641662061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.704628, "value1":0.42388353536260726, "value2":1676230088334959718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781871, "value1":0.7853951443498608, "value2":4677884824803639265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.093020, "value1":0.44592927039430036, "value2":5106772041668223293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.066421, "value1":0.7167334468150509, "value2":916960599462889530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.753333, "value1":0.8855172765987326, "value2":3498076904045993942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.263558, "value1":0.039242126444255614, "value2":1994687158267196250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.529319, "value1":0.11390721956642542, "value2":6770255083967052135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.642302, "value1":0.25917237536080345, "value2":3561054032858646761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.800222, "value1":0.5648139239977721, "value2":7859467413386922358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.714908, "value1":0.3172565417584279, "value2":4018189432904936827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.495891, "value1":0.1699695672081134, "value2":2951696383925672800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_9":"d","key_5":"d","key_8":"e", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.927247, "value1":0.5193998219352067, "value2":7461947996252867047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.465786, "value1":0.04561481590524083, "value2":8852624735734959732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970044, "value1":0.9502884615051707, "value2":5544298343674474830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.224365, "value1":0.7835642667167426, "value2":7989661444310811565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.970171, "value1":0.044357973684653176, "value2":6240172232063272107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.022705, "value1":0.2909464675793184, "value2":6791113566169749215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.791002, "value1":0.30758766636926194, "value2":4087253744537420622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.526835, "value1":0.26218100233396685, "value2":761773481307699191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_9":"c","key_3":"d","key_6":"a", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.057757, "value1":0.5941262990661297, "value2":5203993881422592468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.621587, "value1":0.9726557103153585, "value2":4065263619832011048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.936496, "value1":0.4220278511438201, "value2":537814119030374569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.476922, "value1":0.7453981222005723, "value2":3086149467770264289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.544208, "value1":0.1648752329298873, "value2":1528219915951406823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.480134, "value1":0.5864292944345278, "value2":1915038657623169872, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.634200, "value1":0.06085546420380519, "value2":1586772996290355012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.290157, "value1":0.4479966729250899, "value2":603667588545059551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.523397, "value1":0.8019975659364824, "value2":2263229113306312715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.260677, "value1":0.49446211956319075, "value2":5384294748496824064, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.040570, "value1":0.2634688640512974, "value2":7081899407358309984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.670159, "value1":0.4242108328120063, "value2":1003697559007887859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.181699, "value1":0.7357308047023233, "value2":7674299813279760629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.437960, "value1":0.34081697732260335, "value2":1366760825001014571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_5":"f","key_0":"b","key_3":"f", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.585340, "value1":0.9880202469237767, "value2":5993013748473116382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758920, "value1":0.528974137063132, "value2":5387464413435497334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_4":"c","key_7":"j","key_0":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.190405, "value1":0.24309504115260316, "value2":8699961182370479292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.045104, "value1":0.42560682798506067, "value2":3749517767528292571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.239270, "value1":0.11837048427737668, "value2":4852849985587057450, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.647950, "value1":0.24581023177518593, "value2":7946632180521405772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.969355, "value1":0.36754556888022905, "value2":7897267252653597746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.416692, "value1":0.9421287451192997, "value2":7881806374967586751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.052040, "value1":0.27660174901670026, "value2":5294738319793885165, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.112174, "value1":0.8996109424352536, "value2":5125031699148488781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.696416, "value1":0.42330986040349045, "value2":7061433961041368821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127933, "value1":0.6743423459526591, "value2":5703788929519053213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_9":"a","key_3":"k","key_4":"h", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.421658, "value1":0.6878542321395752, "value2":3553548366690488868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.697493, "value1":0.169783750302659, "value2":7055001682108205827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.550811, "value1":0.35231569219578235, "value2":1455572534223159241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.493018, "value1":0.08084254027780423, "value2":2900266808751124367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.949248, "value1":0.8847972628692381, "value2":4051840894997158475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.142802, "value1":0.3270345064861823, "value2":2685380632911712697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.004387, "value1":0.822755239334881, "value2":7165522026622587078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.620895, "value1":0.40285181555826477, "value2":7279952302942542040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.988553, "value1":0.42488091295486163, "value2":7195769969726296289, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.314916, "value1":0.24528560160904253, "value2":3412797507763244235, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_7":"b","key_3":"e","key_4":"e", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.740841, "value1":0.7678978195926474, "value2":7353627394146413053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.767489, "value1":0.9849856324971001, "value2":4180092134684532856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.681082, "value1":0.08407929345646925, "value2":2247545351887675892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.987608, "value1":0.3158564571415852, "value2":3372134575580371247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.953355, "value1":0.6808029749520165, "value2":6416401290847025754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.934118, "value1":0.3815229371385279, "value2":5398231187188543220, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.349522, "value1":0.008036202059570214, "value2":6632834525194225583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.852636, "value1":0.6082938917169154, "value2":2209452475312580755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.491711, "value1":0.7070391677298287, "value2":5867779644024249420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.329284, "value1":0.23697436114335285, "value2":2747492895656100802, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.217160, "value1":0.013558152161070156, "value2":543215008088311315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.376202, "value1":0.3967940240541657, "value2":4348396634707940652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.134419, "value1":0.2570940106942101, "value2":5495331795379375428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.693581, "value1":0.6383501980446401, "value2":7332225459965791621, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_8":"i","key_7":"h", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.843591, "value1":0.374550163160159, "value2":1957979143426115517, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.127314, "value1":0.16562410193780275, "value2":5514782476287105847, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.564210, "value1":0.47834402715548674, "value2":7864140566183633471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.673777, "value1":0.656108656648855, "value2":6919590747679310147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.071957, "value1":0.3923477172589352, "value2":7954657800672945455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_6":"g","key_8":"c","key_1":"k", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.497858, "value1":0.21899200976683347, "value2":5922902740218283725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.063776, "value1":0.819166087262725, "value2":497846112853863498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.039234, "value1":0.06262728093114268, "value2":976242972769343091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.658955, "value1":0.48196452599993433, "value2":2417102300700963835, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255288, "value1":0.13639741045424703, "value2":7715812700116132347, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.496112, "value1":0.1915457705996175, "value2":7008066197109595718, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.144334, "value1":0.011357361010319224, "value2":7600237475494790795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.129843, "value1":0.45829887966275845, "value2":1540110795133843915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.828068, "value1":0.8516996075016723, "value2":8405984298577097957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.870240, "value1":0.16467566190225996, "value2":1556056753506941522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.199050, "value1":0.7512917767548035, "value2":3201410658564101147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.830765, "value1":0.6115828253610872, "value2":1648965358200873623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.385351, "value1":0.44894635523974136, "value2":2127367263922899732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.555613, "value1":0.566989554108829, "value2":4275387093192470829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.758445, "value1":0.45045350568682, "value2":6540058212119502734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.051410, "value1":0.07872483972360628, "value2":7661444664176600116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.027756, "value1":0.1866583235803357, "value2":4402492796625360355, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_1":"i","key_5":"g","key_0":"c", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.043704, "value1":0.3399629891033194, "value2":2843620218251838442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.218394, "value1":0.6792705695737844, "value2":3717810518285734277, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.489979, "value1":0.2797515798899019, "value2":5603667040320572095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394844, "value1":0.47076120967369356, "value2":2551470543073926967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.176040, "value1":0.28649026862589944, "value2":8089352485313652968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.015062, "value1":0.8503661846365486, "value2":9115866206649167217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.867138, "value1":0.18222497145716, "value2":748036164126661849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.560015, "value1":0.16282129139411827, "value2":7758713833273415824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.782776, "value1":0.3488004874020465, "value2":715341283187680350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.547186, "value1":0.07214699982761937, "value2":8071660169425611932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.383243, "value1":0.8304505195235525, "value2":4079430116382080627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.551280, "value1":0.6676351538773236, "value2":4287835647581016602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.853493, "value1":0.3478301116617329, "value2":7815908195604525371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.511951, "value1":0.7049888722888338, "value2":1948135132689353800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.173192, "value1":0.5458720430431239, "value2":5871330428575335057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.413126, "value1":0.46553694703913845, "value2":7586060029018039815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.781439, "value1":0.705616760521343, "value2":1127274508185282406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.732371, "value1":0.37205742400318453, "value2":5213749986182819618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.569115, "value1":0.03375202087271923, "value2":118632103331380976, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.102614, "value1":0.2822023902913289, "value2":1391807523431966730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.140423, "value1":0.44611401204912965, "value2":828680952801675744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.085991, "value1":0.7893667946565419, "value2":8411361994319110857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.975471, "value1":0.7162325854468116, "value2":79462509030833666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.244787, "value1":0.40407729683075916, "value2":2829176965864438571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.304099, "value1":0.2959757161094293, "value2":5764137406549726855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.837106, "value1":0.4258452743236353, "value2":5341156407592196309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.707606, "value1":0.031525280019894485, "value2":868621351492758022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_9":"g","key_0":"k","key_3":"f", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.441177, "value1":0.3225860911008584, "value2":3070966375799941978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.077150, "value1":0.044429243112689185, "value2":8848387799149411370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.604969, "value1":0.8353491972869558, "value2":5277008629054064552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.537157, "value1":0.4103644112452139, "value2":9148697464094805661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.353178, "value1":0.8020014291715372, "value2":611786907029670311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.505264, "value1":0.10898025782239415, "value2":5937127177824419205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_2":"h","key_7":"f","key_0":"c", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.228410, "value1":0.43346447533111176, "value2":7102784791187505051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.321463, "value1":0.9635846235850585, "value2":7091075060854093514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.999019, "value1":0.15889354502430938, "value2":7074771352556045066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_5":"d","key_6":"a","key_4":"k", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.766619, "value1":0.4943151198279696, "value2":6293058242069710093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.382183, "value1":0.47579561094609324, "value2":2482575383850980092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.054382, "value1":0.7191240626399533, "value2":7914822293313169877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.255873, "value1":0.055171339528865125, "value2":1928364223437432733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.535958, "value1":0.40417404005201557, "value2":8512533852699686666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:01+08:00", "value":0.394085, "value1":0.8317795195864898, "value2":6061937875766891692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.752931, "value1":0.1811338646540346, "value2":3103290849364510613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.049507, "value1":0.11124247410560267, "value2":8253232345223333946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.067068, "value1":0.02792863936331565, "value2":1942738411033963266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746589, "value1":0.43603502537000616, "value2":5153565802559198697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903064, "value1":0.3556713649253779, "value2":3489353708315351817, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.856458, "value1":0.28933348763855665, "value2":5247367677027973150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397179, "value1":0.7130148119357395, "value2":5728198037062323847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.927222, "value1":0.7639936733901646, "value2":101990787192301191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005808, "value1":0.03609120161396754, "value2":9216428818715898202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812165, "value1":0.4866254594918793, "value2":3026112033227979237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_9":"f","key_7":"e", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420851, "value1":0.7402262167603453, "value2":6177484091405440598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836705, "value1":0.0336701243488239, "value2":4778937227074309153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097260, "value1":0.6626429057561898, "value2":7257306969130399155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496088, "value1":0.36910009322506554, "value2":3657880390271983286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870639, "value1":0.7067162570632016, "value2":1102103625009499222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.020747, "value1":0.46587247219742656, "value2":5967339025562916268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.724139, "value1":0.8163768770313428, "value2":2067500216739340556, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.369554, "value1":0.7671983408631192, "value2":4937837819229078201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382589, "value1":0.94426076786917, "value2":575895696952869704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115327, "value1":0.47575759482061636, "value2":2541949700573197783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260734, "value1":0.8252666210219215, "value2":5316874961818228049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.872358, "value1":0.8794099168533623, "value2":5800876034859982246, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.054141, "value1":0.05686166977091221, "value2":5301631175885078161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_8":"e","key_5":"i","key_7":"g", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759774, "value1":0.8076817854451119, "value2":2675452361460215935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921855, "value1":0.0467410376485931, "value2":7851926061831999904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578515, "value1":0.9029496688391901, "value2":3737429676030030627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812804, "value1":0.47887479306222563, "value2":1687173944192158201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472625, "value1":0.8441160499425835, "value2":1515286134892734672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760814, "value1":0.33274037254007843, "value2":1188458814270546919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788501, "value1":0.15942957594654944, "value2":2916736695606434573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098854, "value1":0.8030098284257553, "value2":2336601872331852302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894386, "value1":0.4139981184160349, "value2":7897864995270774175, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_9":"k","key_5":"b","key_8":"g", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117046, "value1":0.5674622805765219, "value2":4791162537839256650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.542843, "value1":0.19807908776500033, "value2":6807107659814067311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_1":"h","key_9":"f","key_0":"i", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703834, "value1":0.5788042510612239, "value2":5007564675868943302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945291, "value1":0.3701740719620502, "value2":2459541142030887319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070410, "value1":0.2627776276203505, "value2":9110732671786561088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.305776, "value1":0.24403108521016115, "value2":3139654072672113070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.417910, "value1":0.549110839930364, "value2":2406040915101429610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410216, "value1":0.8784609507591513, "value2":3467112924963980779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186300, "value1":0.13500098796508653, "value2":4869858265003017538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934357, "value1":0.7572145499314314, "value2":3619701318200032332, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871110, "value1":0.4402673304510715, "value2":5018886154821992035, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.493200, "value1":0.1348835909121323, "value2":3845890133642201057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875968, "value1":0.1364875892800979, "value2":570804136752641105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.469938, "value1":0.5776925075890766, "value2":8313002338216576665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.667608, "value1":0.9950302066492306, "value2":1913698187097493343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497144, "value1":0.7591363060077458, "value2":4304322015184203986, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731147, "value1":0.743025674257032, "value2":3480752580915944185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878760, "value1":0.5947269209561762, "value2":7998247484475194678, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.606309, "value1":0.663213778377581, "value2":3067341192985682212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.207494, "value1":0.8601914108563131, "value2":256790036338793553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.144417, "value1":0.30286956006124033, "value2":5377965606661311252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.134377, "value1":0.609971735912583, "value2":3774215546443285103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.434483, "value1":0.986170039019965, "value2":5064754870308124270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.509040, "value1":0.5522321797777098, "value2":5012812814115566068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903746, "value1":0.20330422291674133, "value2":6986910292987554191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282547, "value1":0.04694512830804489, "value2":6329857494201458927, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354275, "value1":0.16833606088021735, "value2":3136536256813456866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774256, "value1":0.5410085627082251, "value2":3587475793865363544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.762751, "value1":0.806841924275979, "value2":7358998298622408616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584082, "value1":0.6563492986452016, "value2":387593409104388354, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.080055, "value1":0.5509982288474451, "value2":6251069533288673484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695506, "value1":0.8632479367125148, "value2":2061552818267451584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489099, "value1":0.6265315870043634, "value2":4304737020249244240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098523, "value1":0.5339406052485941, "value2":4381041775949231103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945882, "value1":0.23785010902188283, "value2":6729562900414289120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.167154, "value1":0.3701120650676118, "value2":622381534969203518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775199, "value1":0.6184507954864329, "value2":5210419683928766855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_9":"j","key_3":"f","key_4":"g", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.730074, "value1":0.38121631002699524, "value2":419350096079979395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.013737, "value1":0.8747949109587233, "value2":1018960453244065142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.091852, "value1":0.629803885859078, "value2":859062102274314528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166974, "value1":0.47395530253799945, "value2":6411055007256508887, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.524472, "value1":0.9186585978945081, "value2":3492305084628232737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241236, "value1":0.3787867675534582, "value2":4760531059679736084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918712, "value1":0.05290704362542312, "value2":1966436853098488271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636045, "value1":0.9510956232200477, "value2":8704302710827007347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187946, "value1":0.6635616634234764, "value2":1433605612315108919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180096, "value1":0.173261531213523, "value2":3306945339384428282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.453459, "value1":0.11094008182846338, "value2":5724966973695631624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562127, "value1":0.3576118834730829, "value2":3463667757785807622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137341, "value1":0.6255329096036213, "value2":1381955744525597553, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913181, "value1":0.29162415237583095, "value2":2202030046648119531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.443734, "value1":0.3209051100037936, "value2":3274712051745788108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.302887, "value1":0.38589486316701666, "value2":1489273904023316120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205658, "value1":0.07820373463872678, "value2":7665901880569799873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.211623, "value1":0.32477633025580727, "value2":6808357456829544888, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.697819, "value1":0.26244647649977987, "value2":6232754250974380613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853360, "value1":0.3310597786987246, "value2":7565924568483154342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.135295, "value1":0.049613381396899345, "value2":6155724101399152110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701749, "value1":0.9716574010135762, "value2":3704214151049295904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527554, "value1":0.6532141955951636, "value2":2786469958733724605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866043, "value1":0.03422399382505321, "value2":5496039574170303988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.415307, "value1":0.682109162147481, "value2":4924263240012975638, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_4":"d","key_1":"g", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631078, "value1":0.7274897566111301, "value2":8520095366826456731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.549725, "value1":0.7077865773466269, "value2":2977543758339035237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.154150, "value1":0.14439730391589514, "value2":7810498955756317569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007570, "value1":0.8087681893470107, "value2":4017899853890170979, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.055629, "value1":0.44977291321386215, "value2":3453485470431150033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189056, "value1":0.012064619774392913, "value2":3564594888238224552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.545459, "value1":0.412297537176583, "value2":7021178975362987850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.116215, "value1":0.3276649448479836, "value2":3143364922694929733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529456, "value1":0.4708787441244093, "value2":2999813825732785635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_8":"d","key_1":"i", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.966668, "value1":0.04918146120370962, "value2":870818154766064140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601524, "value1":0.4723442759712491, "value2":5773596887056998694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712431, "value1":0.04260252713403969, "value2":4497318416983710614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452968, "value1":0.03699415334331326, "value2":3190104546355404213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.394341, "value1":0.05586282401938057, "value2":2019354346274899062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.084367, "value1":0.35383668982583555, "value2":14041413767842483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227017, "value1":0.7278732120270098, "value2":8901905120302064125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494493, "value1":0.0910027308098097, "value2":3614458258424175747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059547, "value1":0.2958045744232146, "value2":5813283620430694858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.423389, "value1":0.9221509815116762, "value2":6742495498273189642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501855, "value1":0.44198471706869047, "value2":7662744388645110665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293650, "value1":0.512834114916101, "value2":5984393328938374467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614743, "value1":0.8714098760733937, "value2":937734757736172502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406459, "value1":0.17711840072487878, "value2":749048739837121472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711112, "value1":0.5465711310503617, "value2":2943773951314932011, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335697, "value1":0.8515828942170133, "value2":7615557480020888685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548405, "value1":0.13026413723454033, "value2":4513263208882824306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.351448, "value1":0.28264554746804893, "value2":1701229548455066314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.501558, "value1":0.7474271542447126, "value2":4300562906938627060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107125, "value1":0.5220893826927367, "value2":3433346642469038441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968552, "value1":0.32921550916944375, "value2":4204134061287881453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.438315, "value1":0.5786006279022659, "value2":7016921001360584415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.834753, "value1":0.15159677478724748, "value2":3404421917920731295, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454269, "value1":0.5292362536342466, "value2":6241633082661695873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137463, "value1":0.4976726181585776, "value2":7634359338923847776, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744484, "value1":0.7709542443985684, "value2":2984932547540133289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.653649, "value1":0.40168424113567774, "value2":9208485552143870835, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.420303, "value1":0.8834331384252292, "value2":6602388659890449882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.995276, "value1":0.35489802779155194, "value2":3627299399432728520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.609717, "value1":0.32106453173709887, "value2":6997589541390821763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.764384, "value1":0.448976213692369, "value2":2970007228856945436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.548393, "value1":0.15204462152678552, "value2":7646037909450468928, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379101, "value1":0.7807002939639928, "value2":783999596388900221, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.278309, "value1":0.014487780492841778, "value2":1057643982300790379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.016491, "value1":0.40903973612790867, "value2":453932377401006766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347001, "value1":0.5622168932310977, "value2":5705011066230310463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.086911, "value1":0.6494793319729653, "value2":861844073039964568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.997904, "value1":0.8338228010696771, "value2":1349114457177401043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266213, "value1":0.43023507437229846, "value2":2397780341037792730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068462, "value1":0.6058806411980994, "value2":78381759723174420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358282, "value1":0.2854157607268251, "value2":9047800747480831869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845468, "value1":0.009988708005244632, "value2":2440593977353687165, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780415, "value1":0.6187822246822943, "value2":4482602300599359335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_1":"g","key_4":"k","key_0":"c", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700223, "value1":0.018684194191095932, "value2":6529685586040726217, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.792345, "value1":0.9111940168010332, "value2":1575039147844860051, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275696, "value1":0.01410734034670126, "value2":8535583193313743852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458977, "value1":0.23636701604224677, "value2":314843807550918715, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_9":"d","key_3":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711597, "value1":0.1315269488618045, "value2":914748291669479577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003734, "value1":0.48566417416368673, "value2":5809881749485776988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.587116, "value1":0.411572893699433, "value2":582776826400065747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000207, "value1":0.25509895950578154, "value2":5802214523620869062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082969, "value1":0.48435989736978513, "value2":7352735247711683766, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.042350, "value1":0.16157141460949973, "value2":1607874054697622469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.765374, "value1":0.15664293959132114, "value2":3675493379037113802, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.359224, "value1":0.887036256695556, "value2":7786059543942584229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293533, "value1":0.267785564160632, "value2":2723125994900979360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964294, "value1":0.716239726237387, "value2":8651041612345523274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076213, "value1":0.9749455558859894, "value2":5297461003834502762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.896241, "value1":0.7102104646771853, "value2":2795778846362770396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903120, "value1":0.14414851964547873, "value2":7334142904098197, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.843431, "value1":0.8361555205773405, "value2":7976205336110881786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.731615, "value1":0.4179919061811569, "value2":1554283199870929425, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.953374, "value1":0.7537780463182495, "value2":2591672750977463865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400999, "value1":0.7471726171159019, "value2":7300350647929857982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232296, "value1":0.7720730243870363, "value2":1060384392061020703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219798, "value1":0.16805543570986609, "value2":1462620146278820176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364330, "value1":0.7255148218772242, "value2":713753994079127610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152830, "value1":0.13410318940561705, "value2":9140011796352402602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585566, "value1":0.4130103851752984, "value2":2970454230901035575, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.718094, "value1":0.5500539331076559, "value2":8305617631847453693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.624058, "value1":0.9500416769324029, "value2":2935568409193402069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539303, "value1":0.05177136725388866, "value2":7442175860869173746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.172986, "value1":0.4456449462659514, "value2":137655062794989901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631420, "value1":0.9356892224700029, "value2":8946945917645913041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948627, "value1":0.9988907819529511, "value2":7978995836658846886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946323, "value1":0.5613474012283862, "value2":648588536937617868, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361054, "value1":0.4585620050578102, "value2":5573215420419545492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.817729, "value1":0.3241298418342054, "value2":2657202509964663855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_6":"i","key_7":"h","key_2":"b", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713670, "value1":0.7338301625220077, "value2":7217765288314019300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.521076, "value1":0.5030076209647331, "value2":3777966789579493364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.855317, "value1":0.7172152098421418, "value2":8060266815999977878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849239, "value1":0.9561486509896251, "value2":266727525021228946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008199, "value1":0.11740640397171373, "value2":3689690733597943394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936773, "value1":0.795334335887194, "value2":4733404788345141419, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_5":"d","key_7":"k","key_1":"g", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888974, "value1":0.9298030625919304, "value2":6088282595701162644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247054, "value1":0.1558941325474189, "value2":460928608228918469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853182, "value1":0.8511278083029166, "value2":5598797292292719359, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.252753, "value1":0.16156992511009938, "value2":4368635811765896864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309449, "value1":0.5849973411963314, "value2":683197153873995001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.222187, "value1":0.6637363404506509, "value2":201431956551451886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805299, "value1":0.03082292423828661, "value2":3135350466088347105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489193, "value1":0.47023015827549575, "value2":301183049195776618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_6":"b","key_1":"b","key_2":"c", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.058774, "value1":0.7620086190415307, "value2":2178333587571299612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.098933, "value1":0.696442402358351, "value2":427216344556980093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497026, "value1":0.18415993654905655, "value2":8786493859199944811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257640, "value1":0.6674105512961352, "value2":395753979079709412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_9":"g","key_1":"d","key_4":"d", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490944, "value1":0.5114071451614219, "value2":2042645288775439509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455445, "value1":0.7431063214848364, "value2":8165863124378358117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.796800, "value1":0.6414858212054014, "value2":2442066548061613486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616766, "value1":0.17517079750483502, "value2":2134452812266479738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060207, "value1":0.5103041206672632, "value2":6515531846051600666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677324, "value1":0.2542456767596465, "value2":3968114223247313347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.487523, "value1":0.6903207412936282, "value2":1424416428597884416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601220, "value1":0.7953800597000228, "value2":3247496941215159385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.642959, "value1":0.2541441152051556, "value2":7121652419723619875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268832, "value1":0.025435318199453064, "value2":8277309445690585191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591576, "value1":0.98502525758626, "value2":3261712292707439368, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696131, "value1":0.602844278741062, "value2":9150721424292798196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.799050, "value1":0.6397730903214256, "value2":322841440821332477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.703698, "value1":0.7179811740811757, "value2":6911091771320522730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.276968, "value1":0.16623158682521033, "value2":5763688015485978321, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.932237, "value1":0.9953773456479392, "value2":1648531648235955285, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.562810, "value1":0.4786357292651346, "value2":3349388012721867031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496221, "value1":0.10819101220823944, "value2":5211051877655840528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720431, "value1":0.552698162040604, "value2":7021843927329324470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.087213, "value1":0.8472571863880398, "value2":7745061885728922306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.039167, "value1":0.21533794157440472, "value2":3121229544727240655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.309356, "value1":0.1590321315931928, "value2":710479249184112134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.191661, "value1":0.040375310871212516, "value2":1055365248220450266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189905, "value1":0.8222460270476257, "value2":9109111690041797580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517425, "value1":0.7168387126170568, "value2":6945048447223853682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916881, "value1":0.37193142727878464, "value2":2747126651524473231, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.395094, "value1":0.6414572510838905, "value2":2905014786439561176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.163189, "value1":0.7311236407483467, "value2":5070208982215207577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827333, "value1":0.19235777061627696, "value2":1042609530408390018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.107479, "value1":0.11539837467269841, "value2":7629188841800355160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.186100, "value1":0.23799692868617986, "value2":3114291156883869525, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790442, "value1":0.4974549500784167, "value2":3142531162087829163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.797183, "value1":0.252808246395566, "value2":7867371410034903023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269019, "value1":0.4459897405815389, "value2":5769972800266784623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299688, "value1":0.7900084672305991, "value2":4400086899085085411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.295251, "value1":0.4992500267841033, "value2":8698415822608825668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.992903, "value1":0.9703688948654993, "value2":699484569689785049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694010, "value1":0.24590491974037657, "value2":6378096972799959498, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.916779, "value1":0.03667685970441408, "value2":8236016681410946158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.018930, "value1":0.7762719211488223, "value2":5653992369569699642, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673243, "value1":0.9005498092358467, "value2":2541259280078576557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.403564, "value1":0.7066442393226117, "value2":7468478788293853391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912162, "value1":0.4479784585493673, "value2":2451129253364350697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622846, "value1":0.6985884362308626, "value2":1822728881940552123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707469, "value1":0.052057120254889885, "value2":640515677830640949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433576, "value1":0.4097754058261397, "value2":2475392911974322330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374459, "value1":0.1353067477342801, "value2":7368967313707141232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_7":"k","key_3":"g","key_5":"g", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993651, "value1":0.06711618906287087, "value2":1750993237587336816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.219084, "value1":0.5316119999916927, "value2":5790597415731545344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527236, "value1":0.6470232539834205, "value2":5706809590317379695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566174, "value1":0.9745838789323467, "value2":774234656544468494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803752, "value1":0.8708197206158679, "value2":3775873873449515760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.388879, "value1":0.4862649086023325, "value2":8962983929778901050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471944, "value1":0.25578979603582636, "value2":7732635688969517720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017485, "value1":0.4968624551232297, "value2":202940911271524303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_8":"c","key_1":"f","key_3":"b", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.333780, "value1":0.8299780273551013, "value2":895214629529935810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.164082, "value1":0.9321061863901389, "value2":7407307236544459993, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225068, "value1":0.9163062543062077, "value2":8739271132887488724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304547, "value1":0.45077586944816295, "value2":1933302507685789986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.538110, "value1":0.4128481092985977, "value2":3178182543294139274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541173, "value1":0.7483924955860858, "value2":212944257013368531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255386, "value1":0.14438260405328707, "value2":7645966424158688784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.073000, "value1":0.7628193459385832, "value2":4973597494190231390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408853, "value1":0.5564603800067697, "value2":4401411283807625564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447512, "value1":0.8939878640344981, "value2":2693396599053914337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048017, "value1":0.34230394362828026, "value2":3182689568942606164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057151, "value1":0.8042880864594656, "value2":2051969816473375198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746476, "value1":0.7332341667442952, "value2":461488002348760082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_6":"d","key_1":"i","key_4":"h", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.180890, "value1":0.31049348146155, "value2":1020023460971269956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550108, "value1":0.4362790210713837, "value2":7901529415931806303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345598, "value1":0.11823802615751057, "value2":8356166230167153323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804625, "value1":0.5643553567149021, "value2":2154734444228835687, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_8":"c","key_9":"d","key_6":"j", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247313, "value1":0.6975496564473892, "value2":3820052334665018388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.805735, "value1":0.31084346436090265, "value2":4462974662487547374, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364439, "value1":0.1744566217814337, "value2":607910964577919864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589313, "value1":0.37010824428777245, "value2":5796079422541877022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931293, "value1":0.5772458152930102, "value2":1027336122468786461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527679, "value1":0.2977670251954975, "value2":3769527680254605539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.404475, "value1":0.38496439646172187, "value2":2284351654690866371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772590, "value1":0.07775425303960136, "value2":8477700234367515830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722168, "value1":0.17927054370632736, "value2":7192360571641392544, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.712775, "value1":0.576069645932529, "value2":4594581975868935835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.126652, "value1":0.5061094013712955, "value2":3728550795113093627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321145, "value1":0.13357161718693558, "value2":588806792143157793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.101550, "value1":0.6394371793311785, "value2":3045802217092318405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.547834, "value1":0.5303880277608923, "value2":244812322497139670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_5":"f","key_6":"f","key_2":"k", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.022447, "value1":0.813105541632356, "value2":4932627202785941294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_8":"b","key_9":"k","key_5":"b", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382289, "value1":0.3508103161551219, "value2":2376325264202595095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.611464, "value1":0.9618639190404236, "value2":2197184281244482985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.911985, "value1":0.3625352195059055, "value2":211893969967676184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.268188, "value1":0.6899041268799436, "value2":3263903700628732234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229932, "value1":0.7830409765107325, "value2":3581295224187821255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921675, "value1":0.15132831757259205, "value2":8941349375775958121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.738768, "value1":0.28818505448700765, "value2":5311365998633458584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.670459, "value1":0.7891122563117983, "value2":5569764792017894880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.893664, "value1":0.863874518524225, "value2":4812640494718764785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484101, "value1":0.6065100518640419, "value2":3350972220626617309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048572, "value1":0.15248106917852358, "value2":6593923637797377418, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.043823, "value1":0.6456350127530551, "value2":3888964250592362865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109495, "value1":0.6845812758644145, "value2":167613517010834758, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.328450, "value1":0.42643027428907515, "value2":8129659398848977292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.429304, "value1":0.14015794848590404, "value2":4681962038728020905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_8":"g","key_1":"c","key_4":"c", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.776267, "value1":0.3289665984746857, "value2":3318083720052608693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496512, "value1":0.3624675397742157, "value2":6289410424653796398, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441324, "value1":0.9094233421689666, "value2":2912289985138568601, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.650459, "value1":0.3752516427539381, "value2":4398293649161961566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.350383, "value1":0.1741603700667637, "value2":3124160867383954120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.051974, "value1":0.5242492694985263, "value2":6760802540060503552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_9":"i","key_5":"d","key_8":"f", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.291840, "value1":0.5576830705012547, "value2":4155187805029620559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.853505, "value1":0.13344813988799406, "value2":7220245991757698014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_4":"b","key_1":"f","key_2":"f", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.480822, "value1":0.34390971406763976, "value2":4047856794594370906, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192205, "value1":0.46496172552445103, "value2":2904944897204510913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668616, "value1":0.9800566015891223, "value2":3768818727218430516, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.940469, "value1":0.8502278446477077, "value2":5643864232780485221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_4":"c","key_9":"i","key_0":"a", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.775796, "value1":0.26560945264856717, "value2":4386556505603475589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_4":"c","key_9":"g","key_2":"f", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484224, "value1":0.10363743205437767, "value2":7594526116903338826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.889136, "value1":0.4627114619785784, "value2":4680730415934145352, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739229, "value1":0.1719548250883568, "value2":6803717844294234894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_8":"i","key_4":"f","key_6":"k", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.057765, "value1":0.3081555912049451, "value2":8499651141401941047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_9":"f","key_0":"e","key_4":"f", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.450693, "value1":0.8168675860307004, "value2":7949382588313472478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770167, "value1":0.272960033553786, "value2":1543554906153054869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411881, "value1":0.5252975573521564, "value2":7223448923269287286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.892105, "value1":0.15152104472471403, "value2":3531358618763240055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.882618, "value1":0.45516649656010977, "value2":2288125701245215412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.913392, "value1":0.6620512228658078, "value2":4363674990969315932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.835393, "value1":0.7513525395048825, "value2":5883431192279251653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.563057, "value1":0.9008981984383352, "value2":6465724162584986904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007001, "value1":0.9502622268086611, "value2":6989543064471041687, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688255, "value1":0.07451084769674933, "value2":2957263551734111550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751629, "value1":0.2617947338456188, "value2":684618542420066244, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930787, "value1":0.6308597904360509, "value2":6601900249320434792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.363243, "value1":0.8139409689083822, "value2":1497160325123262680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676085, "value1":0.9711732869605988, "value2":1463546357374490230, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.161858, "value1":0.6374277110955211, "value2":6435065381978315913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757941, "value1":0.910443361797238, "value2":2854035357843167779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.331899, "value1":0.6895825352637633, "value2":621471764809456095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473746, "value1":0.2844187612168192, "value2":5704986401969616545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048060, "value1":0.9984260267119236, "value2":4771344600707329825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.030112, "value1":0.9637634957448155, "value2":5568130719531472451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.257160, "value1":0.5940271387669745, "value2":142865511754737819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760326, "value1":0.4334581523321684, "value2":6413870941968139402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827687, "value1":0.8136794802681319, "value2":746965665674196204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447680, "value1":0.3168283761550826, "value2":7144161377877398366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690596, "value1":0.3996428785716089, "value2":3598801694226963836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.009204, "value1":0.21753660582669873, "value2":1790271338088923331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.432520, "value1":0.8760470338201594, "value2":2329654518443264597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558450, "value1":0.5241764449812006, "value2":8062760419700144076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.999165, "value1":0.4211253053940594, "value2":6241537222130424287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.492382, "value1":0.19419908429465416, "value2":8132562796409382183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.968488, "value1":0.8740668460370395, "value2":5462263580744941212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566020, "value1":0.3116629548456918, "value2":1660846180499855539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899292, "value1":0.4455952655758791, "value2":5641720897227183391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213999, "value1":0.7107285930376039, "value2":320006539559036783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177859, "value1":0.6189254938745993, "value2":3916861895859871884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.942059, "value1":0.3464916414346185, "value2":1739217875853687994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637964, "value1":0.4027137396066144, "value2":498622351647583844, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994610, "value1":0.7530828834500288, "value2":5890892402021233384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.727487, "value1":0.28182955033112955, "value2":4630833361364112333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.915590, "value1":0.5470194104324091, "value2":1666728635470642191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757932, "value1":0.5371051329548212, "value2":1493513093655584392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.254302, "value1":0.06977656180754654, "value2":5979425073849108121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.475877, "value1":0.8550799756217883, "value2":624522672976326814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.247089, "value1":0.39743079305559637, "value2":4830779617580090488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015595, "value1":0.2577347123153436, "value2":2406835300488043720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.945719, "value1":0.3129335496013211, "value2":820286347352483926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081969, "value1":0.6051672450326654, "value2":8261558360059795606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948207, "value1":0.7815897187663656, "value2":3429909037501105433, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.935696, "value1":0.909809980891043, "value2":6385383912360129741, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.248852, "value1":0.9247408795385116, "value2":9175721939467544545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.934738, "value1":0.6334884200328613, "value2":327826698570505025, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102837, "value1":0.44571438142580455, "value2":4747306485524936018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.885921, "value1":0.35994074928816305, "value2":1395907621788196782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_7":"a","key_5":"i","key_6":"e", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.949368, "value1":0.7895021421389461, "value2":7949753331449968910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.633040, "value1":0.5458972679689471, "value2":4245189254820710416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.150358, "value1":0.7548359500337994, "value2":8481176564399105584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.557167, "value1":0.37427466994480085, "value2":1133639012241894316, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.093134, "value1":0.29043113295756745, "value2":6947797053182077820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_7":"d","key_1":"j","key_6":"a", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.136231, "value1":0.7872857259243544, "value2":2466956534940008420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.329501, "value1":0.01079935258997052, "value2":4968522926518293278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.825131, "value1":0.6767719760068325, "value2":8529334985450219291, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807971, "value1":0.7351117134307579, "value2":3019346910002173461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_9":"k","key_3":"f","key_7":"a", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017959, "value1":0.6253725541571399, "value2":8835344453005145545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_1":"f","key_4":"j","key_0":"j", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.280599, "value1":0.16481336771423033, "value2":1417072867431315310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673036, "value1":0.8178932673930933, "value2":6530240293167023011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192297, "value1":0.7505738676572953, "value2":6861957449422419505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517236, "value1":0.639000099787322, "value2":5815570450456127187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739883, "value1":0.17493563192408712, "value2":7179940437785343272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.531912, "value1":0.08360001327419847, "value2":2755543678440921421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.452307, "value1":0.8726532834145122, "value2":187164282595967676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.517272, "value1":0.9444444024596198, "value2":5607680802025895328, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289453, "value1":0.15619031798053584, "value2":810705789880173134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397508, "value1":0.9544477853731341, "value2":7641027031682219689, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.812107, "value1":0.9448725135844366, "value2":5145031592575451663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.063548, "value1":0.922662738511914, "value2":2569888172146766439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.959005, "value1":0.7947728545231653, "value2":4794325670292386969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.570203, "value1":0.9426031560649749, "value2":8759901536059040166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.514218, "value1":0.47519036709767065, "value2":1247765910819397634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.541370, "value1":0.3774216285225394, "value2":7667033726860743169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.938282, "value1":0.020047520852319107, "value2":7849085976338159942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874020, "value1":0.47048043771282444, "value2":2196635070664843443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_8":"c","key_3":"a","key_6":"e", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513588, "value1":0.012232986779526358, "value2":8478348320905895865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109700, "value1":0.7338145643706604, "value2":6797416603289211588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.488779, "value1":0.6031093288566471, "value2":5300319596464382751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.931594, "value1":0.42339447548233033, "value2":969762807855801088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.553860, "value1":0.676076748018672, "value2":1192139536623982501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.132006, "value1":0.6728558661354047, "value2":1380355336108501777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.669226, "value1":0.1577792131508693, "value2":4512376080159929307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.921683, "value1":0.24615483388690507, "value2":5797393083888487885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.800085, "value1":0.768303750055719, "value2":6971653838071612447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468961, "value1":0.45803974634578515, "value2":1199483949895079721, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356489, "value1":0.49471640885346424, "value2":4138046086256944727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225992, "value1":0.6741558407603926, "value2":5216648174057731100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_5":"h","key_7":"j","key_1":"i", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.779458, "value1":0.11692394331277675, "value2":9011198363894466670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.683290, "value1":0.48772002686762633, "value2":7212512979389072906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053472, "value1":0.5939009672642634, "value2":1146669122452133702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790089, "value1":0.8261492714018603, "value2":6290707817110359790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145153, "value1":0.6394833078139336, "value2":7849898590496421466, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.367040, "value1":0.6490663681468899, "value2":3967326627567255091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_7":"k","key_9":"e","key_4":"j", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.930388, "value1":0.6376629127038135, "value2":3197193413798224716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.471623, "value1":0.06021884177520976, "value2":4773928340410962251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_7":"e","key_1":"c","key_6":"i", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201531, "value1":0.7136591237382468, "value2":6260011441045155926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064553, "value1":0.2818642070197672, "value2":2495939851413003901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.282911, "value1":0.1452427125541155, "value2":8365230104234658997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279591, "value1":0.8354266947002554, "value2":1254307733558365753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849657, "value1":0.6912085697845711, "value2":1824933885382646361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_2":"d","key_8":"b","key_1":"i", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.092000, "value1":0.053974617818740134, "value2":8465803905922172931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408792, "value1":0.3821176162488874, "value2":4135301501032886839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.728999, "value1":0.9460512775720927, "value2":5232062325949200093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.706765, "value1":0.05808795299595995, "value2":3187473873599955497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689611, "value1":0.07108495627051868, "value2":8720638688812490559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.686555, "value1":0.11535487696002504, "value2":3943663806047625686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815986, "value1":0.6951101939977824, "value2":2103945278295322543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991212, "value1":0.9405521412873721, "value2":3787480236551923641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665833, "value1":0.21165663807431181, "value2":3904059491287927557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_6":"f","key_0":"i","key_3":"h", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183315, "value1":0.5795855388985459, "value2":1854253684581147206, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344965, "value1":0.8721578775869344, "value2":2246100236955108961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671953, "value1":0.31034394563958956, "value2":193813594216774336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.127315, "value1":0.4080665667532778, "value2":360930115537783747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.947191, "value1":0.901998732832304, "value2":1708333379014319670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894585, "value1":0.9195635726221545, "value2":8084121448593915190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004792, "value1":0.9591461919610487, "value2":8675745838539187803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900181, "value1":0.4274569433819623, "value2":6055452196459435249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.948861, "value1":0.954396067309185, "value2":2700832246849384276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_8":"k","key_9":"g","key_3":"h", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656208, "value1":0.4983661330975837, "value2":3291693517576289438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_4":"h","key_5":"c","key_1":"a", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.937253, "value1":0.2620189984734357, "value2":8181737649910995938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867100, "value1":0.06224158128949376, "value2":4580692356241748765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.827749, "value1":0.03871136162211054, "value2":7089705182458313228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241402, "value1":0.4482242754297532, "value2":9114765255689214165, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.602059, "value1":0.29735202117340814, "value2":3162172555777612351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.249371, "value1":0.23918886296603434, "value2":9067747281684729145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725148, "value1":0.8102090769702104, "value2":4626069739522513533, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.928403, "value1":0.3838571333237863, "value2":8647736244601572769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358211, "value1":0.32922133279873583, "value2":8174411590575614648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347825, "value1":0.7167150468238062, "value2":8311537141574084420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.875897, "value1":0.26919747077534717, "value2":9053609159881163713, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392772, "value1":0.2800052196883224, "value2":7203447031464069407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.849662, "value1":0.5009846634201038, "value2":939832331440858045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.024201, "value1":0.9391704477843268, "value2":2885798586864811264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952269, "value1":0.2664477356829927, "value2":847075590517072325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.307777, "value1":0.6456685214669929, "value2":3218983200036055999, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_5":"a","key_9":"h","key_3":"j", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.832715, "value1":0.2591746701265793, "value2":3426357136923132270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_5":"c","key_2":"h","key_4":"c", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.510375, "value1":0.5688430382533638, "value2":7025749889591390600, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.850762, "value1":0.6438552947023466, "value2":1197489157433338286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_6":"f","key_1":"j", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.905930, "value1":0.54095928495375, "value2":67603635153932551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.636437, "value1":0.5854506016456503, "value2":5128251159570110333, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836009, "value1":0.4067978187295729, "value2":6225162557978433985, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.781439, "value1":0.8809907639802731, "value2":1945481468549859562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722225, "value1":0.6928638550983661, "value2":7984089816823103368, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631076, "value1":0.0305606186093776, "value2":3704333676945193488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.515877, "value1":0.03574457632343526, "value2":8116748669267665996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.062029, "value1":0.8880344844442181, "value2":7205869492721817899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.753934, "value1":0.5309770343574366, "value2":3276732533413239990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598973, "value1":0.9458250274999828, "value2":4588112807086674988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284154, "value1":0.6864088859089572, "value2":8516198907569397198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.337595, "value1":0.45387405891882765, "value2":2878690183576728505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743636, "value1":0.5411320007572493, "value2":7926291410401458702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751569, "value1":0.29015226479219164, "value2":550259652101408680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_6":"b","key_9":"f","key_4":"g", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696737, "value1":0.19318283293628083, "value2":8059270019889405703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558974, "value1":0.5527141281446059, "value2":4981805365285955545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.499157, "value1":0.3593055667554787, "value2":8763899128009608550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.725437, "value1":0.38132949820890993, "value2":2593622500148824140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575337, "value1":0.27925006017882537, "value2":4225105729105614711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.168187, "value1":0.8455269167046058, "value2":4706079022430056900, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.223652, "value1":0.14768410638621007, "value2":2848574409182395374, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.655719, "value1":0.26992469602574687, "value2":4754721503564673004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972901, "value1":0.22308996900209793, "value2":4121883961065445917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513268, "value1":0.763713042109811, "value2":1389619580667003266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335708, "value1":0.9988121809461258, "value2":8201212630370607829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.874837, "value1":0.37662051730821194, "value2":1239286216512679885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.761409, "value1":0.08352300660036191, "value2":788197175065166874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993753, "value1":0.08212627471454952, "value2":2431495613386540436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.513306, "value1":0.09225555422185021, "value2":6434312790658522957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.899387, "value1":0.2747071960330611, "value2":3838090571407685488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498947, "value1":0.19881833202736185, "value2":645491056031774982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851971, "value1":0.9432399401550566, "value2":3792284814302154775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.912337, "value1":0.6898029391029513, "value2":8695381256661600831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.060802, "value1":0.4672690888050405, "value2":509110642459148760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.988275, "value1":0.6949519126514042, "value2":2125445866842945317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.537780, "value1":0.9644299177479598, "value2":2844641921809856363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.129171, "value1":0.2870831250353682, "value2":5356758362083554180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.699225, "value1":0.7476204022496519, "value2":4681919372415529338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.798856, "value1":0.5178345021294356, "value2":8950909328927419020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.356529, "value1":0.9689016091606751, "value2":778967583500360424, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.740297, "value1":0.48229504021118547, "value2":319778397891873168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.082023, "value1":0.8672498335471771, "value2":2761738695533665878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.737582, "value1":0.21539065762206674, "value2":1173702480391317018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.203204, "value1":0.04428967569336166, "value2":6796467582468653168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197343, "value1":0.5038157716364858, "value2":5419554006840711744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_5":"g","key_6":"g","key_3":"e", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.166454, "value1":0.4520211285816449, "value2":4140960458683685597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.810425, "value1":0.9556399667218088, "value2":2345821074968960700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708611, "value1":0.36562484484170277, "value2":7482183983302020104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614909, "value1":0.010690625631515951, "value2":4802163912353727023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.529200, "value1":0.23500509922901588, "value2":1518336865591090789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957588, "value1":0.8999494296908893, "value2":5272580661109552970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900198, "value1":0.3844554073160095, "value2":4170365040112200144, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.579116, "value1":0.16747277518786968, "value2":6804977291625507700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250708, "value1":0.8577877748125339, "value2":590691746141274872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721810, "value1":0.7147800394287138, "value2":4035652902850373019, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.922402, "value1":0.0627394141655773, "value2":4185528067707010977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.198128, "value1":0.08114002676829289, "value2":2239252166892691326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.803033, "value1":0.0019520065249596544, "value2":1312973954738137976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.174637, "value1":0.09909477348410087, "value2":93134409084713078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.250984, "value1":0.49581172361364056, "value2":9059996107352612477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381289, "value1":0.5781636662002811, "value2":3051612121774242926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.046509, "value1":0.758530400177533, "value2":67989907532933127, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.933345, "value1":0.9339175035199608, "value2":377586629316226209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963510, "value1":0.7004527478348754, "value2":1390034556068597934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918093, "value1":0.17000680115237268, "value2":4215018843369459603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.569630, "value1":0.20175634343868196, "value2":8381338885007503558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.883668, "value1":0.06695323081792075, "value2":1589024156310802370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.178981, "value1":0.37314887727591756, "value2":518499450705948775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.187208, "value1":0.4599180344115974, "value2":4887550148987301900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952934, "value1":0.8319538072999415, "value2":3671167315474130299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.588275, "value1":0.5426790448893685, "value2":6523903627951338788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.128001, "value1":0.37530546288075156, "value2":5616976404881722053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.004862, "value1":0.8346907316991129, "value2":4019033006145719569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826250, "value1":0.5254457071340125, "value2":972903594117286740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.344623, "value1":0.6999472357226462, "value2":1701492823670778543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639041, "value1":0.2248828857383105, "value2":4810585903388633809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.571165, "value1":0.22946219998225406, "value2":8055176566662404630, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117315, "value1":0.7792786347828345, "value2":8503346841194050818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.646178, "value1":0.29610828190559946, "value2":6707117461343778078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804384, "value1":0.8994527553958773, "value2":3055603759085197982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.503237, "value1":0.038643663095115535, "value2":97929435615191250, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.917565, "value1":0.06570063912325382, "value2":7659557506835508250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.194133, "value1":0.7280519586189449, "value2":607993872873285299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.197005, "value1":0.34702978428971004, "value2":2119181822487481024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.578715, "value1":0.4531709992663944, "value2":1719249508626109183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.221581, "value1":0.5476991652235295, "value2":8006956843834072386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_1":"c","key_5":"k","key_0":"k", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.177885, "value1":0.5137634978637854, "value2":5009892345362966157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596214, "value1":0.2368591303155911, "value2":8792401284340375040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182004, "value1":0.5444756866358181, "value2":1210706677095755681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.972249, "value1":0.9909726775988351, "value2":1108183309913410572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225911, "value1":0.47194616841751835, "value2":3490783413049237610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.321843, "value1":0.5905642407899484, "value2":4893835600551421123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.552992, "value1":0.8654702264058397, "value2":5106927081449112108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782381, "value1":0.8285431175840194, "value2":9168936255854371872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.410849, "value1":0.31656794437122665, "value2":2071986113321022167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.780581, "value1":0.051832044232056354, "value2":6484981440389875558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.072700, "value1":0.698857333675772, "value2":3443549406704488268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689951, "value1":0.5808073564555776, "value2":5048983612386176894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.733768, "value1":0.8029317643963917, "value2":7359595780765172048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.364358, "value1":0.34011958562739014, "value2":999331274737013831, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.512816, "value1":0.34358105240557685, "value2":6714721903573983722, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711860, "value1":0.9821515464895118, "value2":5477696623940889135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.857930, "value1":0.697732523770638, "value2":6661494860267552531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_8":"k","key_4":"d","key_5":"d", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114607, "value1":0.8236051117026364, "value2":4748182480100358673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.628228, "value1":0.168082038026373, "value2":7102000252406916356, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.687012, "value1":0.3638576608952103, "value2":8540460395700338240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_9":"c","key_2":"k","key_5":"b", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169601, "value1":0.3111748596945155, "value2":8571165720141489367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.498080, "value1":0.6187426468242411, "value2":2451438877433776073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.866907, "value1":0.5131483959448067, "value2":8869138925582259535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.396171, "value1":0.28292656018705137, "value2":5391262893377663378, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114785, "value1":0.0649996585433473, "value2":3978180949228199287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.671381, "value1":0.13833508369091968, "value2":5430930553797657293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.923816, "value1":0.8519197270200854, "value2":9127083562371495095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786218, "value1":0.02969493094352536, "value2":7671441472993176714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.774561, "value1":0.2102387127527105, "value2":4039185760762390304, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192344, "value1":0.144819186927003, "value2":378491549245728191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.076506, "value1":0.3187033255519536, "value2":8885020385104236250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.213717, "value1":0.8128476179407741, "value2":2181064442818506560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528442, "value1":0.5936064804578587, "value2":5648031374498714859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.767476, "value1":0.1348520099403021, "value2":4074505568425329493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.449677, "value1":0.09144593933086395, "value2":8719212717808997934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.411794, "value1":0.8990540517914553, "value2":8624156888058647365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.457637, "value1":0.3197628975782331, "value2":16763285997559821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.330356, "value1":0.8095806602517515, "value2":4384365921475656332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.584148, "value1":0.16829289997155406, "value2":380241693738836834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455567, "value1":0.7529538853574959, "value2":1681989614930760275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.844936, "value1":0.6074576588778431, "value2":7722066461223215119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926267, "value1":0.5797545736876416, "value2":7883616781252732331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238789, "value1":0.014736618507623488, "value2":4669225171297242340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_9":"g","key_0":"k", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361896, "value1":0.17876601404118195, "value2":5115665659478830691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756049, "value1":0.04403510631259223, "value2":7660735889331080164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.031957, "value1":0.33256068212920453, "value2":4453878565561794031, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.828157, "value1":0.39237914319665085, "value2":7249969180646160990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.952636, "value1":0.372690613847562, "value2":6192025452112268327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.991675, "value1":0.5494769946982794, "value2":90791903671198766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240830, "value1":0.9786829569747533, "value2":5978162907079513697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.279525, "value1":0.1494973578442592, "value2":6869066553359800738, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_8":"h","key_5":"d", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.939830, "value1":0.37432428722372013, "value2":4226472512491590319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.564559, "value1":0.7159468168647539, "value2":1512961341899043326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_7":"e","key_6":"k", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.048658, "value1":0.5401236641884074, "value2":8347872809477134111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.340095, "value1":0.4533223235157631, "value2":1670286383442367211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293295, "value1":0.4258630060523724, "value2":5674260405699571928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.239916, "value1":0.9150796129317534, "value2":8474870548791768473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744309, "value1":0.6470478918019383, "value2":7704476841600519211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_6":"g","key_0":"c","key_5":"f", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458393, "value1":0.05603920506738755, "value2":1942285693271412968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.720119, "value1":0.6222144605927992, "value2":5009900948732974588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435194, "value1":0.6528463342414516, "value2":2372394551953155351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.847658, "value1":0.12821161771950726, "value2":1481519210091358924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.083324, "value1":0.3648356544882518, "value2":15076769986125901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_5":"h","key_0":"h","key_1":"i", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.696866, "value1":0.5460078372418026, "value2":8482317071463194706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.878357, "value1":0.9158893232321995, "value2":4112830082593015136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.815751, "value1":0.7832798664333082, "value2":6621546193633794291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.036509, "value1":0.7258315650178256, "value2":2697123926599878172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.786542, "value1":0.050969235189444476, "value2":960522431282362262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293909, "value1":0.1637483242537312, "value2":4063809715912746117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_6":"d","key_1":"g","key_2":"e", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960513, "value1":0.15412859037505927, "value2":3609332046279378552, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.845902, "value1":0.9308703210994502, "value2":6609460372849170625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750261, "value1":0.12234055832157559, "value2":8912704681070762337, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.458834, "value1":0.9318510467298435, "value2":8026158416345347023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859979, "value1":0.9301579137721765, "value2":7925354941740095610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.783146, "value1":0.4471773356574157, "value2":8481603802647763313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.505375, "value1":0.5948200403445978, "value2":52874652951483269, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.005495, "value1":0.829951436170396, "value2":3721373420722024974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.867411, "value1":0.6049905245498157, "value2":8221468316724652544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.435857, "value1":0.8205378674820365, "value2":4321340481613778781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.618588, "value1":0.6552813364407445, "value2":3804139748589484208, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.027076, "value1":0.23841518035800105, "value2":7013010516589186218, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285111, "value1":0.5613630899205873, "value2":1517169132984045486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117978, "value1":0.3226349150452925, "value2":5930250973192218816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238673, "value1":0.398077529845209, "value2":8495752825324479706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_7":"d","key_8":"j","key_3":"j", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258260, "value1":0.3301633034128501, "value2":3240643823549476632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.373493, "value1":0.3529131970207001, "value2":3035971070657682432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.665486, "value1":0.1772430533425256, "value2":5999482327853244380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358796, "value1":0.18816869038874143, "value2":5134838279453664617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140177, "value1":0.8387362327402264, "value2":2310199526324609410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_9":"j","key_0":"a","key_1":"c", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.059616, "value1":0.8521328351977854, "value2":8719119695607623072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121795, "value1":0.3605246283964201, "value2":1540950724302287517, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.760661, "value1":0.21327358886251288, "value2":5483717677008529333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.019253, "value1":0.7234842919041711, "value2":3995891303198614565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.468371, "value1":0.026572272284047523, "value2":2431068728438057213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.397658, "value1":0.22598294609730654, "value2":2698499054966120972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755700, "value1":0.09282245685151808, "value2":1822149750384470026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.304716, "value1":0.6358956357038871, "value2":1038515895167880998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.673410, "value1":0.01619219159654624, "value2":999540262235584439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.406798, "value1":0.2801610335950404, "value2":5737646822014765045, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.472994, "value1":0.14109660013219333, "value2":5384594772062199779, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.601871, "value1":0.291815873607527, "value2":1847519318868439297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.994725, "value1":0.6730562186913949, "value2":5489090928652625049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205639, "value1":0.42334867153762545, "value2":1823044221508369143, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.904945, "value1":0.598173556849254, "value2":1029832085092338611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507464, "value1":0.8462370553826191, "value2":8938945324991684299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105783, "value1":0.4362851240411875, "value2":8230814679005715011, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.130819, "value1":0.5311370726009594, "value2":2799369947502242194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.436621, "value1":0.5406911040823768, "value2":5927551572729154830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_6":"e","key_0":"b", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.081043, "value1":0.9897107711937903, "value2":1409810039307886257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.360124, "value1":0.8898589942584547, "value2":1182666162202869439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.473406, "value1":0.9785032169444227, "value2":7933797459692204307, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.711755, "value1":0.8440901040738314, "value2":1266347640739805258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.836845, "value1":0.13202825619743236, "value2":6494324863003452768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.814171, "value1":0.22371325434459086, "value2":3679612694652027947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.707264, "value1":0.43001516107021553, "value2":4022260893191171434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.585576, "value1":0.9028218878505339, "value2":3312996140868409539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_8":"h","key_2":"j","key_3":"d", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.253947, "value1":0.5596795145791367, "value2":9053991858276197521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.793355, "value1":0.655432892066219, "value2":2171965578246650022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.003349, "value1":0.32642930281521576, "value2":4724329284506985861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532693, "value1":0.47384645211929527, "value2":166053326506373649, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.258148, "value1":0.8755464478705562, "value2":1367687629182661630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.284672, "value1":0.969983045672162, "value2":7709886208589610791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656648, "value1":0.7178009692517222, "value2":378528392759859903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.138190, "value1":0.17961568288211016, "value2":6086554307721297156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.755588, "value1":0.7476161070292083, "value2":3339991721000583202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.160231, "value1":0.4667023272863336, "value2":8520356596039544954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.888583, "value1":0.20049555103413333, "value2":2444783554472337568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008015, "value1":0.08008213078983399, "value2":3073434633965527342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094964, "value1":0.7132542439294741, "value2":5312205610465131789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.234374, "value1":0.3178507521158574, "value2":1341935169276159022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.772701, "value1":0.9799507521983674, "value2":4679043722393447880, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.742787, "value1":0.8436883610291797, "value2":8206108321797356735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_5":"e","key_0":"e","key_4":"i", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.285496, "value1":0.02119049506813719, "value2":5643416354242611531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_9":"f","key_1":"e","key_3":"f", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041734, "value1":0.8593746773495375, "value2":7177072845753511018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.102405, "value1":0.258959468834628, "value2":5506364318868451234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.617097, "value1":0.5213134879577571, "value2":7820417886370593685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_9":"c","key_1":"k","key_6":"a", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614285, "value1":0.8586887753126803, "value2":146426955523604107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.094475, "value1":0.3100760743003405, "value2":3216906708607385583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.400149, "value1":0.40305327199936813, "value2":4063666163388559666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_4":"a","key_8":"g","key_1":"d", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.741707, "value1":0.07472258132971145, "value2":4583642984114208044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_8":"c","key_2":"b","key_7":"b", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.289818, "value1":0.21965034622340104, "value2":548484284878385021, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788770, "value1":0.8645491106416102, "value2":6428112142936498186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.589911, "value1":0.8352505870700639, "value2":8807067943589232363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980194, "value1":0.4222199616754522, "value2":1431823521353243628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.347713, "value1":0.3792858568507692, "value2":5699682291213510899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.790574, "value1":0.48263080283098414, "value2":3899441712388650234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.586360, "value1":0.39598813053011617, "value2":8951581838322949271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147821, "value1":0.696249186695191, "value2":5997811343683052027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_9":"g","key_6":"a","key_8":"d", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375267, "value1":0.30594268061015806, "value2":7589869429394133120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.662739, "value1":0.8061361815679017, "value2":5904903629197529499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_4":"h","key_1":"d","key_2":"d", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.447051, "value1":0.37750215516426633, "value2":8371518067848222420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121391, "value1":0.6696130432092745, "value2":3528483712948148367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_7":"a","key_2":"h","key_5":"i", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.114395, "value1":0.9303974176092045, "value2":539479413974722614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.440024, "value1":0.33141470193064243, "value2":492711817812030768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830103, "value1":0.7998316524208386, "value2":4274003622712511224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.343594, "value1":0.948502518427523, "value2":1092062054441297015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.820370, "value1":0.6468806610248711, "value2":5700638394649707417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.169752, "value1":0.427387565910367, "value2":7968698722232869415, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.352398, "value1":0.008797452976395297, "value2":5389560588922529986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.574184, "value1":0.6925729377012323, "value2":7921127567234590177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.708589, "value1":0.6874360024316648, "value2":7727336634389454587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.647250, "value1":0.2053256936728798, "value2":8730436625808972720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522110, "value1":0.7058378270168358, "value2":5838333049897839477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.894605, "value1":0.3141493596388785, "value2":2076790219296988613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.008782, "value1":0.590566630449788, "value2":571655682516576680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.424761, "value1":0.6098530285996386, "value2":7846500926381318806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591196, "value1":0.33824124551827933, "value2":646303391431941929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964204, "value1":0.7366285135213106, "value2":4137944794465811468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643081, "value1":0.3218928504433246, "value2":9121706025453589663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_9":"e","key_0":"b","key_4":"f", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.804724, "value1":0.014940637092911, "value2":7451060690570027498, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.694428, "value1":0.21460306796333273, "value2":2982813118003123474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_6":"g","key_9":"e","key_5":"b", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.122258, "value1":0.49140759630280006, "value2":8832276444519664682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.996796, "value1":0.2474526024511197, "value2":7613005431630356063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238269, "value1":0.8267663978098108, "value2":2724285849945025892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.644529, "value1":0.4649514742650587, "value2":7589769671176102226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.376035, "value1":0.7135202605627624, "value2":4131507979199286159, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958098, "value1":0.5742451996367589, "value2":7655662522626761261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478186, "value1":0.30259713893200063, "value2":7366343481003162441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_6":"g","key_9":"d","key_3":"a", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299728, "value1":0.5700777203840441, "value2":8077274544410191414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.685643, "value1":0.9403232361639245, "value2":809458262071959965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.478225, "value1":0.7234775762369925, "value2":1256131637116314402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.445423, "value1":0.17591193935592764, "value2":6211570042356723652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.362681, "value1":0.09007902579380467, "value2":5590217114587646147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140908, "value1":0.3872422494802931, "value2":3824505998771737796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.312510, "value1":0.26492926145602147, "value2":2580605589364820036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.639094, "value1":0.8695708814560201, "value2":828997494663746747, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532778, "value1":0.8659642362585516, "value2":489767661854306328, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.332818, "value1":0.5085353492466338, "value2":7131627003266463948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.616148, "value1":0.6665482572291218, "value2":9183191141357087143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.807708, "value1":0.03703617074385954, "value2":1634911105603876624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.990782, "value1":0.8721197542810023, "value2":829062251335888311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.678575, "value1":0.8518891534609417, "value2":5490128933307846252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676580, "value1":0.032403043603468545, "value2":1860775688299500894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.011776, "value1":0.3310374930985375, "value2":6454291476355278297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.902226, "value1":0.03335506845841715, "value2":4281002708262490682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.884426, "value1":0.26587448707952976, "value2":2888077545939138944, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.721322, "value1":0.9352434968753679, "value2":7587503328880398082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674491, "value1":0.13470131213667097, "value2":733064028083204302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.422073, "value1":0.34800250717845166, "value2":5657610491232907815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232687, "value1":0.8974445402000435, "value2":5670871752075567567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.603175, "value1":0.8044440847017621, "value2":8279540241317103114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_5":"b","key_6":"j","key_3":"e", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.354995, "value1":0.9370809546532062, "value2":2794379758286031831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.598521, "value1":0.03162604483850602, "value2":8898609192002715752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.489635, "value1":0.09343989179985779, "value2":6354680197733735029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_5":"k","key_2":"d","key_4":"i", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.766631, "value1":0.6557065025532866, "value2":8867906113412686551, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.110067, "value1":0.37336348255028134, "value2":8359961637534300323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.734771, "value1":0.6467506386274888, "value2":1558727129876488311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.751073, "value1":0.5011707406625697, "value2":165276254030404304, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.903806, "value1":0.883542811993649, "value2":5713497784481830170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_9":"b","key_3":"a","key_5":"h", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.068830, "value1":0.41287620615552584, "value2":110550725395415731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.481998, "value1":0.9176610757654294, "value2":3543653242038441064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.206639, "value1":0.6881985031716263, "value2":3991865736711226608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682333, "value1":0.7354938188464508, "value2":2336942143484925177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.909363, "value1":0.6550859218093187, "value2":7848025848676081478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.439129, "value1":0.4735897904277586, "value2":1730101214885033885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.788447, "value1":0.7029400400744759, "value2":7348647096857266019, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.656519, "value1":0.8226813316731755, "value2":8565628989265102630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.286018, "value1":0.1715470955708772, "value2":281139728200192436, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.316461, "value1":0.970829440713552, "value2":8266115606666039974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.131938, "value1":0.06029126490427053, "value2":6088969321338942970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936137, "value1":0.16630484422294922, "value2":368627481682565524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.632956, "value1":0.4597748954914025, "value2":614883467360042989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.441534, "value1":0.3169134259464135, "value2":4894244187289767341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.536363, "value1":0.21988172452701402, "value2":3848642916965059480, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.722069, "value1":0.0035036640993316324, "value2":1784910019753576066, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.189179, "value1":0.5999084267877618, "value2":589928855378135734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.070147, "value1":0.2816071736469606, "value2":1978576359562742327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.474328, "value1":0.12233813345274026, "value2":7757125810591739471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.756802, "value1":0.8734910453077231, "value2":2858305999840238155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.830462, "value1":0.7809071944843656, "value2":6070440638222443699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.158081, "value1":0.8007440801809812, "value2":3893225795949780633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_7":"g","key_1":"b","key_3":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.266044, "value1":0.5291100849459248, "value2":8570983485516004790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.375100, "value1":0.43060898088087474, "value2":5072556087103000232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.433713, "value1":0.4065332038272028, "value2":7498601821703546789, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.293147, "value1":0.2008523459507916, "value2":3469755278652299519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.464083, "value1":0.7193413705182999, "value2":8360590737420028572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.182168, "value1":0.12869784027227613, "value2":2436732966596869973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.152849, "value1":0.915235247509805, "value2":6575468662860533653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.106299, "value1":0.6574795299867987, "value2":5839146977664216835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.842382, "value1":0.6684180756051546, "value2":6255233029731307668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.976893, "value1":0.9921834371308854, "value2":286112572755130788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.747197, "value1":0.6687884895336262, "value2":5026368687057544137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.227443, "value1":0.8444750407098351, "value2":3238045843783198862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575432, "value1":0.04959245001196866, "value2":4763760968874648742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.484574, "value1":0.8794886478305205, "value2":2929187105321944273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.613332, "value1":0.4649488124035846, "value2":8038984692264433406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.156389, "value1":0.013056913554735015, "value2":1073629030912865258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.614657, "value1":0.21062410476220902, "value2":6415897327379032774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.238729, "value1":0.10820884210252545, "value2":9083864450943421396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.241678, "value1":0.22605677165183255, "value2":191747579342742576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958794, "value1":0.08847610578305673, "value2":7897538472323716766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.335228, "value1":0.7181578715519717, "value2":7450823110233453413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.408629, "value1":0.007049664704797943, "value2":1505104843936344580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.558681, "value1":0.17466226992103584, "value2":4472498115626926601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682989, "value1":0.9645272985758268, "value2":1982644188431990793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.179200, "value1":0.6439691808346755, "value2":8186363896835482044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.823991, "value1":0.6478071089364331, "value2":8599197823923500800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.109730, "value1":0.43770117688691335, "value2":8534783631376673343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.659077, "value1":0.2388079100822681, "value2":5323293096195049116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.120405, "value1":0.7373368673794832, "value2":8559881035375574344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.355361, "value1":0.6166673498020887, "value2":5456145421934358189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.824086, "value1":0.4534358999465994, "value2":6384310233648456918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.871013, "value1":0.5576949556795737, "value2":1428733241052481930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507168, "value1":0.14733425604177453, "value2":4932780474472793744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.399272, "value1":0.7455596536595573, "value2":7959578558660693956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.015542, "value1":0.3661445195957949, "value2":5873286557095287645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.103786, "value1":0.04914833432556202, "value2":3473465672536909160, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.201038, "value1":0.09243610446589845, "value2":317387159009776683, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.637510, "value1":0.05764711949860248, "value2":683074398669967457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.000515, "value1":0.27555419865710506, "value2":1427932259617448392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_6":"d","key_4":"e", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.455871, "value1":0.4421289415542984, "value2":2179707427691179241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.816179, "value1":0.0315328103269199, "value2":8955771787657622788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.012467, "value1":0.2583652156574764, "value2":3152397034013397022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.323088, "value1":0.6137213421267609, "value2":6194154265710187463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.361691, "value1":0.2728847707358428, "value2":1410878768807423021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507208, "value1":0.40681095070477, "value2":2031127740762892841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.943788, "value1":0.4688930142953297, "value2":4135242725922960343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.528649, "value1":0.6786255340569864, "value2":5287486612492903502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_3":"f","key_5":"a","key_0":"f", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.225718, "value1":0.4601298705932887, "value2":9052049426787719860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.308880, "value1":0.4295083103049244, "value2":5651329885903965921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.759618, "value1":0.16482488772851353, "value2":7857083585674436753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416052, "value1":0.9111867810311565, "value2":6723934981662609251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.962464, "value1":0.4935260151955853, "value2":3632302155175562257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.771117, "value1":0.5157133736913164, "value2":2578240794064018102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.690480, "value1":0.6806227775589547, "value2":478506980366332127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_3":"h","key_0":"e", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.819739, "value1":0.8780417506767497, "value2":5796860147973283852, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.393173, "value1":0.42102685254727695, "value2":655129410007603946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.648983, "value1":0.7792161980083644, "value2":3901696553081135725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.746096, "value1":0.4415965960336573, "value2":5128962497088709971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.550611, "value1":0.9164668411003939, "value2":8252426555782396131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.026169, "value1":0.7150672668373745, "value2":4764749266652855333, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.684226, "value1":0.7136962442214909, "value2":5607090679190953591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.382560, "value1":0.4327657862928645, "value2":2199828856833521565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_7":"c","key_2":"b","key_3":"i", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.467808, "value1":0.22363679307659168, "value2":2768232122997302049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.575206, "value1":0.7602213610975361, "value2":2779299954374668017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.591751, "value1":0.6270567668817301, "value2":8638942281069975034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.374899, "value1":0.3628294711980554, "value2":2256958770162190672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490274, "value1":0.7562635891941201, "value2":828105904736409259, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.170118, "value1":0.5429488039839387, "value2":2772778506127755922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.240769, "value1":0.2820088361870865, "value2":5639438631779439142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.052386, "value1":0.43149632078191064, "value2":1748438881846630926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.507233, "value1":0.961520455982136, "value2":538372936759435184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_4":"e","key_3":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705937, "value1":0.5267821110187133, "value2":2615203611933488638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.750925, "value1":0.12125960803758137, "value2":9007375284837027095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.631618, "value1":0.07157188871273036, "value2":8155465841105503917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.695843, "value1":0.7797612580730544, "value2":174508687346238960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269295, "value1":0.7420818036952436, "value2":4380281606468315484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.566418, "value1":0.31419999154507045, "value2":1448626920753535043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.125439, "value1":0.23644172282231143, "value2":6454497651916669000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.260012, "value1":0.7675706994384546, "value2":5131372429329822614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.674484, "value1":0.024443192261290648, "value2":1060802392645575205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.870912, "value1":0.40232027987742836, "value2":7398514903536659789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.193840, "value1":0.5573502520764032, "value2":2891807756570316806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.496711, "value1":0.0561679292808207, "value2":7275940738251409744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.007546, "value1":0.4548616529539407, "value2":3391843142730199327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.532516, "value1":0.5229547783776293, "value2":306719517858726688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.879419, "value1":0.738955824150393, "value2":4551796041817468625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.037090, "value1":0.16862399179431758, "value2":4308060124699455866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.748879, "value1":0.537219693168335, "value2":464744057316162498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.610036, "value1":0.35423811263377813, "value2":8058264218672212470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.212290, "value1":0.8726353918052155, "value2":7894596078524999317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.876606, "value1":0.6546517022010405, "value2":6758959777685985912, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.140504, "value1":0.347953522010423, "value2":6223000072814932347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.829533, "value1":0.7297577355885363, "value2":7098834979468843426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.770147, "value1":0.5158958137849098, "value2":7721786354129545110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.960566, "value1":0.08984803039947419, "value2":7183642332674708296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.002511, "value1":0.7991212300299283, "value2":1907144646133978540, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.969474, "value1":0.8490958586264525, "value2":2475634499457331111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.757044, "value1":0.08525717165380897, "value2":6126821447855017519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.826913, "value1":0.3832744407117778, "value2":7549472912902563562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713539, "value1":0.040929240199395034, "value2":8725477692408636856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.235038, "value1":0.21655074702892513, "value2":6340750264489982261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.713507, "value1":0.09698561541865447, "value2":6976532341223849395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.926365, "value1":0.6869464705551463, "value2":1770406309916093131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_8":"h","key_0":"a", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.159778, "value1":0.37025745483297273, "value2":1667176166944781722, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.964175, "value1":0.3460815492793407, "value2":8584884445165219279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.789212, "value1":0.5752530695794612, "value2":7518543931942002634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.946193, "value1":0.9389444654094391, "value2":4798866777286425255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.183973, "value1":0.42689788996190486, "value2":2707564478214913458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.381159, "value1":0.19968138225970908, "value2":8498929039632010906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.535645, "value1":0.9768502318817727, "value2":6273017261118239502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.957340, "value1":0.1584492922790321, "value2":1071530234423723880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.622320, "value1":0.6014449518439046, "value2":5905866866815287771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.839825, "value1":0.6412078194818629, "value2":8581664149275929445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.925054, "value1":0.6499072539736884, "value2":3213784800648371267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.654382, "value1":0.9107398509050536, "value2":6290871463312648713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.795697, "value1":0.8789558932554133, "value2":9118966352973366626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.299533, "value1":0.18449853142128472, "value2":6924501956061845944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.993498, "value1":0.10358257618685637, "value2":949727120096577179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.192327, "value1":0.14944601875044164, "value2":3650903859653754777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.115925, "value1":0.7654222794387283, "value2":1807303188151583637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290372, "value1":0.022817260035797225, "value2":3306384333491756709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.402670, "value1":0.5073298240798392, "value2":4756233885952638662, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.682572, "value1":0.44581853587828163, "value2":8977717914108281095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.176704, "value1":0.0686762673056191, "value2":418784156784186861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.676421, "value1":0.9540024144812602, "value2":5040838132393700704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.851525, "value1":0.4501599756607321, "value2":7291682933774791859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.593343, "value1":0.506206698395899, "value2":622013338659516904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.454600, "value1":0.4498330838292854, "value2":2669650474822317581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.527805, "value1":0.10877922841749062, "value2":2131131607855096268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.196472, "value1":0.04264644660597854, "value2":2656288085872651609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.743799, "value1":0.2248320170707705, "value2":6330060840606217610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.137446, "value1":0.5799142292246671, "value2":4848799016740596142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.405919, "value1":0.3378516880177889, "value2":3532844197361780130, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.085405, "value1":0.11768848717895854, "value2":3011043909687794254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.494421, "value1":0.45496539093555294, "value2":1968423041425508639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_5":"f","key_0":"b","key_4":"h", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.846925, "value1":0.23911281903346318, "value2":1229857216789047456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.320172, "value1":0.7911586278138405, "value2":145982991700455098, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.688187, "value1":0.9272721413003945, "value2":5752958588041193324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.069720, "value1":0.006222818414235851, "value2":7251719396221186260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.974434, "value1":0.7327419748667513, "value2":8892941322690231954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.891519, "value1":0.2559114110516683, "value2":8453661615833220464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.596204, "value1":0.8329616164402396, "value2":6548226786173702319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.539908, "value1":0.7401840588881007, "value2":2823865774842747097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865375, "value1":0.23950780446408748, "value2":6387790804623201730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.121268, "value1":0.4818898421020435, "value2":5213834205313537165, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.865802, "value1":0.3275415536098801, "value2":2802490407634150279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.105641, "value1":0.6210077776605651, "value2":7224328772436945002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592291, "value1":0.821033330430236, "value2":3130486219263774432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.497809, "value1":0.5925199380020667, "value2":341686057277963037, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.097734, "value1":0.7071847698151175, "value2":3681912566029517435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.763574, "value1":0.5357729458141934, "value2":4576393986863014129, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626276, "value1":0.4467400550839313, "value2":8832239215373133572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.626952, "value1":0.8566744061055088, "value2":2681684884383601627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.980666, "value1":0.620719669726131, "value2":3866315867973394400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.970686, "value1":0.492660470887635, "value2":8584579737939983838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.205981, "value1":0.41980273151462594, "value2":1272721076115788334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.643071, "value1":0.9850402079127909, "value2":6866475251892359276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.520241, "value1":0.021531538514302357, "value2":3680419636166133733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.689781, "value1":0.7246711061095096, "value2":9156928465977729149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.936886, "value1":0.30593890582238353, "value2":8650088989944818585, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.064882, "value1":0.9419400037260646, "value2":5139525288533697167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.255204, "value1":0.6562362511494719, "value2":5377088753944330401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.147096, "value1":0.12576537131594923, "value2":696189011986700613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.199366, "value1":0.8703136109731814, "value2":8758678284656449473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.782148, "value1":0.21588917591543605, "value2":7157001910263594492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.053985, "value1":0.9864976225072658, "value2":3999466034196061068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.409718, "value1":0.4052988292496616, "value2":7239010836693528474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.918878, "value1":0.9460279119002769, "value2":8629146616157881411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744254, "value1":0.12263478198823631, "value2":8698623269262602240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_9":"i","key_4":"i","key_8":"h", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.701868, "value1":0.3178667417931335, "value2":8069435536591676411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_6":"k","key_7":"h","key_5":"b", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.794445, "value1":0.7546856348043545, "value2":1056257035145843208, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.318803, "value1":0.8976234734615749, "value2":7848445975050376420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.146350, "value1":0.31748244669278847, "value2":2926268572806049868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.392349, "value1":0.655725592207486, "value2":4190046781739044777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.749690, "value1":0.6239130127907853, "value2":5206683397328774242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.744300, "value1":0.6764999956977692, "value2":1477381647586507515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.758303, "value1":0.5258481908865111, "value2":5235026862755190183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.379762, "value1":0.890935177217856, "value2":6322796509408703364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.592912, "value1":0.6886535422027805, "value2":3968885283796366911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.859203, "value1":0.3806206868492738, "value2":6588306706789304138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.565175, "value1":0.823280011831958, "value2":1840222942885696142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.095858, "value1":0.18658533157232132, "value2":3040729957448861027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.567863, "value1":0.1544724870114228, "value2":5739061219420103119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.739840, "value1":0.5915071280708923, "value2":6986464734573649185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.200755, "value1":0.4500618730121175, "value2":4400580498849630870, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.358028, "value1":0.8986628799372285, "value2":3320447208954210899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.017181, "value1":0.4972731292168526, "value2":5687205385943047330, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.229539, "value1":0.5257635254456374, "value2":1562089048942850119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.171044, "value1":0.8736127789799626, "value2":7151271249368522054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.384626, "value1":0.2370246172687121, "value2":2826036906774872004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.522883, "value1":0.6013577976454938, "value2":890649018657102134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.877768, "value1":0.6077458920362099, "value2":8966895480129060070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.224033, "value1":0.0076863111785604905, "value2":6569047840639595510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.311682, "value1":0.07593318166504835, "value2":3253540135578122766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.490459, "value1":0.4609451528285016, "value2":2994036544656680091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.378898, "value1":0.9545401129487334, "value2":3668542248171100045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.416785, "value1":0.35541532907488144, "value2":94039845626229270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.275554, "value1":0.421200916650073, "value2":5818521501894570082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619269, "value1":0.30772799684273167, "value2":4876402407267868155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.702725, "value1":0.15927682148224467, "value2":1903456341847904299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.232604, "value1":0.7711772747260455, "value2":3866638254479737513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.217772, "value1":0.8540182257688077, "value2":7309608218029816756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.705505, "value1":0.48163362162995105, "value2":1550283929814058088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.900193, "value1":0.23760307179291948, "value2":8646186763820142769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.208626, "value1":0.38931880434870053, "value2":5483530341831301025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.568347, "value1":0.2801336860166125, "value2":9164397534745927432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.117458, "value1":0.981405746517363, "value2":6802896340427379587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.963035, "value1":0.7214166769862517, "value2":8440456704015434339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.700137, "value1":0.5934611351717767, "value2":6333818613628602547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.145303, "value1":0.6922047329264945, "value2":3009832451157760657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.345925, "value1":0.7232346453151443, "value2":4791014890475627712, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.619327, "value1":0.9932237622196707, "value2":3624811523135333449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.269730, "value1":0.7866373764087783, "value2":3746435711914598124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.265782, "value1":0.05991121042342668, "value2":1951849832097700576, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.175543, "value1":0.5444637318639066, "value2":5220283889625062347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_8":"h","key_5":"i", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.677825, "value1":0.904620506580413, "value2":4669605336529676512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.668600, "value1":0.17366696171797397, "value2":8249554071220606769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.041222, "value1":0.3254693615668756, "value2":5262353711290630554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.243762, "value1":0.8759302538831941, "value2":7311945236881805486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.958625, "value1":0.5492731854343449, "value2":5435276136514133338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:02+08:00", "value":0.290213, "value1":0.7949670770789082, "value2":7111766404096808279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025988, "value1":0.1026128579407924, "value2":1755616401770874869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595099, "value1":0.18775955592545962, "value2":1656134813479826833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.671098, "value1":0.8079739199189518, "value2":3665225572712988915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.542084, "value1":0.5181312401815146, "value2":1481804189008155276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.784956, "value1":0.40357178109169595, "value2":9035114082415618141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.276517, "value1":0.24301724893563487, "value2":4602891919019880467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064974, "value1":0.37706581069603073, "value2":7053383221578853097, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698510, "value1":0.45184858839742487, "value2":2258520036203869092, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.218133, "value1":0.3113727295782559, "value2":6841422843138500205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668881, "value1":0.25909755219748914, "value2":1185172632185740020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_4":"e","key_8":"a","key_3":"e", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.049479, "value1":0.9359900704241361, "value2":8443878057371326705, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.674394, "value1":0.5330986303950987, "value2":4597086230952392696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195073, "value1":0.23095513043718424, "value2":886484372856556987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_6":"a","key_1":"e","key_5":"c", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.380352, "value1":0.9222632147295795, "value2":2011984636052581606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.302156, "value1":0.7967664965647748, "value2":1936857391665696956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673174, "value1":0.8085420465867998, "value2":6974688906284239573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.013224, "value1":0.01982064999844403, "value2":3955016096628456095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090855, "value1":0.01304461484505885, "value2":8778318340262765164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.795784, "value1":0.12217080131703652, "value2":2014947645475742270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284442, "value1":0.42738762978686595, "value2":2584606560267930457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.741760, "value1":0.5561052082820417, "value2":7263332550390986826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035524, "value1":0.908173032025188, "value2":5685807943763240264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025764, "value1":0.20925710637592332, "value2":6562668387404626293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.530188, "value1":0.6447955243050295, "value2":3888614173619999623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424157, "value1":0.17487330381635957, "value2":5301061833157028268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026934, "value1":0.0422857387218024, "value2":8303721108125129087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841847, "value1":0.5206472086096076, "value2":74348635869177909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.414469, "value1":0.4278094238360654, "value2":7010327929309184782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491788, "value1":0.747156270778759, "value2":4790193170750716406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399141, "value1":0.47388249427808166, "value2":8836306836420904183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.160250, "value1":0.29496035919361546, "value2":1937679940636334558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787511, "value1":0.40198955034703965, "value2":5313833463463440565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684988, "value1":0.6177512623201419, "value2":1449869359073755283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_4":"f","key_9":"g","key_3":"k", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803253, "value1":0.8970896940692594, "value2":6178067064755804966, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964550, "value1":0.8199555931693112, "value2":140777809986023618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456407, "value1":0.2401625723920763, "value2":4030355648738753679, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849585, "value1":0.2629587147468046, "value2":7076050191155576404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.824331, "value1":0.6892589926802947, "value2":6399755277819070915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_4":"e","key_6":"h","key_2":"k", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388915, "value1":0.8127879446503501, "value2":5588278675440700300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782971, "value1":0.22771808116681516, "value2":6261899644042013732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712419, "value1":0.2905286238822736, "value2":5614218841386062674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_3":"h","key_9":"j","key_1":"d", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249696, "value1":0.9445558673553629, "value2":1404486883141712140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601814, "value1":0.6990549633969345, "value2":7198014296771075924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543813, "value1":0.07857387425059083, "value2":1217712980435541695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123893, "value1":0.07983470162468347, "value2":7171526612504871775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_4":"a","key_7":"k","key_1":"f", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703648, "value1":0.36447696895349185, "value2":5146632909527815894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166353, "value1":0.7098874569397392, "value2":5324193865568739754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.399594, "value1":0.7745072356158436, "value2":7831862313374173936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.439724, "value1":0.2037239485962775, "value2":3755069786706852308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_9":"c","key_2":"i","key_3":"f", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839998, "value1":0.6732355725193216, "value2":2639271267504971364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228435, "value1":0.897840654678523, "value2":5116662644663764475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973287, "value1":0.9461886626837847, "value2":5279391211817478931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.441360, "value1":0.9419535299333034, "value2":2098864241711696584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.780978, "value1":0.7212145622817746, "value2":8825784157879516761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.342306, "value1":0.40763808526506856, "value2":615440271373894957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.964992, "value1":0.8428995412660205, "value2":5458378648341607708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386901, "value1":0.3242872455426699, "value2":1659876068547712992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.960102, "value1":0.2464238639205716, "value2":699750419933475491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585099, "value1":0.11986390379255613, "value2":3678288424535052375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.154037, "value1":0.7785920099800233, "value2":8275166075671164173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_4":"g","key_9":"j","key_3":"f", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211892, "value1":0.7853941226491261, "value2":6369532531235861115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.746677, "value1":0.6664817644749024, "value2":4658328841108884689, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.941862, "value1":0.846994201539624, "value2":1274645788690532949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.733827, "value1":0.13169457879746155, "value2":3221287182313964764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870127, "value1":0.37466507316122794, "value2":8856106508914542017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798298, "value1":0.22462880312469063, "value2":6798911070080579970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993366, "value1":0.9961738964349245, "value2":1193549068611970903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.157412, "value1":0.4452570786392375, "value2":3487651503828456956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.529021, "value1":0.26905395113611574, "value2":2025155869621977272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.701922, "value1":0.8730175680632801, "value2":997347931310956157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_1":"a","key_9":"h","key_0":"b", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234577, "value1":0.7588051800901922, "value2":5319874977525060911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943591, "value1":0.47603601658101, "value2":5946410545801626105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.292747, "value1":0.03173464098642328, "value2":7226691638812613138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_3":"b","key_5":"j","key_2":"a", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608463, "value1":0.2957253043607161, "value2":122362680185330457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.643245, "value1":0.7167827619023723, "value2":5186237060875400709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.456051, "value1":0.2188755629521717, "value2":1845588654842798990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231176, "value1":0.6418253580281563, "value2":3690123298454825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_4":"d","key_5":"d","key_3":"f", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158253, "value1":0.7945876103242618, "value2":9004322525035166688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234079, "value1":0.5202371436378874, "value2":1591535835129382659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.875339, "value1":0.37493709867362873, "value2":8328159676060260451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907034, "value1":0.20583691616211497, "value2":3631192064418800289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.518876, "value1":0.16335464977880865, "value2":4643059165781979124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185787, "value1":0.018176867671764584, "value2":3270163687436586406, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_5":"e","key_9":"b","key_4":"h", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308787, "value1":0.4773555846410255, "value2":3253306657765008963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297154, "value1":0.246942999840671, "value2":6872861714776356785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_3":"k","key_9":"j","key_1":"c", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.817582, "value1":0.8165235911801363, "value2":8310469403306376604, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_3":"a","key_7":"j","key_2":"d", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493041, "value1":0.38500411997916434, "value2":6747567659837060435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_2":"g","key_5":"c","key_1":"c", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224106, "value1":0.9994238326374754, "value2":7099587552820760691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031476, "value1":0.09066884358892856, "value2":5945790442760569800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311356, "value1":0.5806991883130863, "value2":6895176067401972683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653725, "value1":0.15503072276739358, "value2":7449412216901454949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.598231, "value1":0.20373471056711764, "value2":1437658094331870355, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798108, "value1":0.6097451467316525, "value2":7133130757829716058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709054, "value1":0.7371923755130838, "value2":7543405219351917468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104790, "value1":0.21083306139991156, "value2":7929604552171608387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.187877, "value1":0.5608708053678212, "value2":3368623446076987813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749130, "value1":0.657437311445075, "value2":8949805346712651308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582481, "value1":0.42362389624895797, "value2":8717288037606444575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.558438, "value1":0.370755803762819, "value2":1107999166234163628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802675, "value1":0.8677299277452752, "value2":5915000168574046056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_3":"j","key_0":"j","key_1":"c", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.208039, "value1":0.38781051864576976, "value2":3375198001539598542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680278, "value1":0.886048145499295, "value2":4081527324669320081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162287, "value1":0.7118801419064047, "value2":2390093012206667849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.092015, "value1":0.9942758150465311, "value2":4638994738698984564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039345, "value1":0.641165676244722, "value2":2976540841974698815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.408130, "value1":0.5042712547968512, "value2":8947846924433036861, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259088, "value1":0.514500988249709, "value2":2770449998520564629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_4":"b","key_0":"j","key_3":"i", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293773, "value1":0.36521689212647834, "value2":67527608081891502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646136, "value1":0.04048325982754749, "value2":4743191660759249600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.047497, "value1":0.5502497475964416, "value2":7169106217640232030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876800, "value1":0.3799501707452096, "value2":6757458314368411993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968057, "value1":0.7499783045753542, "value2":1200059614173097893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.871681, "value1":0.6987940576652765, "value2":3787158661733344974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.998419, "value1":0.7974184353878208, "value2":3011281681090708365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663978, "value1":0.1875336848417326, "value2":7682956450262415743, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799777, "value1":0.13431952299825461, "value2":8472008336913527353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224565, "value1":0.7432835827699946, "value2":5363247813194187485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896272, "value1":0.1238866666650729, "value2":1249324477450532329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630720, "value1":0.15283582580094462, "value2":1819923308050662735, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405798, "value1":0.9413697480070442, "value2":6170928651103063980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249174, "value1":0.3594131764729996, "value2":6196624819261733193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628627, "value1":0.5009434579166352, "value2":4913727766417338788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_6":"d","key_9":"g","key_2":"e", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949656, "value1":0.677579274449204, "value2":4321897610912596942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.452302, "value1":0.8465391778403211, "value2":8767695810058898547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.224741, "value1":0.9419255467202373, "value2":3942608565088456699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832311, "value1":0.1844332411242929, "value2":7897621420940762914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_6":"a","key_1":"i","key_4":"c", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364450, "value1":0.6062623207548122, "value2":3048338263269528616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458762, "value1":0.8964243000562225, "value2":6980106557138925502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090342, "value1":0.9353992150482529, "value2":3024974544306240484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.680282, "value1":0.32391766175077186, "value2":855225884463637074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949004, "value1":0.8992840018801868, "value2":5343754317476027418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.564072, "value1":0.04591172153502947, "value2":15119713343690272, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.015169, "value1":0.2775250663576938, "value2":8329492228577328961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.035102, "value1":0.8955268703368635, "value2":1413978015063203049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_8":"f","key_2":"b","key_3":"h", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.922310, "value1":0.5335365949387284, "value2":7752415045685811238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.862402, "value1":0.328052481246616, "value2":1049471938369225232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.632372, "value1":0.702701110691373, "value2":736041481181376478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886888, "value1":0.8185435772395079, "value2":6423113939133946645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087072, "value1":0.4700081672384728, "value2":4642202730229975792, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.936650, "value1":0.18762496890040795, "value2":3355401686890613388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032044, "value1":0.12307064568471555, "value2":5991756874915773532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093295, "value1":0.22401921911686248, "value2":6120641321792430058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.777098, "value1":0.29744828015347885, "value2":3084065102603938152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.437450, "value1":0.2884592585353877, "value2":4044254904136084327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.017135, "value1":0.7608822818277434, "value2":1205572766308158879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_7":"a","key_4":"g","key_6":"d", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907629, "value1":0.7081298253215954, "value2":7688268403734929948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924934, "value1":0.7192898599410195, "value2":1421354730652485196, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.736050, "value1":0.7502596301905261, "value2":2582210329738662673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515235, "value1":0.09310016531489594, "value2":5123514648664672005, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.117213, "value1":0.07556633746315061, "value2":3501827326756011617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.485929, "value1":0.9272991096008794, "value2":5159915940333201308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_7":"h","key_1":"a","key_4":"b", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296263, "value1":0.11108812397005534, "value2":8753717861080819638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990089, "value1":0.527982415726353, "value2":5638704340740792965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835455, "value1":0.062168150269998286, "value2":1944114908616022523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297201, "value1":0.30562887567201646, "value2":2937626694886920620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974272, "value1":0.7221303752638616, "value2":4841762343070531864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.477606, "value1":0.011147194457821816, "value2":7938710785429203576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_9":"h","key_0":"d","key_2":"k", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129918, "value1":0.5408385148641213, "value2":9195358942783956586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364995, "value1":0.0773648729840119, "value2":2987992944358913043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.776833, "value1":0.8273262149027085, "value2":3110126394788152042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.610517, "value1":0.6781728258681893, "value2":4793737503812454856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.896976, "value1":0.37188254064396126, "value2":3144974162911084078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202562, "value1":0.5271477787382647, "value2":6411589275983706377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906770, "value1":0.4476129119528191, "value2":424159247916997254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_9":"j","key_0":"e","key_3":"c", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289456, "value1":0.541155755467242, "value2":2999265325007368522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.860878, "value1":0.4758705080993404, "value2":6690006483968082589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.365448, "value1":0.027420735129761006, "value2":825754437394215422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_3":"d","key_4":"i","key_1":"k", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212551, "value1":0.16839070098231337, "value2":1942763578810483030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.870900, "value1":0.4994077584006896, "value2":5993483889801645858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.455124, "value1":0.26138016691309135, "value2":4427691071851307843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.569012, "value1":0.4778238812129043, "value2":8556858598621845387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.362641, "value1":0.2203054974232932, "value2":3477406531290022646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493850, "value1":0.4787384781915333, "value2":2198408185614770688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.513028, "value1":0.6429388402116987, "value2":7593443154020639176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386729, "value1":0.17475131034566962, "value2":3647348637356138676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053464, "value1":0.6091636844532319, "value2":5763485176228727883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.628348, "value1":0.023668653461306262, "value2":6178270391876561387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913114, "value1":0.637992847841559, "value2":8248068010165238913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.413158, "value1":0.5132505144116425, "value2":3488568643829613122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517474, "value1":0.21449952384417484, "value2":4829637720497890669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_8":"d","key_4":"i","key_7":"c", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.659125, "value1":0.35348852774479717, "value2":929980299829000703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071642, "value1":0.5598789974197187, "value2":7463399784012985552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993416, "value1":0.6475276350489354, "value2":3252439000616730752, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003658, "value1":0.897666564581801, "value2":508174253835823210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.629566, "value1":0.49532352920736916, "value2":3384274756820723223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.084906, "value1":0.40471552937588173, "value2":2358566109120320700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.108543, "value1":0.8814759719911462, "value2":940297244819349653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138889, "value1":0.48057446422385713, "value2":5651682080133821762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.468024, "value1":0.6607643026338559, "value2":3567273041505939519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.278010, "value1":0.05185404011612328, "value2":9174087231001182435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.095328, "value1":0.4998002617147629, "value2":4028395944074542796, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.962889, "value1":0.7966578010574328, "value2":907724426763786137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.434695, "value1":0.596944451662117, "value2":6599947217349296930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.695515, "value1":0.23670726317490795, "value2":2923186045419354751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830693, "value1":0.43683284863611044, "value2":9110976313628947704, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.311809, "value1":0.18924644468054935, "value2":2283072307874475430, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405030, "value1":0.5026514010742291, "value2":7499615277819756210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.985393, "value1":0.2214787811403981, "value2":1167173892132804942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123017, "value1":0.24050302556695696, "value2":3512829503479749438, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.275825, "value1":0.8989518053152799, "value2":7332940306078599750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_3":"g","key_0":"k","key_1":"e", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.935513, "value1":0.422470893258827, "value2":1965139573445622612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_4":"e","key_2":"a", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697239, "value1":0.9715985007012725, "value2":2441260821113474270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297829, "value1":0.4121340516781959, "value2":7327020552945137669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.488215, "value1":0.22392461646214185, "value2":5911938621135820572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.206243, "value1":0.020325317058324376, "value2":9207086370751028426, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183572, "value1":0.23206347801714602, "value2":6792315880866236659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524533, "value1":0.9124779133103079, "value2":3221463708308920869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.893528, "value1":0.4848003839426891, "value2":3814492558819679029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.752371, "value1":0.8170507918067189, "value2":6527155317426998401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916551, "value1":0.342137272729241, "value2":1602041662641372111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.037592, "value1":0.6500639685761466, "value2":3567065955735150085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902253, "value1":0.7531115895009749, "value2":2543364566980292046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683579, "value1":0.30275430657154007, "value2":7640650810082868472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956614, "value1":0.5403322951696091, "value2":332198111972672347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202908, "value1":0.3832878669702297, "value2":880756788127038920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.820586, "value1":0.34286605495032285, "value2":6526500882733787563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930022, "value1":0.3539159969493719, "value2":370073818816792799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023457, "value1":0.047546161290410185, "value2":5398553854250083599, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303770, "value1":0.7186586727923895, "value2":9169471131017267259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.756889, "value1":0.31103630013784395, "value2":3594098461981248414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_8":"a","key_2":"k","key_7":"j", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212371, "value1":0.3882294279634762, "value2":2428299028227660876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.472113, "value1":0.19389271171622363, "value2":7737936344522555466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051213, "value1":0.12920469147183916, "value2":7837057207029745863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.907093, "value1":0.017956122904157993, "value2":4878755396476119431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_4":"e","key_7":"a","key_2":"g", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.525258, "value1":0.49782305829726053, "value2":2466466392623808778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913492, "value1":0.4704351769152427, "value2":7908561685667216260, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290702, "value1":0.4198914182070395, "value2":6819583912532498504, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125217, "value1":0.04410603241691789, "value2":6832161683827996201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004937, "value1":0.13145801093930243, "value2":5314457688060891725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.656896, "value1":0.41192120722520353, "value2":5457893604153821743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_5":"k","key_7":"e","key_4":"a", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.700497, "value1":0.8047394500892703, "value2":8180560267702238361, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553716, "value1":0.7484558334038893, "value2":1003966921986785454, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966183, "value1":0.8167158291471999, "value2":2250434045285686743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663750, "value1":0.7127353578019723, "value2":3741760060639503306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686363, "value1":0.9712701187599743, "value2":3622116724705792870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_5":"i","key_2":"h","key_3":"b", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.103723, "value1":0.4724493721726449, "value2":8181583911299132150, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.484649, "value1":0.18448759636607337, "value2":4053994543260017135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.925440, "value1":0.06382834538328273, "value2":6916231632621539461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939805, "value1":0.22424399111258897, "value2":4351472309231431136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792665, "value1":0.33436392964606154, "value2":2696421529224538356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054274, "value1":0.9272819232934056, "value2":7264556561094806515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.983823, "value1":0.7989127551970725, "value2":5239695063979297186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714766, "value1":0.6892792624795272, "value2":270252571976728726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.263106, "value1":0.64122047400478, "value2":3421795383581628735, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720260, "value1":0.8087795736115097, "value2":8426217720617766298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839070, "value1":0.5917446619321536, "value2":390571547986932413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966990, "value1":0.2432955344040808, "value2":8461111581798764026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.808072, "value1":0.3216500857620197, "value2":4440941489040196963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091958, "value1":0.3995806295500445, "value2":1180768478663237716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139366, "value1":0.11507703668400583, "value2":4516311843825616388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.251512, "value1":0.7632852678548858, "value2":6418768969046377858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963533, "value1":0.8213013619231073, "value2":6101164365690068241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.466433, "value1":0.28350424710105476, "value2":5563469458031181794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831271, "value1":0.8821293582204699, "value2":3430349356503249989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.003553, "value1":0.7323647142880576, "value2":5739925461894436779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.241144, "value1":0.3850622637508932, "value2":6137203197960360313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.872100, "value1":0.3184658084196574, "value2":2061373587207508318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.900532, "value1":0.2888524955130242, "value2":6600516250776964704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_9":"c","key_3":"g","key_4":"b", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142058, "value1":0.7572218923237818, "value2":5437617110907280837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.167933, "value1":0.20087813542991115, "value2":8497531403193171323, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977657, "value1":0.49027491644969856, "value2":6846007575513730281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.337497, "value1":0.7535889949814326, "value2":4191915280375173080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.091305, "value1":0.41827531366800946, "value2":3553824659208645760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.260475, "value1":0.39071909544867817, "value2":3828966522631037754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421117, "value1":0.6070501451115596, "value2":637677502764251310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_6":"d","key_3":"a","key_5":"a", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.949074, "value1":0.8719785824911194, "value2":739855289497042435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005847, "value1":0.564214486086032, "value2":3879428464931256295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608985, "value1":0.8120552257475274, "value2":6222621751128276018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.505971, "value1":0.44491083173920337, "value2":2966233843840872053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_6":"b","key_1":"d","key_2":"f", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686524, "value1":0.910573536000882, "value2":7259657495340557823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.386181, "value1":0.5690887314353372, "value2":7696797346941049468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004437, "value1":0.2837421940276855, "value2":3630048754825451133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197835, "value1":0.3478392366591849, "value2":8924518323717745322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404754, "value1":0.14818758453959469, "value2":3841840031846521039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.847397, "value1":0.10611042816524652, "value2":3749767974486199531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773493, "value1":0.16710420925739453, "value2":2156267597938791257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253265, "value1":0.39770730374803015, "value2":8078016986748379936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.720298, "value1":0.3023956927195079, "value2":2183933305568131133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.306860, "value1":0.13870609943567333, "value2":5783110454957795198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189190, "value1":0.3825441585131095, "value2":2493897460883724129, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842956, "value1":0.712749117246484, "value2":1337306979661530320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045437, "value1":0.9278250040101162, "value2":6116936767056337825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.606083, "value1":0.4732072236705937, "value2":6967738612762397759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094844, "value1":0.9269374913746184, "value2":2591132307871233919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369784, "value1":0.09045009486516034, "value2":7834710052863465672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032637, "value1":0.16606039490952637, "value2":2737666598235173614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.394163, "value1":0.47507719343340277, "value2":313537922371752138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.987576, "value1":0.5780445815848603, "value2":4108970207058212125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.906463, "value1":0.03465970899762972, "value2":6841259554497483309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.266096, "value1":0.6742131298888875, "value2":7025803245311237704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.963062, "value1":0.5904971275955457, "value2":2254362569087327315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.130304, "value1":0.868734941900672, "value2":1538504759997860509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.955104, "value1":0.5994289988135004, "value2":3881947199005484386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.480748, "value1":0.49853130606809243, "value2":727938692981473193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.682757, "value1":0.38003013064353247, "value2":8479678523613035623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792753, "value1":0.16576560662385773, "value2":7797109252869584262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.510440, "value1":0.6049726150607012, "value2":8669482832353600257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.638136, "value1":0.15014964585075946, "value2":6225706505785608080, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.522674, "value1":0.3512111717300567, "value2":3641814037368457348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785473, "value1":0.4544503987969124, "value2":4635364210597620767, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_2":"f","key_4":"b","key_1":"f", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110745, "value1":0.5221211963164464, "value2":4903849187407455848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.195405, "value1":0.08594736781118724, "value2":2859597321511204836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051298, "value1":0.8256101721258909, "value2":290470632378621656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.284220, "value1":0.27211501738111415, "value2":2134420797562982003, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792536, "value1":0.9372077558354763, "value2":7927244557057049645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.681776, "value1":0.4477409740262762, "value2":3995977364357841936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.231810, "value1":0.6078487538143919, "value2":3261959160546744959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.131613, "value1":0.7340176818759, "value2":3565658759712151137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.908393, "value1":0.28199967714832963, "value2":3350748733246898915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698846, "value1":0.043919603046489966, "value2":4515802998753688491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.803882, "value1":0.9014217952934828, "value2":8345388974528376469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_2":"a","key_3":"i","key_0":"d", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501652, "value1":0.45069419646062914, "value2":4400331011414194610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.240902, "value1":0.34849466754791153, "value2":3738192721082184330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.950263, "value1":0.5119937423488377, "value2":8919897235633479114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.443897, "value1":0.08318284858972097, "value2":8878946431124018702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.653076, "value1":0.2711114129019499, "value2":4904832384185588956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199562, "value1":0.5179430768249581, "value2":2338622690587545147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902254, "value1":0.8456128690164353, "value2":3512723913452353513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.424568, "value1":0.358235963026908, "value2":1951463002559117594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.891020, "value1":0.899730727943035, "value2":384925737643795292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242327, "value1":0.8194521049409685, "value2":1651591731883214718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.064957, "value1":0.5094969888256025, "value2":1137790773894543912, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032406, "value1":0.3455331481317415, "value2":2958859262258749395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553817, "value1":0.484360827667417, "value2":3796071195831724862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197274, "value1":0.4874943149488065, "value2":1044578434864087492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_1":"k","key_5":"f","key_0":"j", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.384646, "value1":0.3107814255944672, "value2":1519235564474675279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286271, "value1":0.42853641527210906, "value2":4722523490137371754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175308, "value1":0.3946055220580569, "value2":6954032044064595773, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.793904, "value1":0.06297728493572644, "value2":5259172994496601557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.062223, "value1":0.013288825435356888, "value2":6140160587654305146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.768622, "value1":0.39922765836242, "value2":6629678017270316706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_5":"b","key_9":"g","key_4":"k", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.203380, "value1":0.9727752066385791, "value2":960040697676306380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_8":"i","key_0":"i","key_3":"d", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.845192, "value1":0.9218669222513459, "value2":8161692681657646744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625176, "value1":0.7818690860470178, "value2":6286486510404426651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943535, "value1":0.004848635402133272, "value2":8498014405632225004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.204456, "value1":0.5833134681081519, "value2":7410659829376107064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.576114, "value1":0.9192372118815992, "value2":6657353407785586037, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118284, "value1":0.8313517670516266, "value2":7764880758391828110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.535569, "value1":0.24827312684994104, "value2":2040106521001592824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.549323, "value1":0.8166897779780931, "value2":1564430022376434258, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881818, "value1":0.09926733907609549, "value2":3471220368243877954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_4":"b","key_2":"i","key_3":"j", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374110, "value1":0.5149369120619428, "value2":2786214942305343351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770127, "value1":0.7580522569953325, "value2":4637323015723129783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178499, "value1":0.146709310096475, "value2":7936042836162333987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274162, "value1":0.4372447015799375, "value2":9083170531856859501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491205, "value1":0.022796016225098064, "value2":8123705023600354614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016451, "value1":0.9256503393490508, "value2":6970208660239633828, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.087308, "value1":0.4462899441715343, "value2":1749521530182375024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.094183, "value1":0.3475409881711034, "value2":5390535090747181247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.735505, "value1":0.53372762504523, "value2":8372678000097135865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.114823, "value1":0.029693120979849812, "value2":6638472287910475525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.242928, "value1":0.3050035672228757, "value2":1792917800424529821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703291, "value1":0.31694612370345837, "value2":2183523770285388049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.331904, "value1":0.47208239473287594, "value2":6204971001548635721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.345432, "value1":0.4951178136444847, "value2":3855507055566939240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136145, "value1":0.6933172444157865, "value2":1571591012651895136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303587, "value1":0.05226903625691088, "value2":6994464805121874342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.604568, "value1":0.14657045529476723, "value2":2698838307937260948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376282, "value1":0.8133509413304958, "value2":8456674940436282992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843511, "value1":0.6514108937273068, "value2":7171882174493021649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888039, "value1":0.34434154548426427, "value2":5546478000801091154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663092, "value1":0.2145932369532521, "value2":1924108819168508153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178424, "value1":0.3811642005135661, "value2":5277232346465674780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.982420, "value1":0.509903018966164, "value2":1366973490493465213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.789914, "value1":0.5590817594064089, "value2":2545283978114147454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.858245, "value1":0.2990011327668893, "value2":7402472987921684090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.045358, "value1":0.4729812652892598, "value2":6922132388747645101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212994, "value1":0.6311902604147308, "value2":4414936740257073237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357341, "value1":0.06207502931784455, "value2":4475293686563973956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.975194, "value1":0.41868388111482774, "value2":1854059337783469990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841014, "value1":0.21676180216590327, "value2":4591196137073503812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.978140, "value1":0.3174552368999545, "value2":1901352595947591853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.267125, "value1":0.18049579402909974, "value2":4625546630316797154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916965, "value1":0.5612711107070505, "value2":6029229351915879062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.927166, "value1":0.40285609194050886, "value2":5993132840270480501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791545, "value1":0.4639522865587751, "value2":7463548107279858536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770063, "value1":0.26886367149111556, "value2":3097459774572125541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077189, "value1":0.7572977836842385, "value2":8298753827870878088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.287375, "value1":0.621000003066286, "value2":5005851683916784910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.517180, "value1":0.6660153705066499, "value2":8387087977559252312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.675034, "value1":0.9813881401454424, "value2":3741676304938251291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.554896, "value1":0.7715073370679849, "value2":6589060977722716090, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319288, "value1":0.12279901479898898, "value2":3422770912327659239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703895, "value1":0.8358860148624778, "value2":8257794540400854550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.493064, "value1":0.048118109532117544, "value2":4134373887037968422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799328, "value1":0.11593192650373021, "value2":1597084067805135642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.848966, "value1":0.3986720724086452, "value2":2405658508790196207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.792447, "value1":0.5248994141540482, "value2":6715402324504357457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376734, "value1":0.7992683845538152, "value2":636776368616526730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_8":"b","key_2":"j","key_6":"i", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.646082, "value1":0.7040359351145827, "value2":8792341972386350386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.358717, "value1":0.15149972457126396, "value2":8525880284771508381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038854, "value1":0.6775307407388963, "value2":4497884453622086771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313173, "value1":0.6408664944912432, "value2":9170385529548968990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.209148, "value1":0.8824832965439914, "value2":6994782182036294339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.349829, "value1":0.6332729838723153, "value2":8895300460695986955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877356, "value1":0.9629884067987685, "value2":5874587318078108417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175639, "value1":0.2215504394755138, "value2":1645626442028267495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.867334, "value1":0.17245948139280137, "value2":7635160136876279780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.304089, "value1":0.4417775050588339, "value2":4663940359268319500, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.377260, "value1":0.3699273314519082, "value2":3098095013643812538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.687645, "value1":0.49899496293822504, "value2":3483082050221607102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.764258, "value1":0.5854336662763441, "value2":9096847150126021331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.395360, "value1":0.4621587456489886, "value2":4751829496417501448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702903, "value1":0.41598100403233135, "value2":3929201336867975918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107090, "value1":0.7914140452826858, "value2":3589076543012868088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.487563, "value1":0.4474861481593704, "value2":2779456470910748695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_6":"j","key_7":"f","key_2":"i", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512119, "value1":0.5021386207405886, "value2":5678715906223819621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_4":"k","key_7":"j","key_2":"h", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346070, "value1":0.55706872148054, "value2":2039287696528568559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.574280, "value1":0.3713448367655447, "value2":3929829805998447613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.892079, "value1":0.6942759872823322, "value2":7729227663415283404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.344002, "value1":0.42092378887559295, "value2":4794883782773019081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078411, "value1":0.6154878270099419, "value2":6707716797669517904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.571429, "value1":0.1378751349573672, "value2":8074751840319910755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.040055, "value1":0.6408943986817053, "value2":1079211030051415138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.162869, "value1":0.5000621894976172, "value2":5126515351324361210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.390768, "value1":0.5289053805740587, "value2":7495442762687396988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259180, "value1":0.5649142133543891, "value2":4328071162032108601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.842704, "value1":0.21495927580985347, "value2":2932451180465678632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650755, "value1":0.6984634581926075, "value2":5766024562422825714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612080, "value1":0.22851599558447847, "value2":5344041318095855549, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990865, "value1":0.7354893379622883, "value2":9025665650100041943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787926, "value1":0.33568512834020503, "value2":6804173344109850974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172355, "value1":0.9139245088414795, "value2":6155807217840657379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.979541, "value1":0.24199196392953898, "value2":5042790445163279739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.314666, "value1":0.3110154918771059, "value2":8906652806658780575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.357408, "value1":0.8063999303874789, "value2":1651751776595868392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.288676, "value1":0.9293974960471005, "value2":7670397636585827003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.293262, "value1":0.7818217756805212, "value2":8606918715162923765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.083534, "value1":0.9670843857632683, "value2":5887729702260327460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149981, "value1":0.6552447379991068, "value2":6725961755903338686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.492842, "value1":0.7092364133108918, "value2":7796995215212529618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944648, "value1":0.6256326288941183, "value2":4831063196252879183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.895337, "value1":0.4995752655149396, "value2":1408597026195247818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.944023, "value1":0.634711000263352, "value2":5584697240807054271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809484, "value1":0.7426498850085607, "value2":8020687455958940501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.289743, "value1":0.008820277060811819, "value2":6337782472142563494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.537111, "value1":0.14436676295527084, "value2":1783901321543975972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749042, "value1":0.9467021656046419, "value2":279529980240550876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.261740, "value1":0.20334209964882055, "value2":3677408684840846866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.272200, "value1":0.5140769192808836, "value2":3965788713393972569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391452, "value1":0.5620808926889774, "value2":3432925103242153786, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.797201, "value1":0.24306630830309864, "value2":2212488327265931040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.296686, "value1":0.4958413853936345, "value2":2745010941408731624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.508150, "value1":0.1764099610149601, "value2":6087888705783067229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.150030, "value1":0.6475061379725164, "value2":4152410772475568903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406997, "value1":0.8552543065924282, "value2":6943121315248371505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.798595, "value1":0.4482032174944134, "value2":4591949889203793788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427353, "value1":0.23427032816286456, "value2":8455901174434850105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.426947, "value1":0.2157937977395769, "value2":4812396978108020858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553964, "value1":0.6569888020417926, "value2":4304633456065882312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.830280, "value1":0.6753558122351391, "value2":5709360383833750840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335737, "value1":0.6229413264866215, "value2":774141312393774432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934503, "value1":0.3184401780744072, "value2":992485213419150513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.286280, "value1":0.19701195292124982, "value2":5131579316119627914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.027527, "value1":0.5595644624118316, "value2":7324008691917344768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.445835, "value1":0.3878002605330551, "value2":5283397242149349986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.317903, "value1":0.34865399024368093, "value2":919918660600814082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.988293, "value1":0.520877683344731, "value2":1362287203583282026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451918, "value1":0.17797518451009597, "value2":8069259017842130934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.416614, "value1":0.49438798293161773, "value2":8221530176741871409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930143, "value1":0.11046223267799947, "value2":1569220098038190448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.841002, "value1":0.2351074608491269, "value2":6987153620614536862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.172031, "value1":0.30378326240838155, "value2":3438948688273621480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427034, "value1":0.3726321157197697, "value2":3892410333661915590, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.996688, "value1":0.06148106153681236, "value2":7012776674540552021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178139, "value1":0.24652480255103007, "value2":7264782641186044646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.641890, "value1":0.18896088271258277, "value2":5281363384350384982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020470, "value1":0.5427453076719968, "value2":4211018551963613162, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548627, "value1":0.9467235670877983, "value2":1356346215173760766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923887, "value1":0.16868132666008845, "value2":2419587552380208808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388273, "value1":0.34607395217688514, "value2":6360018437765039879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199475, "value1":0.48700761251474795, "value2":4207130869528000977, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212632, "value1":0.7298887109862953, "value2":3855225371708173636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869081, "value1":0.3040832001677732, "value2":1636252957386132843, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_3":"f","key_8":"b","key_1":"b", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458358, "value1":0.25499479265263875, "value2":7951534915845928364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.673427, "value1":0.3560914688953808, "value2":4486505974707646981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.934304, "value1":0.9086314287420252, "value2":6454407382033994812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.077472, "value1":0.272412312785447, "value2":4055844210948982160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.391470, "value1":0.4094306269697527, "value2":8737172085844354362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.507196, "value1":0.4293932426928758, "value2":7752579176339832618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.354922, "value1":0.8782836749723913, "value2":3092074071663841238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290283, "value1":0.6466038651665884, "value2":6445966108572410248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930993, "value1":0.2534798517547614, "value2":2212538268968967382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078571, "value1":0.766186870171555, "value2":2276988139922811436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.512143, "value1":0.26492881415396713, "value2":7960469334919381452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.664663, "value1":0.9293891578195701, "value2":8363525489715122179, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.859809, "value1":0.5317709711783597, "value2":5646146960428890136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755810, "value1":0.5033479266486215, "value2":1442438201922477703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.014944, "value1":0.2736949295034319, "value2":4543402002213224798, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.019842, "value1":0.06301734338783647, "value2":7848421926410748778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.006460, "value1":0.7283128915704968, "value2":8696821133231766119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.495665, "value1":0.22357458621454834, "value2":5013935622275517771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_5":"j","key_9":"c","key_0":"k", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.829438, "value1":0.08516153321035196, "value2":6346747105122069788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120479, "value1":0.6330301971897442, "value2":3074605270601257967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.406377, "value1":0.9646902889346889, "value2":9116844958473667659, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_3":"j","key_9":"k","key_0":"k", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258382, "value1":0.9290313183449712, "value2":7925410787408077827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340705, "value1":0.7460138699765932, "value2":1834174806073680705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.660263, "value1":0.9250659950303504, "value2":6999248688568055694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_6":"g","key_0":"h","key_5":"f", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252290, "value1":0.718659365662016, "value2":414864145410727491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.404912, "value1":0.5143974391105166, "value2":6178244170296726344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.355231, "value1":0.7285417682392774, "value2":4099314188219635809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232613, "value1":0.9628251247534773, "value2":2753347640662156751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042238, "value1":0.40260777202369225, "value2":6405687740970757268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.059614, "value1":0.6706802975294311, "value2":155105754975301806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.335427, "value1":0.03875943420721546, "value2":2621496673126808846, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791994, "value1":0.4069226188854093, "value2":8911123850930154192, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.152311, "value1":0.001001700535447483, "value2":8160805255392424399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.806126, "value1":0.09279090865200551, "value2":5165528109888127485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_8":"k","key_9":"f","key_2":"k", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181230, "value1":0.34297422042642206, "value2":5952935302808125089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308758, "value1":0.39399952685268963, "value2":2291587630268260397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438229, "value1":0.008760206538330227, "value2":4899262456651209905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253853, "value1":0.7708129134752543, "value2":5591299897276344295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.388133, "value1":0.4749440177332357, "value2":1978415349188199488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612871, "value1":0.07573141562457633, "value2":5953047090784791116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.481277, "value1":0.12674805322449534, "value2":8950535310944284362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026417, "value1":0.13620073115890582, "value2":5838422267573225081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.491782, "value1":0.6697421435610075, "value2":5470323240276494634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599889, "value1":0.2218252683274155, "value2":3863404473966363047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028452, "value1":0.8531657224160458, "value2":1800467248475944001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234267, "value1":0.2783481987951185, "value2":6997165241525177240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.369705, "value1":0.5636516367093837, "value2":4958650393346745090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201031, "value1":0.628281679414999, "value2":3983210515283198963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.519582, "value1":0.950842451530494, "value2":8333725115642196357, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.800667, "value1":0.47362573201727787, "value2":5531339313881866007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.022715, "value1":0.7456986400451252, "value2":6375649752818307396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.026963, "value1":0.013441045212029653, "value2":8879122156886303116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.849167, "value1":0.7209297073484169, "value2":5571977114960029707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.916214, "value1":0.07478238462060836, "value2":4957974530052652715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.801842, "value1":0.5497676040427942, "value2":7068572572815929980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.180884, "value1":0.16934771421243305, "value2":4129529446885753081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742335, "value1":0.17996128783258833, "value2":1612889506435491518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.124131, "value1":0.6211285761421723, "value2":6936437753872174981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_4":"j","key_2":"c","key_3":"h", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.626664, "value1":0.1412142921060844, "value2":7340286601023012024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_9":"k","key_7":"e", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534599, "value1":0.1824543867650024, "value2":2710596260731528329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.198428, "value1":0.563515863993397, "value2":8658671925349980639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939981, "value1":0.5602221119588965, "value2":1332664463057731952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749870, "value1":0.27530155912076365, "value2":3294589175109845443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501991, "value1":0.28276078133217086, "value2":6074437039967216201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886807, "value1":0.3878328960046618, "value2":6751632027412168403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_8":"e","key_3":"d","key_7":"f", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.911289, "value1":0.6411363324230984, "value2":8252067497790790379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.696807, "value1":0.43753398310350267, "value2":3790908907624313334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.319319, "value1":0.8437649419939369, "value2":4536112071098596928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.734427, "value1":0.7768336856428384, "value2":577814262800235815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232482, "value1":0.4639181971383516, "value2":915233793101361418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033045, "value1":0.6840737140076493, "value2":6077817095892263388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427451, "value1":0.7412294644798527, "value2":5067651533453787714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.086307, "value1":0.884916959850894, "value2":6213157740517107791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032588, "value1":0.7692661642362982, "value2":3255905758159811447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129133, "value1":0.9868818324607139, "value2":2840159926392322636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.005933, "value1":0.4856659261691267, "value2":5401836471710156191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.078678, "value1":0.43872499297363293, "value2":7610348477992750069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.101709, "value1":0.6553052079447295, "value2":5590613299241898040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.691765, "value1":0.5500719022328135, "value2":2311787078857040546, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.609841, "value1":0.20437362444327167, "value2":5976709159513367946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_7":"g","key_8":"g","key_5":"c", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.262383, "value1":0.988890785552926, "value2":5073782211732867167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_5":"g","key_9":"g","key_1":"e", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382099, "value1":0.7056400640121613, "value2":1362786206799762754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.910924, "value1":0.740118438943497, "value2":5204241364776707180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526567, "value1":0.33926045568377805, "value2":8282435047009117311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.259549, "value1":0.6968073477407744, "value2":5508984811117632066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_6":"a","key_2":"i","key_4":"b", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.575379, "value1":0.9204583134801098, "value2":7328623658034172463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.021181, "value1":0.8037794671210114, "value2":8763012613490416638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666802, "value1":0.735290222214112, "value2":5898346647756703163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836023, "value1":0.9774841401477888, "value2":262883488175479807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085239, "value1":0.2006523822964799, "value2":6927806981819194391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722778, "value1":0.29750991877070054, "value2":7269146382542606407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.585823, "value1":0.7790122698056057, "value2":5454870737254023995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966399, "value1":0.9342783027092735, "value2":1736810747318512793, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.363378, "value1":0.5421736888212692, "value2":8680345426115136675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.548518, "value1":0.6180502927475818, "value2":576215651567739722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169252, "value1":0.8808696517971845, "value2":3266464067570846970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032461, "value1":0.3854322648479189, "value2":5351736110848205547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418352, "value1":0.9282836374954182, "value2":4950655664039540565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.567245, "value1":0.06221685577996024, "value2":6376294794655595743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.607825, "value1":0.8284877360321535, "value2":8836357162970050736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.002720, "value1":0.9429757239354525, "value2":3712886085226721402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772087, "value1":0.8045038867848936, "value2":4759138460922621996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.515031, "value1":0.2984015395169622, "value2":2069584478283962675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.141150, "value1":0.6666900797494949, "value2":268184099180001198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_6":"j","key_1":"g","key_5":"h", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.102173, "value1":0.08736073078451814, "value2":3669960405293392951, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.825677, "value1":0.883585003117732, "value2":4900464079888733154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.613326, "value1":0.2003226722632331, "value2":3796686759340146598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_4":"a","key_8":"f","key_1":"a", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.965080, "value1":0.24041472099907274, "value2":7454089011589464567, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.139395, "value1":0.8846967167208682, "value2":3451086131264105278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.709618, "value1":0.40080113749800483, "value2":7347906568423400039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.490979, "value1":0.10232414632757307, "value2":6898384399737921493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.419810, "value1":0.9521194603097142, "value2":4165710553126975748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_9":"e","key_1":"g","key_3":"a", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142251, "value1":0.3656369505236563, "value2":2656042717790241170, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578775, "value1":0.42161623774225904, "value2":7959202830983535548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340081, "value1":0.47208224231080925, "value2":1119972655203312039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_3":"e","key_6":"j","key_2":"k", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.201301, "value1":0.686282703890728, "value2":1347056528311053195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.905450, "value1":0.1504198445233754, "value2":4153183079049561671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.774660, "value1":0.04197441149840889, "value2":2892741854923361469, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.223828, "value1":0.9913278205978917, "value2":2634921019433133187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145135, "value1":0.9764589636119458, "value2":1863333686324511619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.051457, "value1":0.9664880803510422, "value2":5468675095205300977, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.471202, "value1":0.9438594012185837, "value2":8711551131933841419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.438432, "value1":0.7332734301616686, "value2":6736601196370014167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.018444, "value1":0.6166410041335356, "value2":7430067770903432071, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.042785, "value1":0.3083845760668554, "value2":7529797289952981250, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.500528, "value1":0.41360737530731173, "value2":527946418421906650, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.639029, "value1":0.28120003518234893, "value2":7333246850823447562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.118602, "value1":0.22780592472876143, "value2":5562449967641903101, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773447, "value1":0.34556648930141964, "value2":3454919255414457289, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.751050, "value1":0.5691145183911931, "value2":3420985612149611112, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.967122, "value1":0.010774686710818307, "value2":1857363403020110299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.211203, "value1":0.5071144540997159, "value2":3201415543625346596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.966034, "value1":0.10396047428400347, "value2":9014157472738440520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253319, "value1":0.43609058248995825, "value2":1183645881443278471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.104187, "value1":0.4324653404306875, "value2":2865322183450789108, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185518, "value1":0.8897028996083935, "value2":7566651396044601727, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930391, "value1":0.8041371194633276, "value2":7871648174569158725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163568, "value1":0.9301616644769536, "value2":7748526202149251971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.869278, "value1":0.88349271636991, "value2":554007324522749523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.213408, "value1":0.664832495384472, "value2":2758153864408048603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.451136, "value1":0.3702265627491039, "value2":494580882184796857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915009, "value1":0.6519575592422996, "value2":6479471856073874185, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.581364, "value1":0.0703141245253348, "value2":899491222964044388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142057, "value1":0.9957531092014932, "value2":5996532724034214303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618561, "value1":0.31275866348570913, "value2":4261560321529985020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.526426, "value1":0.8632813331612903, "value2":2280841193196363442, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.640741, "value1":0.4671010753968709, "value2":2523475617435456826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.686774, "value1":0.37684937337218677, "value2":6568317226871778750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.547284, "value1":0.9412532971618462, "value2":6396494980432088364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.630465, "value1":0.10697292615695254, "value2":6790210135628014818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156319, "value1":0.10115374361141051, "value2":2774605952238778168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.742080, "value1":0.4345761934130475, "value2":6896687242300333410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662119, "value1":0.3886014719849511, "value2":2171378209788759856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.158805, "value1":0.4473032074837117, "value2":586039993535786710, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.025398, "value1":0.0651048756141861, "value2":3136658443010387115, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.189314, "value1":0.11574796728414168, "value2":5756732023607570312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_9":"d","key_1":"i","key_8":"g", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469000, "value1":0.3187212262703272, "value2":4880377874531438837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.698910, "value1":0.9021085577061128, "value2":8314934978791640839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228153, "value1":0.7939593629687762, "value2":5721748474729655997, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_5":"j","key_8":"b","key_1":"j", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.174833, "value1":0.47180057752688764, "value2":74850886504396865, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.373682, "value1":0.9894401032981868, "value2":2443814427210362559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.230070, "value1":0.840516361148932, "value2":5295744989016708862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595645, "value1":0.2788762608865926, "value2":898774770804749270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583440, "value1":0.38764558337295657, "value2":4879715356875855821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.818979, "value1":0.08550208893092236, "value2":7063465379400979205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.990629, "value1":0.9735905702213057, "value2":9121417987396463994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_4":"d","key_6":"c","key_1":"e", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.904697, "value1":0.06007475161136332, "value2":1399260524832392153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832693, "value1":0.9560818456694564, "value2":6313771830503199817, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.273330, "value1":0.5911156951363433, "value2":6957059227202260296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.069691, "value1":0.43486640701864104, "value2":5573463606213032969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107074, "value1":0.28194511495703845, "value2":5008401247473276713, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.133554, "value1":0.6572805795366599, "value2":1430282776047161882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.994425, "value1":0.7317293344785567, "value2":2161864319170825977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.703844, "value1":0.05169084281485511, "value2":7389845001390437940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667112, "value1":0.912614805848696, "value2":7494408396730454879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.175192, "value1":0.5654088358678295, "value2":7414573375849683263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.205436, "value1":0.5764549187518897, "value2":4315302504250390302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.816073, "value1":0.3021595298462732, "value2":7835612651483818643, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465800, "value1":0.7224511737802353, "value2":1475815608612889931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.836860, "value1":0.13043025990273924, "value2":8725370796816365579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.110112, "value1":0.2182570483232002, "value2":4212625314776126266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.401613, "value1":0.3660560702023913, "value2":7136594683517469774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.772305, "value1":0.6420871129325768, "value2":346199260296482441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.142556, "value1":0.18492252698973582, "value2":2410884666534888495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.710999, "value1":0.03498389655141747, "value2":3857815905441742992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588844, "value1":0.6843616917435196, "value2":925508006135613377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.902010, "value1":0.7892717951683124, "value2":7453201615304707478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071475, "value1":0.8597541524726022, "value2":5247483360729978449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.886399, "value1":0.896477399685476, "value2":3245685435537096080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683519, "value1":0.7125828488615462, "value2":9221328263547905112, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650600, "value1":0.8976371405804583, "value2":6562020796772331341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.316077, "value1":0.8989654150875539, "value2":2315139769444859937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.844010, "value1":0.027932169490326376, "value2":8266191397214577403, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_3":"b","key_1":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.732165, "value1":0.9768852287807686, "value2":7871357409427659276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.644252, "value1":0.3652027068824268, "value2":4777422302644847840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.016877, "value1":0.4774488788727276, "value2":1359637185519527638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_8":"f","key_9":"g","key_3":"e", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.121368, "value1":0.07637107971325026, "value2":225307864650781063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.661741, "value1":0.9526993459994305, "value2":1751890497684728484, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534867, "value1":0.5672837246804853, "value2":3948370290656301760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.400506, "value1":0.312527577003677, "value2":3200609992091774001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.023204, "value1":0.9498281171738407, "value2":1775257641053296989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_6":"c","key_7":"f","key_1":"a", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.650859, "value1":0.7200243703698949, "value2":6459489446120996262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.125434, "value1":0.7513756216852647, "value2":6502722962310909561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_5":"k","key_0":"k","key_2":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.861634, "value1":0.22132024628333183, "value2":4213082281103637151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.616718, "value1":0.7492815151768418, "value2":8785198245169443750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.618495, "value1":0.5367662090666573, "value2":311068731789452160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.090805, "value1":0.6461806005354355, "value2":1567126231539851190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.704866, "value1":0.6742157058257261, "value2":8258394004417852586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.178045, "value1":0.9347807954648532, "value2":5391643132437615590, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.903981, "value1":0.04205578457439769, "value2":4267367541005494603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038592, "value1":0.9632414356901877, "value2":7847330231249131972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753078, "value1":0.025682952887214512, "value2":4407969986032631485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.551387, "value1":0.9127538521307936, "value2":5532887277300530226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.340315, "value1":0.357568695323674, "value2":5839322022227051810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.253088, "value1":0.37455084749823464, "value2":919408856650004667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.061612, "value1":0.08807754228035809, "value2":7674978240888726818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350886, "value1":0.9187800273401259, "value2":3985979173089262193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.997832, "value1":0.3332675485121975, "value2":6884650671561667205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376107, "value1":0.882485292013383, "value2":980580007402095853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_5":"h","key_6":"h","key_0":"b", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.666200, "value1":0.543727834482791, "value2":7683502472635473948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.690345, "value1":0.9227402770528923, "value2":5963421474121503099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.831896, "value1":0.8899731417411955, "value2":3119002755449635021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583901, "value1":0.46723052632915907, "value2":4738569017001620646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254234, "value1":0.07205706866452416, "value2":1464209096722519553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.940932, "value1":0.4880590550999508, "value2":3249029228406137695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.819723, "value1":0.9180382594935601, "value2":5531777988899534137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.899553, "value1":0.5017555686141989, "value2":3526732457056334830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.364242, "value1":0.726219019872585, "value2":7463421321863859910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591997, "value1":0.18380773538675038, "value2":6489374408484526555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_9":"d","key_0":"e","key_3":"f", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268627, "value1":0.43767329135389277, "value2":7591255891965320507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.668426, "value1":0.8905993514345967, "value2":8025287728753470033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_4":"c","key_2":"a","key_3":"b", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980089, "value1":0.9629455057472474, "value2":8613343366791814003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_6":"g","key_3":"i","key_4":"d", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.561494, "value1":0.11038616417110533, "value2":5718549668479051731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.031316, "value1":0.5572446441362652, "value2":9073355869031665981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913579, "value1":0.40513173097972344, "value2":1392267543875933322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.469802, "value1":0.7197223556739805, "value2":8312522501295372114, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216239, "value1":0.38424010939993836, "value2":3429512093250859609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.053128, "value1":0.12167032010014549, "value2":2500133776481013473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_9":"g","key_1":"d","key_4":"b", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.755093, "value1":0.1432079670350677, "value2":4460076765766388831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.939117, "value1":0.022396858430100586, "value2":1677736517466813137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730771, "value1":0.5975163721595746, "value2":8564236690483063617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.763913, "value1":0.11111651544548351, "value2":2128389252611471929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225483, "value1":0.1857169909861624, "value2":1459664257900170929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.557521, "value1":0.10580492023702627, "value2":9211924519949290338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.398250, "value1":0.8985634717833737, "value2":2690743666091944740, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.523236, "value1":0.8737716900990798, "value2":3717036232567418933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465788, "value1":0.8763409298655005, "value2":1486778905892801100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.185627, "value1":0.4834833602007871, "value2":8283437701946785636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.129438, "value1":0.4506377660858689, "value2":8577339860951120030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.937075, "value1":0.6005010987020736, "value2":4798896188528165681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.387681, "value1":0.8777928078176396, "value2":4677891530420984908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.074680, "value1":0.015131030176877407, "value2":2652924707092709319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.032439, "value1":0.7762590461709376, "value2":5555968285860120909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.107619, "value1":0.0013683674938652786, "value2":8724767287173452904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.199777, "value1":0.2705573241568207, "value2":5731172518242608315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.883455, "value1":0.440980354957089, "value2":717450023283511313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.065153, "value1":0.7805040031236675, "value2":8249819238828017763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.028771, "value1":0.4266508385350679, "value2":3525220539697375602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417030, "value1":0.9344530676800743, "value2":2840008251192450241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.169783, "value1":0.4944137362673325, "value2":8143727104076341090, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309365, "value1":0.8233798931001785, "value2":4143752746550875358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.405563, "value1":0.36473772798628423, "value2":8368477347330201864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.080493, "value1":0.45312759136632275, "value2":5757367910796479605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591589, "value1":0.5574497820130245, "value2":4692942231335997930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770417, "value1":0.20880923062348417, "value2":4939040557737590500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.923414, "value1":0.8006055497749137, "value2":6979326850811422524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.249012, "value1":0.022621459630026125, "value2":6570512292324514521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.787566, "value1":0.5692664113038692, "value2":3307063324843670478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.236668, "value1":0.706981031305401, "value2":6240895241125781235, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590979, "value1":0.9325091299580149, "value2":5714058795401825599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.782982, "value1":0.8159590900452445, "value2":3981533983292904484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449106, "value1":0.2421958898621009, "value2":8819094433559346153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_3":"d","key_8":"c","key_1":"i", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.721732, "value1":0.7320938316381037, "value2":1840371677400026121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.417920, "value1":0.3777544081714696, "value2":2517494055558796825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.467240, "value1":0.07736271112081412, "value2":3355581288729910614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.802307, "value1":0.06164218213969723, "value2":6962215396306940249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.599871, "value1":0.2948698648725146, "value2":1065059716477496547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.109220, "value1":0.11737528756979723, "value2":4468678220083400352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.156222, "value1":0.6294465098457002, "value2":2835781199255992964, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.697592, "value1":0.9317953948461911, "value2":2221981181398322475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.785752, "value1":0.4235402956347778, "value2":5956435747949559681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.943686, "value1":0.28355998213096456, "value2":3638916038813769869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.010104, "value1":0.22773696044954786, "value2":2360395284842638669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.239178, "value1":0.14419564768805598, "value2":5315118297850048019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.136093, "value1":0.3244550022291072, "value2":164222067931855660, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_6":"i","key_7":"b","key_2":"j", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.968733, "value1":0.7486844786821671, "value2":6287914493893452990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.252999, "value1":0.24537563867085085, "value2":1626574969302079091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.885743, "value1":0.9307243682752692, "value2":2858030878483913913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.631011, "value1":0.7084136481624964, "value2":2044467985498488215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.853026, "value1":0.04425979450750048, "value2":1653452661642691666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.991236, "value1":0.40886253057738015, "value2":7871621921656029433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.207156, "value1":0.37687587966259445, "value2":1943347287070225574, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.708313, "value1":0.5785225454675275, "value2":1380941757138302107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770913, "value1":0.9188180749639426, "value2":3884885787383779114, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418857, "value1":0.9998412287287592, "value2":1886115929568662728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.850257, "value1":0.9796804114853653, "value2":4397109710918249601, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339971, "value1":0.5089698391144044, "value2":8237713822108701891, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.913742, "value1":0.9646995309003717, "value2":1173067889283022862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067798, "value1":0.6899692743360695, "value2":1520636933221389194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533696, "value1":0.8606713340189989, "value2":5737213954118900238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.212415, "value1":0.39075235526474594, "value2":6592432468398327445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.054486, "value1":0.34784858262092483, "value2":7843113252021213719, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479147, "value1":0.5614107726843738, "value2":8690368331747046125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_6":"d","key_9":"k","key_5":"j", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.622976, "value1":0.06775340365098863, "value2":5564636394288500497, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.315983, "value1":0.33546786831791003, "value2":3930900696554115736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_6":"j","key_7":"g","key_3":"h", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.370627, "value1":0.779233269421853, "value2":3716543858406653649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.969610, "value1":0.006014494345107452, "value2":8710808689518483016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.724048, "value1":0.07768942947746518, "value2":496409043717037404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590602, "value1":0.8288610653786054, "value2":1102610595846648990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.297543, "value1":0.37578423768641916, "value2":5536977376920161050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.155394, "value1":0.7083524353328224, "value2":4002932567237644088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.459881, "value1":0.4873646265203241, "value2":5916271319076231574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.791505, "value1":0.5808298895011346, "value2":7841892156688279555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067541, "value1":0.09121699649729094, "value2":8699612203169033522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.138436, "value1":0.8850890106644654, "value2":9069485496265005186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.812668, "value1":0.7098616610527902, "value2":8132915524494658053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.190668, "value1":0.2761598493860578, "value2":651770389362313830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.220527, "value1":0.17936744276680264, "value2":310375881133353905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428315, "value1":0.49075022964169346, "value2":6493345006460964913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.595999, "value1":0.2755442227044263, "value2":4862778030716507189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_1":"f","key_6":"c","key_0":"j", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.719052, "value1":0.21758648924177262, "value2":9168976451471290619, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_4":"c","key_8":"i","key_0":"j", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.166119, "value1":0.5187037640605351, "value2":7936720027148899998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.361344, "value1":0.7201073761748152, "value2":8725593566170852352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254415, "value1":0.07641284618924576, "value2":7978515171853518726, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.981898, "value1":0.027541030165662613, "value2":6121611839007131360, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625954, "value1":0.22759978693770738, "value2":2989185053134172992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.350547, "value1":0.566849398731457, "value2":4327099372915065254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.799208, "value1":0.11746266545752859, "value2":7451515689230242023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.322064, "value1":0.26907348347198895, "value2":262434464387240754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.919974, "value1":0.6405108138252255, "value2":7284316699831173435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_9":"f","key_3":"k","key_7":"c", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.534467, "value1":0.9814271230191226, "value2":9214896283898892364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.291823, "value1":0.7119821636486502, "value2":4137668561036791084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773725, "value1":0.05006867968524374, "value2":4350082823034910080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.642706, "value1":0.8326077028765216, "value2":6301127653055186823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.624652, "value1":0.6013140948710498, "value2":7150151669380473161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.663653, "value1":0.6471420893487286, "value2":2842026748176794650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_6":"b","key_1":"e","key_3":"e", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.533665, "value1":0.40332997931033865, "value2":8939087210798770840, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.915959, "value1":0.20834521315367086, "value2":7895351758501362895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.238383, "value1":0.5265667505785955, "value2":2676293106518263900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.582878, "value1":0.3420886897063641, "value2":1233598437288047014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.489464, "value1":0.6718160824398975, "value2":4468628860088387255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.649028, "value1":0.7293826474439736, "value2":3127060444172041547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.254010, "value1":0.8457444172560414, "value2":6200016452741633767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.536370, "value1":0.16874770298054162, "value2":7575698105239291032, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.809584, "value1":0.343753090691686, "value2":4854239248106071203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.334375, "value1":0.5540567040963329, "value2":2646076573445947055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826524, "value1":0.7273180695893363, "value2":6564419565708307637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.071563, "value1":0.26385268835278003, "value2":4070457585628644142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225965, "value1":0.9321283899737617, "value2":7702159669474089190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.458779, "value1":0.10195409940683517, "value2":5987602857042259052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.501592, "value1":0.07688705594071264, "value2":5259725214710983339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.385240, "value1":0.8254376912417642, "value2":6760238975249610189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_6":"f","key_3":"c","key_4":"g", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.760329, "value1":0.7530171422146449, "value2":1097173296466796516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.597269, "value1":0.2403481802375547, "value2":5050041855629937310, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.149349, "value1":0.43132198586517284, "value2":3287400838900732872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.499626, "value1":0.737892470781718, "value2":4461721794139931373, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.449953, "value1":0.8639558735483506, "value2":2903703899322529602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.912253, "value1":0.01605090964788225, "value2":5781647102967996198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.4054065908363409, "value2":6440329633133258472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.038844, "value1":0.3763702930516151, "value2":1302491802347098686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.453739, "value1":0.6265318797324549, "value2":819075302260626614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.590829, "value1":0.5479451127212884, "value2":8557856578535483978, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.714164, "value1":0.8513319596110408, "value2":3981392010073089983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.767546, "value1":0.9500048124431112, "value2":2193055476612398128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.685203, "value1":0.9241278767545394, "value2":5146494836784446946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171752, "value1":0.05193751579817584, "value2":2407542541423061968, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620585, "value1":0.9449477088124408, "value2":293003481714135625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.197927, "value1":0.9081143587822024, "value2":336470947794228646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_6":"b","key_1":"i","key_2":"d", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.730165, "value1":0.4106378302625472, "value2":2305406403284031522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.993455, "value1":0.5015628664485756, "value2":3942941750281172936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.376150, "value1":0.08846010494510266, "value2":451409939393211294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.578966, "value1":0.5184884584460681, "value2":5013311361860229047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.980632, "value1":0.6414794039381226, "value2":4661775189685128413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.543153, "value1":0.7779597819383781, "value2":1678453405944867509, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.728157, "value1":0.4026694195455883, "value2":1247937331490084335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.039033, "value1":0.6078130901686731, "value2":6572519819620597387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.020089, "value1":0.16501689332992148, "value2":2135299756770180375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159279, "value1":0.06812322597368, "value2":1853312335979487824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.225355, "value1":0.31761717143878, "value2":1547635249117783016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.579591, "value1":0.45080215259792744, "value2":830280592135153306, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.989155, "value1":0.3246650009893744, "value2":3832412454221483471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.898794, "value1":0.9353227689941251, "value2":5019805485542488656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_4":"g","key_1":"d","key_2":"f", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.325402, "value1":0.40452074217950085, "value2":3779976710102146862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.846682, "value1":0.8078556003817017, "value2":4563926776477066326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.538107, "value1":0.6916244463696708, "value2":2616499684497809047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.601707, "value1":0.0005961819411869942, "value2":8421789457526552579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.474772, "value1":0.40247038234287125, "value2":691809847413019723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.835874, "value1":0.24276779175523852, "value2":6899854861109897252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.366826, "value1":0.8778001737651254, "value2":4321676554804962434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.826448, "value1":0.8031889549998205, "value2":5554188972077808449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.432681, "value1":0.15017093541000798, "value2":7513115457016757405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.353975, "value1":0.23702527674889587, "value2":1810989557365945672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.303934, "value1":0.2677306312444, "value2":430799962318472061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.163809, "value1":0.8138323281483592, "value2":8246241283973870498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.662355, "value1":0.9144882781553422, "value2":4359470640610896382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.120568, "value1":0.8056479342778862, "value2":6170593949410616781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.541822, "value1":0.248047143451303, "value2":8344606419798048221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.147354, "value1":0.35385149944297156, "value2":6006537614889971434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_8":"j","key_3":"i","key_4":"i", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.116235, "value1":0.6774547184266492, "value2":5503266725598369859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.563395, "value1":0.978792023344712, "value2":8139355376515169099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.202304, "value1":0.42453838538082794, "value2":8767087562530261434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.553168, "value1":0.813931872578598, "value2":838817774250793063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.476339, "value1":0.2705647850316454, "value2":6004383053661805919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.067348, "value1":0.23127791420697405, "value2":614040873255409897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.063340, "value1":0.04494057741571453, "value2":7945808906854676018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.877369, "value1":0.188685408716376, "value2":5653852277013510880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_6":"f","key_9":"d","key_3":"h", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.145318, "value1":0.6394711610065485, "value2":6391486639411558835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.171038, "value1":0.9902721723985418, "value2":3163235560331954355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.773084, "value1":0.40123815528773027, "value2":6772197516975478771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.976169, "value1":0.608513336182585, "value2":3898075135558136236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.460540, "value1":0.027884469593113505, "value2":1806632468596114826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.411909, "value1":0.6437101253853217, "value2":4628507915366383990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.055892, "value1":0.4209111843259915, "value2":8498738002897413709, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.684315, "value1":0.3608627471653678, "value2":8911984339628672502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.181887, "value1":0.04106030144724243, "value2":912973087706429869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.127221, "value1":0.8723975492127802, "value2":4688994221703849298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.881735, "value1":0.8013556912079172, "value2":643258613153646569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.093171, "value1":0.9983142632275949, "value2":2242609892773836818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.702516, "value1":0.9951260414368723, "value2":9110402286568364937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.712584, "value1":0.10494403348655837, "value2":4745554958401980793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.699467, "value1":0.9010413033988799, "value2":9007036540377523860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.132646, "value1":0.06467587338211019, "value2":1060827532336676254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.552157, "value1":0.4330362437559917, "value2":7496787358324320393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.888799, "value1":0.2576583993305694, "value2":4018360458617386281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.498269, "value1":0.2224059001256479, "value2":4065063745853139952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.588721, "value1":0.14516718870277823, "value2":3512331805580372297, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.583233, "value1":0.7489522543167118, "value2":5610699138607696953, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.290155, "value1":0.44273304014865483, "value2":427744180869204041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_9":"e","key_5":"a","key_8":"c", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.183829, "value1":0.6345267428968718, "value2":1846381248733979645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.323098, "value1":0.012439339538142696, "value2":8869073391299009493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634145, "value1":0.9267919956221178, "value2":2118172993893609219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794582, "value1":0.8353549954749553, "value2":1351674467301929869, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.008550, "value1":0.6040059322178531, "value2":753170312236461842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.268043, "value1":0.31922185771526074, "value2":2365194637514291305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.857933, "value1":0.8196275302693926, "value2":4478227043901542114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.645585, "value1":0.17157442528490047, "value2":3055405974571687053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.277791, "value1":0.8508212157393535, "value2":475170132890875463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.123450, "value1":0.35015180460716977, "value2":7931493690032322348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.918435, "value1":0.30789613557477924, "value2":4634204529169845529, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.747396, "value1":0.7119174217020244, "value2":5113967535733338451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.346547, "value1":0.4105703222986521, "value2":5634326652610778742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.873833, "value1":0.5726619572582797, "value2":4834039219162125836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839553, "value1":0.3474160702051541, "value2":5912402851303338785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924943, "value1":0.25193375971882537, "value2":3157228766646586798, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.309520, "value1":0.4856065585160012, "value2":12512784025771269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.612385, "value1":0.06742962293903079, "value2":5561454087904294314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.634744, "value1":0.8158802826640159, "value2":4407063105915178233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.524442, "value1":0.05056002141365119, "value2":1069310785755082516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.122461, "value1":0.8648592352685478, "value2":2607760955410972754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.308553, "value1":0.8340608354185821, "value2":5091217112123016682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.073014, "value1":0.412293314616913, "value2":1754188674597018639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.382506, "value1":0.7914084871078703, "value2":3113230598922027169, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.494019, "value1":0.22683994673921629, "value2":345509562584529791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.770833, "value1":0.36946692588064045, "value2":3634543347045698698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.374724, "value1":0.7674203492653544, "value2":409087874753033650, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.234424, "value1":0.33760436450813625, "value2":6378216485000854161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.753898, "value1":0.08146706117124274, "value2":4619087622758440080, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.058417, "value1":0.0895913841094616, "value2":7775148933653129917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.889748, "value1":0.9025957275835974, "value2":9060698938664882878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.429099, "value1":0.45935248830386766, "value2":5813431686543708617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_5":"c","key_8":"i","key_1":"g", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.415280, "value1":0.8845089283715811, "value2":6384109853170568567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.956220, "value1":0.6163283411653032, "value2":7939487848850431764, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.813125, "value1":0.3306736514104797, "value2":717112968732186535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.464126, "value1":0.23519387446018714, "value2":2061991374504688132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.336194, "value1":0.6334359034532542, "value2":5464410230276800258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722970, "value1":0.6104432497384834, "value2":3298076355265173187, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.246842, "value1":0.8069861206050694, "value2":7793368021912415700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.465645, "value1":0.6879012241509205, "value2":4196934016416565115, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.876004, "value1":0.06724546167889978, "value2":7203308436755461322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.667298, "value1":0.6826712318664662, "value2":6402839614176163878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.329694, "value1":0.7040366253423608, "value2":1262392626609016984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.591566, "value1":0.9312106031712911, "value2":6293442498282971529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.559072, "value1":0.4448009704050545, "value2":2669159126008899565, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431852, "value1":0.8238849009316198, "value2":1021818236630467563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.722762, "value1":0.5948459056041813, "value2":6253806025921861085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339501, "value1":0.14816011629555237, "value2":2437202342857552183, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.313849, "value1":0.14517234363613402, "value2":3032490535727906731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.974064, "value1":0.5282428591902778, "value2":7306937940345849875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930949, "value1":0.6595519497499615, "value2":2818227237339798252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.995836, "value1":0.41346641045143234, "value2":457793590293209131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.527926, "value1":0.17089664849686367, "value2":5825724904068162921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_7":"d","key_1":"e","key_6":"j", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.258906, "value1":0.5587901020638738, "value2":2247971244413895345, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.232248, "value1":0.9182710496214209, "value2":2114818465791862400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.075755, "value1":0.36489084685089246, "value2":294204895726685309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.726685, "value1":0.6513663450138257, "value2":741257611118540876, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.794966, "value1":0.07728058455296861, "value2":6115920270410886698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.954148, "value1":0.6576477115837028, "value2":8442517348621889353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.305446, "value1":0.8417215600157095, "value2":3960819244598684350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.749210, "value1":0.905397326364665, "value2":7392448658664215427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.479499, "value1":0.8192850944227448, "value2":1109448037735485747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_8":"b","key_4":"h","key_5":"a", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.228850, "value1":0.7549298352802428, "value2":2475887390541907125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.256344, "value1":0.6191375565525401, "value2":6629221972448932333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.843715, "value1":0.4048974244394111, "value2":2052496354441233400, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.257771, "value1":0.11131625744600283, "value2":5527432282921695963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033616, "value1":0.18479886541727453, "value2":9028612437715086928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.832552, "value1":0.6959404377768793, "value2":2173084661524542898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.431933, "value1":0.7816557853866045, "value2":7424533234329790131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.264080, "value1":0.7382347817860379, "value2":5213561271316161382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.085647, "value1":0.4523244522101, "value2":7889921400021958159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.004893, "value1":0.6556889205138186, "value2":3025561804921198898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.339616, "value1":0.2999169933424259, "value2":8184676717410157627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.216679, "value1":0.42313229610434316, "value2":4008780382662754625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.418495, "value1":0.49555776117242517, "value2":1018253790078366875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_7":"j","key_0":"a", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.033786, "value1":0.9209475895817386, "value2":5569575246133894540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.620702, "value1":0.6494468637932249, "value2":4953667604093058532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_5":"b","key_7":"e","key_2":"a", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.706504, "value1":0.12760690080118256, "value2":6290715519152270329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.483588, "value1":0.5095141564714446, "value2":6862972202733799902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.159429, "value1":0.90417660237305, "value2":8461823351624136781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.868532, "value1":0.20009311827615117, "value2":2269303027016825167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.030853, "value1":0.07427016371278365, "value2":6524107618104112365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_7":"c","key_9":"e","key_0":"b", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.625191, "value1":0.3705374970999065, "value2":8940487375802886850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.274579, "value1":0.43745692673872216, "value2":7664930411577220439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.428630, "value1":0.527310522539046, "value2":6084859156132058763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.594595, "value1":0.21322470607998698, "value2":3416879710307779950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.920870, "value1":0.47127845699263055, "value2":4564621285620268941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.930512, "value1":0.41802427118606883, "value2":8549501771062307593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.977996, "value1":0.1610907563584878, "value2":6578022034712623821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.973727, "value1":0.9224145581231933, "value2":8668945457313971984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.421891, "value1":0.40172827078543016, "value2":2817865116264178423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.683482, "value1":0.8428158825760677, "value2":1087743773064912864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.001802, "value1":0.5075672608227982, "value2":3748937813625482336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.924538, "value1":0.6184538471340483, "value2":8049426804122873881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.427400, "value1":0.23484522887445405, "value2":360795956748828928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.371453, "value1":0.14713938237360089, "value2":2782450449945692027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105401, "value1":0.7787234859135739, "value2":7103390400724679707, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.608837, "value1":0.9083492888699478, "value2":591947190434298439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:03+08:00", "value":0.839636, "value1":0.20678896707817307, "value2":1816652983048222981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.798496, "value1":0.27959153848419394, "value2":6453301245071446957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_8":"f","key_1":"a","key_2":"g", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079294, "value1":0.9921346825877959, "value2":2610009567825131673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_6":"a","key_4":"g","key_5":"g", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.932174, "value1":0.616531393243407, "value2":5793240958146632836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352378, "value1":0.5817027431697587, "value2":1001435181122015935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_7":"a","key_8":"k","key_5":"f", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.892678, "value1":0.09115452403074104, "value2":5100882205034466347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.976748, "value1":0.223689480612581, "value2":4832346905388106952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348707, "value1":0.4785116775888147, "value2":1474405574460091730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285197, "value1":0.2545105724094511, "value2":2069757700743153534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_8":"a","key_3":"e","key_4":"e", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886496, "value1":0.6267242677514145, "value2":5390965713876988864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_3":"k","key_8":"b","key_0":"h", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.106653, "value1":0.29720904047717794, "value2":3776030286792185916, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.360157, "value1":0.5546347320924953, "value2":330359320098225031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229710, "value1":0.36539460584181066, "value2":2852256551896569615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.107272, "value1":0.7519447135299673, "value2":479042341091737730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.653206, "value1":0.055664103121505173, "value2":4715105139689145584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339724, "value1":0.65279778916842, "value2":5051290581444035933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605595, "value1":0.7185823471699857, "value2":6840084614633305448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923752, "value1":0.4375825544173804, "value2":5512879974255226786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_5":"d","key_7":"e","key_0":"e", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259124, "value1":0.1976251344530473, "value2":3865204942280496136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833243, "value1":0.1327195116975274, "value2":25036382112222312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181954, "value1":0.03893696644874014, "value2":8620048848181171214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241452, "value1":0.08765547856551259, "value2":478115772614972254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.346035, "value1":0.8594527006005106, "value2":7112082294884150846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_3":"f","key_9":"c","key_2":"i", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171651, "value1":0.03206744023244803, "value2":1423544456047900511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863325, "value1":0.8536761474388656, "value2":6060590565202496674, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.441838, "value1":0.172156773217081, "value2":5954759661339252215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_4":"j","key_5":"b","key_0":"a", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960980, "value1":0.32980876505437323, "value2":8725964433492010874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_7":"d","key_1":"f","key_2":"c", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000707, "value1":0.24709283071336913, "value2":6676514692655151346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.618373, "value1":0.9009517094483285, "value2":3595072304978000605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.170508, "value1":0.9280407408957537, "value2":5488749324303924425, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.863252, "value1":0.12661256202486373, "value2":8664732043269436820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_7":"c","key_1":"f","key_4":"e", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.849403, "value1":0.16431470929214623, "value2":7019138744327165116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464027, "value1":0.09323138351556215, "value2":7725624855401413056, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_7":"e","key_9":"c","key_6":"b", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.236399, "value1":0.8723974444748736, "value2":4852776678069519279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748433, "value1":0.955013940926474, "value2":1647976132077310759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519277, "value1":0.9878333472759755, "value2":1234045371574302701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098817, "value1":0.01452665827731407, "value2":1567075824080468956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.086812, "value1":0.507976874584688, "value2":7088565570837880582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.442740, "value1":0.05234017389082447, "value2":6744325328771713826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200886, "value1":0.26486555470244233, "value2":7063921451775904700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542647, "value1":0.44652572463811874, "value2":881040498652574470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403848, "value1":0.15638403887386948, "value2":881480808583221270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.456051, "value1":0.8059172501323808, "value2":7272729849107639168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529307, "value1":0.34486654778342174, "value2":392684728493023069, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_8":"a","key_4":"b", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171436, "value1":0.3729515434430422, "value2":3944928681545148043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308638, "value1":0.7632957285940241, "value2":92442464729948625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515220, "value1":0.6925288324736131, "value2":8180048442965609063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584191, "value1":0.021892941959738798, "value2":8483219855752350520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.062243, "value1":0.4180089249967854, "value2":6354312606503764778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_9":"j","key_1":"d","key_3":"h", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341779, "value1":0.7379607607767219, "value2":5150239222005969615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843413, "value1":0.6687750316343205, "value2":2316529591090571268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.402005, "value1":0.9141363896275393, "value2":6270246938486716288, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128379, "value1":0.5150067708119237, "value2":678394980878451314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108482, "value1":0.3129644434816646, "value2":8384633845422523721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_4":"a","key_9":"g","key_3":"c", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.594874, "value1":0.2519824660631957, "value2":6886759477377939773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445205, "value1":0.903159708784011, "value2":7034171128847343546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824198, "value1":0.6204070552794421, "value2":1059263092990498226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.159620, "value1":0.8686561618779289, "value2":5404253444446120581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.365387, "value1":0.4572890031222697, "value2":7934471101097198056, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521775, "value1":0.36902024174358433, "value2":2258863231460147617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.553157, "value1":0.44513850096108715, "value2":6217475414573430294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141433, "value1":0.21220874698804243, "value2":93944351046750038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.811456, "value1":0.8670934692477834, "value2":7403345830074134000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003892, "value1":0.4239005643048954, "value2":7762289626422779835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.067882, "value1":0.18864986614817061, "value2":2907310668904743062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.750570, "value1":0.6373780024843834, "value2":3686123513663951346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.386866, "value1":0.2003273945539795, "value2":5992425398301380670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.515245, "value1":0.516857446811694, "value2":4876668434287590384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389071, "value1":0.9340489387136409, "value2":7504250901331893981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098316, "value1":0.29683731723021556, "value2":7480572590262205533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906295, "value1":0.7596148064915852, "value2":2005574389285214762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.469199, "value1":0.3474919541045514, "value2":4513611972684272623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735221, "value1":0.44321964186413737, "value2":284056865289241089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382330, "value1":0.4698238258381697, "value2":6544780705530113911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644228, "value1":0.34547957427274706, "value2":5643558758908257061, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025533, "value1":0.9831418878764304, "value2":34382825795091331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564530, "value1":0.1974264877985391, "value2":7202346148984902707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_8":"h","key_4":"e", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.582038, "value1":0.6658499962262118, "value2":4779411307587076137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.048223, "value1":0.28118182352648263, "value2":2406664371776445226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.277281, "value1":0.3293088133074837, "value2":3125287577587960916, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711593, "value1":0.6513318644160435, "value2":6304109109655854952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.136044, "value1":0.15479605681668362, "value2":4561542839689997911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.143594, "value1":0.03881037667597162, "value2":5145183720039449728, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766270, "value1":0.5913740900789849, "value2":2880750585927778789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.330272, "value1":0.03345193869225257, "value2":5177206809108614692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893588, "value1":0.9545617077887952, "value2":1994751413642673810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.200065, "value1":0.7615854624227617, "value2":2555245387882162412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.637143, "value1":0.8697167932948927, "value2":3163242108044176287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.640389, "value1":0.5659009743641423, "value2":7954937488899706120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.558497, "value1":0.928513605246244, "value2":1207841403052993714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.244339, "value1":0.9485629290996225, "value2":8066024682557174519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.368292, "value1":0.1771040772082859, "value2":5158146682102067527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883369, "value1":0.26796315495485606, "value2":500188854907829642, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.603831, "value1":0.695040513930677, "value2":7670486818087410343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635402, "value1":0.17089841071375036, "value2":2188177332357604180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246584, "value1":0.638326848995027, "value2":2467157573588247428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_7":"j","key_1":"c","key_2":"e", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297753, "value1":0.4028871413838985, "value2":549392368948968713, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061298, "value1":0.28566196711825703, "value2":4345219419785285366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468603, "value1":0.15137756224435941, "value2":5857705640176057374, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705772, "value1":0.9900652870935787, "value2":4160023456685330050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628560, "value1":0.47755802914007767, "value2":2886069569872931687, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.201527, "value1":0.3765200270511815, "value2":8192740497964394547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405600, "value1":0.16568713498682006, "value2":807681981221494025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.070083, "value1":0.5411063354146068, "value2":6815859051389366015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.339848, "value1":0.09610559077312085, "value2":5872072911320364504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571126, "value1":0.2497057720663749, "value2":924893218806837423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659128, "value1":0.026188484488162033, "value2":8931856805131420281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_3":"d","key_4":"d","key_2":"b", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.127455, "value1":0.3964141576554178, "value2":2662798392903843365, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780076, "value1":0.15040609462861912, "value2":7522359721454017998, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.108734, "value1":0.6578186778027336, "value2":2079921522814480295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.476594, "value1":0.5388243902425934, "value2":1295082543190183614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377611, "value1":0.4899107617022405, "value2":1810844261495017790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.980760, "value1":0.052036560099483146, "value2":4808591658339710229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.662657, "value1":0.739342249293886, "value2":4457016924188070830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573154, "value1":0.043565004929414355, "value2":2934360743315418976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.792186, "value1":0.7194337918198027, "value2":8482147182498656703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687869, "value1":0.5159874448621534, "value2":5870856463348491189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.522440, "value1":0.8824876555801262, "value2":2534178016362914654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132515, "value1":0.8599678891940701, "value2":8395948226155967317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.096926, "value1":0.9245913183435254, "value2":2358549994381165596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_3":"e","key_7":"d","key_2":"d", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486752, "value1":0.2842029485021584, "value2":2801356184403378281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.475238, "value1":0.22293224326621178, "value2":443811864922674372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.132657, "value1":0.9371037777036672, "value2":4385576391999515299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_5":"a","key_7":"h","key_3":"e", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.549121, "value1":0.03204732941506213, "value2":8714811939867543319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796275, "value1":0.21070900388130828, "value2":8667819799137478248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927343, "value1":0.9159887689007843, "value2":7689292400204431331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685507, "value1":0.7348923960578333, "value2":330857683592078866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.865459, "value1":0.24066397037006473, "value2":203169051837200909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.827976, "value1":0.4488303067264579, "value2":3646968045887419113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900145, "value1":0.31889183176683844, "value2":5999122195912397909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.251489, "value1":0.5593865051855272, "value2":5772018587784459447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191087, "value1":0.2891985736646055, "value2":4399230413942175266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087488, "value1":0.5425699812510858, "value2":3965285909604337765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680134, "value1":0.18266599397918218, "value2":5571070206339456089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_8":"h","key_1":"a","key_6":"b", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551156, "value1":0.8876027936303559, "value2":6010745863587972423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.803161, "value1":0.9394184911877158, "value2":571658979107886488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.054336, "value1":0.03957759543141472, "value2":3918244459790702522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_3":"b","key_9":"e","key_0":"g", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685390, "value1":0.5264564714839759, "value2":2232108588919897592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.161082, "value1":0.32998817001502934, "value2":8460874534662447333, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778535, "value1":0.06811024060980113, "value2":7835773491677016452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832955, "value1":0.028669939437680664, "value2":4299578964430488535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004645, "value1":0.8188081000238006, "value2":9221293293613853860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205474, "value1":0.40545870686171354, "value2":1575377844766226698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639268, "value1":0.8908404617416188, "value2":1540158992274106467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.139661, "value1":0.3668772992670227, "value2":6483649363648949947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924909, "value1":0.16190741337415684, "value2":5257368157508278570, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297500, "value1":0.4330469601614637, "value2":4632520395480129342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101765, "value1":0.4685291307083107, "value2":1614399458247460286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291465, "value1":0.6213831794241761, "value2":7754123020970248013, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369394, "value1":0.5153486584517465, "value2":2263160774029528717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719680, "value1":0.5383036464533613, "value2":3589028105702605678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262982, "value1":0.02336562179973987, "value2":2500396082328773649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.790458, "value1":0.7125005920534319, "value2":8453423485925643300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.937011, "value1":0.9029577396659265, "value2":3493444148478497784, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815751, "value1":0.34331005780857, "value2":4896770691763395578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.732239, "value1":0.5781653462685661, "value2":2815554048397289034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073195, "value1":0.6318398759739543, "value2":1443128294631046938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.335280, "value1":0.2900799960272814, "value2":319439076295666505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375292, "value1":0.8416100389726322, "value2":1104228901216289966, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543223, "value1":0.30539264418826334, "value2":2529051273510780065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850347, "value1":0.9475422757893678, "value2":3815335775316587120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010486, "value1":0.9233328326166781, "value2":1132448560381334143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509503, "value1":0.43078826324145314, "value2":2585538196535057958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440516, "value1":0.36575868540627, "value2":764298189191108299, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.576334, "value1":0.9992971397552057, "value2":8607203945347812848, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.970781, "value1":0.43725394836722203, "value2":8039899278253467744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746791, "value1":0.5653040341344706, "value2":6676447946324724349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636245, "value1":0.8420862976065959, "value2":1562613302672270132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285262, "value1":0.32142916834577895, "value2":5289001385309911194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.417392, "value1":0.049995803596490404, "value2":6607894685909801752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584296, "value1":0.2409216093884783, "value2":1637965971530230928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325720, "value1":0.028777043283946674, "value2":963675377332588860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566004, "value1":0.9002123352889874, "value2":4604125644132750665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.882236, "value1":0.33330368402055666, "value2":156824449049228711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.767642, "value1":0.13763101488639068, "value2":8064121309553799557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_2":"k","key_9":"h","key_0":"d", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.719598, "value1":0.8986180062766446, "value2":1742955026017794842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886873, "value1":0.010775899740746808, "value2":1351720915728762737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_9":"d","key_0":"a","key_1":"d", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484279, "value1":0.5158913513515657, "value2":6695543448765155807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.120818, "value1":0.38000311042969326, "value2":8387696033998291, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.471169, "value1":0.5449952216814127, "value2":7977583153767662789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.692704, "value1":0.644651424950153, "value2":1344814886763463146, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434917, "value1":0.5336620566427158, "value2":7974250102837601477, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.231472, "value1":0.18429722211173638, "value2":239717814769298968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351874, "value1":0.3302574440246314, "value2":116725965202686758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.652256, "value1":0.8832602328157859, "value2":2443207413955073666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392058, "value1":0.3666546013150111, "value2":5949156227430002451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940471, "value1":0.038777989285728266, "value2":2871780422161853348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412070, "value1":0.5197207843322247, "value2":406042026849178989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.571008, "value1":0.7027610078055025, "value2":1383746034138340292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900415, "value1":0.17910816553895145, "value2":401749706711859248, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.718726, "value1":0.2276914412180115, "value2":9221447498523708978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_5":"f","key_6":"a","key_3":"g", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.381086, "value1":0.9677704240195594, "value2":1303001585301279474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893466, "value1":0.2579622668275943, "value2":3971563895105696581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_9":"e","key_2":"h","key_4":"a", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.400237, "value1":0.23652480428212666, "value2":6009422694154041952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481908, "value1":0.625595272995919, "value2":2204722405376257638, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.936595, "value1":0.2680275935509345, "value2":324738255121972023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.983646, "value1":0.3650479381459544, "value2":6655499432411034428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_9":"d","key_6":"i", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.489964, "value1":0.6666717547984726, "value2":2225549269590513897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641209, "value1":0.578327662161616, "value2":6547750503066251015, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134486, "value1":0.67209109592655, "value2":2885152331577461710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073911, "value1":0.08294753363850706, "value2":7590236594974272032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181170, "value1":0.5500056163737558, "value2":6771546940482209584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.029045, "value1":0.6184324246909119, "value2":8362437134419045707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162843, "value1":0.4243438443026443, "value2":8729664805668824103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.808018, "value1":0.9740191657641797, "value2":811057613633969138, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.509186, "value1":0.5860682567407373, "value2":4715370569695436166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664283, "value1":0.9354682602818816, "value2":4771667149292350278, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220707, "value1":0.9895245632099, "value2":7384968594835669289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.389342, "value1":0.5213334242916959, "value2":9064052527115170337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977547, "value1":0.9968413793490465, "value2":1018668683107683896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122093, "value1":0.22725801593699438, "value2":471678573457243344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_6":"k","key_0":"k","key_1":"j", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474960, "value1":0.6106407332478643, "value2":6142821675445007328, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770612, "value1":0.38366813133853406, "value2":4771686788392964587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166046, "value1":0.6731562334089229, "value2":6765933549203517189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_7":"e","key_2":"f","key_4":"g", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000435, "value1":0.7725469520964668, "value2":1526254098614108268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.596567, "value1":0.33143109459424, "value2":6988963074743099921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_2":"f","key_0":"f","key_1":"h", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832348, "value1":0.5082718126919952, "value2":7366669828299681751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_5":"j","key_7":"b","key_1":"j", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.933461, "value1":0.2177858524105436, "value2":1927680313692641282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_4":"a","key_9":"h","key_1":"h", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800386, "value1":0.37752422688739345, "value2":3623415676274096956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602632, "value1":0.9591290646171173, "value2":149144865605034806, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805853, "value1":0.46005104238615857, "value2":5399494644643471050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023064, "value1":0.2589954384060476, "value2":2161234059148173478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.663052, "value1":0.4340276216577473, "value2":1651362675477804080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425648, "value1":0.5284587146267693, "value2":2204338428823542625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.297837, "value1":0.9930278173662268, "value2":1153109222914733395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.661230, "value1":0.3189372627835483, "value2":8950530915947476877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997529, "value1":0.9931406569786665, "value2":3171376634461957806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785991, "value1":0.6115707472871859, "value2":4480386318218501447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909727, "value1":0.29199552201380746, "value2":3686357880544622730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155119, "value1":0.5470008889969509, "value2":7858820522318450111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_7":"e","key_4":"a","key_5":"k", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.530629, "value1":0.9873625937302516, "value2":5471095001682678938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512132, "value1":0.14465321479404997, "value2":8852396394387214520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.643903, "value1":0.81607168234715, "value2":7750000008183770029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088371, "value1":0.9834997161657205, "value2":8265123019714389041, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.840291, "value1":0.31720704974294733, "value2":8838522133270072113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607394, "value1":0.6158114593013463, "value2":3920524769932432062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.891225, "value1":0.7260707014711243, "value2":8545845285837175547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155524, "value1":0.5324837252402642, "value2":6050562650984829349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463368, "value1":0.3636429297340659, "value2":7366387480034568615, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.051267, "value1":0.21240586224005645, "value2":3520112151921939439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.713935, "value1":0.6148054965944443, "value2":4435199374933360163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229994, "value1":0.877849746015665, "value2":4328772383519778289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.394088, "value1":0.7419214909083843, "value2":1533570891823401045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311200, "value1":0.6547222273874411, "value2":603076909255570468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593460, "value1":0.33723617074445394, "value2":4429464301654712087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016002, "value1":0.7850512802013139, "value2":9218293660844009579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293893, "value1":0.9024192531760731, "value2":874814059564514510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197015, "value1":0.6346018344137921, "value2":2275287342249871781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444015, "value1":0.07476763645648904, "value2":6294420894284163263, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.137036, "value1":0.7149572150590823, "value2":5890716035999924313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382203, "value1":0.2373007200794848, "value2":4237441552449229366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927338, "value1":0.5067232894842505, "value2":7166709656823836078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.856975, "value1":0.7551172442205076, "value2":3114426699624764241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290618, "value1":0.461652630545808, "value2":6870191814019559455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194719, "value1":0.28972301156720964, "value2":2266210405139781750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.930146, "value1":0.8041428167626514, "value2":3758081748375193976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.876259, "value1":0.8601408243336739, "value2":8153013110856241749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544542, "value1":0.4588432483941638, "value2":1078220017088765885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861984, "value1":0.33210767773987954, "value2":6423637803003548004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_9":"g","key_2":"g","key_6":"f", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725768, "value1":0.13821724766383348, "value2":2685436575194046233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_6":"f","key_0":"k", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422624, "value1":0.6174306852475553, "value2":622178413482526248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.749470, "value1":0.7804428540523304, "value2":6875918761098582661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.903503, "value1":0.7328633216838554, "value2":3622697451467218798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.116528, "value1":0.9193757905595477, "value2":6857103271674313745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318972, "value1":0.35822522818261543, "value2":8829601973990551809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433538, "value1":0.5317306440222773, "value2":2556858263334639046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497594, "value1":0.1404686557588262, "value2":5182788796116063454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.731708, "value1":0.08352722566425953, "value2":57349948699707298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052659, "value1":0.5013772188087474, "value2":4075548779807119976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680970, "value1":0.7475466041318248, "value2":804362071276609030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644235, "value1":0.13991370513704515, "value2":8012476351028838411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.430709, "value1":0.07096170614998987, "value2":810033186472659399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.283029, "value1":0.49501003784947184, "value2":5335533898113400696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.213964, "value1":0.06985861165056967, "value2":3236599269070483646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899939, "value1":0.8432631992057813, "value2":8331656940556283499, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447604, "value1":0.4633606486187383, "value2":3282862092550905414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986819, "value1":0.31082999792855004, "value2":5075360107904774840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.375656, "value1":0.6617351656082016, "value2":8930874302935418560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_5":"j","key_1":"c","key_4":"h", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787774, "value1":0.5242892599116226, "value2":4186364548758456678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918288, "value1":0.3268848136936015, "value2":2614642728372801254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535191, "value1":0.176268908097137, "value2":653040557920632015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.529661, "value1":0.35212458808041835, "value2":1525600014751099598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.361165, "value1":0.4714568379001177, "value2":4598533494631122989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898032, "value1":0.22769956750018774, "value2":5631589457641828317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_7":"c","key_1":"f","key_5":"f", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483853, "value1":0.4317121876757055, "value2":564210499810148536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512418, "value1":0.14841146100443744, "value2":6375007633580482412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172575, "value1":0.4457725144006962, "value2":940411418055250112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_6":"i","key_2":"h", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109679, "value1":0.17465206890579033, "value2":619954528132439983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.016784, "value1":0.6172432789077167, "value2":8694611599799563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405055, "value1":0.9156938137116516, "value2":1197968795715040120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.153068, "value1":0.6193599211433017, "value2":3101256038412073082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041261, "value1":0.0029965595633104653, "value2":1337872063966920132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.852796, "value1":0.021368920853982197, "value2":7987040863826958067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_7":"e","key_9":"k","key_4":"b", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921132, "value1":0.47595655072486737, "value2":1700855145658274194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.383399, "value1":0.26371142871266584, "value2":6832011082222328508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.332123, "value1":0.8824714441278855, "value2":4889159017232292596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_5":"a","key_7":"j","key_0":"f", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847690, "value1":0.5825485094693434, "value2":8301349950043802905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004022, "value1":0.963420472883698, "value2":4714062367056435117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178319, "value1":0.21282157659328568, "value2":5741804038235426956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226035, "value1":0.614117409288991, "value2":7916317937107113091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188200, "value1":0.6715893678504037, "value2":7136230473959043902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709476, "value1":0.7492754619759127, "value2":6914951663040962797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_9":"k","key_4":"k","key_7":"i", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879839, "value1":0.9373684799176515, "value2":3333434004719572493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174084, "value1":0.14484188554360505, "value2":2035242236177706450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.505125, "value1":0.57329431354178, "value2":5119683575480371621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742853, "value1":0.7629472821449486, "value2":5737967107040353117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.226226, "value1":0.17730887465740228, "value2":4673386690110010077, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560859, "value1":0.9766459300795906, "value2":1996912721458005085, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262060, "value1":0.05094335959612934, "value2":1245357682330480297, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432269, "value1":0.8390455137158405, "value2":1878222408225434076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113677, "value1":0.7363256343069756, "value2":2885521712019062646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.444967, "value1":0.7955117384805845, "value2":264780152192144506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.162729, "value1":0.6712123871834628, "value2":1119778679298263778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.911266, "value1":0.9891407670464004, "value2":9103652755878870157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379007, "value1":0.2889256375011702, "value2":5327006092087239478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.092408, "value1":0.21632368500444782, "value2":3442219206298318473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032852, "value1":0.524214031146113, "value2":6530887848447144232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.382209, "value1":0.7113041138812657, "value2":933816269978698137, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.334638, "value1":0.7695287307455443, "value2":7489040093668157103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_4":"d","key_7":"f","key_1":"e", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372671, "value1":0.7439130206835773, "value2":4959199765009131376, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900273, "value1":0.9052840812209041, "value2":965461062104746729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.654196, "value1":0.8121385984212723, "value2":5665296699467259244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178571, "value1":0.19948342559708582, "value2":5100240681564871548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.770922, "value1":0.3886846963277322, "value2":619589477307797843, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.623448, "value1":0.010709272715713756, "value2":2405417022197491275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.501430, "value1":0.49322743442420075, "value2":4840878662359361238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268501, "value1":0.9623480970104531, "value2":7490234459966550271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_7":"b","key_9":"c","key_4":"g", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.403202, "value1":0.7953121912476196, "value2":6585060526257826515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078706, "value1":0.6781344722473363, "value2":3204877843782827661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.343846, "value1":0.7227864686817432, "value2":3271852738757870545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.026648, "value1":0.47137422416076813, "value2":3734918729615183416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.691254, "value1":0.7660887564670621, "value2":790090719852690753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262583, "value1":0.666841123381339, "value2":2785906959504169261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.234934, "value1":0.28467055157973764, "value2":5751420159217569904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110004, "value1":0.6775088488593821, "value2":3553287660015547262, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.766422, "value1":0.23227174275211976, "value2":623571748203397381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.800936, "value1":0.31353934764817454, "value2":4270432397290565810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_7":"e","key_0":"k","key_3":"e", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.709087, "value1":0.45637800363407555, "value2":6299983262198737196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756471, "value1":0.4807492151985086, "value2":7279712801686741955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188086, "value1":0.36885137660554584, "value2":7300782007111153833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.948483, "value1":0.37900022039799186, "value2":659834049221243616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.589486, "value1":0.790378840630986, "value2":5696488550645422430, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783617, "value1":0.7580291430060272, "value2":3320953914407054988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321932, "value1":0.6227238552522466, "value2":8556762730182657496, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585825, "value1":0.8340787864264869, "value2":2874883662014238947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.425246, "value1":0.4329750194122766, "value2":6011155334107441536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023097, "value1":0.474467449495024, "value2":9096564500742879948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625423, "value1":0.1682038891195739, "value2":4431847776626169211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289832, "value1":0.9685809921059807, "value2":1593648503211538110, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613185, "value1":0.10083138549838662, "value2":6012379324790188749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720225, "value1":0.22688014499451828, "value2":533494308206781076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.925929, "value1":0.3145810371686551, "value2":2521880979655219825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_5":"j","key_9":"f","key_0":"a", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073559, "value1":0.8510652378781342, "value2":2771699030834759801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540880, "value1":0.19703714704465763, "value2":2229904355406996428, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_1":"d","key_0":"c", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358372, "value1":0.47865214913286414, "value2":991745116280869642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.985891, "value1":0.5712880806403865, "value2":7108232781064624938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.787182, "value1":0.10623777547287752, "value2":5757177518524775613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.725343, "value1":0.4881539523572101, "value2":9008775172737657154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.276129, "value1":0.7963976864976252, "value2":4233311333311619132, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.296009, "value1":0.9362536518865696, "value2":6472012225611436212, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572284, "value1":0.5256827892320936, "value2":6967364447645723612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490559, "value1":0.5321215426269078, "value2":6272951345279554009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_6":"c","key_8":"i","key_1":"b", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.897929, "value1":0.9148681755770618, "value2":8519086271296505773, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438881, "value1":0.9657537469163081, "value2":1825676918798292475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.878233, "value1":0.44284605438890395, "value2":948269923000784878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_8":"g","key_3":"b","key_5":"a", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.240505, "value1":0.2373413838092857, "value2":510374534141290052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.041417, "value1":0.17592672383423122, "value2":3473808065501665173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_8":"k","key_9":"a","key_1":"j", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.191879, "value1":0.3751219146996357, "value2":7106698191372720163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694969, "value1":0.8445486295845246, "value2":672504742133677359, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974994, "value1":0.6654201856510823, "value2":4512443800515268537, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.265020, "value1":0.3977309051692021, "value2":987280456712575618, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_9":"d","key_0":"f","key_7":"k", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.837282, "value1":0.6667916772962499, "value2":2649317741597325862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584752, "value1":0.8184998685164273, "value2":6853243233010504233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956207, "value1":0.20740750871245983, "value2":3701599145151889680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785977, "value1":0.42059150301577825, "value2":4300206568693523992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206794, "value1":0.4065971742289675, "value2":5055750687883280777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.466356, "value1":0.24692397834834437, "value2":6503690772355582431, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_5":"e","key_0":"j","key_3":"d", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374485, "value1":0.27439926954774063, "value2":2257827689674118963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859317, "value1":0.15734285025101918, "value2":2188339349886556422, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.888728, "value1":0.2275635074824085, "value2":3959472993254545104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552286, "value1":0.8140316248314068, "value2":28562363718590634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.101732, "value1":0.9986972418263012, "value2":6625998065774179710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.064398, "value1":0.4719546471748007, "value2":5016107261432878232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540095, "value1":0.05564657854486056, "value2":2892021965316178106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052034, "value1":0.6085312696710864, "value2":267981962928859635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479153, "value1":0.1864971302120239, "value2":4274890765687650257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.285637, "value1":0.8618179927379825, "value2":98273395449385741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160949, "value1":0.1665709809492651, "value2":3549197933610586412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479807, "value1":0.9792753435060556, "value2":544874846395596701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.715679, "value1":0.955575677088658, "value2":7351930305259561505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655220, "value1":0.7716326625888155, "value2":490280663710319116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799754, "value1":0.26438748028137354, "value2":2889376751051793411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_3":"e","key_2":"a", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.625103, "value1":0.1721904108442133, "value2":7991543743244408523, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.690861, "value1":0.007633155868195746, "value2":3073219921658712367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308852, "value1":0.12391196742809665, "value2":1115580635235450437, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.013227, "value1":0.7267200692886595, "value2":1429240573369570759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.268822, "value1":0.6684636760714637, "value2":4339881778709198457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_5":"f","key_2":"i","key_3":"i", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.160477, "value1":0.5396173638949372, "value2":4906545989468275563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599858, "value1":0.07893489799204395, "value2":410345370022207222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584892, "value1":0.8068626341311561, "value2":4979004915398024459, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.987465, "value1":0.8935603600443448, "value2":8149710572361609309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227032, "value1":0.3687576237691273, "value2":1391051851919204878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833201, "value1":0.32203470987301264, "value2":2081318068679055279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291063, "value1":0.034071004249405484, "value2":5342977357585868177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.290178, "value1":0.3044083195032883, "value2":4841141697891251443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109841, "value1":0.9228991323104464, "value2":1691775942527270832, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587504, "value1":0.5332728718746681, "value2":1334529969932810461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.920462, "value1":0.18350284649923848, "value2":2590094361323084553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_1":"b","key_9":"j","key_0":"e", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.966666, "value1":0.25923537198192, "value2":6116285159120956507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480100, "value1":0.8741323699909098, "value2":1633286453079731557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.175764, "value1":0.46174408884851265, "value2":5588956365079776400, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_7":"j","key_1":"i","key_5":"h", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357949, "value1":0.9471421832392611, "value2":2020538223652392065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087613, "value1":0.8391096943297279, "value2":1883071449916289062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_6":"b","key_1":"b","key_3":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883487, "value1":0.5961374967262479, "value2":6748555704132728387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.514904, "value1":0.43082646153613147, "value2":3827154946167603504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.899576, "value1":0.926252451999718, "value2":6912098010507428750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.708901, "value1":0.5214095428253637, "value2":8239976493295716264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.784267, "value1":0.7119074977387614, "value2":5875315435697946397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.318238, "value1":0.2857683698624022, "value2":2168440855638261641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434779, "value1":0.728469975577827, "value2":4027120624532411802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.432104, "value1":0.9159655292186285, "value2":1266477315347168519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.453551, "value1":0.9820731220635184, "value2":472902679452044251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.239100, "value1":0.05657570742681277, "value2":1082500030056759801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.918697, "value1":0.5552870083342735, "value2":3402695849932536811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689901, "value1":0.372073062318352, "value2":4891331242898882829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.924258, "value1":0.5283619937827609, "value2":559122472611611400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.172686, "value1":0.4344219735503553, "value2":1248792109975059381, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894193, "value1":0.43426860175202114, "value2":3497813026288966485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_5":"h","key_7":"h","key_2":"i", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.327909, "value1":0.3878605121485491, "value2":5047756483827083362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220825, "value1":0.07655058465784402, "value2":5252985818354538541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.742213, "value1":0.38430981024473343, "value2":5620787295533246773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.733232, "value1":0.5555223164703845, "value2":631270712021422829, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.940662, "value1":0.3098977203238577, "value2":5954879855194614083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.649184, "value1":0.3354932964670884, "value2":6051779552546236650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.832762, "value1":0.9063274954717605, "value2":3724265762255192701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.385652, "value1":0.5036946685314467, "value2":5109085475601380385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.722579, "value1":0.772640832685559, "value2":3780594965713240464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655329, "value1":0.29356054638145535, "value2":4596604760791157242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.459104, "value1":0.8777447945394818, "value2":1186981345641271839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.399975, "value1":0.025674080171056102, "value2":8964167090414725476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.123918, "value1":0.9272811881045043, "value2":7218529778562540903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747920, "value1":0.9264629051459597, "value2":8464913590880556582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914960, "value1":0.32245869851475545, "value2":9109063864698425563, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.328190, "value1":0.7652891376156531, "value2":3024374861601192187, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.593397, "value1":0.17904346621073747, "value2":3899031855173978925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229837, "value1":0.5384685110923013, "value2":5235158233416481121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.698328, "value1":0.5982638259890453, "value2":6740778632490166175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812614, "value1":0.1666081159883058, "value2":2800734738787135789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352669, "value1":0.33792886940688555, "value2":7496164581714310216, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.128711, "value1":0.2991783007626336, "value2":7338677993190210677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.552539, "value1":0.06968528327878533, "value2":8300480680068139329, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.581708, "value1":0.35231544880285015, "value2":1384864101595986439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778952, "value1":0.8415964882413857, "value2":1745115595251159346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.695991, "value1":0.5753223257206643, "value2":1886601101964605359, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.664273, "value1":0.6725491194351899, "value2":5574699884022596470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.711367, "value1":0.2595939895449639, "value2":7814193120722379636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.600545, "value1":0.944128951867823, "value2":6711650903238481221, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452146, "value1":0.9554010846729131, "value2":3169390990279464268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.140642, "value1":0.16345235341026784, "value2":2148518750069655597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388132, "value1":0.5979152819074895, "value2":7002872128153528233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078323, "value1":0.5697531979956287, "value2":980455479278929153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499938, "value1":0.388179798040964, "value2":2192712143484318261, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761109, "value1":0.2600517751619912, "value2":647733851046806648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_9":"d","key_6":"d","key_8":"g", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.088032, "value1":0.2162359479396891, "value2":7960761017907576840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886480, "value1":0.21950291960892157, "value2":7881441150954657573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.538255, "value1":0.36627249562354336, "value2":2697185681710870928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.956144, "value1":0.09482810661100612, "value2":3142048329987868522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.610158, "value1":0.7330550786147262, "value2":4897289423318138878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_5":"g","key_7":"h","key_0":"b", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641699, "value1":0.27867439634792107, "value2":621712637914424018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.218950, "value1":0.1329184934440547, "value2":3565116595465876544, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.247888, "value1":0.7166338804544903, "value2":5367509956565676215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.214416, "value1":0.3332699601933243, "value2":4699030477990361670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.023652, "value1":0.5060732821933588, "value2":1384885559250896583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907328, "value1":0.1694015036022409, "value2":3592921690010847117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.642558, "value1":0.737568796988017, "value2":2406767446522063712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273826, "value1":0.8447613680154935, "value2":6080438111270024688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.984506, "value1":0.28262866688772725, "value2":6480278416681893463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_9":"f","key_4":"g","key_6":"b", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.868136, "value1":0.8910201029103536, "value2":6867531047341490726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687877, "value1":0.8829884505686191, "value2":2249445178190997362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.850677, "value1":0.6276341510695712, "value2":2252184121487700107, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737726, "value1":0.12597210262772834, "value2":1348511802277935140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_6":"k","key_8":"b","key_1":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.810752, "value1":0.2102822839010783, "value2":4831043683200695850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317105, "value1":0.817061302570146, "value2":8324179065679406640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.426069, "value1":0.09982797254913842, "value2":6519984536301213031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_3":"f","key_7":"g","key_2":"f", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.121073, "value1":0.5197148618479782, "value2":3010915938164137229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.264976, "value1":0.4225812260629186, "value2":5303229136813956536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572439, "value1":0.07380111555778088, "value2":4428887607089365223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526101, "value1":0.2368198468823957, "value2":5676247054773494672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.794040, "value1":0.8488571887152928, "value2":7926265788784400661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682501, "value1":0.6419011526811798, "value2":6127958830288275598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.508833, "value1":0.08505936393863323, "value2":6956791791775335476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.485450, "value1":0.24440134625086388, "value2":8675077803244715182, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.341184, "value1":0.7204080689080778, "value2":7288860462608592695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300497, "value1":0.587674524187528, "value2":2136914643668219995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646074, "value1":0.5687487761791935, "value2":5117858885820847437, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.405215, "value1":0.63703146651586, "value2":8601605955587466200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040361, "value1":0.9056350649746174, "value2":4395139702119619272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042190, "value1":0.25258468888663055, "value2":6275160558122438530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436823, "value1":0.3052140189316623, "value2":1176588437754725430, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_9":"c","key_2":"d","key_4":"c", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.885585, "value1":0.996098364991821, "value2":7425087259003526051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.667291, "value1":0.01567259898271256, "value2":688757342279745926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560861, "value1":0.8815928724937075, "value2":5127701164424889645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.921488, "value1":0.5280525030917018, "value2":8702967906065448733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.906624, "value1":0.204252621066632, "value2":7869988567138398448, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.855702, "value1":0.956230561465019, "value2":157946491688041108, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367342, "value1":0.6684623495570281, "value2":329650868812150871, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.445102, "value1":0.6894360816538613, "value2":4920916512388188612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.609045, "value1":0.0027756305160863397, "value2":5637928110041308291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.632075, "value1":0.04166500110036427, "value2":6157318199778992604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972254, "value1":0.31480609981503055, "value2":4364758259634809973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.776493, "value1":0.13104837145124149, "value2":4732079918184171170, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_2":"a","key_5":"d","key_1":"g", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238222, "value1":0.6559225899889957, "value2":3576156993215909053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.434447, "value1":0.24817502461178453, "value2":7546679218765777183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.040709, "value1":0.1860292914746309, "value2":1580899927050287482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.273335, "value1":0.9526744406123745, "value2":5086267667987270481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.052913, "value1":0.33164061518472737, "value2":6494564179935068135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093729, "value1":0.8324601543372498, "value2":4840824205416424699, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_8":"g","key_5":"h","key_6":"e", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.167756, "value1":0.8292588795543363, "value2":926691352131787790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.602891, "value1":0.7468923005026717, "value2":1499084894678948825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939687, "value1":0.9093420550399373, "value2":7624268916058145495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.859436, "value1":0.018264976223293757, "value2":8966202800040990455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.540853, "value1":0.15398968210956118, "value2":1169217513145417422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.927943, "value1":0.6497527028232101, "value2":8964902118693660858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.032864, "value1":0.5440150103669782, "value2":3398396102381979841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_9":"f","key_2":"b","key_3":"d", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440858, "value1":0.5805592700724092, "value2":598543749015746918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.220522, "value1":0.7634462305829404, "value2":4744586200886838973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350814, "value1":0.8753641286301881, "value2":3594959380973533940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.689230, "value1":0.11698455183819528, "value2":3362979353229964321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369880, "value1":0.9959022990686378, "value2":913069268809019275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499935, "value1":0.42435230326696954, "value2":5642082683494178997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.126493, "value1":0.24855180288733794, "value2":2553926153696879833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.241812, "value1":0.19715345400514303, "value2":7966035951898297683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.282709, "value1":0.26568871987724985, "value2":5614603759761174315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634974, "value1":0.5611667195638738, "value2":719923428562838874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_9":"f","key_2":"i","key_3":"b", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.437429, "value1":0.044098595487335565, "value2":7598428545433157434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986084, "value1":0.15643080062821366, "value2":876340157580973000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.350215, "value1":0.5350647081004255, "value2":2039458830523373741, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.263790, "value1":0.13663992546983536, "value2":4887595679013193650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.761252, "value1":0.31778054800300737, "value2":4963237153707981360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_5":"k","key_0":"k","key_1":"c", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358509, "value1":0.4977744678036884, "value2":2417156657544845857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.825502, "value1":0.7927543298826962, "value2":7803607945365925463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.463820, "value1":0.25407613728207273, "value2":7298874514736284603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_7":"d","key_5":"f", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.815985, "value1":0.03419039527127868, "value2":2983524479323907938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648209, "value1":0.5111390924658735, "value2":4469943396931617311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560198, "value1":0.3027175063306067, "value2":8582749896805213520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.293707, "value1":0.7381247559612524, "value2":267296433337218156, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436607, "value1":0.1678795867934854, "value2":2882160820519571353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516040, "value1":0.02439990616294445, "value2":3459503829097802670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.861717, "value1":0.8549050337904404, "value2":5330118776081944380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181801, "value1":0.014687212605387332, "value2":1741497345355098408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398203, "value1":0.3791649455492745, "value2":7827883443566154722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.004152, "value1":0.4627140852343998, "value2":5127693472090561791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_2":"c","key_3":"b","key_0":"i", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289305, "value1":0.9473077760244797, "value2":1972308969161346102, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.407619, "value1":0.22478804543381714, "value2":4393947727507143193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_8":"g","key_5":"c","key_7":"g", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500355, "value1":0.9148638467317476, "value2":2930801640270658847, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390931, "value1":0.03756752782865913, "value2":6450238118865267877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.436518, "value1":0.7302171451231968, "value2":8892033149379515881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.352126, "value1":0.6345536180749114, "value2":8076417892766647115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211476, "value1":0.7869548859510007, "value2":2137142575900025284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.620465, "value1":0.04777740137317064, "value2":2493346722675519040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.166994, "value1":0.5908047667032886, "value2":2133480026426783424, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.799110, "value1":0.7447928303219921, "value2":6367274514210109366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.258756, "value1":0.5720021685376423, "value2":7819250772844902436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.178485, "value1":0.22728962622611987, "value2":5810329748220466185, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756847, "value1":0.09118290676810949, "value2":2438579961381457911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447596, "value1":0.2813020748043202, "value2":4446085339672311465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_6":"b","key_7":"i","key_1":"d", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.507148, "value1":0.8635348228135215, "value2":7235169187543778395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.412107, "value1":0.6168652299086078, "value2":8460560637760416654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.916980, "value1":0.20506020965429841, "value2":1044309486752695204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.358540, "value1":0.5251096014647666, "value2":6638181287796109637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_5":"b","key_9":"c","key_2":"k", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.199047, "value1":0.6942630163772678, "value2":3410541935645339151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_5":"h","key_6":"j","key_1":"g", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.754827, "value1":0.9837922147041048, "value2":3690131697027385202, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_4":"b","key_5":"h","key_2":"a", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.329509, "value1":0.8271404181330777, "value2":4227974754451880800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.513229, "value1":0.672862073275747, "value2":3810363550665232995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910574, "value1":0.9390557529752215, "value2":5133678849432201178, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181846, "value1":0.49307852075676845, "value2":6137025771035845818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.572559, "value1":0.2408765387072259, "value2":8298872358883983458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_6":"i","key_1":"c","key_2":"k", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.078241, "value1":0.08393417297668128, "value2":8837983950648920911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_9":"e","key_4":"k", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.056551, "value1":0.9755466271988745, "value2":2207490383904730952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.814175, "value1":0.5043585864165596, "value2":5350947527006645760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.066736, "value1":0.5177229679064868, "value2":2703347438865819514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.560402, "value1":0.4478041012053205, "value2":117552149576626326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.079515, "value1":0.133747701259907, "value2":5643720599026556636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_9":"h","key_2":"f","key_3":"e", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.246265, "value1":0.8787847052412423, "value2":8596628070583162060, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.011510, "value1":0.3733460731999948, "value2":6601446014598669145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.735379, "value1":0.9116679871732262, "value2":5605502556155572766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.574257, "value1":0.3506079241295861, "value2":1800090074854288106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.900580, "value1":0.8930823791186644, "value2":1843480248185265031, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.744935, "value1":0.754485340555253, "value2":1527411985430067052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.369195, "value1":0.7284643223167715, "value2":2551804296404453319, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.500109, "value1":0.1715238288854149, "value2":3468590424262559119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196638, "value1":0.4131391791311713, "value2":2036494112995161182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660034, "value1":0.5162081878070082, "value2":683174465375000181, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.324311, "value1":0.6392052032119463, "value2":6854067358310903979, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477405, "value1":0.9223995048777075, "value2":8678575474954356403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_5":"k","key_6":"b","key_3":"e", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.894983, "value1":0.2879379950878977, "value2":1760029872301364905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_8":"b","key_0":"a","key_4":"b", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404612, "value1":0.5964910019926303, "value2":5352075776414002891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_6":"e","key_1":"h","key_2":"j", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.406123, "value1":0.9911934759109439, "value2":2505572631856818583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_5":"c","key_0":"a","key_4":"a", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.098160, "value1":0.29265324156352274, "value2":673331618670269188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317156, "value1":0.5284207406945622, "value2":5801235018377104530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.278954, "value1":0.8339844494736254, "value2":142783045099103297, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144316, "value1":0.5981739630713123, "value2":4376705426457644175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.533900, "value1":0.7048057967645066, "value2":7668900249344587055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.655047, "value1":0.9572743468227054, "value2":8615116979377517835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.164063, "value1":0.12955196466904195, "value2":8720744967395487030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.211413, "value1":0.4645239019949383, "value2":1341011821671689315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.254073, "value1":0.9282656526373411, "value2":5675449932048560089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.939659, "value1":0.3774328021259814, "value2":7756643154892601396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.789551, "value1":0.7757594275737888, "value2":8309777310097157940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_6":"a","key_4":"c","key_5":"j", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.256014, "value1":0.7599489616826112, "value2":210625827724423434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883947, "value1":0.13842755442158153, "value2":5303830768512577958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.516070, "value1":0.17914029301657106, "value2":1056091608874344104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.446988, "value1":0.27173267141582086, "value2":6180352463990379834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.622518, "value1":0.8672352642207358, "value2":6931279306288268314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.002662, "value1":0.25623553544743577, "value2":4114415105877902794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.250822, "value1":0.1691133676662619, "value2":8315894568290440082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148935, "value1":0.5989068031663785, "value2":916798823552491100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.404397, "value1":0.3854563629426573, "value2":1729425108588138938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155114, "value1":0.42561573956387977, "value2":3412690494545228649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_6":"f","key_4":"e","key_5":"k", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.185911, "value1":0.7064647650423553, "value2":6268941778110729170, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_4":"i","key_7":"b","key_1":"g", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.349779, "value1":0.9255472682273418, "value2":1166345991907977547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.982678, "value1":0.6891602632892445, "value2":1740831383850752805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.914669, "value1":0.4165113125409203, "value2":1826795354292330747, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871167, "value1":0.7086334187759732, "value2":5543325343112219047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.762690, "value1":0.8505896053437381, "value2":5927978325036218166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.292162, "value1":0.6672535143315814, "value2":1327915299266347941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.311132, "value1":0.47941008598606394, "value2":7437258572827323548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523170, "value1":0.7404188223668129, "value2":5505665204112874175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357326, "value1":0.9538155003965737, "value2":2702198679896121556, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_3":"b","key_6":"a","key_0":"j", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.364597, "value1":0.17680019955559403, "value2":1753707621503560793, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452318, "value1":0.2992929781674707, "value2":8362411304781786581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518625, "value1":0.8647905294441987, "value2":5431066155038766988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.355524, "value1":0.6116936780714586, "value2":1234552975837284329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.542514, "value1":0.8351923165774915, "value2":1733954151005989610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979827, "value1":0.2892109226113143, "value2":6049795733820800226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.883185, "value1":0.011744889551642781, "value2":141159892430595908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.223860, "value1":0.4073953516179897, "value2":6458792499529133548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049058, "value1":0.12482723086065665, "value2":9009567363074084595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.298389, "value1":0.4827417824302383, "value2":4841281853843733128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.629693, "value1":0.6591071306029623, "value2":3355181768485869219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.147730, "value1":0.34536474665893674, "value2":3191505597112674507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.179092, "value1":0.7637896583466016, "value2":1179889369576374895, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.658439, "value1":0.9209377727035774, "value2":3422449294756383390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.071622, "value1":0.28143127213081115, "value2":7766511653375780058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537938, "value1":0.6024534595610256, "value2":8547099742960318907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.646788, "value1":0.20378541750257417, "value2":7978976103929383445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.747803, "value1":0.10859041477345463, "value2":1109588376642858177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.564874, "value1":0.23647658766118684, "value2":6863917686945126643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.042244, "value1":0.3078158837684314, "value2":4419488498667340703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.284250, "value1":0.9387293705744738, "value2":6769306683790898588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.479123, "value1":0.4851789805412465, "value2":6507016270927900733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.025704, "value1":0.8538216321323174, "value2":4020894753483999383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.833612, "value1":0.2626123684082926, "value2":5033990782030407302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.881634, "value1":0.935752736774716, "value2":6384512508877447191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.474427, "value1":0.8815963618627093, "value2":8929077315195001710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321721, "value1":0.8079053030903706, "value2":7509708901019477626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.854474, "value1":0.6950357389111533, "value2":6524710922297840763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229889, "value1":0.6795085741944863, "value2":5609613021607849758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.988904, "value1":0.5223586661895601, "value2":89977508053442787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.374340, "value1":0.755589597933608, "value2":7955537057313043569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.639189, "value1":0.7408346971613415, "value2":1902500838340597252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_9":"d","key_4":"c","key_5":"e", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481547, "value1":0.5004511492728786, "value2":164083172923357290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.518019, "value1":0.729605886464792, "value2":9068451454110926149, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_7":"b","key_1":"c","key_5":"d", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.363805, "value1":0.7382344993077278, "value2":7045722716338732866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.630886, "value1":0.688064557117447, "value2":296768996976239880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_8":"k","key_4":"g","key_7":"g", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.845466, "value1":0.13213135497669065, "value2":4879659946939633689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.660200, "value1":0.2229370900047786, "value2":292379935200020145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073732, "value1":0.8115899146610889, "value2":8577867893469971833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.019908, "value1":0.49801059455223595, "value2":7683222481005280397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_4":"h","key_7":"i","key_2":"e", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.347447, "value1":0.9276573673104045, "value2":2091335224644651701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.321409, "value1":0.6126771681057066, "value2":1893407571657392417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.738125, "value1":0.3569881382718092, "value2":4029761814756595416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.665842, "value1":0.05170172426704011, "value2":8174581603042814007, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.449347, "value1":0.7466823594518665, "value2":6257307241351449210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.588421, "value1":0.4753472682300776, "value2":4728406440983577982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.257494, "value1":0.1386821629125743, "value2":332192645116862048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.728108, "value1":0.43976058351020564, "value2":1985435892430278941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.209628, "value1":0.8966889425339122, "value2":5787236380684241489, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.638851, "value1":0.9617304712840572, "value2":4707670589409962460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.484422, "value1":0.020940662405038862, "value2":1545134502951937463, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440131, "value1":0.17343525627253645, "value2":6009630316902812876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.229422, "value1":0.02146978522387302, "value2":2111892615484907893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.780591, "value1":0.6377996056471382, "value2":8896668703689799570, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.124061, "value1":0.7970190759525996, "value2":7452754965853045737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_2":"i","key_4":"g","key_0":"j", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.450958, "value1":0.8563337518192949, "value2":1606820680288550386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.194998, "value1":0.8092586961600053, "value2":1104653518579575826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.746061, "value1":0.888221791516023, "value2":6881584298104407367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.550870, "value1":0.45584434378222843, "value2":7394447363524184561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.684022, "value1":0.3067771551792505, "value2":1245627384806850593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289452, "value1":0.4582593754084005, "value2":5465272523370779338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235677, "value1":0.8840876989101745, "value2":551093905717196985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.890313, "value1":0.09198263253620918, "value2":6028451346704401787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.168914, "value1":0.40840563117271195, "value2":3931349792536134233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.717876, "value1":0.7410792258916561, "value2":5169350845698434472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.202605, "value1":0.5290546996961457, "value2":3475693437398263961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.636633, "value1":0.824130818871441, "value2":5903284123492948390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.615008, "value1":0.04083331973299552, "value2":7577674506757874978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036559, "value1":0.5696805792367668, "value2":4633771544842561674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367723, "value1":0.355783015228386, "value2":5675601540429514938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.907306, "value1":0.5698956093759964, "value2":9122457118633339454, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685953, "value1":0.4276033626449204, "value2":7616472412744453525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.587193, "value1":0.9520532910468696, "value2":2039460897796769985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.322100, "value1":0.9372598841781506, "value2":8025865217929956411, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.155211, "value1":0.2725517811852112, "value2":7680103519195376114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964045, "value1":0.49419337991273576, "value2":8605447057400658561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.423455, "value1":0.28782941494899417, "value2":7631463815755268967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.433388, "value1":0.6722143467332055, "value2":4743404486443339981, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.073044, "value1":0.9919783220125814, "value2":3546009072923452494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060903, "value1":0.09931972482798261, "value2":3396822536822006988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.952732, "value1":0.3775686488599951, "value2":7121618792612707640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.521690, "value1":0.5851273414170087, "value2":4662473906180780029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.227468, "value1":0.11958534947502218, "value2":3788112644660727919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.196785, "value1":0.8966564462732959, "value2":4357533587354590088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901366, "value1":0.5571789697475874, "value2":7759726578542538969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.633498, "value1":0.9521589976099354, "value2":3412636563124759184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477196, "value1":0.6345321514502629, "value2":2968047284799701894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.245970, "value1":0.8602524837567829, "value2":239564384927138993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.923521, "value1":0.1090517952569125, "value2":8200798429042044530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_7":"f","key_8":"i","key_5":"a", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648242, "value1":0.5170091370557403, "value2":2725553204805480592, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.812069, "value1":0.8884437199979248, "value2":9154279472010215199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962857, "value1":0.2140414709499265, "value2":5080056778998450413, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.561764, "value1":0.6433379926959468, "value2":1246414196804721612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_4":"f","key_8":"k","key_3":"a", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288689, "value1":0.8910605919928337, "value2":8094720078703937078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.419833, "value1":0.6705423885099231, "value2":6631047350002617679, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.085812, "value1":0.03831227212231745, "value2":2735730201464302258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.046896, "value1":0.09940708362873886, "value2":600170391537990700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.884000, "value1":0.5332485748572247, "value2":5278799395808920103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.942254, "value1":0.6825570254740958, "value2":3505708958239891309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.010039, "value1":0.007884797092278711, "value2":3745225427592325971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.712360, "value1":0.2908960224193339, "value2":9147580185923316377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.109158, "value1":0.3741670076640341, "value2":1005531285446043532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680722, "value1":0.010722209112304005, "value2":826737562607372966, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.439540, "value1":0.4713372686055574, "value2":6754252699406092830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392281, "value1":0.9898544758145399, "value2":2353492636354645276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.205843, "value1":0.6788358030320953, "value2":712272724855746801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003145, "value1":0.7114317643988356, "value2":3716874444298014530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.133514, "value1":0.37586422274898074, "value2":2416361460381490648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.904105, "value1":0.18756095304630882, "value2":6928964256284052382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171387, "value1":0.6820596249231228, "value2":172259903859496202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_6":"e","key_0":"i","key_4":"k", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.510204, "value1":0.06734756409519246, "value2":7819819964334132673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422546, "value1":0.1465987153240839, "value2":3085770726305992826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.039220, "value1":0.39594354666033404, "value2":6309816101603497959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.739730, "value1":0.8530985778840952, "value2":7379770578925142492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.672747, "value1":0.7865961192571739, "value2":7774331371169359686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.769896, "value1":0.802526565041674, "value2":3838614021162591095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.304485, "value1":0.6949310789221889, "value2":7501685449967872481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.678592, "value1":0.7504248678928968, "value2":7218481023042482207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764761, "value1":0.20348098581370722, "value2":5988835601145003290, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.968921, "value1":0.21895244723583734, "value2":6407608012483731307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480313, "value1":0.3910222274917921, "value2":935230396616102910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.680717, "value1":0.7679413590875351, "value2":6737105603635406237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.843792, "value1":0.928935788622385, "value2":8032690240756723112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544961, "value1":0.3803375288284628, "value2":808064213573953017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.974943, "value1":0.2873027943668844, "value2":420303650121753052, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.351352, "value1":0.9011332762961933, "value2":5787708950335762277, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049110, "value1":0.920403506389589, "value2":3542056558848585785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.473874, "value1":0.20499658001386006, "value2":6473623955229286943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_9":"e","key_5":"f","key_6":"c", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.705527, "value1":0.3369786416226785, "value2":5565389573084278676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977283, "value1":0.5753145958771464, "value2":5509805379215659395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.969055, "value1":0.7944991097395847, "value2":5227490476089768493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.260822, "value1":0.37247993171967786, "value2":9100748121410350084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.392300, "value1":0.9170516775216153, "value2":3337181235028553814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.917458, "value1":0.7005088931842404, "value2":4249344436835260528, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398079, "value1":0.4412594689419285, "value2":355338602844953023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605493, "value1":0.14953313696992326, "value2":7522955956227719413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.947612, "value1":0.24504972733419753, "value2":19900356272350185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.657164, "value1":0.7554151847082919, "value2":2083009730678034907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.303410, "value1":0.9047440140278514, "value2":2665949305315254597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.720136, "value1":0.02233304294884983, "value2":2148351116720732280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.605342, "value1":0.06576250199061187, "value2":3149195196917185744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.249421, "value1":0.10981969871005968, "value2":1489417694228791945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.490002, "value1":0.2365349300258502, "value2":5413143921150744151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.537535, "value1":0.056469545325025704, "value2":2942905392018720189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.748755, "value1":0.5086783291906242, "value2":4857376881146282553, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.235928, "value1":0.6784255332464089, "value2":3029395751611067716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_9":"d","key_2":"g","key_7":"h", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.401233, "value1":0.753375539931517, "value2":8169340970187425814, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960595, "value1":0.5301594317579601, "value2":3908827718931366422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.113359, "value1":0.7603991477763598, "value2":2119709520548445193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.875780, "value1":0.49587314465392923, "value2":8685987546300538378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.938401, "value1":0.9152835192021896, "value2":1076417540165823423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.869630, "value1":0.7973627027092416, "value2":2126356962780308110, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.388566, "value1":0.6382739532774706, "value2":159979324198275701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.275145, "value1":0.5927046376681012, "value2":7080867181269495541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.523278, "value1":0.5267214411412359, "value2":1728237597853111534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.584632, "value1":0.5083502145633392, "value2":7227399514745308579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964267, "value1":0.6658213068549924, "value2":5519764789585080716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.824564, "value1":0.9264931547852524, "value2":3998899671907293853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.986213, "value1":0.027645492661625626, "value2":3185626691636394685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.634127, "value1":0.858418006506526, "value2":5813619502423233874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.099856, "value1":0.5750664046005645, "value2":7125055104530490351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.685619, "value1":0.3275765619786257, "value2":3507230098472264785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.174236, "value1":0.36236865121536765, "value2":2097215901740320742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093835, "value1":0.8733207807127527, "value2":5480486627610559582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_7":"k","key_2":"f","key_6":"d", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.687722, "value1":0.42815747753178873, "value2":2463297564216975626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015930, "value1":0.035417712050281464, "value2":6849011969803492014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181519, "value1":0.9822679258679019, "value2":408242883289676228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_8":"a","key_1":"b","key_5":"h", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370465, "value1":0.9916450645571185, "value2":3540916962390370495, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.557960, "value1":0.814086825170977, "value2":975702025360269586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.785667, "value1":0.18117401736664177, "value2":8964384630261548826, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014126, "value1":0.6836156690970117, "value2":5150749937802379773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310715, "value1":0.8554125170001826, "value2":6356794838014021178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000193, "value1":0.43591866134299384, "value2":3781675008214883017, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_5":"k","key_7":"e","key_3":"d", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.310472, "value1":0.20656655062551638, "value2":2155883927623577685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348781, "value1":0.1874457704512197, "value2":3156445090401189095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.737752, "value1":0.4107965308340836, "value2":6831391234474352496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.864771, "value1":0.7087377270632053, "value2":3852759604751603741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.377175, "value1":0.302211293966416, "value2":3671141882815450126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035362, "value1":0.11868658008033231, "value2":4402958858442974997, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206005, "value1":0.9050640840246325, "value2":1571018380191846388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.607719, "value1":0.5276900307886362, "value2":2938372021759409514, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.847851, "value1":0.23589306520821895, "value2":8130082209207560079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.181853, "value1":0.3318615047159984, "value2":2814234275379593797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.093707, "value1":0.8735019416998069, "value2":7839357347839415099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.050872, "value1":0.045154096442812874, "value2":1387216811650422069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635715, "value1":0.7372871363614325, "value2":2653034997948603272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.807390, "value1":0.06360776205204009, "value2":8705617981394656683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_4":"k","key_8":"b","key_1":"j", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.893750, "value1":0.033458142425902254, "value2":2550969485909611240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.599717, "value1":0.9713785482184008, "value2":1528762568291094955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.544670, "value1":0.0942081326101987, "value2":5208646740238503092, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.460630, "value1":0.30899325368963804, "value2":6955190987114158639, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.348159, "value1":0.8338319484328468, "value2":382561011167415899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_5":"f","key_4":"d", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.866844, "value1":0.4559148181796855, "value2":4921190547959215054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.779864, "value1":0.8408779744303566, "value2":5151627244176235100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087888, "value1":0.9444160436820308, "value2":2636681313922893724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422949, "value1":0.45533312838966733, "value2":8719456700476032056, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.909073, "value1":0.4871022495216044, "value2":7470894518121500308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.526308, "value1":0.6556527723106481, "value2":2061496006850665904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.483995, "value1":0.01917969989791312, "value2":6594173174887504625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_9":"g","key_1":"k","key_3":"h", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.704096, "value1":0.7335839167893562, "value2":7683462035683402688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.796245, "value1":0.775721209508694, "value2":1762994312464074505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635089, "value1":0.08634955165534107, "value2":5435729701416464796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.146656, "value1":0.904887030211023, "value2":3845389016759808576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.975641, "value1":0.06011625753112877, "value2":2167561119971885977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.027722, "value1":0.7606346737865884, "value2":7806658536996851894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_6":"d","key_2":"i","key_4":"e", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.255754, "value1":0.8709289771166605, "value2":2946981794368069885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.682006, "value1":0.164502402573321, "value2":5375762287724896423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.901286, "value1":0.32721693073724145, "value2":1600743249293213477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.844121, "value1":0.4592683330441191, "value2":5355971233932247551, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.756958, "value1":0.06556927134836166, "value2":2324918632583888734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.686437, "value1":0.5613972309334896, "value2":1964277954793319022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.280286, "value1":0.210108301927216, "value2":4762745887015662113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300106, "value1":0.3451768742896802, "value2":5756080702879459521, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008833, "value1":0.8860643993351358, "value2":5562038867537669443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.964560, "value1":0.5679670350248304, "value2":7270655908716143592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.259598, "value1":0.1779142648395649, "value2":1989791279542261185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_7":"b","key_2":"a", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.289316, "value1":0.27910110845131275, "value2":9108304266352627509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.380786, "value1":0.03821154554853608, "value2":5351444755572082025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_3":"d","key_8":"b","key_0":"c", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.619088, "value1":0.014955635682113628, "value2":1506343674187146948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.991863, "value1":0.4507410741879421, "value2":700372855016807730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.819751, "value1":0.47361005562943986, "value2":397573677309043949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.962980, "value1":0.5040052190830692, "value2":7266525751259633187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.451541, "value1":0.1442134636404948, "value2":561640599502466121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.387998, "value1":0.5301085061271325, "value2":9134048967019410288, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.300013, "value1":0.7373601287356207, "value2":342878676774549894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.008200, "value1":0.7728904842469639, "value2":6539748237741452295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.697595, "value1":0.02404028274324282, "value2":5913828782152003967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.726484, "value1":0.45889774925126936, "value2":8474247967864647236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.696138, "value1":0.8390712125923527, "value2":8640587498190477408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_8":"d","key_9":"a","key_4":"c", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.053954, "value1":0.3443648011087006, "value2":6899937907932201153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.702864, "value1":0.09419193983638496, "value2":1509375003037818758, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.379925, "value1":0.10612710322810476, "value2":7235965724589430822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_4":"b","key_8":"d","key_1":"b", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.065471, "value1":0.03922963726603652, "value2":5920886218601602154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.087291, "value1":0.4838942531586474, "value2":535199259720805783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977724, "value1":0.6375345212159459, "value2":4619853255519698247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566854, "value1":0.21378747510647453, "value2":3467239940916600883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.110862, "value1":0.6797718237586716, "value2":7914262457466633074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.440246, "value1":0.30120840846619557, "value2":3776365109572133674, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.797803, "value1":0.7797287504617298, "value2":5037915665887516284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.586577, "value1":0.1709193163609849, "value2":5198104224414552394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.879594, "value1":0.7450124911318364, "value2":8773848846938448870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.886367, "value1":0.551654132389125, "value2":358026416808351667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.464288, "value1":0.7461865171369122, "value2":8085418698319569937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.197248, "value1":0.8352566024408136, "value2":5751810192674740519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.217337, "value1":0.19284687675681011, "value2":7864252915572676984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448294, "value1":0.42640546323275524, "value2":168095719577724557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.591254, "value1":0.037748263913037684, "value2":2863811536714848498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.372170, "value1":0.6660431538229836, "value2":8567505344595141514, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_6":"g","key_9":"f","key_3":"k", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.317243, "value1":0.18445456220952994, "value2":4074456277125729011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_3":"f","key_4":"j","key_9":"k", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.267738, "value1":0.2691217808870719, "value2":159261331127552033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.585076, "value1":0.21078242008152992, "value2":1239129613801043878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.979016, "value1":0.5956340156534701, "value2":9183614150234962352, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.447286, "value1":0.10494392100402684, "value2":3770086671966420800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.105069, "value1":0.2794799410983503, "value2":3469689160956336636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.173597, "value1":0.6860882854882017, "value2":6360441874685617414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.835787, "value1":0.7543174676127191, "value2":6129127352506150313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.772599, "value1":0.7516717167947417, "value2":8027123646004983290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.115846, "value1":0.3569203207491387, "value2":8576278949637442153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.551211, "value1":0.9246333110829817, "value2":898669648143321253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.230459, "value1":0.04953444106043317, "value2":2309701856093198744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.060038, "value1":0.7972052364471702, "value2":2490346122315313464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_9":"d","key_4":"i","key_5":"k", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.512371, "value1":0.09068245544499785, "value2":3847395208192505439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.783506, "value1":0.7070750057247079, "value2":451775490327949567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.000319, "value1":0.9107891782905447, "value2":4820343479119697886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.628626, "value1":0.016138857727039164, "value2":6452512787294668321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.063279, "value1":0.39758013318800806, "value2":2380900021298276106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.480402, "value1":0.014403992075819017, "value2":6901656754464212004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.462954, "value1":0.20570595420817753, "value2":4308695238369802814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.308777, "value1":0.46154273334740287, "value2":533630259579000143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.006531, "value1":0.41769996010310556, "value2":1003539425076610908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_5":"c","key_7":"d","key_3":"k", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.606413, "value1":0.8392935388898198, "value2":7138000920468723420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390008, "value1":0.9457032273317953, "value2":8816745864308273066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.545540, "value1":0.9980077865004171, "value2":3431998048103898458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.771291, "value1":0.9269154791448218, "value2":2868685560372963447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.035244, "value1":0.15635792222626285, "value2":745780555759481875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.802078, "value1":0.49326419263144117, "value2":7061825021138296948, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.141863, "value1":0.6394029445292002, "value2":4052750041261099856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210519, "value1":0.4447889112985626, "value2":4491375706281220187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.014644, "value1":0.5845639893426985, "value2":8504751940020912407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.457311, "value1":0.39232937571361776, "value2":8948083137978870620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.497330, "value1":0.0554441906288732, "value2":8552872946126060990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.778960, "value1":0.3021332369906045, "value2":5664412036675630494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151648, "value1":0.7180248405530221, "value2":4322565027390602745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.286112, "value1":0.7877440826847747, "value2":6988799327817384951, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.131604, "value1":0.8048208447034889, "value2":7951591877124931974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.122845, "value1":0.26609349854031233, "value2":1608820578917270500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.059135, "value1":0.8932421413801345, "value2":6863045131855298597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.119289, "value1":0.6625474883897694, "value2":3938725492057076257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.114546, "value1":0.1285766450638473, "value2":3866539491041344116, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_7":"e","key_6":"i", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.134976, "value1":0.9092879262740383, "value2":3783610158682087147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.676253, "value1":0.24518334285373874, "value2":5037863892800296388, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.613311, "value1":0.5221731263503148, "value2":6112150162991672535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.448627, "value1":0.09620166167463585, "value2":7262951598026367151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.370708, "value1":0.9034926573578455, "value2":6264024668154911646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.694827, "value1":0.4562036337293121, "value2":5132858458452108988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.409213, "value1":0.2400040814231921, "value2":5692228309415552332, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.272515, "value1":0.512618446397202, "value2":4245362498789762636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_4":"a","key_1":"c","key_3":"k", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398497, "value1":0.9796136787594169, "value2":2854381927687220104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_3":"b","key_5":"g","key_1":"k", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.573421, "value1":0.25717373724181153, "value2":6071818930618476006, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.499699, "value1":0.5015171773350958, "value2":6590026308842608922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.390872, "value1":0.15108026004608982, "value2":4054685864574240684, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.187721, "value1":0.11524534470163803, "value2":4129310514528385220, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.486124, "value1":0.9297806553036565, "value2":2076740459139610542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.506484, "value1":0.48473773192721936, "value2":5279382479053069968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095317, "value1":0.5302104024468535, "value2":1006511502199967697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_4":"g","key_9":"d","key_1":"g", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.659995, "value1":0.7173304922658666, "value2":7272260713937588084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.206009, "value1":0.21353139637435428, "value2":4614697991046896838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.049549, "value1":0.9736103016824452, "value2":713014430698092731, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.764793, "value1":0.5877828915841818, "value2":2743371147788843703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095972, "value1":0.7802447999330678, "value2":1247537151507568078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.210056, "value1":0.13206812499303777, "value2":5119237508780620544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.477450, "value1":0.6302618943375251, "value2":3505999540579430287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.182076, "value1":0.5504514321159135, "value2":2799023815129557532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.262688, "value1":0.9279334258185475, "value2":3960329791115549039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601942, "value1":0.9333313963421735, "value2":8488152958276516414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.095627, "value1":0.2423663973685341, "value2":3147375434225291522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.325183, "value1":0.9698662748883076, "value2":8348906475599130816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.535337, "value1":0.9108408536175355, "value2":548221601502028995, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.465394, "value1":0.20049512890521093, "value2":6918344754347341650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.805179, "value1":0.04912411768594746, "value2":7498908729880858823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.635014, "value1":0.5227012296024662, "value2":6384611575231594987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.481932, "value1":0.2492598145582209, "value2":7063605156324549396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.458503, "value1":0.3120534424622104, "value2":219352200445992919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.144186, "value1":0.6043714345133836, "value2":5343022569080095368, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.148408, "value1":0.7539243961545181, "value2":8912511444056366708, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.357993, "value1":0.9968001216010738, "value2":3427619790755264753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.519356, "value1":0.2148487452898843, "value2":387702087539510555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631948, "value1":0.4990937312036263, "value2":936214783188826913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.631191, "value1":0.03578918325727108, "value2":2415177266799716415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.915725, "value1":0.6672040350791707, "value2":7911702372383761591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.898111, "value1":0.2610115750745007, "value2":6604319492118960080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.871853, "value1":0.700295668840288, "value2":4280817186446527854, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.601683, "value1":0.48046941918756864, "value2":4803180873024223807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.036853, "value1":0.059813074330959816, "value2":4421336201693669164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.870282, "value1":0.09570589043315565, "value2":2716621407072678469, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.543069, "value1":0.31121787546995405, "value2":1276512731094194139, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.367830, "value1":0.09434652170549433, "value2":2256603050688647078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.468827, "value1":0.8402294714187805, "value2":299914447978500025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.910520, "value1":0.6890435117531037, "value2":2224012689460052929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.291500, "value1":0.2581942233541333, "value2":94642526795868830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.972763, "value1":0.39798358703466546, "value2":711133600064208053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.398873, "value1":0.007238280134283497, "value2":414930643350925093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.003583, "value1":0.3709596459020853, "value2":7158538565937127848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.288974, "value1":0.0033772263476846164, "value2":3505644260463770547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.989217, "value1":0.7234413065529468, "value2":6102862603791704696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.151126, "value1":0.962243743917813, "value2":2796295067408464675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.313076, "value1":0.4725232716261173, "value2":5800110791744456113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.169414, "value1":0.10364598232077965, "value2":8013370755699143486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_4":"d","key_7":"f","key_0":"d", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.641143, "value1":0.3874568519220252, "value2":1865317425913683745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.960408, "value1":0.6444887127846117, "value2":2021509390881885067, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_7":"f","key_1":"h", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.188655, "value1":0.9671102461325771, "value2":4499002439335266005, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_8":"h","key_9":"a","key_7":"e", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.438594, "value1":0.23438356733095106, "value2":5880329494012383454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.043142, "value1":0.8137602822079195, "value2":5072736510183548575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.566884, "value1":0.3037390436862004, "value2":1427943088173652054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.528095, "value1":0.06744248220950605, "value2":2617279385140322030, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.075656, "value1":0.6248261098594748, "value2":3102580216650467270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.644300, "value1":0.11077089127760144, "value2":252971746503923271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.061068, "value1":0.18796880019702752, "value2":1108553918392505010, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.996996, "value1":0.5752434742054809, "value2":8128574670569848168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.015917, "value1":0.06928565567804344, "value2":6283279207443485305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_4":"e","key_0":"d","key_1":"h", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.238217, "value1":0.29468486256767473, "value2":4879109468201269125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.302561, "value1":0.16374126420414117, "value2":1267840542849218667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.879665, "value1":0.06081772906299747, "value2":7735358902544562393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.436414, "value1":0.9145077406345081, "value2":725040121642504252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845550, "value1":0.6355646354104764, "value2":2412964869373168042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918386, "value1":0.3307533061620177, "value2":4567564826294647319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310744, "value1":0.5825111373150833, "value2":6000767898591093764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206794, "value1":0.8760021186600452, "value2":3359095928804782209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.237359, "value1":0.7515468979685085, "value2":4507098732286155978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247625, "value1":0.9212103536180063, "value2":1210374309278913705, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.531076, "value1":0.9294917705494895, "value2":6008294356031388873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258474, "value1":0.4801885775502877, "value2":4809296905208303773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687251, "value1":0.5503479981842399, "value2":1159057662539665025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294328, "value1":0.579524047000884, "value2":8591427275775614498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030329, "value1":0.3897745465299769, "value2":8195510768796559399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591813, "value1":0.4311576035013457, "value2":57188656194062821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_7":"e","key_3":"f","key_6":"k", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258460, "value1":0.19968970719511664, "value2":6006706732580329556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153359, "value1":0.6163826197572554, "value2":8075354818413450461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.977991, "value1":0.47088424321623057, "value2":1437620300807340754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.554048, "value1":0.1312819884002532, "value2":2639581932761358442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.829720, "value1":0.9986118788512084, "value2":3816923447231661055, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.648184, "value1":0.9987080219677139, "value2":7493304631161246772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.171705, "value1":0.6479656070812448, "value2":940500361907630226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.452962, "value1":0.7379240782521741, "value2":8607242240788556904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.422715, "value1":0.29055682202125643, "value2":2609014884729200526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_8":"a","key_1":"c","key_7":"h", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.997247, "value1":0.03493075047200949, "value2":1341694184124582251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_5":"j","key_2":"d", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:04+08:00", "value":0.454967, "value1":0.4083872840808467, "value2":7327414310312171249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268085, "value1":0.9598378795022791, "value2":9180787761997570552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.185765, "value1":0.869366023599698, "value2":6065348312952390593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954707, "value1":0.29066212545734305, "value2":6500179214577769106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134999, "value1":0.0746058596733823, "value2":3954294580008112843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_7":"j","key_1":"b","key_4":"h", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.056701, "value1":0.08887774822608922, "value2":2624540757429278414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503714, "value1":0.7939975169235949, "value2":1543165523760841555, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.407434, "value1":0.5263350258249397, "value2":6207484012298867707, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.216887, "value1":0.6135120862192558, "value2":6706252188296828465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950767, "value1":0.6258842267416237, "value2":6943474392706233335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409499, "value1":0.39019457036616395, "value2":5543165311657950313, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.064967, "value1":0.25118206025688317, "value2":4933404061205221336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503859, "value1":0.6260829223715964, "value2":2553798156705274576, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.239884, "value1":0.0646597153433716, "value2":5357886478310061690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.560239, "value1":0.025368350153306324, "value2":1834804451034467956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021890, "value1":0.4250913620299391, "value2":7901663683744276123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.720024, "value1":0.7102429898685084, "value2":8848853087379978689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_1":"b","key_6":"f","key_0":"b", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721708, "value1":0.267345226146574, "value2":8204105067810932755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.688701, "value1":0.8124970825519422, "value2":5072280932532136578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227719, "value1":0.19432503349760097, "value2":2763726347992488568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398501, "value1":0.6725319245300637, "value2":2029542129845625509, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220707, "value1":0.07663396351127535, "value2":6862161694126124253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464464, "value1":0.8033021261492177, "value2":500776341255510367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.369494, "value1":0.6874411246839779, "value2":5472077652556256405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429691, "value1":0.7205162840867915, "value2":2379554410345555228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.003693, "value1":0.5301205012344029, "value2":7339147645001280473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095495, "value1":0.0031080960946583136, "value2":3948518200809126624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.649621, "value1":0.9796250725140294, "value2":5076285879574984318, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.701028, "value1":0.4664187080974912, "value2":8061147057085676917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024773, "value1":0.7030139999067455, "value2":8201442408389043582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570617, "value1":0.6283295712663881, "value2":7893659755244222642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780271, "value1":0.6223497445918998, "value2":4413418515678745434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904547, "value1":0.28169993275009253, "value2":1702651611066126658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.039402, "value1":0.42131084562328136, "value2":3560241450438323083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.612707, "value1":0.19465390726157364, "value2":3161717756488318325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729159, "value1":0.25669937307380997, "value2":1479715553122219203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813686, "value1":0.049715056995271165, "value2":2079826454687289078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747376, "value1":0.7275373884923515, "value2":2435962270026982603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354388, "value1":0.8636816800782944, "value2":6027722589314572513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505532, "value1":0.1663611987871669, "value2":668594150031699725, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087699, "value1":0.09888054207021492, "value2":8612173434517336899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.118211, "value1":0.9755664430251321, "value2":3670406131876856616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.918355, "value1":0.2320424600489978, "value2":8375963807463485059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106082, "value1":0.2496270382815134, "value2":3218140633568902603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.356200, "value1":0.22905701139904852, "value2":4580449910037652861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272156, "value1":0.08246894655199775, "value2":8520120895086499443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.445288, "value1":0.13062843592283024, "value2":1544733522378030955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189215, "value1":0.023728801354414537, "value2":7950194843845547325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525278, "value1":0.8120711270842138, "value2":6926776820889632931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532234, "value1":0.2803691083519268, "value2":2750609942680894663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_7":"k","key_1":"j","key_4":"a", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169390, "value1":0.721935860273012, "value2":323165999073131824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465184, "value1":0.14065806381420595, "value2":6373433541582043482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211347, "value1":0.5119890992803398, "value2":437881008737730578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.326310, "value1":0.8982944342798628, "value2":7550941116286319434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767054, "value1":0.058056603449873775, "value2":8172199064443221808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098754, "value1":0.802732462605452, "value2":5531373846445331861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.626701, "value1":0.9658445273400498, "value2":6144529458195478799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382019, "value1":0.6200649361393201, "value2":74370737764383341, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_8":"f","key_4":"e","key_6":"e", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.162075, "value1":0.773701976515762, "value2":3415021911489973103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452128, "value1":0.4165931264471384, "value2":1592331302702594637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546168, "value1":0.9818897837453856, "value2":2770075967419206536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794037, "value1":0.6329308941123749, "value2":1855788356162220067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267186, "value1":0.5183944510585616, "value2":2971715621482767583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.386007, "value1":0.7925028746106562, "value2":5519530031313056293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194807, "value1":0.9560872102829179, "value2":8899546063621825706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786449, "value1":0.25289379431092024, "value2":3323685962729555247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_7":"j","key_2":"d","key_3":"a", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722033, "value1":0.791566877264509, "value2":8569419440028565177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.527726, "value1":0.5123053424894405, "value2":5410088201011323097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_4":"c","key_3":"e", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991248, "value1":0.00835285017592764, "value2":7876085652270552879, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971469, "value1":0.7516893928865965, "value2":1615677872423733394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457203, "value1":0.2802029059377931, "value2":3774880897765407379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.429963, "value1":0.4064794421282188, "value2":7288375695455796287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807745, "value1":0.1346588472416809, "value2":7295335243976756666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247446, "value1":0.18437680636683432, "value2":2763564622590822709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.505494, "value1":0.36367826246909607, "value2":279237135102501393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570322, "value1":0.807767632694814, "value2":3489957523135662213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.770864, "value1":0.12825646293816148, "value2":559050377523308246, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723654, "value1":0.48061854827645123, "value2":2625015563945376475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939025, "value1":0.31592719165163524, "value2":1661597397006530804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935809, "value1":0.16925319668616004, "value2":8449044609778795224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336027, "value1":0.7017500784708112, "value2":8428599073105193397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.904159, "value1":0.7924061448124604, "value2":4266452884457119467, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847458, "value1":0.27361557683078364, "value2":3345769475113460399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378261, "value1":0.7380019020377752, "value2":432938298263940108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799977, "value1":0.25999970547607815, "value2":7119589132321429408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786435, "value1":0.812994267772834, "value2":2642960787391817015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.587571, "value1":0.5078453077544904, "value2":7413095241087719577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_2":"a","key_3":"f","key_1":"e", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189572, "value1":0.33029349839366906, "value2":182717731110127548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128493, "value1":0.3005744554380485, "value2":1813809549706584614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.751426, "value1":0.43021252312166497, "value2":1048575931846703963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.824593, "value1":0.9962985612048794, "value2":3451457313304562908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913497, "value1":0.19667409773488792, "value2":324965724484330012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280371, "value1":0.6558253377067158, "value2":3238482029594886215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953879, "value1":0.11691624934505847, "value2":3461549771600466582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.189513, "value1":0.9019750283572279, "value2":7575759643142018621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.465264, "value1":0.15533711497405195, "value2":8078121216216233303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857857, "value1":0.2731351567904969, "value2":6433553377051334332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.114548, "value1":0.9868338794536605, "value2":6036587149016739858, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734426, "value1":0.4227117888273388, "value2":1455241552057769722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075679, "value1":0.2701709613339434, "value2":3857058056766739646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104255, "value1":0.981970010025583, "value2":3682624724640104305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653572, "value1":0.008103832880105273, "value2":16321050615205215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.790159, "value1":0.6178089725023603, "value2":2820923101957050608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274981, "value1":0.28628503444016773, "value2":5532572594206077059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274235, "value1":0.01138331892670461, "value2":4146948379716629700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280495, "value1":0.19902196639234795, "value2":7564853449905093412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.272293, "value1":0.017257126721765374, "value2":91860645774780383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447052, "value1":0.6776141143437516, "value2":5897324288007611430, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.209396, "value1":0.14657117940198489, "value2":3976211106646613048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850761, "value1":0.5201711080016932, "value2":7354069105129952055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.300233, "value1":0.1860571279843043, "value2":470481327297902326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.310707, "value1":0.25674808205958144, "value2":2662172304685648211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766353, "value1":0.3692045132202804, "value2":7757321942348800299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214351, "value1":0.6936702854905948, "value2":8248415695462939576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.005940, "value1":0.9564433372149446, "value2":8800540347032950677, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009379, "value1":0.5129617748107491, "value2":2230603715203037945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867507, "value1":0.6850688842908961, "value2":4341430047057401636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_1":"g","key_2":"b","key_0":"h", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640527, "value1":0.8542276865628686, "value2":6337085201397795074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186612, "value1":0.2520653493154805, "value2":3597439324405314019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168374, "value1":0.9686674775113324, "value2":5086009893745928969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.930110, "value1":0.9065446675547199, "value2":9084895148279172527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385337, "value1":0.5310871446244799, "value2":3634604672435791412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_6":"k","key_1":"k","key_4":"a", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273397, "value1":0.6978323941286791, "value2":6270777752769360572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_8":"c","key_0":"f","key_5":"f", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.644440, "value1":0.7390124331493296, "value2":5633558971491993000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692014, "value1":0.5735092134024398, "value2":4018757059763515593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307407, "value1":0.9523210688679733, "value2":4664291077378357176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.864207, "value1":0.2504856063622225, "value2":5125796995165750818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.891946, "value1":0.8905322399035085, "value2":3790600090207208565, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.196262, "value1":0.02637661052612273, "value2":3345947703698603019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609259, "value1":0.9964561610316119, "value2":4297199572790537149, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.479133, "value1":0.573177630994298, "value2":4352137818057928535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.212122, "value1":0.9374174976714222, "value2":2883686750556684691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_7":"i","key_1":"c", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143194, "value1":0.5493618690016547, "value2":8675397367930704936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.911116, "value1":0.8323148117854684, "value2":3727739579448171557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.100156, "value1":0.7882098153425944, "value2":6270723454145661182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159904, "value1":0.5891414895820671, "value2":7892306327256327213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.561342, "value1":0.575909380567069, "value2":2087294011397500886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.542768, "value1":0.7419123808449162, "value2":5892135507379591161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977295, "value1":0.2537719509110862, "value2":23083079770681507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.690704, "value1":0.08136832639928646, "value2":8018440792149947537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.681388, "value1":0.6173470752976815, "value2":4725502695033185312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045833, "value1":0.9052130764103621, "value2":2335561799986333486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.621313, "value1":0.5578705772605427, "value2":5423853801041319612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362143, "value1":0.058506006415061124, "value2":1952601720041809520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.019544, "value1":0.2454839257683829, "value2":5439837990435419484, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414507, "value1":0.8898999762410835, "value2":9399980232748751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.946393, "value1":0.5231814332055934, "value2":8044307915684488670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640533, "value1":0.1401932575466956, "value2":3489584691123939236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.641620, "value1":0.7793690564908118, "value2":6745229093735363060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256954, "value1":0.9792655297711959, "value2":8340241164033802673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.578274, "value1":0.6786422376056035, "value2":2268577288780537910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168131, "value1":0.7011560783500894, "value2":5754899077173699372, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261168, "value1":0.41241864943729767, "value2":6926145892542625665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.142353, "value1":0.026897262060108916, "value2":3407492707727317952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453717, "value1":0.9480329042934568, "value2":5909842310603335433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_6":"d","key_9":"b","key_4":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.498253, "value1":0.2696408414861873, "value2":9093122739488143709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152193, "value1":0.829008206269259, "value2":7872930012981859271, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147195, "value1":0.40564509426795664, "value2":388387455399110782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.756476, "value1":0.08296241719338777, "value2":8819867312676478909, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472139, "value1":0.10622575406945087, "value2":2389837147036501120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553362, "value1":0.4640810980824681, "value2":9129773762938561987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138568, "value1":0.27153851873742235, "value2":3922539618739320056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629854, "value1":0.06560725045609712, "value2":5242856550347246211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532533, "value1":0.6685532806932011, "value2":4403464406211940116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.102100, "value1":0.30269603070417783, "value2":560972484995423105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_6":"i","key_2":"c","key_3":"h", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.594212, "value1":0.3436785616541505, "value2":6118258015340076117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182345, "value1":0.5829131946786057, "value2":316161802601703652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865350, "value1":0.9510270552078569, "value2":5815202444060769343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.417157, "value1":0.07805271825644472, "value2":1671886271645798176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453428, "value1":0.07722736953435244, "value2":5062577774198478117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923875, "value1":0.3869476521576109, "value2":7644772358804477804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170062, "value1":0.771360835653392, "value2":8029933370964328416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.341390, "value1":0.538519995660261, "value2":6637857221246165272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.267222, "value1":0.4666279704151431, "value2":716146669296183590, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562383, "value1":0.35102191528024995, "value2":8428363533739190009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767969, "value1":0.6173626174937589, "value2":6290463280158957319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451339, "value1":0.546379610337649, "value2":1547962669719183472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464199, "value1":0.24671185662577808, "value2":3561721323195761229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_5":"c","key_2":"j","key_3":"e", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.154391, "value1":0.45596856325620033, "value2":6560907831281448338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.709403, "value1":0.6109692366076535, "value2":3486468545935536219, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570798, "value1":0.7335030585660903, "value2":1572258901091495894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.198469, "value1":0.7425214958348296, "value2":3904425769094656355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315912, "value1":0.9245437278546984, "value2":3047808486948520796, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.186365, "value1":0.8721078635865479, "value2":6305488287785962360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759050, "value1":0.3487806615217454, "value2":5791750426523247447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_5":"j","key_9":"j","key_2":"f", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.991504, "value1":0.5895957831739977, "value2":6282966318152098641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355229, "value1":0.507857444797784, "value2":295494213133771851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551655, "value1":0.4307208831464999, "value2":7265410065885482832, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.525491, "value1":0.6404292728685821, "value2":1060635693413562315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_3":"b","key_5":"a","key_9":"j", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008415, "value1":0.6916001000931801, "value2":2353592246165483773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400680, "value1":0.36661865825780576, "value2":9104767548881126156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376594, "value1":0.3981424874885838, "value2":7118511131751820911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.553143, "value1":0.6460744131761798, "value2":1887476981066696157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473253, "value1":0.8348910772414322, "value2":7455007366058138980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457579, "value1":0.4708610456466168, "value2":4593338028669686470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433566, "value1":0.16598645827046257, "value2":5257287213674383265, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.357007, "value1":0.7860515830836307, "value2":2972497673348348004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_7":"h","key_1":"a","key_5":"k", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.461293, "value1":0.35370822925539525, "value2":355204159426372577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.975565, "value1":0.6491904274322989, "value2":2226145731856801788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951226, "value1":0.8792346577420341, "value2":5756323624906456991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.143238, "value1":0.9818710036776283, "value2":4180983964934518595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874423, "value1":0.3287251148209575, "value2":381928105696021842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648299, "value1":0.1842978254216889, "value2":5567179544229053293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055528, "value1":0.2649298342733898, "value2":3063386077105020906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.551525, "value1":0.008929798118195076, "value2":1932022596647743318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078106, "value1":0.6249963076831733, "value2":3775614794809061222, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595048, "value1":0.6556888188689401, "value2":263555282139003693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270487, "value1":0.15652423199780005, "value2":8005167724994681049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986526, "value1":0.44168299905409525, "value2":3616684978189935787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.648015, "value1":0.6935400265504532, "value2":3554010850084269714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082643, "value1":0.4464596047653067, "value2":7393355921025489008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.297812, "value1":0.8411130245970256, "value2":3060908524323233968, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.023535, "value1":0.8979883834169406, "value2":3955075066530261651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775710, "value1":0.803115138047979, "value2":1956023105923958421, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276355, "value1":0.35534087510852963, "value2":70554362711904210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.227561, "value1":0.3475937884638202, "value2":5770217876057854108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.997058, "value1":0.9920212102015171, "value2":4179786645072950325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_9":"i","key_3":"e","key_4":"j", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330001, "value1":0.7117987721576648, "value2":1416148178901813673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_4":"k","key_7":"g","key_3":"a", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.859492, "value1":0.07088427876105753, "value2":5628499584516778303, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.758514, "value1":0.5315142010423931, "value2":3999275359211756172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_9":"a","key_0":"f","key_4":"d", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567291, "value1":0.5976846753929923, "value2":3568009536081310459, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914132, "value1":0.8979176888984427, "value2":4778372357738152583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.470387, "value1":0.2136647847272661, "value2":4632422950916722092, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.180567, "value1":0.7349455061346736, "value2":4238367012667671950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444766, "value1":0.35029664162897833, "value2":4149159141941833270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.351876, "value1":0.719839928132912, "value2":8311035304279403168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.343283, "value1":0.5403744774398413, "value2":7831018924301288402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202600, "value1":0.07323861368397915, "value2":7512140161978615708, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_8":"a","key_9":"d","key_4":"e", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.953729, "value1":0.4072430540967121, "value2":6162477020927495810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206750, "value1":0.7276384026815652, "value2":3867378392166470570, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313521, "value1":0.19760060640640484, "value2":1845943836170488560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774075, "value1":0.9632484532837077, "value2":2186178755453106945, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228120, "value1":0.1138891785769252, "value2":2119625663561385331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.355712, "value1":0.12321943936422765, "value2":5732137663228356419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.004820, "value1":0.1110884226197388, "value2":2265343653685159151, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.630751, "value1":0.6269969213669998, "value2":3359610757917670818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.015500, "value1":0.20581219173107831, "value2":452968813923389028, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646040, "value1":0.14066092038312122, "value2":6343927209062846133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.220902, "value1":0.6865529990306172, "value2":1648919416099165702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.700811, "value1":0.4621690976315305, "value2":3551937779376072568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607198, "value1":0.8914891421096152, "value2":246252330281719887, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.574035, "value1":0.7865743679720645, "value2":5713754277685792561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_7":"e","key_9":"e","key_1":"i", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.473497, "value1":0.13845795308548414, "value2":259488391880499746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944779, "value1":0.9609062478569093, "value2":5705930854457323614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775443, "value1":0.1259913054905557, "value2":5080990926603035088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010365, "value1":0.4903104589413477, "value2":3314614042137410585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.913172, "value1":0.7864192987989552, "value2":8344919384469197086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.796557, "value1":0.037094581426906255, "value2":9011770804417776349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.314769, "value1":0.8877112795540667, "value2":6683425457896895076, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.806104, "value1":0.5962447685862431, "value2":5435268152791547936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579388, "value1":0.6885293468734471, "value2":9064761548833010698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.391022, "value1":0.5900734892123236, "value2":4669259037774416678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762980, "value1":0.7685532202125763, "value2":2315953471058650483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.135687, "value1":0.32474681440848086, "value2":5755607255133419339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104934, "value1":0.14589803567481788, "value2":126178294464524762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.052690, "value1":0.593421743400015, "value2":5119197034484564414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870007, "value1":0.4673411129128983, "value2":2860543686633489085, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.629308, "value1":0.4498616996591619, "value2":5328613189051877451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711415, "value1":0.8849559918426787, "value2":5903543494874730971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901063, "value1":0.5349473283169682, "value2":4797032563086773247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.651731, "value1":0.3127106327077303, "value2":5700343521566497813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825358, "value1":0.7283436835171048, "value2":7844311848127264742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.956544, "value1":0.8853976152911399, "value2":5661939812745981161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235737, "value1":0.26141150008124164, "value2":1680083033155058708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.537374, "value1":0.4021256295776066, "value2":412217863647126208, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238448, "value1":0.20048610554803628, "value2":6383291537020572135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506422, "value1":0.3812390668975348, "value2":184701845674968962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.456772, "value1":0.12479606082790685, "value2":7770738457381111114, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.789484, "value1":0.7412786023699602, "value2":492664655073083816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931307, "value1":0.2536425784942127, "value2":3030103981590386173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610031, "value1":0.5842291904640166, "value2":5028658995770055300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.068107, "value1":0.6578126156370956, "value2":4352952521986270134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477620, "value1":0.16120256759137608, "value2":8502715215777515370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.124418, "value1":0.2689168702571952, "value2":404701828083799050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.779829, "value1":0.4388821199327946, "value2":4899529018433185166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800101, "value1":0.2714233939432663, "value2":584157730559703393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_7":"i","key_0":"g","key_2":"g", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860807, "value1":0.4135766617320191, "value2":3872756991712201227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.917667, "value1":0.2149289211585331, "value2":1070350813804397058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.214604, "value1":0.6792613272879433, "value2":2691750153580462441, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245813, "value1":0.6913342601180135, "value2":6216928639197666035, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.087487, "value1":0.9301668438033497, "value2":6232780192940116715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.069143, "value1":0.6293446581971062, "value2":7108419812354921929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635591, "value1":0.9539262040674916, "value2":2954917330417044389, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672064, "value1":0.8389645710809651, "value2":3571122666192720690, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818753, "value1":0.0576648702049322, "value2":450051931472461981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270657, "value1":0.28617962738097397, "value2":7674542934415081242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_6":"e","key_0":"g", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726745, "value1":0.2340744755556526, "value2":82189729669291091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.897029, "value1":0.2638345630149327, "value2":8238518585440831118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721815, "value1":0.7091199604872773, "value2":6366310396364171134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.130557, "value1":0.49724111456500786, "value2":5852311186811706792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_6":"f","key_8":"d","key_3":"b", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447829, "value1":0.6230142507141527, "value2":937535691807945647, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.802396, "value1":0.6394461194574114, "value2":7242948197847068875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.424144, "value1":0.6000896127174374, "value2":8028530164864078540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.426986, "value1":0.7003205100320555, "value2":2322106140405745021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942361, "value1":0.016353745384231323, "value2":4629676907790639064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653679, "value1":0.37062297658466353, "value2":8208981100462487617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.992030, "value1":0.7072168752768998, "value2":3820991654654691772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.346953, "value1":0.4620611667835753, "value2":4760146202938852105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321463, "value1":0.9135126280713665, "value2":8417775919956392807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168908, "value1":0.05729213648419359, "value2":8181061490241314804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.230852, "value1":0.17085551083436254, "value2":3688016011930153595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.066979, "value1":0.33115248245947465, "value2":398734641986366420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825110, "value1":0.29481497745263574, "value2":7623192343447837676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.979130, "value1":0.18961515202216345, "value2":6037869636552702023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692641, "value1":0.4102441559202999, "value2":2079215473977001869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459527, "value1":0.38418621904384953, "value2":8455919021737289595, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924379, "value1":0.831174718392, "value2":8455941435197243693, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.433952, "value1":0.6847864876960775, "value2":6021961695231181621, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195096, "value1":0.9137826039265785, "value2":1951159443698721147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.392032, "value1":0.7140691740516873, "value2":7387270195217478014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711603, "value1":0.30553043110809663, "value2":5447610537284294754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024988, "value1":0.636153947206049, "value2":6011662688198487566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140076, "value1":0.9829818790209834, "value2":1992400557046890911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_6":"k","key_8":"i","key_4":"f", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.591682, "value1":0.734180708126354, "value2":8165041750078897088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.254962, "value1":0.4530139927706582, "value2":792701874387399785, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653418, "value1":0.6853469695508817, "value2":2445415548541311714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.406935, "value1":0.49945705352987035, "value2":4326827190201670587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.140154, "value1":0.19200910536315854, "value2":7788114053604406936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.774910, "value1":0.775205817556658, "value2":1778011502106771934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224005, "value1":0.4109368947961367, "value2":5090636250181538696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025070, "value1":0.8639998386130291, "value2":4097020029155871346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.110800, "value1":0.46475105418540896, "value2":7930781849523032659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502796, "value1":0.19123829931308983, "value2":4926709577891896623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_8":"g","key_9":"c","key_4":"e", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.483311, "value1":0.21080006327277256, "value2":1394785449775081074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.194479, "value1":0.7780668170411426, "value2":3729639098545975029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_1":"c","key_4":"b","key_0":"b", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.385848, "value1":0.4299644347219865, "value2":7191790268699040877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211519, "value1":0.32521468058588077, "value2":2791560527192035681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228801, "value1":0.83757037869911, "value2":7191574037900046957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_3":"e","key_7":"e","key_0":"k", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.752930, "value1":0.8040696176893675, "value2":1942969218455296906, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.951018, "value1":0.07747976080254226, "value2":1630225104393013299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_9":"g","key_4":"k","key_5":"b", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.750256, "value1":0.816008410350239, "value2":6724933613410762759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583564, "value1":0.4508316797935446, "value2":6917556384524072348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697470, "value1":0.8171672522091267, "value2":2060386169134326462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.974221, "value1":0.7951082158998446, "value2":153021997721922670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134841, "value1":0.655263428147034, "value2":6943977496854991893, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_4":"b","key_9":"e","key_1":"f", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.673465, "value1":0.16441674918821836, "value2":3664243165148767961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_6":"c","key_2":"e", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152562, "value1":0.26528241288165916, "value2":1119343360475809102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.420604, "value1":0.2336675869017092, "value2":3279324159460647609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.622036, "value1":0.6846769209119129, "value2":7351022317760422092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.687001, "value1":0.06651364619191985, "value2":4264970580734046753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.106345, "value1":0.7730270909594562, "value2":6259010316309812350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018243, "value1":0.39163099713401783, "value2":2052648194611616106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576138, "value1":0.9602266945474953, "value2":5355810905366063269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398686, "value1":0.6503809877595728, "value2":8153069195959796847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.367754, "value1":0.6043902819598663, "value2":3335160234939115824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.089576, "value1":0.7877453973247192, "value2":7617166012064737014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.112529, "value1":0.42004273729990893, "value2":2084869905228279863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262886, "value1":0.6795217484416415, "value2":8937461554206260283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652433, "value1":0.15404027924570762, "value2":16114496573329591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.285612, "value1":0.36648611710364465, "value2":4517711436918873094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.213223, "value1":0.283682948749034, "value2":1960872184236727731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_9":"a","key_5":"k","key_8":"j", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096677, "value1":0.1578537002758255, "value2":7008414275581832390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_6":"a","key_7":"d","key_1":"j", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.900633, "value1":0.993822419434507, "value2":3303474850432526688, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862995, "value1":0.2925883436045491, "value2":4890951234376924728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.038886, "value1":0.18850439286651008, "value2":7964663822924622191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_8":"d","key_0":"e","key_6":"f", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742880, "value1":0.8063312642154098, "value2":8220352304335077837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_7":"a","key_9":"k","key_3":"f", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545517, "value1":0.023471717745134867, "value2":5984820965643364202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.079979, "value1":0.6088556886509169, "value2":8961288376687773441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409232, "value1":0.6445899053375183, "value2":992192951010465319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.224358, "value1":0.13573773649022736, "value2":1165195770092489984, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.211580, "value1":0.8798345576210722, "value2":2259526712170036956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.137602, "value1":0.5470546586447269, "value2":2196698895749095736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_5":"d","key_8":"c","key_2":"f", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.950011, "value1":0.5109493788687889, "value2":7352128446573662528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.098941, "value1":0.972587647595324, "value2":4702835811693986573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036022, "value1":0.7940355478266105, "value2":3489006783771959904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.257286, "value1":0.6629731808259061, "value2":6607686064851284173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283256, "value1":0.4735441861177576, "value2":690002256366468336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729363, "value1":0.5152988869648636, "value2":7191544072945571987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.671685, "value1":0.7719299111978523, "value2":8611060565403129118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585682, "value1":0.23795953447079246, "value2":7946332098229864997, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.187797, "value1":0.09090886526352324, "value2":470342401691160594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.617428, "value1":0.5223785659162559, "value2":5891162691146013349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.053786, "value1":0.9753973493719631, "value2":1732499838549362952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.830052, "value1":0.2400470925507307, "value2":7569204709479310112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566485, "value1":0.9031315965779712, "value2":1637870392526502935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104166, "value1":0.8068833505635166, "value2":5557577539110699594, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.457581, "value1":0.31154508147136156, "value2":3306826589254236098, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740978, "value1":0.6517923386066168, "value2":6929527434345999196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398554, "value1":0.7163746669385104, "value2":6895927508923594471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174253, "value1":0.37937402384141433, "value2":1418065909790834443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.168613, "value1":0.8552112788507821, "value2":1201668238332263018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_6":"i","key_1":"k","key_2":"i", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.376103, "value1":0.7428463858250676, "value2":9210155210384521495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_6":"c","key_0":"f","key_1":"c", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.798751, "value1":0.16215960908856522, "value2":4916478620663149838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.588227, "value1":0.7494081995223059, "value2":1969994590907491188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270909, "value1":0.8815916954125361, "value2":2264317244685656881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.466310, "value1":0.09496768292246209, "value2":5092383460942749308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543412, "value1":0.9329719667021983, "value2":5844328151679816255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.834516, "value1":0.9248915367132075, "value2":5869247588035820642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_9":"f","key_0":"j","key_2":"a", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.916922, "value1":0.052803170907048405, "value2":105752679780838825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.583307, "value1":0.04534276867452472, "value2":1081545276806182474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.957758, "value1":0.1893001037569564, "value2":4530859851365168030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723091, "value1":0.22925973916873355, "value2":5896808329469448104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_6":"c","key_8":"c","key_0":"d", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.719902, "value1":0.5038773612977684, "value2":5428690516490754673, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025587, "value1":0.7184099655587061, "value2":4108913234892685798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907580, "value1":0.24492386825368684, "value2":1156962833669313399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958121, "value1":0.9818175985414233, "value2":3857893912179537366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.657555, "value1":0.17952027005330515, "value2":3409773579631767064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829662, "value1":0.9863896179595923, "value2":7017412361402418209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.382356, "value1":0.13377486891358994, "value2":2261611253942664564, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.735596, "value1":0.3414280491857885, "value2":1656516704640942493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.365266, "value1":0.08360581432684039, "value2":751089660510969379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.236047, "value1":0.6527140947106955, "value2":7257801694535836049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_8":"b","key_1":"i","key_2":"d", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.563096, "value1":0.4425122630203769, "value2":8109435442419953734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.870160, "value1":0.7097748830786019, "value2":6394016408878658736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853311, "value1":0.7865716590956611, "value2":9110725490965710227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.103882, "value1":0.9153251610630555, "value2":5340001815935707095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.319691, "value1":0.32338872565142723, "value2":9182777252636363483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971522, "value1":0.5621974159988574, "value2":6803454508761360177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893225, "value1":0.5781000116088169, "value2":3230113190945788558, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_7":"a","key_2":"f", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.910026, "value1":0.6955188069638236, "value2":3766609848094762934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_8":"k","key_4":"i","key_5":"i", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.680766, "value1":0.24230859153208487, "value2":5617809008491748996, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.493463, "value1":0.1697890841270668, "value2":8556153147754124462, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.849699, "value1":0.7552917967316483, "value2":2395400148768820203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091646, "value1":0.5496353344673094, "value2":8512525001992418076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104484, "value1":0.4174390270830882, "value2":8422771009020841833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210780, "value1":0.8954537390033984, "value2":4547239076695924457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.882314, "value1":0.5142219879570834, "value2":7919864591569880176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.983791, "value1":0.42885136319724737, "value2":7969816758490279476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.747300, "value1":0.4699241249924091, "value2":4491569276746468237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939701, "value1":0.6800950779540557, "value2":3198358625582737117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.261442, "value1":0.4575968148914831, "value2":6477186909414818810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.256111, "value1":0.693918133492274, "value2":1509539897942915838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893394, "value1":0.31628967903294547, "value2":5501152751179175693, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.742786, "value1":0.9162968056386509, "value2":9070857057311832045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.642040, "value1":0.9183764345745785, "value2":1183434137529208194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_5":"c","key_0":"i","key_1":"a", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.045625, "value1":0.9635794700047972, "value2":8991017817592303034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.847013, "value1":0.07723071092189422, "value2":6180620156015906931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532846, "value1":0.5943541460929466, "value2":2462231763403080171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514157, "value1":0.7898774054205837, "value2":9059207414611927294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_9":"h","key_0":"i", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.044776, "value1":0.9033139820464094, "value2":6478344621702437696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043799, "value1":0.8974102840071708, "value2":4685725453168510298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.138572, "value1":0.5940072973710391, "value2":9150962581658325077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_9":"k","key_3":"b", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.215236, "value1":0.35464434283570645, "value2":2728457088873951162, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.545063, "value1":0.4562894763140951, "value2":3160291795997224895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.277361, "value1":0.9620522510409097, "value2":2528750129940465586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477478, "value1":0.5247853848955957, "value2":7252619165572966740, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_8":"h","key_3":"g","key_7":"d", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.940911, "value1":0.4288772636544275, "value2":1739941452377564494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.488981, "value1":0.07801581197604958, "value2":3785496084335402474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.037817, "value1":0.20985274999200432, "value2":3757407018866875990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.477559, "value1":0.26133928393072436, "value2":4699711244003996922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.107051, "value1":0.5035109860681579, "value2":2406818127100942026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.692353, "value1":0.44174492856895586, "value2":5849623391254459092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.948736, "value1":0.6131752371911697, "value2":8355568973826156875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.562536, "value1":0.5314554437999132, "value2":2325922727970832181, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336595, "value1":0.8477764608771982, "value2":2899112579581457318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.609668, "value1":0.7984731186839654, "value2":1035702079046481045, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.446414, "value1":0.8536378880203014, "value2":1450863797869138094, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.345841, "value1":0.005289676671501025, "value2":9055262701472338992, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.665956, "value1":0.5725507407225954, "value2":2388492838265413703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.972916, "value1":0.6568884959214099, "value2":4110169872571045957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.370968, "value1":0.9171054847132539, "value2":9195948027770005199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109600, "value1":0.6976295242986092, "value2":3260150427416206815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.722444, "value1":0.756396263144578, "value2":5473984714896425934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.011414, "value1":0.35518020333634837, "value2":5432640933038728748, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.442479, "value1":0.9135774445018765, "value2":1658872104403529722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.860707, "value1":0.1419422977742929, "value2":5745627115489673481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.925502, "value1":0.19711051262279727, "value2":4597556979704859536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.636414, "value1":0.8593669141574183, "value2":3782467267185587990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157759, "value1":0.4115235495685266, "value2":3223572108511746336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399657, "value1":0.24629418245237128, "value2":4491685955930394692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.941994, "value1":0.17062270403788604, "value2":3592023630547953731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282191, "value1":0.8843261061566057, "value2":6691527928807377675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938895, "value1":0.7167094005927336, "value2":1624441217258843945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492025, "value1":0.09243530358924243, "value2":1564407566445130438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837030, "value1":0.7918808546682014, "value2":6330206212330805579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.123530, "value1":0.594358060271596, "value2":1086840755761366155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.515405, "value1":0.8774866546466452, "value2":6471463500843130242, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_7":"h","key_0":"b","key_5":"g", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814527, "value1":0.46973250770262637, "value2":617895520080191082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.712491, "value1":0.8366069393700927, "value2":8124214803409242134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.598705, "value1":0.13529834276697159, "value2":4048137400971693133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.115099, "value1":0.013374612744122648, "value2":4497239763994228002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.871918, "value1":0.5477513867264745, "value2":6929880807654898865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.600230, "value1":0.06263242199896547, "value2":7099533171627178830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.832053, "value1":0.9159517384120022, "value2":6068004913866310526, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497324, "value1":0.7077689276515862, "value2":8496454581274124817, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912067, "value1":0.9966402570764005, "value2":6508382435383037912, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394175, "value1":0.4056622098754871, "value2":7483291783499825043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.931738, "value1":0.7229091089946994, "value2":6319491958713592651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.778862, "value1":0.3546846056067059, "value2":5823244617744051085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.571315, "value1":0.32603322678558494, "value2":617900084765432670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.022106, "value1":0.09634190796880056, "value2":1538435119318752571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777759, "value1":0.9991415497234644, "value2":6469274665586859562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.813159, "value1":0.6009731720422214, "value2":5681496788640905646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_4":"d","key_8":"b","key_3":"d", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990295, "value1":0.866629146873839, "value2":454920947769684349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_0":"d","key_1":"e","key_2":"f", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577898, "value1":0.4068627360956291, "value2":8354382964548721361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.652554, "value1":0.25976343730256174, "value2":4258649045980411819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031275, "value1":0.4835922460947118, "value2":1965749848088972278, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535728, "value1":0.0741614372394035, "value2":2365543114658538202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.725451, "value1":0.08897749963143643, "value2":2568717794069312737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517633, "value1":0.31412074219954, "value2":3756199754285644161, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.170595, "value1":0.31501680305424884, "value2":835864889280146724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.839334, "value1":0.8860221287236891, "value2":7263801537900469082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874807, "value1":0.37435025928000243, "value2":7922739501753662605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.914081, "value1":0.7766285758384013, "value2":8262805777914063347, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.668141, "value1":0.6740415300239384, "value2":5791687526396481300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.302177, "value1":0.7738583815242437, "value2":4861664383009090324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_8":"a","key_2":"j","key_5":"h", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.711096, "value1":0.314472985957645, "value2":8509002774444373125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321149, "value1":0.44826985902573663, "value2":5704588376153110492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.438786, "value1":0.9611261969930821, "value2":8417802456083541652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894152, "value1":0.47571072484930443, "value2":22587301710957954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939556, "value1":0.9226156153333962, "value2":5568759137623866023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.159870, "value1":0.6737683215814319, "value2":23284244914642959, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273159, "value1":0.7150778132424469, "value2":7795879578675436289, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.116050, "value1":0.7176538032312242, "value2":6236044932175094049, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_4":"c","key_0":"k", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274847, "value1":0.2989651835291124, "value2":2096206477458603718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764340, "value1":0.5947263013675924, "value2":949703240129275173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_3":"i","key_1":"a", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.073476, "value1":0.1345576666087966, "value2":2417122739684294792, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418445, "value1":0.8647442373572726, "value2":4787011153359227960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_2":"a","key_6":"c","key_0":"f", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721780, "value1":0.31241733378602804, "value2":2882725760348945089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338609, "value1":0.30106180327338455, "value2":4145007555047407199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.109899, "value1":0.7817730958310003, "value2":5914588161487452188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.547269, "value1":0.29274410504915255, "value2":3689334620249634917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741862, "value1":0.6895772402009777, "value2":4687778247837546339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_5":"b","key_9":"k","key_3":"a", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.744160, "value1":0.7533050071878981, "value2":183319309026503278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.880890, "value1":0.8337197335863336, "value2":5434700933804867558, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467450, "value1":0.7739450951598513, "value2":8969122970020127825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.036765, "value1":0.05604892859180446, "value2":5414621267015229038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458853, "value1":0.3999783567682559, "value2":7084993602991838778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.144253, "value1":0.945939865721221, "value2":9151551595196013169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_3":"b","key_9":"f","key_2":"i", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.519078, "value1":0.28562220040037745, "value2":3466327056358787385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.425448, "value1":0.1561471042921615, "value2":8555609213379362480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_4":"k","key_9":"a","key_1":"h", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.071015, "value1":0.867937414704974, "value2":7843796710543352776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936460, "value1":0.8572536817998362, "value2":3953769786680934648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886358, "value1":0.2289778721765615, "value2":7617650297619769040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.029905, "value1":0.0015729191075447354, "value2":2294660572707247909, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.777761, "value1":0.9646894479758846, "value2":530724906601952281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_6":"e","key_3":"i", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294885, "value1":0.6419477950120894, "value2":3626154680940200317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_6":"g","key_3":"c", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210708, "value1":0.9629469371323777, "value2":1825251501924013732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354794, "value1":0.9560891897559984, "value2":7402084087841622412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.007599, "value1":0.8488357845508476, "value2":9129548229641808509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.949198, "value1":0.3153555717941824, "value2":7434568718980757764, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373802, "value1":0.3277196318709405, "value2":3450473392216255589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.815211, "value1":0.19640953195392433, "value2":6991753246444170561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.532006, "value1":0.8004824485602414, "value2":6264834568239277372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.745472, "value1":0.20136170467216696, "value2":5514377914099787271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.296097, "value1":0.9552161212557331, "value2":1452145138149816534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350623, "value1":0.4834905958063239, "value2":3177892637454509046, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894862, "value1":0.26274803670489, "value2":5904879356296841126, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663989, "value1":0.4617229653712207, "value2":5839673609391532826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920142, "value1":0.4202645422145252, "value2":6322617270554106528, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.174524, "value1":0.7616258127751694, "value2":465160921360548556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.829492, "value1":0.9553481357587528, "value2":1445392053101953093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.419698, "value1":0.7725216991307152, "value2":4285351248890366822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.235468, "value1":0.9784345273084265, "value2":3668509819811778630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.935515, "value1":0.5182554276152802, "value2":3295309486606700899, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539034, "value1":0.37544115144770396, "value2":7306450144895271272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.729718, "value1":0.8597111754781319, "value2":3238304594948466970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538038, "value1":0.32408768170822555, "value2":2186207806521299657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.566784, "value1":0.4892156459829451, "value2":8596226688481136458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.980628, "value1":0.8327067880954203, "value2":8561137068463144173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.153522, "value1":0.15467129518207473, "value2":8602088790336601513, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748640, "value1":0.18204671422495655, "value2":6797988693835752844, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.247682, "value1":0.09249535842888819, "value2":5067988675442272893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.936986, "value1":0.9192193881284415, "value2":5123985662043884348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.378293, "value1":0.815846067412878, "value2":6070693823656037795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734247, "value1":0.49161995996865615, "value2":624464230830574462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.661797, "value1":0.2056464067499432, "value2":6156118294329348494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.033052, "value1":0.35405594685495784, "value2":520776464703947921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.575933, "value1":0.9335712679806217, "value2":425476291317360309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.732828, "value1":0.4700815629744225, "value2":8550463063060596020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.899613, "value1":0.4385944862087238, "value2":7654768751774889730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165683, "value1":0.16899343532059313, "value2":5688582074969409894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.986964, "value1":0.253512537787597, "value2":8166110434329136902, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.766787, "value1":0.6248495968383331, "value2":8035594691046262470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.338227, "value1":0.6533144293221477, "value2":3922786069578896774, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.912364, "value1":0.888149605868496, "value2":7372081699617979857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_8":"f","key_1":"a","key_4":"a", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.223399, "value1":0.21329948027080436, "value2":8415945829680890770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.922973, "value1":0.6987095535905381, "value2":8304144963505893865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.484292, "value1":0.6786625244755433, "value2":595838710724334047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.977202, "value1":0.8656898577077133, "value2":3529785897914282727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.231129, "value1":0.9035229811237685, "value2":2477457900157688998, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.565357, "value1":0.40499024658110055, "value2":1356497801667980388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.691153, "value1":0.5098765403884086, "value2":2450630350992474672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.459909, "value1":0.07976566115765178, "value2":3732709013710166299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104179, "value1":0.3445131016649922, "value2":528513889221012225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.764200, "value1":0.1348858138724795, "value2":409727276676244812, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288182, "value1":0.46853874054271116, "value2":6728001446241512599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_7":"k","key_2":"a","key_3":"c", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.502298, "value1":0.31577497007213595, "value2":8859554815564990855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.990152, "value1":0.3102588503519357, "value2":5723742327078842798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520176, "value1":0.15090023272414005, "value2":5438285255049886987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.155921, "value1":0.6850691286515489, "value2":7357616240486203605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.350044, "value1":0.8819289168133723, "value2":4656926419487785926, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.383040, "value1":0.8254228746494352, "value2":2036646364854508134, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_4":"d","key_7":"e","key_2":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451480, "value1":0.9507122214170446, "value2":4089393907406910121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516899, "value1":0.7707382433506376, "value2":5274734231531309553, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.467177, "value1":0.603964532592086, "value2":1716897544081104224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264870, "value1":0.9573825263948318, "value2":3412285010274210994, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.556842, "value1":0.17542611082882992, "value2":1043370942216175800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.009549, "value1":0.21407922779664104, "value2":5715745607073532896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_9":"b","key_4":"c", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462287, "value1":0.8132271496462169, "value2":7467232826614980432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_9":"k","key_1":"j","key_7":"d", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.538632, "value1":0.7627133328751289, "value2":2219523491939336599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.217749, "value1":0.9344741716490049, "value2":8166313022724995390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811552, "value1":0.9190437296914767, "value2":5885018846694177935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.814548, "value1":0.8390440310277583, "value2":5493133492272371657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.817102, "value1":0.6598894636593393, "value2":8537596977440503466, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.067200, "value1":0.263575315223174, "value2":2125629267937428501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_1":"i","key_5":"h","key_0":"h", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.082507, "value1":0.6268250434274911, "value2":9058686535109969227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.734195, "value1":0.3167788157158676, "value2":7432987222538757392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.795085, "value1":0.4488172699242242, "value2":4473525864538142538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_6":"b","key_8":"i","key_4":"d", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.332775, "value1":0.8505535842615966, "value2":6047205731470921861, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727338, "value1":0.8876718108712484, "value2":1531222095185940701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.387878, "value1":0.5788599279069843, "value2":8604920509885625807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_6":"e","key_9":"d","key_2":"h", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.233954, "value1":0.6310172938155442, "value2":8925576745188878375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.805738, "value1":0.9330396354121838, "value2":1424947937953535481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726267, "value1":0.32286499179099665, "value2":3766446177817631940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229570, "value1":0.9916323313225612, "value2":3888482529661188248, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167559, "value1":0.08178718223938254, "value2":7520256881259787943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_9":"i","key_1":"e","key_4":"g", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845474, "value1":0.6808433459458884, "value2":7844247683138918907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.400844, "value1":0.39452808620697233, "value2":7357453563119638881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.206854, "value1":0.9379245513406866, "value2":6598367899784187154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.640341, "value1":0.11560748654575319, "value2":414342029075207599, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_6":"f","key_0":"g","key_3":"b", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497616, "value1":0.26256799403609016, "value2":7366324474884701561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.822083, "value1":0.6049405008185211, "value2":6275944038502753172, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.819954, "value1":0.373947872013572, "value2":3355608188213748317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739894, "value1":0.06143600864431769, "value2":309096149476303416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_6":"b","key_9":"e","key_1":"d", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.684612, "value1":0.7692595721499597, "value2":2629612954063953162, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.958165, "value1":0.6185066606297357, "value2":2824340682692220488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944290, "value1":0.3864309642344103, "value2":6500366333535522458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.576232, "value1":0.6603219832087598, "value2":7174737056650755203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_6":"j","key_1":"f","key_4":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.047735, "value1":0.4501836136879949, "value2":7043766590114442160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.282665, "value1":0.3628575018664628, "value2":3673366331802014852, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.590235, "value1":0.30332064856581503, "value2":5473820163155790307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995108, "value1":0.1854914610326227, "value2":3604467487566582522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270312, "value1":0.9632251653093951, "value2":4816282908135882739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541752, "value1":0.2581250654809775, "value2":6451610961696076892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.535737, "value1":0.3163976646275719, "value2":2322386485534186745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_4":"b","key_9":"f","key_3":"a", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.610988, "value1":0.49088683208094147, "value2":4479251775302671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.111345, "value1":0.016089752487428322, "value2":4062002565116218346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.380428, "value1":0.8878960076695425, "value2":7482838775142667379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_5":"a","key_9":"i","key_4":"f", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.286191, "value1":0.5281244570210905, "value2":4742826162899756346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.633864, "value1":0.28655049708989444, "value2":2281246548041472554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.517718, "value1":0.8760337085208192, "value2":6876760617398624261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275264, "value1":0.8763572333888452, "value2":2040006649964380357, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.133688, "value1":0.03683553631174895, "value2":879315984438783745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.178775, "value1":0.7921271023115032, "value2":8974452178625132408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.559621, "value1":0.6842414958291543, "value2":4473030242509416772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.933147, "value1":0.42656042046254644, "value2":6875784776532081765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.265632, "value1":0.6577298952553573, "value2":4117148991099157160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_6":"e","key_1":"d","key_5":"k", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.291705, "value1":0.35063088013808896, "value2":6022154315116551060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.486399, "value1":0.9066996869066141, "value2":3890435706555388518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.539080, "value1":0.6597581253474917, "value2":7149394823928068405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.172124, "value1":0.8972702809236033, "value2":5889011931565346539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.399820, "value1":0.22420487156545466, "value2":3632173066141869984, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_8":"j","key_3":"j","key_7":"d", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.857491, "value1":0.658023411947534, "value2":5044439445561970794, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.960983, "value1":0.8072960706443908, "value2":4799697070254932404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.672380, "value1":0.6734321372300892, "value2":7080832382541851744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.699212, "value1":0.7095725169760027, "value2":1829131464356530776, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893830, "value1":0.2302554104612147, "value2":9188062021573390442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.520337, "value1":0.16075557090608333, "value2":219133269848494495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173477, "value1":0.6892053752715795, "value2":2151318569219964336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229017, "value1":0.9691975492787838, "value2":1366609608627051856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659208, "value1":0.7460794334063994, "value2":817106321355418532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.361925, "value1":0.05687743883366658, "value2":7921175806067397073, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.523858, "value1":0.7866547024502999, "value2":4071070330904780510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_7":"i","key_2":"e","key_4":"h", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920001, "value1":0.5740949320374887, "value2":487306006337280717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.775196, "value1":0.1341448972829432, "value2":7272783478294294123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000330, "value1":0.1260519175980482, "value2":9041625229579242321, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313318, "value1":0.4146999970169919, "value2":6937035767989914959, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_7":"h","key_8":"c","key_5":"h", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.147057, "value1":0.5790734522844233, "value2":1184517563846638764, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.331707, "value1":0.5826950424479324, "value2":7357100402735593295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_8":"i","key_4":"h", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.476945, "value1":0.09337572406541937, "value2":3528304154285323988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_4":"i","key_5":"d","key_2":"e", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.660357, "value1":0.9211165869524813, "value2":6490630229378478544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.978786, "value1":0.24445422171222897, "value2":6316462580906113749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.031375, "value1":0.9435579557750952, "value2":3775564649397357096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.786489, "value1":0.19831604296196023, "value2":4519338814156406008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.845707, "value1":0.0733198408835947, "value2":6043258187561628613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.607379, "value1":0.49574791837525684, "value2":2511607289885086712, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.541151, "value1":0.9239913172285873, "value2":8906754784907447893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_6":"h","key_0":"b","key_5":"h", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409187, "value1":0.347121325872609, "value2":4022279358676190140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179171, "value1":0.9901703571238136, "value2":1971122679978818549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807000, "value1":0.9794430062314226, "value2":4140173177331652200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514307, "value1":0.3368836183017325, "value2":6646214711319824003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_8":"d","key_9":"g","key_6":"a", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.430324, "value1":0.6783643243115747, "value2":8700387930430110357, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.245716, "value1":0.5777847683942137, "value2":1772788621154859179, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.825752, "value1":0.8229486912139793, "value2":3218217186376093534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.550373, "value1":0.44484600976320454, "value2":1214217372893594474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810847, "value1":0.6410630380778536, "value2":1107368725861373561, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_7":"e","key_3":"g","key_4":"b", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258640, "value1":0.3699636915738955, "value2":3778046401086414746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.354190, "value1":0.2787621713653213, "value2":7905127146501563870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.276300, "value1":0.38144875313061755, "value2":279189270439050245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030267, "value1":0.09524338073167776, "value2":624319768896255519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_8":"i","key_4":"j", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395031, "value1":0.017165936825319472, "value2":1839096275149454289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_3":"c","key_8":"i","key_1":"k", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.920079, "value1":0.3695378285006256, "value2":3642800979833317953, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_3":"f","key_9":"f","key_1":"e", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.055296, "value1":0.5098042037064671, "value2":1508579590160432139, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128994, "value1":0.8793028393199556, "value2":5718919044160675205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.418223, "value1":0.7389480744126634, "value2":5430202156875304420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.458004, "value1":0.7589090833562657, "value2":7035246718540970070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_6":"k","key_0":"f","key_3":"b", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.492634, "value1":0.3450062925676948, "value2":289715553515433637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965681, "value1":0.9053649041017894, "value2":2294309054504906499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234931, "value1":0.0732565141042219, "value2":2755939599945126397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.939989, "value1":0.8451432422468432, "value2":2656355912924482468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862255, "value1":0.3071521473170743, "value2":4029052487902069438, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027859, "value1":0.5418227496728708, "value2":5130775345772381962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016707, "value1":0.5241900456864456, "value2":1703509093458181611, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.129222, "value1":0.45379187899537626, "value2":8646708304907761402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.167400, "value1":0.4118569437426427, "value2":1933032658312744458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.448154, "value1":0.9880224154363099, "value2":1958598830665944865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.095756, "value1":0.7737922723291526, "value2":1554270443791925823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.669733, "value1":0.05998044134164805, "value2":4233304263515802857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_3":"b","key_4":"c","key_2":"a", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360452, "value1":0.021777318888489317, "value2":1462350357926539483, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.791525, "value1":0.24698569461323847, "value2":6782513953965205791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.136814, "value1":0.13312227926717296, "value2":2973379075607635624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_5":"a","key_2":"i", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.611322, "value1":0.06800531937511199, "value2":2253428998305264546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.000702, "value1":0.40505920041369187, "value2":7444308564790803777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.317483, "value1":0.5921456568045917, "value2":1159233331687655633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.207714, "value1":0.03623125609776327, "value2":4050612595565072329, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739699, "value1":0.1967627573726683, "value2":5128621003255593133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.938178, "value1":0.8074789236886436, "value2":5329361240090086591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.075581, "value1":0.36183126859676484, "value2":8380824340175200195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.840434, "value1":0.36564465907617033, "value2":4138989352760782569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697538, "value1":0.8089117668002023, "value2":2706669265618979267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.810618, "value1":0.9822561514547918, "value2":6363847253975708775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.453492, "value1":0.7521891954121415, "value2":3415954419502830788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.246679, "value1":0.291250645374521, "value2":6787758008675320095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.717578, "value1":0.21885147116003636, "value2":8249910093492361560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.584635, "value1":0.267173208281922, "value2":6350566579717368150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.288596, "value1":0.8034133849819789, "value2":3057165042866020348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.043225, "value1":0.8891915956366093, "value2":1172085376591869222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_6":"e","key_0":"k", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514557, "value1":0.7993701790896678, "value2":3530709071532758425, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_8":"h","key_3":"k","key_4":"d", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727549, "value1":0.7581402483767911, "value2":2882149021889454970, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.944767, "value1":0.03125902081908963, "value2":5930819025045052350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_9":"c","key_6":"c","key_7":"k", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635131, "value1":0.5563692033064526, "value2":3139117266540547008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901432, "value1":0.08725832596843655, "value2":2516400281682537507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.965447, "value1":0.24555624361218226, "value2":8420031154067673423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.653005, "value1":0.6015476287649423, "value2":7179718486091286242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_6":"f","key_2":"g","key_3":"g", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.251270, "value1":0.6303898680513278, "value2":1976335650844570524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.902243, "value1":0.42829690884430943, "value2":7763758072113593479, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.748351, "value1":0.5394339139974091, "value2":1947184765867041268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.496788, "value1":0.0776914163879659, "value2":4120402058111565140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_6":"j","key_4":"a","key_5":"g", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.202592, "value1":0.3950986449717796, "value2":7867550414487529994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.475860, "value1":0.5847506627916105, "value2":1395577688027634835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.307279, "value1":0.66953692642835, "value2":1884810173101069714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.447867, "value1":0.30049434187635843, "value2":2021494635928386331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371456, "value1":0.9583003775042523, "value2":8408660678843228458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.321951, "value1":0.673208957772375, "value2":4484934316922285119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.654317, "value1":0.34301031853223307, "value2":1377167639028225502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.258833, "value1":0.5754272342729656, "value2":3672436572873719874, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.894800, "value1":0.27153721678184795, "value2":6771295009871301863, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886915, "value1":0.4211098948169063, "value2":327314688390276361, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464071, "value1":0.8401442859431041, "value2":7237906700870505791, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179548, "value1":0.35527262545135796, "value2":6929428209196710992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348853, "value1":0.7390477554491065, "value2":6017303276519989534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.228799, "value1":0.7190691653435217, "value2":3466847624797872404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.676189, "value1":0.8701176990093307, "value2":8139200890464740272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780301, "value1":0.31331610868424237, "value2":7970934533227613450, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.763684, "value1":0.6117316228567153, "value2":1342308618516991103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.510130, "value1":0.26253831176449904, "value2":223167711418991037, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.054906, "value1":0.30113096258283606, "value2":9029709188884415357, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_7":"c","key_1":"e", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.101830, "value1":0.6535137630775729, "value2":8047896116508601877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.816267, "value1":0.6779088610569585, "value2":402736258268648092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794465, "value1":0.6385825716360082, "value2":647146760807700438, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.342961, "value1":0.08518129482765982, "value2":3079348210933738649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.503723, "value1":0.5000975675164184, "value2":3087069466664576618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.552740, "value1":0.46623198603217897, "value2":5991311418948024851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.634526, "value1":0.1623989693910568, "value2":880550310997257962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409701, "value1":0.18776946541701378, "value2":4672037814003639717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928628, "value1":0.5544683022868725, "value2":8786609556345104074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422305, "value1":0.3339761609696324, "value2":2037434470333278554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497260, "value1":0.21504973222613463, "value2":8744267736203582723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348792, "value1":0.7267165846871734, "value2":2029452385366977955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.767788, "value1":0.7268443539701483, "value2":5120204611450631315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.313022, "value1":0.3937734460488266, "value2":8789465407671191549, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.373011, "value1":0.5815734596824621, "value2":7433149153695507174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.403959, "value1":0.19547385196106162, "value2":4536904833017601954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702004, "value1":0.787480554646893, "value2":6826871522585433391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.238346, "value1":0.3158260333418349, "value2":4890876749664266809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_7":"h","key_9":"d","key_2":"g", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.184806, "value1":0.1658734659451667, "value2":2103368389172466197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.646125, "value1":0.7611256397426871, "value2":3779878057324979416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.963683, "value1":0.6784369755611176, "value2":7198319968843792884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_9":"h","key_6":"b", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.264784, "value1":0.6944677799630911, "value2":1361718740081763677, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.606235, "value1":0.9767553574110414, "value2":8172694375569168120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.555524, "value1":0.0757100455425729, "value2":6369412142150141094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_9":"d","key_1":"f","key_8":"i", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.683577, "value1":0.23488908846392256, "value2":7496440285542052380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.741203, "value1":0.6152800537206294, "value2":4336041330603516829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.076957, "value1":0.46349128481126883, "value2":1995149205719248029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360568, "value1":0.18125853395781905, "value2":5023669000017633099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.127735, "value1":0.2064987900720652, "value2":1090691939971047635, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.182440, "value1":0.18035240548503653, "value2":2377567143003176008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173358, "value1":0.9386364543056086, "value2":6498536821606623122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_9":"k","key_5":"j","key_6":"d", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.749875, "value1":0.9851379736422945, "value2":9068316297781013801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.262923, "value1":0.6215747239294878, "value2":3444654966165617334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.471762, "value1":0.9687744980352465, "value2":4008980207107557770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.449800, "value1":0.43369424684152247, "value2":1796590317391088733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799032, "value1":0.7408688395558841, "value2":1701592597588021265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.727561, "value1":0.14048934174685151, "value2":1964669731520342830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393461, "value1":0.22927403588573142, "value2":6637169452408438633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577492, "value1":0.9899876418682123, "value2":417258805492276216, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.149873, "value1":0.2451895383714133, "value2":6783012468215477739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.794818, "value1":0.2978983436952165, "value2":6898137216027059682, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826276, "value1":0.03031011408512658, "value2":6243267082953045335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.853432, "value1":0.8319004837974954, "value2":6916103698866916518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.336756, "value1":0.6573141098461979, "value2":7552682537825884423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_6":"e","key_7":"g","key_3":"c", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.280458, "value1":0.9944062499165491, "value2":4515137518875248587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.723258, "value1":0.8389015105022624, "value2":4385100343250672688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.518510, "value1":0.262846594625773, "value2":8956848533096958437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.398441, "value1":0.29945983222092276, "value2":7583507905789284927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.169305, "value1":0.3509582334049355, "value2":4486670807718872528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229006, "value1":0.9454115215233667, "value2":4148482506114005261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.108860, "value1":0.7792992725899739, "value2":6662551108461731004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.270948, "value1":0.22595556811111142, "value2":4858846807835096111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_3":"e","key_6":"b","key_1":"e", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.739267, "value1":0.10553288145788361, "value2":3903915559852076766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.771104, "value1":0.2713630569195371, "value2":9127087478037486061, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.234974, "value1":0.4512849051156243, "value2":388475201207159224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.021067, "value1":0.8525575906494762, "value2":6112677337472259317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.360331, "value1":0.5466498354493777, "value2":8416483456048971785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.083431, "value1":0.6268377803161493, "value2":6874204752716819531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543079, "value1":0.356809340191254, "value2":1479162719217476511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.543913, "value1":0.1823988812868871, "value2":1772497490870724163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.697354, "value1":0.6008728382547858, "value2":4808659248566150951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.173629, "value1":0.11694918608631227, "value2":8374732016838634205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.478338, "value1":0.6523910738528493, "value2":6127719310482278347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_8":"d","key_5":"k","key_7":"j", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.195226, "value1":0.11728848022547718, "value2":2692984308355874383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.472792, "value1":0.3849335854410186, "value2":5551198795362400796, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.088581, "value1":0.6479908764028293, "value2":6646669667589759935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.051281, "value1":0.8063603427465644, "value2":4677280644621004022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.421107, "value1":0.8098711885881693, "value2":781451797112683390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.702263, "value1":0.7459154166325007, "value2":8763722224980194522, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.105837, "value1":0.6082412929946549, "value2":9076065007808782827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.229179, "value1":0.5208679934438682, "value2":1614625428456495501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.452972, "value1":0.6602375068241643, "value2":3665988082745333789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.570733, "value1":0.8539881179697112, "value2":8291534021329215591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.854105, "value1":0.19867014748060782, "value2":4707537791933933479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.134473, "value1":0.013248508886480496, "value2":8975225813346771480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.401924, "value1":0.8131686031327187, "value2":1570626633965770939, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.462042, "value1":0.33875887576127867, "value2":1531594524235268547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.780840, "value1":0.6167438470784343, "value2":3111876176210806342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.157314, "value1":0.024850360062745134, "value2":2155248434369533975, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.892341, "value1":0.29151476400849524, "value2":3929095109066478763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811920, "value1":0.3960145984061985, "value2":8079803277636061346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.010049, "value1":0.1522516905011452, "value2":3887799589835913190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.511030, "value1":0.1096312618999367, "value2":4683382419730786111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799775, "value1":0.8107638537946521, "value2":1548400788132615580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_4":"k","key_8":"e","key_1":"h", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.034128, "value1":0.8145920212010633, "value2":4095930474475755353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.506330, "value1":0.8145793344024634, "value2":7248985564172971873, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.632617, "value1":0.530965707458104, "value2":6582545459218253298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.427368, "value1":0.5488520970953542, "value2":7764537337432663671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_6":"a","key_4":"h","key_5":"h", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.128543, "value1":0.339288607631015, "value2":8135936826126620967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.783911, "value1":0.4072868133390787, "value2":398383887746388268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.030437, "value1":0.40039567718797253, "value2":5009000449434898292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.301665, "value1":0.2851345919898927, "value2":196381052074952545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.740740, "value1":0.481454729816193, "value2":2929970498837276050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.867428, "value1":0.8246737181844912, "value2":4477437696490734479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.708771, "value1":0.44011710908297325, "value2":7995836793967148678, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_3":"h","key_0":"c", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.663309, "value1":0.07002506566825563, "value2":3582676294388926669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.628344, "value1":0.9801112824460718, "value2":196546046309844748, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330627, "value1":0.6390220685465513, "value2":5252094592319160612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.014786, "value1":0.4666494400039896, "value2":6746944521794840076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.736397, "value1":0.1650175517955452, "value2":4725786271252183916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.971200, "value1":0.08835643831453004, "value2":1662264677868065737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.804675, "value1":0.3583297226255054, "value2":6764608743691309744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577888, "value1":0.7784009799351271, "value2":7389846578443042716, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.942762, "value1":0.5281041922180323, "value2":225078423814016880, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.807869, "value1":0.12159890893033082, "value2":2007944198207870435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.393243, "value1":0.9174108973827193, "value2":5956246567273801920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_8":"c","key_9":"d","key_1":"j", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842305, "value1":0.6881483813172635, "value2":51917468498479892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.625958, "value1":0.7332068726661788, "value2":1595636028517738482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.738053, "value1":0.19091798634208768, "value2":2237197021871432008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.268129, "value1":0.8406719661444901, "value2":2842521222444292729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.865313, "value1":0.8513846203817129, "value2":3859037819752068385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.762176, "value1":0.0431523780543075, "value2":6794482877795635511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.658034, "value1":0.12118411547914006, "value2":7727119685187980691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.497498, "value1":0.8736017830767735, "value2":5281383221003217421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.016157, "value1":0.45635245796699264, "value2":6588970499031475943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434398, "value1":0.5982494402461963, "value2":3451343743898655419, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.995659, "value1":0.41609794124038807, "value2":2577003956591436853, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.402057, "value1":0.3168156284522602, "value2":7664962841417781702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.874274, "value1":0.8588869341047012, "value2":6471105672211793135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.193561, "value1":0.7716291581375219, "value2":6758054142720243683, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.298108, "value1":0.8739782093698222, "value2":7600130946423390938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.008667, "value1":0.15272944664848967, "value2":28862629936648465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.924564, "value1":0.2262144335029645, "value2":2100662153491567952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.250052, "value1":0.2945787294595236, "value2":6018457540699249059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.395074, "value1":0.2290685251999151, "value2":4865312816831141048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.414183, "value1":0.2892057090700859, "value2":3284381379197405662, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.800711, "value1":0.20023612385737624, "value2":5612718418445245977, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096882, "value1":0.6190652975917784, "value2":4456707593278158335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.811596, "value1":0.1974893316007636, "value2":8350883557246751407, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.328602, "value1":0.3577823211406552, "value2":3856849717607100839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_4":"h","key_5":"f","key_0":"b", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.165294, "value1":0.1558044782512875, "value2":2819862014599071323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.934329, "value1":0.6967191883920082, "value2":8317801698439570519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.826310, "value1":0.5889684966352713, "value2":8904148697557725595, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.544062, "value1":0.3036793591749508, "value2":1172054639314476559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104925, "value1":0.0006382055160369547, "value2":4527189243213341516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.577496, "value1":0.8262646875704296, "value2":6857852785226312453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.304163, "value1":0.23448830223910028, "value2":118297432460308257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.201626, "value1":0.8649386204518231, "value2":2335277383532252247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.072447, "value1":0.305504173455374, "value2":8879950191665801641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635410, "value1":0.8646407642266409, "value2":4794888988052345760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_4":"d","key_0":"k","key_3":"j", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.799765, "value1":0.020261599778331852, "value2":6569370350090311777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_7":"i","key_2":"i","key_3":"d", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.643397, "value1":0.25580026537842243, "value2":8727368389050428851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.886737, "value1":0.709498290535563, "value2":159638875141661569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027896, "value1":0.4252750468577108, "value2":5985063599490557923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.659322, "value1":0.8283450566415441, "value2":3829413923759281391, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_2":"h","key_1":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.323182, "value1":0.05823156526332208, "value2":158181159624437675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.721491, "value1":0.058149447228806335, "value2":4425192040964093188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.715378, "value1":0.1468510162162874, "value2":957193015456701660, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.759517, "value1":0.11145345073332971, "value2":9085390279667186751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179380, "value1":0.22290378424729548, "value2":4452524214324184228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.018314, "value1":0.08677941054726214, "value2":6248669636718030352, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.579864, "value1":0.1843770522783152, "value2":2395237031433429826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.907863, "value1":0.19391713305151562, "value2":1831578948753070965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.567206, "value1":0.1200213703180807, "value2":281170079691225837, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.908312, "value1":0.16873762365247053, "value2":3349277004368724008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.203346, "value1":0.3461718122656804, "value2":2053673338515188112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.252421, "value1":0.8594831603279349, "value2":8129734681001143206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.862108, "value1":0.8625875861161819, "value2":266750263503917399, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.394606, "value1":0.1284747365418792, "value2":511162186129513778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.104334, "value1":0.348212621993861, "value2":8809834773329001166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.390817, "value1":0.056538788821091236, "value2":959306944656093342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.371946, "value1":0.0354556141213121, "value2":5315204481196619882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.027057, "value1":0.04051346889125732, "value2":19281603192892033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.315646, "value1":0.29470865205235175, "value2":3350853301006328333, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.516736, "value1":0.32421705083072555, "value2":6409364233498955235, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.409843, "value1":0.8431693191414327, "value2":2309002455167863238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.290066, "value1":0.769846734088458, "value2":1354615135861452763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.025458, "value1":0.04383062091902626, "value2":6270990659808009038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.422265, "value1":0.4215091845870574, "value2":903506811871753839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.954147, "value1":0.1703656776121607, "value2":8625109899283360680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468022, "value1":0.8656906467989927, "value2":8844353092623218329, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_6":"d","key_2":"g","key_4":"j", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.572694, "value1":0.03657142931464268, "value2":7269928716214871698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.035238, "value1":0.2174690883774512, "value2":2760303323859671187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.500319, "value1":0.8848932111671852, "value2":124147493136831346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.324572, "value1":0.32024379123233365, "value2":8082835280902735033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.585321, "value1":0.18246288567364422, "value2":7470239488455160357, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.726932, "value1":0.8209711897357084, "value2":4430579260139045397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.851627, "value1":0.9988163498901884, "value2":7782482596272329873, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126247, "value1":0.7338638289875997, "value2":2035846164975007267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_9":"c","key_0":"i", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.923987, "value1":0.31023714156918913, "value2":7969614988653223555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.797525, "value1":0.9308157523651853, "value2":2008465742081738773, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.595532, "value1":0.6306393412084162, "value2":1794355738915834222, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.078930, "value1":0.46045211861703167, "value2":2231560481693103072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.835058, "value1":0.49083826368226746, "value2":947523171682771279, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.757956, "value1":0.5783027668846169, "value2":7939213319454038876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.431201, "value1":0.249271140930431, "value2":7943702047022612152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.674800, "value1":0.9050148884978706, "value2":4701267498128748767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.294430, "value1":0.37838796407920183, "value2":6812226255195471591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.837164, "value1":0.41192774227492235, "value2":1535119829477772546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_2":"d","key_1":"j", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.464005, "value1":0.1600487895075015, "value2":1434123145157650204, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.546806, "value1":0.6061159682210883, "value2":6208205100336254753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.096572, "value1":0.8332138223552116, "value2":7164548384493446541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_5":"a","key_8":"b","key_4":"h", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.325386, "value1":0.6865864080335002, "value2":1198200374614070582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.179713, "value1":0.590117470581001, "value2":8089464430800358371, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.273193, "value1":0.6452720470474251, "value2":1981356061798226474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.330589, "value1":0.2830579363626061, "value2":6390191335007008364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.468768, "value1":0.2265890199157533, "value2":4363523400851143524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.901505, "value1":0.3297489212069802, "value2":7168425838004946986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.850173, "value1":0.7443546478795617, "value2":1215583272150299781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.968362, "value1":0.6932020029271382, "value2":4461729221210949302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.210225, "value1":0.36249305348248906, "value2":7617198529682822125, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.001816, "value1":0.46528315796295283, "value2":553457115403327004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.564478, "value1":0.42774941008998635, "value2":6190959325547605268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.126782, "value1":0.6691920311814687, "value2":8190037233750158351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.125382, "value1":0.3161902537409363, "value2":2607443448109856155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.782262, "value1":0.25500427404714787, "value2":694236189793698587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.635388, "value1":0.4760006261995292, "value2":7236218797911995877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.481920, "value1":0.5382100231194429, "value2":6783652184521772255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.818398, "value1":0.6906166285200835, "value2":1593698347526099227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.283475, "value1":0.9162718353521024, "value2":131978695455153917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.434017, "value1":0.8865622180830747, "value2":5499929802234516147, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838878, "value1":0.8831606268003614, "value2":4413611476125545911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.451279, "value1":0.9923758573970507, "value2":7842864722613769419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.362386, "value1":0.09704282298545679, "value2":6868263842667298833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.274837, "value1":0.5887807861238907, "value2":1280869300914331508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.915666, "value1":0.6149407272649627, "value2":2848990718411372775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.024227, "value1":0.1623145961115003, "value2":2455593750882693548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.695944, "value1":0.6887899812661492, "value2":533109273214059474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.528843, "value1":0.5075436862092302, "value2":515274582595904032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.842262, "value1":0.5447726756017138, "value2":411822948007690047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.444248, "value1":0.5454758220348956, "value2":1028316813231771868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.604063, "value1":0.46187992572513836, "value2":8158929971855121147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.275994, "value1":0.8154001283675578, "value2":5366649133302343998, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.348933, "value1":0.903584155771458, "value2":3421400845314883405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.041062, "value1":0.1190413277495334, "value2":3602483838893891944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_8":"k","key_1":"k","key_5":"h", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.928372, "value1":0.19081223162638847, "value2":4430971211717726323, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.677971, "value1":0.7830672620168732, "value2":4412058601784711710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.152188, "value1":0.6024595453233776, "value2":3457200300695401373, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.838789, "value1":0.8421167420494099, "value2":690533435719409387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.578903, "value1":0.6328751611123744, "value2":3916711663864490625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.896839, "value1":0.9070308716530836, "value2":5545334900650073897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.712901, "value1":0.5273783009751816, "value2":7859089933688216880, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884939, "value1":0.07741816375541602, "value2":2787557417309799857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944440, "value1":0.9501843438293957, "value2":4306730282944265532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814029, "value1":0.12358405285849303, "value2":6299011328560473443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965702, "value1":0.9397419047313924, "value2":2182258508456140601, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_7":"g","key_3":"g","key_5":"e", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045751, "value1":0.6945070980136847, "value2":6892165864647238687, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821781, "value1":0.27908605223075134, "value2":5742625392746876485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.514196, "value1":0.8720271384105083, "value2":6651696455700077053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.705476, "value1":0.06250737971386458, "value2":1193722866498852150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.603276, "value1":0.8453600274943766, "value2":31981742286968982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.682527, "value1":0.2427465956249123, "value2":7462455737298824829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.605338, "value1":0.7488120008677794, "value2":4273030954599275940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.893961, "value1":0.0012093187527627421, "value2":3798553194034218364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.593870, "value1":0.4389808952680249, "value2":4760750743940894178, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:05+08:00", "value":0.091153, "value1":0.8390848719802647, "value2":4663558817694651969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.014239, "value1":0.6803598889667898, "value2":4768916054002910461, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478582, "value1":0.8497920155872134, "value2":6014547020899484560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895949, "value1":0.2900471481778915, "value2":9139898160138156870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.613215, "value1":0.18238778666189503, "value2":1572248498405458162, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.817510, "value1":0.7988916675957994, "value2":5551320908811649706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881135, "value1":0.18477673938398376, "value2":5754586889701314224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128674, "value1":0.2730542267310359, "value2":3723967523716120986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_8":"k","key_0":"a","key_6":"k", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392852, "value1":0.5342690346158169, "value2":2053063244515016190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_5":"d","key_2":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931934, "value1":0.712284281615868, "value2":5743978094943818789, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537147, "value1":0.023259389444222435, "value2":5526408740424742448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.216805, "value1":0.8594083029021016, "value2":5158284428621389401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837342, "value1":0.5180882037372726, "value2":7808493880824756093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144558, "value1":0.054388678344141024, "value2":7174534666209181912, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599909, "value1":0.09725816948703289, "value2":4397869661367210515, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.880028, "value1":0.4672046328070093, "value2":7512677570982024303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271509, "value1":0.8816366985657997, "value2":7431242132698500082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.752197, "value1":0.6078880045612306, "value2":3606538184952837708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_5":"b","key_0":"a","key_4":"j", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392689, "value1":0.5702352441472875, "value2":327376335260250215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.201991, "value1":0.3880870455267555, "value2":2539019737033922024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.894353, "value1":0.3785370438906491, "value2":8100099805481858212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_4":"e","key_7":"c","key_1":"f", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.193824, "value1":0.1316387900370704, "value2":2566382490326741291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.735887, "value1":0.9017592259731758, "value2":3204066872434851855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891488, "value1":0.8557864633508439, "value2":7321454426994500720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868424, "value1":0.21022389029059355, "value2":4239400393420255730, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_9":"f","key_0":"i","key_1":"h", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.624822, "value1":0.8941824508080144, "value2":4130019486643335780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.292285, "value1":0.2379306348830461, "value2":19245718469181538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690737, "value1":0.07804300554340773, "value2":3416010458091576918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404288, "value1":0.7382908404371364, "value2":4673144760418942783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.333079, "value1":0.7119882858599182, "value2":2804563282517597579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.472391, "value1":0.08143361630646322, "value2":8016722515581298003, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_3":"i","key_7":"a","key_0":"c", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.373563, "value1":0.6580401651760573, "value2":6974099035575699967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078367, "value1":0.04245208361261986, "value2":3309677777480755698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.312294, "value1":0.08781290868459149, "value2":410348162808240186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768785, "value1":0.8804562203614901, "value2":3554533373089833157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636131, "value1":0.6764401768679541, "value2":5380729408367504508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041277, "value1":0.013798250982638604, "value2":7112024095753417757, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867267, "value1":0.8747601920631648, "value2":1340507463961878602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776242, "value1":0.08520211967435457, "value2":4488809966272125582, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634645, "value1":0.8571991096706362, "value2":4813432822336330164, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415856, "value1":0.16181998754042196, "value2":2843492259629836672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466761, "value1":0.6961378523412571, "value2":7745491307176492894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.594554, "value1":0.04099735827317489, "value2":8913619907287518073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.170785, "value1":0.5648256194335608, "value2":476929125931913132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.184821, "value1":0.7825477180517281, "value2":4452433023228691120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906243, "value1":0.48861587452050464, "value2":8430953017453652958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.667172, "value1":0.06605403171722925, "value2":3065000063305702983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.192260, "value1":0.12827053824062132, "value2":4944252317941317234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400652, "value1":0.728695995485893, "value2":1367272622265070542, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535047, "value1":0.4567196818143048, "value2":5331291578601537074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.466316, "value1":0.5659706744457615, "value2":5221382170536673746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487405, "value1":0.8907853085096481, "value2":2583510249445277455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.375762, "value1":0.8536987085513627, "value2":1666025213570716259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.165432, "value1":0.43405723604720065, "value2":714036789144069944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175401, "value1":0.9859093721286644, "value2":4617822411248224956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.059065, "value1":0.378208510942557, "value2":8575694693763974484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_7":"j","key_0":"d","key_6":"c", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.073374, "value1":0.7874709956095631, "value2":3297271830714214500, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656159, "value1":0.11136055751977454, "value2":6742238162891439594, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102008, "value1":0.8937221842663475, "value2":6378107019082012182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.177632, "value1":0.36894176860576355, "value2":7704992361178834991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335734, "value1":0.6980181667187008, "value2":260944621673155935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.855348, "value1":0.9199066328884838, "value2":4741986190169422412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965490, "value1":0.12178040424602839, "value2":8326386976538257273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090240, "value1":0.23602165280654655, "value2":9043605027949953616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.028464, "value1":0.05960386743156977, "value2":3549887217756566974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.956988, "value1":0.5498920550278433, "value2":6597949147890816377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.171316, "value1":0.0904948299110727, "value2":3471748119473561940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_6":"b","key_9":"a","key_1":"g", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.523091, "value1":0.4454856417468202, "value2":3973537432672533989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453397, "value1":0.7881010562880141, "value2":448816938066732131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384576, "value1":0.9045802469513693, "value2":4349360112470402658, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.929660, "value1":0.1904824371420524, "value2":4775599930437109515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642052, "value1":0.9792443542954197, "value2":6092407690974464405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194609, "value1":0.8726282980309354, "value2":7591391705732812177, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095352, "value1":0.384168214137503, "value2":7516656275093877057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_9":"d","key_3":"f","key_8":"h", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.868497, "value1":0.924966560225317, "value2":4429888702948279946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.146490, "value1":0.7150691128822396, "value2":205950019316022568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009297, "value1":0.4940665543165739, "value2":924283179287111091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904057, "value1":0.23580267125603968, "value2":531910251609622124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728011, "value1":0.8321936268339676, "value2":2170557262229675184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.944000, "value1":0.7690990294985828, "value2":8227626091745901, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_9":"b","key_5":"e", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.871354, "value1":0.21098039407901223, "value2":4374012870679696344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.538323, "value1":0.681484837485245, "value2":6775701390216802286, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230276, "value1":0.36173491575946426, "value2":8235898214428447094, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552468, "value1":0.8892513220522287, "value2":612692529250917070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782746, "value1":0.3267438552083453, "value2":5821122346950557164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908039, "value1":0.8750317891455761, "value2":6904198848902253780, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124465, "value1":0.27489616104124, "value2":1813939809215341473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.739988, "value1":0.11884426917626255, "value2":6375710418065432574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_4":"g","key_1":"c","key_3":"j", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246847, "value1":0.065994349338974, "value2":4501641519889306838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.675544, "value1":0.4346958284003153, "value2":8656554552944846961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907951, "value1":0.3861999534665598, "value2":7376195997991690698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_4":"b","key_9":"k","key_1":"h", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.601240, "value1":0.09029192218287085, "value2":4084656981993173700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327311, "value1":0.46909482167142375, "value2":6484192245523357545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453608, "value1":0.634255139121383, "value2":8880783116370612057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_8":"e","key_1":"h","key_7":"c", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562417, "value1":0.4260014133745824, "value2":1052849307102914712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708174, "value1":0.05688677429464904, "value2":5475041244330586198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.774531, "value1":0.8132424157243254, "value2":1362266175582132598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.106500, "value1":0.7014281430749205, "value2":7134854011640912176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.486559, "value1":0.28063120575400446, "value2":3192436055979504417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.864997, "value1":0.3477928996994262, "value2":735248433592549787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_1":"k","key_9":"f","key_0":"k", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.657367, "value1":0.65265411185833, "value2":2443852584375783883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.172271, "value1":0.4417461489558258, "value2":7942690633839505457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419664, "value1":0.16011001601248667, "value2":6549588207213967629, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416761, "value1":0.6429956000093604, "value2":8493895113638046625, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564282, "value1":0.4457588529183665, "value2":6725016002434781716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430750, "value1":0.3037956469116298, "value2":7998283396935875899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827125, "value1":0.5250484976450595, "value2":6524724358656282389, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395896, "value1":0.12287086144958013, "value2":1930681199692368788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851159, "value1":0.23103414004696357, "value2":4914748620478090577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.736709, "value1":0.6408817571256187, "value2":9105122753792897591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984808, "value1":0.2905624937515174, "value2":7851451601162680954, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487306, "value1":0.605006354442018, "value2":4276465105843409598, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790850, "value1":0.6268022197129945, "value2":9220780769142386741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185700, "value1":0.566519118535341, "value2":3705647574979636584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451877, "value1":0.0460137249105004, "value2":7915985022414197030, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150718, "value1":0.35693559742029707, "value2":5014219743291793375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.294260, "value1":0.9247064360015096, "value2":5127524279022159080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_6":"c","key_1":"a","key_2":"j", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995623, "value1":0.21178547000119013, "value2":2431494318968387519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.256959, "value1":0.8916548214121515, "value2":1002635579417294841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_5":"c","key_1":"c","key_2":"g", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110758, "value1":0.9737478776363947, "value2":7247818685878473810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825871, "value1":0.2917878380844755, "value2":6516114778766465158, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910566, "value1":0.3957632746819297, "value2":7166657573372241684, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.178800, "value1":0.5649796089766207, "value2":7775035464817162126, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_9":"g","key_2":"e","key_6":"d", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634489, "value1":0.6090054970934514, "value2":6647498876825886588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610499, "value1":0.5292322793488414, "value2":8830154316280741019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574572, "value1":0.5304362659437236, "value2":3949756347058852883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_3":"a","key_9":"i","key_0":"i", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380651, "value1":0.956019198700713, "value2":9135146156795796096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778218, "value1":0.2121518375707717, "value2":8667556184145489676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078961, "value1":0.06325131155030916, "value2":8555129638818277403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_4":"d","key_6":"h","key_2":"i", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.047017, "value1":0.1027203958612216, "value2":5246272405737501952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439829, "value1":0.7417681560989317, "value2":7749536771402273968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_5":"f","key_8":"c","key_0":"f", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203072, "value1":0.5621215132192839, "value2":6118954075279232029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_2":"h","key_4":"a","key_1":"k", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758671, "value1":0.48389569392567205, "value2":2705225096795559234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_2":"g","key_1":"c", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665522, "value1":0.8426429373047708, "value2":8279168363918064894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_6":"b","key_8":"h","key_1":"a", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.345535, "value1":0.20088437097171843, "value2":4950747031440638845, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.124655, "value1":0.41827026587025645, "value2":3430038763553236823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977760, "value1":0.10421019819450697, "value2":733644826366835807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118861, "value1":0.7989242448499801, "value2":7548777360232504083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318842, "value1":0.009939081834750191, "value2":8375328698470876119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590619, "value1":0.03636791737366475, "value2":7582770650474370737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045468, "value1":0.7763100607726895, "value2":2803435127241577664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.553650, "value1":0.9987121300161536, "value2":3948733587296676032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790022, "value1":0.793158890727088, "value2":1174928966366390949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.232763, "value1":0.07728571888784644, "value2":3240453509192284959, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_7":"e","key_3":"c","key_4":"e", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074815, "value1":0.5297469977127046, "value2":2019972453697918898, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.126329, "value1":0.8018605847489219, "value2":4857597305010910280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.465083, "value1":0.1752053660866726, "value2":7163605506157251603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.714319, "value1":0.8220755414438086, "value2":876552025810846680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392782, "value1":0.6635667135366609, "value2":6624504880778850428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404283, "value1":0.5416570514867133, "value2":8266175224671672272, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.734325, "value1":0.7575887007274229, "value2":8864761125492843547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.164845, "value1":0.7328611642697666, "value2":1939960544899646141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_9":"g","key_1":"k","key_2":"d", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.087713, "value1":0.6472001702780127, "value2":7064434641814632478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636375, "value1":0.263542227747472, "value2":5091288498651627633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439658, "value1":0.7518898990734011, "value2":6126681364245747402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580431, "value1":0.28444432068121084, "value2":8147884504685238694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408258, "value1":0.5563100440073113, "value2":4573231178010278468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_8":"b","key_9":"f","key_4":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923759, "value1":0.7035415767681118, "value2":4284377617122095281, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069625, "value1":0.7755797987036898, "value2":7005063200108318654, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475808, "value1":0.7602915679203158, "value2":1889690102234767528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.045338, "value1":0.4860219702134258, "value2":5422255506416156729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.985908, "value1":0.7866996153890412, "value2":4784461751340030748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937856, "value1":0.5239212561479751, "value2":3109818040142871590, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.376519, "value1":0.2587179526287473, "value2":1263076833301844915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943956, "value1":0.6806333311736701, "value2":6002705815114995201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.653771, "value1":0.7947376720839073, "value2":2247611778314354742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290979, "value1":0.13605627013098143, "value2":859676226210675359, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166492, "value1":0.4455170530582711, "value2":3676647784290217622, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.625732, "value1":0.26531203043868007, "value2":8360465694110411228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033924, "value1":0.027884795524999093, "value2":4290921120785429469, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.199865, "value1":0.6930021146351104, "value2":8033400918877373777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.387257, "value1":0.5315645824113636, "value2":8487597923574953049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.385317, "value1":0.687478534046926, "value2":5788685556231917492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.632370, "value1":0.4013559424653022, "value2":8640639439561304540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226297, "value1":0.2440055184343977, "value2":5133913946019355967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.763037, "value1":0.20026459422324144, "value2":8670140335638909516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695304, "value1":0.16276770698378185, "value2":1188100569829740721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595366, "value1":0.4506718132549764, "value2":5734081257378041711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.833543, "value1":0.5734402814385565, "value2":922425738240504967, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_5":"a","key_1":"e","key_4":"g", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709630, "value1":0.8692041834415107, "value2":1960359817596413212, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.732788, "value1":0.23444382672590003, "value2":2854071547092864254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.412896, "value1":0.19053537293020936, "value2":1082762509249072310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988462, "value1":0.4293280866481845, "value2":4780102507614458134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_8":"i","key_0":"h", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823600, "value1":0.01981381195792859, "value2":8743022558864727512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_7":"i","key_3":"a","key_5":"e", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.092886, "value1":0.9862388684646872, "value2":3622022818199205736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.018231, "value1":0.4242288910229569, "value2":880886547836969584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062605, "value1":0.9179292075855371, "value2":3743117943661894467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.759025, "value1":0.5166884948154693, "value2":739459076946893696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.907932, "value1":0.2882739675567565, "value2":3650155370836487214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_9":"d","key_4":"k","key_5":"b", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068172, "value1":0.4347373436897239, "value2":1372308133830179280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514459, "value1":0.3578934602691126, "value2":1369826174473414234, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282115, "value1":0.4836091160007032, "value2":7304317831220994830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652653, "value1":0.7715198420930932, "value2":1854799303037917400, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.873426, "value1":0.6674125872681426, "value2":1713253808702182629, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.253882, "value1":0.35618452262868266, "value2":1451362150616905038, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058435, "value1":0.11619135407507769, "value2":729304989898004351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711979, "value1":0.19795958240207612, "value2":5044852283450257784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.620015, "value1":0.7494674771784868, "value2":9072157623838715380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265268, "value1":0.6332467233229876, "value2":7660280405186635297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471797, "value1":0.9586561875915051, "value2":1400450692063171507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715350, "value1":0.28992759308570154, "value2":4420882374996204645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447264, "value1":0.32845931032025727, "value2":3764283351468138749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_7":"g","key_1":"f", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799817, "value1":0.8344281598160882, "value2":3194977322046448615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225092, "value1":0.832673789719215, "value2":8242369633095170423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_1":"j","key_6":"k","key_0":"k", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261842, "value1":0.1783779388932828, "value2":7591102613464952913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885476, "value1":0.33420965888644627, "value2":2231088099862071194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_2":"f","key_7":"f","key_1":"h", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015564, "value1":0.23747285224855552, "value2":2211582269309113296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.415709, "value1":0.43459757217905465, "value2":2297522408220839681, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671302, "value1":0.5521783843346304, "value2":8797575493863836447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809774, "value1":0.3232411464090472, "value2":689408551095099282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391492, "value1":0.44145120690127054, "value2":741707985323247529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593752, "value1":0.5004004561210201, "value2":7899920698831884885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090704, "value1":0.9099972336756517, "value2":8993930752293016892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_7":"c","key_2":"g","key_6":"e", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.365212, "value1":0.17134030372562564, "value2":6300597784300096913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.458419, "value1":0.03537414872175956, "value2":4931151600790868120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.266288, "value1":0.2710772740926763, "value2":5168762762336913335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.846075, "value1":0.8530137052878358, "value2":5846881912829979057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.441246, "value1":0.895274800378327, "value2":6555884777228809545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181487, "value1":0.09635807954147779, "value2":5139650932097495606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_7":"g","key_3":"a","key_4":"k", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.283526, "value1":0.6765717041541169, "value2":717191993149931255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690428, "value1":0.5868291690096701, "value2":4142969563289744689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461183, "value1":0.7388746066577521, "value2":3907321818804232131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.078559, "value1":0.5509982085563974, "value2":8489285868587130769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.006442, "value1":0.17964435576390814, "value2":6385046429151027537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803653, "value1":0.007944203568736737, "value2":8630069900982836811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843384, "value1":0.5245784602574561, "value2":1611187771800002701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_2":"c","key_6":"c","key_0":"a", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.322734, "value1":0.26583004470195515, "value2":5891979565833996742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019366, "value1":0.9471429425977943, "value2":3105898999363337716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583535, "value1":0.3000091753953168, "value2":1671214988273742274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_9":"d","key_5":"g","key_6":"h", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.528263, "value1":0.6167724953698004, "value2":3571326627931246323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.353615, "value1":0.09339691407017821, "value2":8955717188172767409, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.638995, "value1":0.5498958167694153, "value2":6537844553335776020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327054, "value1":0.5183869093313511, "value2":4021636019817176776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391759, "value1":0.6107482510959784, "value2":6639243589639002168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.083515, "value1":0.6947157647744896, "value2":3642097726125624072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.586405, "value1":0.683079171453159, "value2":414358593011525856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847208, "value1":0.004363913601366179, "value2":6293076524070827579, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.583066, "value1":0.38103391426791644, "value2":7168389465606666058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.361829, "value1":0.0757878691848108, "value2":18451857992724418, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_5":"i","key_0":"k","key_3":"i", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481695, "value1":0.5254185806253874, "value2":3155631786417589238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.072077, "value1":0.8787002152127282, "value2":5274515234654561765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.267961, "value1":0.8648926070157235, "value2":4125897469271708547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.245076, "value1":0.06458749862097861, "value2":3727526843973668028, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602905, "value1":0.9902842013208221, "value2":3395310217472956562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.776585, "value1":0.7219469790990183, "value2":77676005392087147, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741224, "value1":0.9110757287517638, "value2":3235394638378837669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.616438, "value1":0.32846413845355765, "value2":5558119133284488407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.617955, "value1":0.12562192604901234, "value2":603576184957547937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.651221, "value1":0.8976398132733482, "value2":7308768939039920124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.610616, "value1":0.3225584230939945, "value2":8416609441712941807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.688197, "value1":0.39642287994645425, "value2":7290339184191009077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.599842, "value1":0.6105446645640619, "value2":1208112926522863577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867582, "value1":0.1840371797963492, "value2":2177170750841832105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098316, "value1":0.966593809823329, "value2":2548324970162406510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694537, "value1":0.3376604030915601, "value2":607328136959332742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748191, "value1":0.9957584594865343, "value2":67877367036436991, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.064163, "value1":0.7248743828192261, "value2":2749527480847272451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_3":"a","key_6":"a","key_2":"b", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502667, "value1":0.16495830272715267, "value2":9030464128428456991, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990661, "value1":0.3949843558188797, "value2":7201429976848106189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987854, "value1":0.29870559261727064, "value2":348149959926675176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.481753, "value1":0.06548607806382464, "value2":1031140396971019018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_3":"j","key_5":"k","key_1":"e", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.630820, "value1":0.8992403438153617, "value2":5146298573769468519, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_7":"g","key_9":"h","key_3":"k", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804082, "value1":0.2519724232639516, "value2":1298310485258981735, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_4":"h","key_5":"j","key_1":"c", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368623, "value1":0.647450377107345, "value2":1284622928379296522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_1":"h","key_3":"a","key_9":"d", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.541781, "value1":0.3261143352667153, "value2":130140210100844559, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543424, "value1":0.06226681703581004, "value2":726618431953897465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988533, "value1":0.49745860881748044, "value2":3406264037344093572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080362, "value1":0.7895574305241833, "value2":152509341700687015, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.150372, "value1":0.9343525151823494, "value2":6076357966504332576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489577, "value1":0.16062395564859527, "value2":3405065082227979672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.225570, "value1":0.28122658630857733, "value2":8458136042003760574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862952, "value1":0.7649680950314343, "value2":1756495878088169871, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.009186, "value1":0.7999050849241861, "value2":3718209620537893555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.327636, "value1":0.23708513628211542, "value2":1522368463913294016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.551176, "value1":0.009439114458257377, "value2":2361018304142309387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.633123, "value1":0.2821310200838197, "value2":6052402151738260293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.128752, "value1":0.3359757178563009, "value2":7472471602725669166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.456386, "value1":0.46133506790277695, "value2":2296093836300686089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_3":"e","key_4":"d","key_1":"g", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420349, "value1":0.36664549905260974, "value2":3546467223240204480, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.938528, "value1":0.007977829048726685, "value2":4244809297577031047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.226579, "value1":0.6725646472034443, "value2":913735509091846801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_9":"j","key_3":"b","key_5":"g", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.850850, "value1":0.23068766044782393, "value2":2300681752566773332, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285218, "value1":0.6989337075616671, "value2":2607116938388175821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.221331, "value1":0.11523512208048978, "value2":8241016534243475750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_4":"c","key_8":"g","key_1":"c", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886294, "value1":0.8815954846610522, "value2":1129122380251841667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010095, "value1":0.4468942822052692, "value2":7305659252479204819, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007461, "value1":0.3287186449948172, "value2":8533612311279585337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841878, "value1":0.06768912562638665, "value2":3915275733365598237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239318, "value1":0.2752287175449378, "value2":6104494420782937095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364905, "value1":0.7269412172343553, "value2":7167814392288649336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030862, "value1":0.35386987370196926, "value2":568029767114417610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945737, "value1":0.8998415941144967, "value2":2346717810373152754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.093117, "value1":0.8244480275524787, "value2":3248267576826263944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.422568, "value1":0.5719502852011219, "value2":8260911644483203383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.823166, "value1":0.2849312986012331, "value2":3755798306625123375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268641, "value1":0.33070121758142046, "value2":3503668812330101597, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564550, "value1":0.4481321012388454, "value2":2552310395319729245, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417743, "value1":0.047809954457076996, "value2":1081553625885385604, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.595860, "value1":0.2921995022802213, "value2":5059986207137621431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.505814, "value1":0.9601390700122682, "value2":1325398133306866350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357820, "value1":0.3304365801280413, "value2":5155828582049823863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963899, "value1":0.8305280855606907, "value2":2952790462431625685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395768, "value1":0.05940274489209035, "value2":2206736622008483109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_2":"f","key_5":"d","key_0":"b", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.994837, "value1":0.28131858851364544, "value2":6377323293251471217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825885, "value1":0.9524553792547288, "value2":5686625986396675519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962844, "value1":0.7824414856459302, "value2":3176085690721005555, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525393, "value1":0.2894500361431452, "value2":9170274733300331931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066894, "value1":0.9750264781896856, "value2":6481943120958967565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118777, "value1":0.49186253332263474, "value2":196535145535309391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.011455, "value1":0.25527316956832247, "value2":23216573243095801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.516739, "value1":0.7031419417768016, "value2":2051319192678332676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621518, "value1":0.7948563798954263, "value2":8394265809911073481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.008794, "value1":0.671348308630362, "value2":8908504416251388849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804142, "value1":0.34185386259040573, "value2":7569056971355045232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971215, "value1":0.795050202357287, "value2":7291081553392495608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_6":"d","key_7":"k","key_0":"b", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285984, "value1":0.6233623903797153, "value2":6444347877884163801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.436168, "value1":0.122788509000057, "value2":2542694660610691923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545038, "value1":0.897272884042895, "value2":3518180018317716924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.377957, "value1":0.26500726028503324, "value2":918021549502007270, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186659, "value1":0.8861606285044316, "value2":7428746411605199070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.706851, "value1":0.8539941323885553, "value2":5095874822126979293, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.552666, "value1":0.8209937326987259, "value2":7918997962757156260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.708678, "value1":0.20172873469302396, "value2":473140516571282958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.588929, "value1":0.8992364975321605, "value2":1107792164310220135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.396543, "value1":0.9159358105409485, "value2":3240110351377657654, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949484, "value1":0.3395753649788759, "value2":1556081794159850294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627633, "value1":0.6064813434025144, "value2":504106831033400051, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760461, "value1":0.027052105627838302, "value2":2547821024594975963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.690642, "value1":0.29842739674487334, "value2":4347369482397818828, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.271622, "value1":0.9232891397814524, "value2":4902825584314369050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678674, "value1":0.8612335750248717, "value2":2676726884600182684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_7":"b","key_2":"f","key_5":"f", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.789269, "value1":0.7153455595388135, "value2":8629094676547921616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_4":"k","key_5":"e","key_3":"h", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.274447, "value1":0.04852479796186674, "value2":520491486142586388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517924, "value1":0.5065900646461697, "value2":1305736770624753229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440510, "value1":0.8079290707038177, "value2":745163355638334416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.992658, "value1":0.33022695610085456, "value2":3745647814397920317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.142855, "value1":0.7836474840216356, "value2":1701655973854226818, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.718918, "value1":0.9106879340813463, "value2":2642697174847166539, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358697, "value1":0.6889718382775613, "value2":2396285724783610411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.782243, "value1":0.10018629733131312, "value2":3557685057170687759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.290843, "value1":0.5213600910147824, "value2":5522593432675792373, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243393, "value1":0.9922299657128142, "value2":267617765686382025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.827775, "value1":0.17424595518809072, "value2":3247150599498567700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954439, "value1":0.6395234710891515, "value2":3700591746218368196, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543518, "value1":0.7123005069599985, "value2":8170425126276779400, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_9":"c","key_3":"h","key_7":"j", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862097, "value1":0.7928466963741031, "value2":1388049342872581158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861104, "value1":0.44830540845614686, "value2":3638392887423846691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996353, "value1":0.9606274415719652, "value2":6260542395914742363, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269620, "value1":0.27973905679528643, "value2":1556926452504115363, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118090, "value1":0.5428339606791369, "value2":1853095038687144617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.843203, "value1":0.15162943121847733, "value2":978719322151022878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.168392, "value1":0.30548225941021784, "value2":7137186554911304535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.197151, "value1":0.3975385059702385, "value2":5863175014471717006, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565354, "value1":0.9234247531019719, "value2":812121677319261736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030909, "value1":0.8792093662826526, "value2":8559138722753842843, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.484600, "value1":0.23944552081323173, "value2":7751158766892162613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.320620, "value1":0.12230291625693078, "value2":6192516194919614511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_6":"b","key_9":"a","key_3":"e", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.935691, "value1":0.28006447226692, "value2":6784015217669799303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.336215, "value1":0.23094104388429443, "value2":5569809866177597446, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988466, "value1":0.5462607441657077, "value2":8311104273733417101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447804, "value1":0.4867937694484817, "value2":5062168010270449210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.190570, "value1":0.8283372201936341, "value2":5874711067636169206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904281, "value1":0.8553852749887187, "value2":3311030506444553065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.751439, "value1":0.4266991113414808, "value2":4999173498909581042, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.125309, "value1":0.5563950948301023, "value2":1187994145431678821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.973142, "value1":0.11588338763957572, "value2":8318570385943791860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.116231, "value1":0.2526143185693312, "value2":782968109827902547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.187535, "value1":0.27872170170772304, "value2":8766589428595489268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945637, "value1":0.08854405995381304, "value2":3412460749402337910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.270152, "value1":0.3638550762746485, "value2":2218032724303888987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051113, "value1":0.40794935925664466, "value2":7831097119693907093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265496, "value1":0.23954206323753308, "value2":63253225412010244, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416624, "value1":0.17710696129812786, "value2":3803821152719869396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607195, "value1":0.28582291116541503, "value2":7284275038719896556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.326362, "value1":0.17094485896565303, "value2":4663361033102261072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_4":"j","key_0":"j","key_1":"f", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438839, "value1":0.3486395787421312, "value2":8809206276934706557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033597, "value1":0.8575270834812074, "value2":1940304743017854051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.065029, "value1":0.9427613360719956, "value2":271108060306002877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.488385, "value1":0.12779971800108084, "value2":3316011689829646532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961814, "value1":0.5428137306648136, "value2":1514601534044579978, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263207, "value1":0.6949685012843101, "value2":895006576805426648, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282635, "value1":0.10777955836055363, "value2":564953042266607985, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.001269, "value1":0.042927028237879465, "value2":1466958494603226347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682457, "value1":0.8478178196386861, "value2":107121152123493591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.159543, "value1":0.9604877997752348, "value2":7593923445754669936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015347, "value1":0.7626139882833062, "value2":5647986907617063275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209254, "value1":0.8450839759887067, "value2":1886374721985796728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066244, "value1":0.07791428229072243, "value2":919090579972011287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.699052, "value1":0.06643124534628293, "value2":2711140899074023727, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.631133, "value1":0.143383362490665, "value2":971805873892273782, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_6":"h","key_2":"j","key_3":"k", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426673, "value1":0.7738540415601494, "value2":4397130911534361286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.389568, "value1":0.33397631468501043, "value2":4052528315967262589, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.622610, "value1":0.24246003195798918, "value2":5479722481130894419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904392, "value1":0.9774495314687007, "value2":8793165354063487461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.793581, "value1":0.4190858782174469, "value2":4926610192238705598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110855, "value1":0.2884411940593678, "value2":5227068360933879219, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_7":"d","key_0":"k","key_2":"f", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135653, "value1":0.402267966757566, "value2":6977955000760069723, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.307152, "value1":0.8927574650950808, "value2":5803177489389723115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.737685, "value1":0.21404400943023025, "value2":6732954103348577362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.121533, "value1":0.18872391644762126, "value2":8989825709899786188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_9":"b","key_3":"h","key_6":"i", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.972223, "value1":0.023522846364320095, "value2":7146680715055172496, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.923388, "value1":0.8860383778342971, "value2":8303048822044107475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.851782, "value1":0.32631560891796313, "value2":1989916996789168449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391839, "value1":0.002802043190783618, "value2":8629895271323501696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_1":"c","key_6":"c","key_0":"f", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.066714, "value1":0.031995170191600183, "value2":3319906268065732292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421535, "value1":0.635220505499552, "value2":7200157785718775016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_9":"j","key_0":"e","key_1":"b", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287471, "value1":0.8948051177891431, "value2":2176610475830643902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.804659, "value1":0.28415985878229416, "value2":33052675971427371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996360, "value1":0.6152217383986905, "value2":544588703505765788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.798458, "value1":0.8665558307064753, "value2":8115268483760058936, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_6":"i","key_7":"e","key_1":"c", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448744, "value1":0.3917339620374744, "value2":9198148727913546136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.861092, "value1":0.9826532618696505, "value2":4765520374138069290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.202532, "value1":0.0052591320469092834, "value2":336652236719043103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_7":"b","key_8":"d","key_4":"g", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.524950, "value1":0.8143167251197552, "value2":6529413407694528834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.485366, "value1":0.5907312441951306, "value2":3337798525793799244, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558504, "value1":0.3641804008571667, "value2":4575793560178915745, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_3":"b","key_2":"k", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642936, "value1":0.623697886602654, "value2":7776504717328671778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043344, "value1":0.9623993994613507, "value2":6602176914068250275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.241555, "value1":0.23253929846791072, "value2":685349515600255534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926727, "value1":0.21223983767190985, "value2":3951472227334441759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.131046, "value1":0.2851063666145672, "value2":6180530959986721149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.247299, "value1":0.5130082295270245, "value2":8139956405448602641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.758053, "value1":0.49876246690774434, "value2":3786706615107859566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323322, "value1":0.4313958567662574, "value2":8854072512884634153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_3":"h","key_4":"h","key_1":"j", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748573, "value1":0.9590027921453678, "value2":3839028144294969465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_3":"h","key_6":"f","key_0":"i", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.384213, "value1":0.010562139959872605, "value2":6878855550977960785, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180644, "value1":0.6131439659014073, "value2":2706285599198878081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.593573, "value1":0.4120750238333436, "value2":1200854625465685501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.995840, "value1":0.297035809872462, "value2":6168541710158809840, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_9":"k","key_3":"e","key_8":"d", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.511756, "value1":0.006206978177219643, "value2":1958577823681147315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.565859, "value1":0.4442276087687805, "value2":39986647781639917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715853, "value1":0.9381008994826828, "value2":3556376195988394411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236468, "value1":0.20999691756150096, "value2":9161072742831597145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.161371, "value1":0.738745986035695, "value2":5138216362948975345, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445330, "value1":0.14739850543555172, "value2":8667736730671884444, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.982188, "value1":0.40813076015564426, "value2":3022991256121544350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.984748, "value1":0.6214792221342914, "value2":5933123653156672217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562273, "value1":0.8568295281307874, "value2":5414869267292754213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476811, "value1":0.49675643916102746, "value2":5664619325942193065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.891490, "value1":0.8866201621688181, "value2":8214607251036840472, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.931525, "value1":0.8288823187078326, "value2":3069955632009854668, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455511, "value1":0.8774391931652025, "value2":3046641762023967525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.400651, "value1":0.5254465108261448, "value2":4999803900412377548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666971, "value1":0.9993076586927376, "value2":4982431527333428358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.8618210720028192, "value2":761627529379782309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.915374, "value1":0.37053901712348225, "value2":3944125570135845867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.960156, "value1":0.9374154078802892, "value2":4747973404867949779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210720, "value1":0.8468677886051782, "value2":4852924773263065270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.332500, "value1":0.6465642869647131, "value2":1008458018130353532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.788169, "value1":0.3773654642642006, "value2":4191084511110280464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.545303, "value1":0.25852749290398136, "value2":2783025552475099417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.143623, "value1":0.36024800648413563, "value2":4932311496806548620, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895647, "value1":0.0053042457769262655, "value2":6923819500937313799, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_5":"g","key_8":"i","key_0":"e", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932434, "value1":0.659079228202596, "value2":4313811336647625065, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.260771, "value1":0.403057123436948, "value2":4416521488550693116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_5":"c","key_1":"a","key_4":"h", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238541, "value1":0.2952177348030342, "value2":1538460010819351929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958540, "value1":0.5291106177059647, "value2":1249052435364791122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_8":"a","key_2":"d","key_4":"i", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.430589, "value1":0.8147197427498966, "value2":5526371695265553363, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431544, "value1":0.9355303369766881, "value2":4469095563720030593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640478, "value1":0.48586313377391904, "value2":7959068707072142404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.144376, "value1":0.00848981726968387, "value2":5068460906328461885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.416953, "value1":0.6734644377643653, "value2":2766510942957347250, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885086, "value1":0.21498485778798626, "value2":8278007933868416694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_2":"i","key_0":"b", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537244, "value1":0.3154976316557619, "value2":2639365305979929980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_8":"b","key_1":"b","key_3":"f", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.020210, "value1":0.453949847740753, "value2":2024272373950233485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.514022, "value1":0.3475698755294227, "value2":1372329110659095675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806258, "value1":0.9274447352712736, "value2":3920648533163431087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.209427, "value1":0.014587282647209867, "value2":8695997195066251972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.971326, "value1":0.2797692384280631, "value2":6419079357930277367, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.652286, "value1":0.5905911533568637, "value2":8712000656540499616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878040, "value1":0.48362174266483915, "value2":2926563391771284063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.990329, "value1":0.5049151964579438, "value2":3650817862757599534, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049823, "value1":0.838067458662296, "value2":8854264061775325606, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017442, "value1":0.8431632474037919, "value2":8852347705624487512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.544821, "value1":0.005649842126714668, "value2":2282778264986625535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.597518, "value1":0.2045781468759336, "value2":8575639431654539646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612206, "value1":0.3075892336013964, "value2":7673450384556490732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.471060, "value1":0.40918086270080856, "value2":6747291739813622106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.513402, "value1":0.7422854995723118, "value2":4083993911078701361, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.965008, "value1":0.6372798813181509, "value2":1707224457136697946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.354283, "value1":0.9108473180297934, "value2":1020266209800871174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806140, "value1":0.14778437030634853, "value2":6105158525255417708, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194962, "value1":0.3473291379662418, "value2":3795992189004973455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.974776, "value1":0.09118542426518424, "value2":5134289787707010070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.339938, "value1":0.13774510006333723, "value2":562171884148417195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.550551, "value1":0.384009626532502, "value2":8609730106908869554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.654960, "value1":0.2639517045663951, "value2":1862456670074602807, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.030932, "value1":0.5143312749002444, "value2":1494847764422163119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.695169, "value1":0.2423970054677799, "value2":8509443740065644494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713656, "value1":0.34108742339653203, "value2":4598279459368093139, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_2":"f","key_0":"d","key_1":"e", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.794261, "value1":0.9458347874797615, "value2":7302724440323717945, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.364772, "value1":0.2365974372946532, "value2":3654435595389536462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.395099, "value1":0.15838919107992194, "value2":1939247105684031089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.335629, "value1":0.39545025723069543, "value2":2672917998276624419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.074949, "value1":0.7876350399335282, "value2":625303995684394443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536138, "value1":0.5616610785403638, "value2":8164874977498368721, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697782, "value1":0.23834340441765955, "value2":3951692500137347815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.671649, "value1":0.6501720201004972, "value2":2955880776784904551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.787079, "value1":0.378893964304627, "value2":3193906322405958964, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.031359, "value1":0.29145414339441095, "value2":5615459484207384702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.306294, "value1":0.34557951146905197, "value2":2931839249030464715, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_4":"d","key_7":"e","key_1":"k", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357655, "value1":0.3923960009663459, "value2":8112783273863341075, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_3":"a","key_2":"h", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.961896, "value1":0.23361672067408637, "value2":432132564805318676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.098797, "value1":0.18420600660031175, "value2":4984332870487251645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.071876, "value1":0.6659197757567412, "value2":1553450733606247594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186866, "value1":0.7000117639690743, "value2":7329872261968800728, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406584, "value1":0.13046586013887515, "value2":1622087582581823179, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.440301, "value1":0.11646408872647017, "value2":1011606322554570900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.136166, "value1":0.4328392754883894, "value2":2919499144416670917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_8":"b","key_3":"c","key_6":"e", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.318957, "value1":0.7075576973107754, "value2":398072267285532828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.012179, "value1":0.5668221134322494, "value2":6110074131405385081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830949, "value1":0.3334949924022553, "value2":7572722630631231675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_7":"c","key_8":"g","key_0":"k", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145965, "value1":0.365420169950548, "value2":2535895288587113465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_6":"g","key_8":"i","key_2":"h", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.649060, "value1":0.16445860625759065, "value2":6921600715877525051, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.517227, "value1":0.07192874427493294, "value2":6297114124078920008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_3":"i","key_5":"c","key_9":"f", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832587, "value1":0.5150936291796077, "value2":181711691089923280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291455, "value1":0.5623557100802268, "value2":8762626146094505466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_5":"h","key_8":"a","key_2":"j", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.149447, "value1":0.5534139568709165, "value2":2510872908080100775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476658, "value1":0.4486907399118968, "value2":8992668719445662670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236318, "value1":0.49075032225313237, "value2":1924725427300127433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.561765, "value1":0.9668124792198439, "value2":990391456732695590, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.884668, "value1":0.0011015591734681801, "value2":5327656656973278324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.175200, "value1":0.9826652864577513, "value2":2876633651705447757, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379413, "value1":0.15362330796009416, "value2":7353421640256094445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.319517, "value1":0.5202924305313171, "value2":7456692117889227019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.122103, "value1":0.7187092003738956, "value2":7131563361103346240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_3":"a","key_9":"k","key_1":"a", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.678103, "value1":0.7543751799594591, "value2":8526553775473008293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.643972, "value1":0.19798980984364278, "value2":5929329615236343098, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.831121, "value1":0.1528427519418499, "value2":6184723596853858850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432219, "value1":0.5515160206615604, "value2":23121424938113426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.058751, "value1":0.6338015087216923, "value2":7816911191202995864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790739, "value1":0.8762741406918414, "value2":4555556160832468314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797544, "value1":0.8139645999303083, "value2":2761547225024668528, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.404353, "value1":0.9860244138424634, "value2":2908826677832473108, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016900, "value1":0.053160518991546266, "value2":8380107749249883562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.407973, "value1":0.7843730354485248, "value2":6824583973593836053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.987174, "value1":0.7619687243629265, "value2":3319925462827235787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_5":"b","key_7":"i","key_2":"c", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577835, "value1":0.20813952139985942, "value2":6437982890007555330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755320, "value1":0.3636597821442561, "value2":5470963049552133157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.897987, "value1":0.07151356373469021, "value2":1477083928227057783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388580, "value1":0.20932748154080844, "value2":3392543048891869159, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_9":"c","key_4":"i", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.025802, "value1":0.38577745314564454, "value2":868566582237921681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941032, "value1":0.17111301155402217, "value2":6961085515450408974, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_2":"d","key_3":"f","key_0":"f", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947055, "value1":0.9780343555878157, "value2":133683558419674847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.954822, "value1":0.08800460018752372, "value2":7497584262448035095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063420, "value1":0.8100121364093217, "value2":3755200469512293547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438294, "value1":0.3424655143965952, "value2":4316249625206240732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.809627, "value1":0.9673131880891396, "value2":4132748723398010856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.439925, "value1":0.8075664782512888, "value2":864263057207692643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.084104, "value1":0.3818348864598292, "value2":2553923579282526899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.647912, "value1":0.46216103925581126, "value2":4639093581287452200, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369627, "value1":0.3422458784570339, "value2":4254456412295026385, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.614963, "value1":0.5220645586983701, "value2":6072161543938690608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.321895, "value1":0.3049038907635511, "value2":6325538472569574229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464290, "value1":0.22336524838033933, "value2":9076126601794953657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.166932, "value1":0.3553203900309477, "value2":5982106259976109422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_0":"f","key_2":"f","key_5":"a", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803898, "value1":0.5280548625824253, "value2":4063002262098608883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.927974, "value1":0.17895618154031318, "value2":547646730434307817, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.303832, "value1":0.5511007572679634, "value2":2862350870063686751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520504, "value1":0.39724544886891017, "value2":6931266301104819935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941095, "value1":0.14786172741452286, "value2":3205668266543533832, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.656997, "value1":0.32075936991205234, "value2":7391478133987747835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120921, "value1":0.3939864396610821, "value2":2909689659050189847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.958790, "value1":0.3269314306247572, "value2":6783238272107071770, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.386136, "value1":0.861947664801586, "value2":5805659227499250652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.449365, "value1":0.5799629274092946, "value2":9193070830904141987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_3":"e","key_9":"h","key_2":"f", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.785656, "value1":0.3165816948400473, "value2":3116393083198778433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.520711, "value1":0.6656585396817016, "value2":1017823568348313071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.650203, "value1":0.22215575308437716, "value2":1902945447598147226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249002, "value1":0.2961338302748422, "value2":1415432970559418672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_5":"k","key_7":"f","key_1":"e", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694015, "value1":0.7506751146797324, "value2":6142650591148786794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.304466, "value1":0.37747841635497736, "value2":5375036774405769501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_9":"f","key_3":"a","key_8":"c", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.641004, "value1":0.46446471911344284, "value2":5955856288590241375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.684973, "value1":0.907579891922439, "value2":5346005034595738366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.111153, "value1":0.49826719629056415, "value2":7497650141562337241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.830722, "value1":0.28960521156096847, "value2":6154503575103623089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392721, "value1":0.6834318849474948, "value2":2628637152649849276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969704, "value1":0.15429907011430974, "value2":5405475101757554186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.536252, "value1":0.7000830425153579, "value2":406966410660924195, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.807572, "value1":0.2012419938894414, "value2":4358282132524848204, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.820107, "value1":0.5076022682087846, "value2":4017461578732969062, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996611, "value1":0.7422015873263831, "value2":436292068764222211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.922123, "value1":0.051072363737232486, "value2":268646986926660655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.729895, "value1":0.8676962023633001, "value2":2065960180781249750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636047, "value1":0.4996316174329004, "value2":3156670443682802457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_9":"d","key_2":"h","key_6":"e", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.431520, "value1":0.5204212362312931, "value2":8019726224533633908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.903149, "value1":0.5859452724568418, "value2":2288060944551638278, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.070771, "value1":0.9932429756726906, "value2":6690408728176272233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.314453, "value1":0.3472880048665214, "value2":58548757977887750, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.627243, "value1":0.7139819549630214, "value2":7080915148891558502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_6":"b","key_3":"e","key_5":"k", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.755294, "value1":0.05408051807422248, "value2":6931097273017939069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.282331, "value1":0.3203397893756164, "value2":8256062300842673383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.034396, "value1":0.9531256831279978, "value2":7765396458031448915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.115811, "value1":0.13116244832703935, "value2":3543488806097224100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.666851, "value1":0.6887850450312126, "value2":5462261380989469479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.525431, "value1":0.3139759113605816, "value2":8160395548303928540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036488, "value1":0.7437550776292858, "value2":905546975072552710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.246868, "value1":0.7173148070188747, "value2":7273545640281703287, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840343, "value1":0.20321210783475557, "value2":336636579624758589, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_2":"e","key_7":"c","key_1":"h", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100735, "value1":0.8393194019944642, "value2":291716047034557387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853981, "value1":0.5202422200638643, "value2":5204943321202939039, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512132, "value1":0.018314329801792806, "value2":961979132289563000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.487612, "value1":0.7033581894769818, "value2":4186345519226619201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778169, "value1":0.7839413953257911, "value2":7874569661704504858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.587363, "value1":0.24066050930994098, "value2":2238921997040090404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_2":"d","key_3":"e","key_4":"e", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.863823, "value1":0.22725313032865457, "value2":4922946319365439720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.462195, "value1":0.6013044688497777, "value2":137484525088955387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_5":"h","key_8":"d","key_0":"j", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.967535, "value1":0.21528129199568857, "value2":5907496598149934999, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.263462, "value1":0.3398984210102334, "value2":3627872884296903263, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.778010, "value1":0.2649025290720317, "value2":2665113842043817940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.895866, "value1":0.8505969892692369, "value2":1773042500322059709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.475234, "value1":0.08631156102550058, "value2":519973765404949203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_7":"b","key_1":"g","key_4":"i", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996762, "value1":0.6378946736901862, "value2":9195196376217348512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537168, "value1":0.2006763700535789, "value2":9067766312454029883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.585179, "value1":0.8887141585928169, "value2":1447763822787404407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.937668, "value1":0.18798426398408305, "value2":5531072459849624180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417573, "value1":0.018969474714960195, "value2":8739232693787930523, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.432784, "value1":0.6772697684929758, "value2":3670574248400915486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.945707, "value1":0.2017640539145947, "value2":6866599884426415644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715992, "value1":0.2770810284692845, "value2":6065196798538107388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724512, "value1":0.7694937233145257, "value2":5461354547379838063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555480, "value1":0.4116895522673444, "value2":6999095526200651536, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357696, "value1":0.9694808723879291, "value2":2308259417403012034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.925321, "value1":0.9804626384591183, "value2":1344780667969714366, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340972, "value1":0.8564397089342566, "value2":3335346544187347307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.634021, "value1":0.2824134389967874, "value2":827846054793909627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535849, "value1":0.8383622181777869, "value2":3210193933415043724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.949231, "value1":0.3423558587998688, "value2":8018588983650318493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.604636, "value1":0.6545573982070936, "value2":8784079580099061471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.041080, "value1":0.6815494814472329, "value2":2413918152267969152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206623, "value1":0.7407991857310414, "value2":8688412282907699410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697192, "value1":0.9920334806937902, "value2":4058886921829417026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.748394, "value1":0.3435945058384803, "value2":6002972956903494678, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421611, "value1":0.8151473391165547, "value2":261810376504255149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.027674, "value1":0.32343755952478886, "value2":8467224123375485048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205103, "value1":0.26583420276460434, "value2":912318083515976688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362886, "value1":0.6918507204150298, "value2":3042551716463426665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_6":"a","key_0":"b","key_4":"c", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.181414, "value1":0.9588817871484018, "value2":5500996639169097947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.607786, "value1":0.1555201931458164, "value2":3667544046344919475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.447991, "value1":0.6322575842993106, "value2":6581489005452708007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_6":"k","key_7":"k","key_4":"f", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.602539, "value1":0.3803193364932477, "value2":501846031619014460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224149, "value1":0.14287189157740537, "value2":4390766248301994532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.019349, "value1":0.9275903306872896, "value2":6065489708075344215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.753905, "value1":0.04103970388034392, "value2":3647389822252762296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.808552, "value1":0.6912948158348138, "value2":7168224379303111769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.383522, "value1":0.7213839418172744, "value2":8917533656021329315, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.988681, "value1":0.4652567645908903, "value2":6088381044289659824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.426511, "value1":0.8194664026578082, "value2":3767723603035206676, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.291449, "value1":0.11519619446576408, "value2":7042710433748403097, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.495051, "value1":0.7541019265512129, "value2":4805842413127851386, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.498957, "value1":0.7657156651926107, "value2":4284045357936772257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.920671, "value1":0.557277837350798, "value2":7618404853124250629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.325130, "value1":0.8555228162118138, "value2":5568850179022711136, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619588, "value1":0.8294092067729972, "value2":1498942661428438729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_5":"e","key_9":"d","key_4":"c", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932410, "value1":0.9716095495296341, "value2":6631794618463446087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.257618, "value1":0.6931656613099204, "value2":4999381616225061323, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.328148, "value1":0.3114535269637382, "value2":7689977371490904120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_3":"i","key_4":"h","key_1":"b", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.280989, "value1":0.9111540107476489, "value2":2001844404871692409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722254, "value1":0.17067710980514048, "value2":9024628709036843159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367790, "value1":0.7784645051385055, "value2":4706281866949049190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368237, "value1":0.6741959290593132, "value2":2438020324428284656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885688, "value1":0.4581933301264591, "value2":977306309852850973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.856287, "value1":0.24855090559153642, "value2":5403228793051986733, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.305675, "value1":0.6170232598989743, "value2":274658269233518106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.908698, "value1":0.5064010761390361, "value2":5817465185742290417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757988, "value1":0.04843336165427696, "value2":5801707099278996481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.933565, "value1":0.22461787050531695, "value2":6104462847804923186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.720615, "value1":0.12587001891222954, "value2":7755893877648671182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.642145, "value1":0.4752800242516813, "value2":1097468219850109153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975649, "value1":0.9950222857947861, "value2":323860279126996239, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.571701, "value1":0.13176848875241834, "value2":8974250013315108425, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841017, "value1":0.5108584547409708, "value2":1015654371149607255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.499539, "value1":0.7323245432905996, "value2":866168553464820252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_8":"i","key_0":"a", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.137678, "value1":0.9837633616038727, "value2":1980815750999371252, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.713500, "value1":0.6844141285754438, "value2":5215363796900492290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.564182, "value1":0.6959446753092476, "value2":2124568431454851658, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.548476, "value1":0.8904291651425387, "value2":8960234619080614404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.334127, "value1":0.5777624482637982, "value2":3890652700139772457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.141095, "value1":0.3019838758248612, "value2":1159300889479734121, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_8":"g","key_1":"d","key_4":"a", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800239, "value1":0.4131750973298026, "value2":2344128018741531119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369249, "value1":0.1589896462920189, "value2":4160545898250072075, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_5":"h","key_8":"k","key_1":"g", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.198713, "value1":0.08298578373908365, "value2":50980726393447907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.773734, "value1":0.7496268790368541, "value2":7804146534404388324, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237426, "value1":0.27543463085841935, "value2":3694238253557813150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577321, "value1":0.6586006530161179, "value2":641077213271202791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.368897, "value1":0.6785289242869411, "value2":2009007121053273154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.801586, "value1":0.7989756868311243, "value2":3470745118950983568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.035889, "value1":0.4089400550263047, "value2":3320771957585601823, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.760511, "value1":0.7386595086100225, "value2":3332011099556778058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.359596, "value1":0.8203562587923371, "value2":574506838747954548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.063436, "value1":0.2579978917934866, "value2":6993045686428025271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.369331, "value1":0.4769488047439803, "value2":3318100445225134934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795599, "value1":0.9012136284147806, "value2":5770204186306544532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.219070, "value1":0.20974721576307698, "value2":2356228020734101377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537221, "value1":0.1725299480275798, "value2":5073957380468101443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.015458, "value1":0.4025573823516011, "value2":3282394548862748845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.580078, "value1":0.9823965911617399, "value2":8476500551084420415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.979266, "value1":0.5236855845499949, "value2":680393934209420766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682327, "value1":0.844770708190905, "value2":8298068650764370173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350010, "value1":0.5806359147839931, "value2":1527219564263667157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.206279, "value1":0.4978598735396454, "value2":8931241509064706089, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135137, "value1":0.7138645301142236, "value2":2633600346218814000, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.981064, "value1":0.7353726130780065, "value2":268395459435449482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.526357, "value1":0.3584205029962212, "value2":7926609514114360701, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.145680, "value1":0.817655724537724, "value2":2809466879616878308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.877503, "value1":0.17147641272949588, "value2":1475184649319875768, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.049475, "value1":0.3904708182990629, "value2":8178189937335677971, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.419221, "value1":0.2498991121797316, "value2":2019493757980421565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646852, "value1":0.2981893785163364, "value2":4127223489573331187, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.194796, "value1":0.9706301430727536, "value2":3907775771119307356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.135232, "value1":0.952467511486142, "value2":6824235396463011996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.392387, "value1":0.19563387556471776, "value2":6531634041022343859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.133702, "value1":0.7965289882627238, "value2":6086508658030257940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.691192, "value1":0.21849926406658296, "value2":2500546592704430602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.238250, "value1":0.8547288670113503, "value2":1751658935211452093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.259613, "value1":0.7399127039208734, "value2":3758431842842501346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.959851, "value1":0.06348104065773419, "value2":5803029266892181407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_7":"c","key_0":"c","key_2":"a", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350885, "value1":0.07399715612950311, "value2":9184522919237034506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.821915, "value1":0.9143789840874811, "value2":3150076904171453287, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.554775, "value1":0.406185744411068, "value2":4390857727102845864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.026241, "value1":0.576960195810922, "value2":864775792792522430, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.118029, "value1":0.6740072102143805, "value2":2738633295377346249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.669898, "value1":0.6063573387513779, "value2":8118600251510115903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.406027, "value1":0.5408212514385392, "value2":836959541970766944, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_3":"j","key_0":"f","key_2":"b", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.205752, "value1":0.9617267357194531, "value2":2302570688726041304, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.683346, "value1":0.7764943186065553, "value2":1516613683034572821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535753, "value1":0.8501812602308251, "value2":3781935852497505404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562975, "value1":0.5762183849456854, "value2":8803839397694965730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_9":"j","key_1":"c","key_8":"i", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.224509, "value1":0.6945846689247714, "value2":8790065402136541035, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.120301, "value1":0.5693297769803048, "value2":8817337009559896419, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.535111, "value1":0.7168638232585879, "value2":1459446787633025168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590881, "value1":0.052466731276145914, "value2":6984133752001938547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.800340, "value1":0.19161367226547096, "value2":8470840993920678933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797252, "value1":0.22382372006161758, "value2":960932677896506756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.502545, "value1":0.4567752766501535, "value2":9200307528794857259, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.919995, "value1":0.3074462546242729, "value2":1729230620360441809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.273301, "value1":0.5542454650469373, "value2":2007525431527262950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_1":"g","key_8":"i","key_0":"e", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.249396, "value1":0.30289020495328595, "value2":7647941374943306916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_5":"i","key_8":"h","key_1":"h", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.847062, "value1":0.8617179899648977, "value2":2309423222399622901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.770003, "value1":0.8185102377578417, "value2":8900535665947339098, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.612076, "value1":0.2520488653737615, "value2":534199497930559417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.867000, "value1":0.3056461531330615, "value2":8074358922929649800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.543352, "value1":0.1267819654500433, "value2":2282337611780682090, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771337, "value1":0.3860477352422603, "value2":6614019503509668230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.724939, "value1":0.35759761476037216, "value2":7730807476801277254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.769189, "value1":0.06006911990731757, "value2":6135075757488436084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.750837, "value1":0.22593301727381723, "value2":815096043795563480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.733823, "value1":0.37434887115879495, "value2":8767359811379324612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_7":"b","key_4":"g","key_5":"a", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.110462, "value1":0.7613769242381963, "value2":679247224492133418, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.207493, "value1":0.26270378513285625, "value2":6127960652597929191, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.757551, "value1":0.17246010749021554, "value2":3746123411387113183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.476665, "value1":0.530111421321598, "value2":7929787788154212077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_3":"e","key_5":"k","key_2":"d", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.080232, "value1":0.24610687758058403, "value2":4382073893358318099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.518795, "value1":0.13825391880428892, "value2":6023983814814871189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.411487, "value1":0.7443212678339678, "value2":3163265105788950613, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_8":"g","key_4":"b","key_6":"k", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.230729, "value1":0.6799575270403697, "value2":6569848391065499405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.251807, "value1":0.23368831551353111, "value2":9121098592064787356, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450371, "value1":0.8049793990951134, "value2":6271606339009618863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.268506, "value1":0.7393978258409317, "value2":3084814221880218023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.453066, "value1":0.7167056491982889, "value2":8359177712079217325, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.840677, "value1":0.3999771937764131, "value2":5907070339651841877, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.853610, "value1":0.4459636741761099, "value2":2591090489482088505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.050602, "value1":0.9347582889921432, "value2":3586868729035531903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.460673, "value1":0.8573683555697056, "value2":77436103148660020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.791642, "value1":0.7214035249481627, "value2":4712458830503182070, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.711339, "value1":0.7704360355595858, "value2":4818602735855589639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_8":"e","key_2":"g","key_4":"c", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.709838, "value1":0.980850892357579, "value2":5814687934592979738, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.448259, "value1":0.6911929910350553, "value2":2331566896810690473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069070, "value1":0.7715792084831908, "value2":1237753144790060173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.728905, "value1":0.6453677424318518, "value2":4827712214727464044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_8":"e","key_9":"b","key_7":"b", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.581880, "value1":0.2677360178055146, "value2":8844278477287710309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.969799, "value1":0.5783967942752313, "value2":5905445599124832757, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741372, "value1":0.012956038256400619, "value2":914084143172531236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.862619, "value1":0.5947373167351414, "value2":3630517863485151782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.139536, "value1":0.8372408825864636, "value2":6952590048486505005, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.803976, "value1":0.43722974429572603, "value2":2176830109855416805, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.534405, "value1":0.7241541542820893, "value2":4511693404596905586, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_6":"d","key_5":"a", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574402, "value1":0.7055278131957177, "value2":1238394241461479434, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.579476, "value1":0.06494326977178205, "value2":3563720913525659633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.878052, "value1":0.8371531564279497, "value2":587508487226380908, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.672424, "value1":0.26126599404383394, "value2":3460643419757907518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.408361, "value1":0.5841391902007207, "value2":8050508010125067469, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_6":"a","key_0":"e", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.743507, "value1":0.5292956990399417, "value2":5743834222350004253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_9":"i","key_0":"k","key_2":"k", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.024743, "value1":0.5111340297086123, "value2":4739422872896352154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053127, "value1":0.41791773967804086, "value2":5388222563610877961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.489667, "value1":0.6344084078946665, "value2":6127655871996983541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.876031, "value1":0.49328901667737785, "value2":3143559155074364531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.278465, "value1":0.23241882776321474, "value2":805438734033729133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.380252, "value1":0.6353489769585325, "value2":1619270428062881618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512159, "value1":0.8152126527698539, "value2":1368040827327524990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.317649, "value1":0.9550761539015656, "value2":8914758126172814906, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_6":"d","key_7":"k","key_2":"f", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.723807, "value1":0.7854700255052053, "value2":7095402949367847209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.660856, "value1":0.4053530100702817, "value2":2145190666873464855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.659680, "value1":0.12646331821035406, "value2":1576238191679234703, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.993554, "value1":0.8499325393155558, "value2":5900620451042065379, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091338, "value1":0.41503145937749936, "value2":2042239402577329647, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.299870, "value1":0.687218145400586, "value2":7560464390267142060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.731227, "value1":0.15009967640134209, "value2":1886018024243215805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.348963, "value1":0.6737669308977404, "value2":5404336119559355867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162168, "value1":0.813527799284827, "value2":4016392262793002099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086162, "value1":0.7399808700194177, "value2":8264192680187392458, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.975595, "value1":0.11433087569708594, "value2":4671281305443756113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.799363, "value1":0.3067546199025452, "value2":225546722481809384, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.113667, "value1":0.6424657658728049, "value2":8718201548571703569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.904642, "value1":0.5465983180036486, "value2":4877073522652743329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.467203, "value1":0.8662610389904551, "value2":399387690820391383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082862, "value1":0.5088605240671744, "value2":1531739105872671512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.039708, "value1":0.10189606169579954, "value2":5185895909677369244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.068495, "value1":0.2805843145287995, "value2":486469505433183294, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.357082, "value1":0.75740796150096, "value2":2690075476895328471, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492715, "value1":0.3004718942084548, "value2":7383744941741065497, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_4":"g","key_0":"k", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.968147, "value1":0.5004304919533921, "value2":1731582924085558465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.991776, "value1":0.7394075904771168, "value2":4141740040514370033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_4":"k","key_6":"a","key_3":"i", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.420254, "value1":0.45767543208547934, "value2":1964890506496503777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.285702, "value1":0.1366965878727093, "value2":2681549871870198885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.492204, "value1":0.9414045318691766, "value2":1575036830396652846, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.086526, "value1":0.5271825907095232, "value2":3742001028067515529, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_9":"k","key_1":"k","key_4":"c", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.296194, "value1":0.6916437451586811, "value2":384054583246095269, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.478834, "value1":0.5030459095566234, "value2":3918028956922111738, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.029769, "value1":0.2164261260808611, "value2":4102211333190644591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.102553, "value1":0.42618668443550417, "value2":7568518633836620405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.681434, "value1":0.7225843033115362, "value2":8160060706717748819, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_9":"b","key_5":"j","key_6":"h", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.210135, "value1":0.21812369739137172, "value2":5702739804858144278, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.786060, "value1":0.33904050622912296, "value2":1017560684015420231, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558541, "value1":0.14437656270299287, "value2":7701298697010262063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_7":"g","key_3":"c","key_6":"e", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.818721, "value1":0.15017675533914024, "value2":7186230665665602451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_3":"c","key_2":"a", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.910629, "value1":0.09672885609565206, "value2":6203887741849145968, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_5":"k","key_0":"b","key_1":"h", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.069131, "value1":0.6075489323724108, "value2":9025102416378601029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_8":"h","key_1":"j","key_7":"g", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.362866, "value1":0.8018878313034695, "value2":8037732655719146713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.682018, "value1":0.5433657256031907, "value2":6040793575397278438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.797323, "value1":0.5032038038329079, "value2":3803498450979968327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.829279, "value1":0.605237212493161, "value2":4367127372886022859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.999162, "value1":0.5082854850214861, "value2":5031512432939201812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.269625, "value1":0.0007161270371379666, "value2":6545591527308584857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180859, "value1":0.6040920298819816, "value2":8926725137631299725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.090602, "value1":0.5326464117623615, "value2":7752649715252147157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.191121, "value1":0.12836456615806674, "value2":7198874487475192980, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.686863, "value1":0.708342981200594, "value2":5465607373577957408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.527995, "value1":0.10447991595171255, "value2":5974156320762011232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_0":"c","key_3":"d","key_8":"b", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.123930, "value1":0.6200031138885586, "value2":5406077264893957847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.356501, "value1":0.44396993176396354, "value2":6255319076363219685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.048169, "value1":0.08153644488046811, "value2":8582968949378456970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.112743, "value1":0.6933778275610673, "value2":4587548588265206284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.323148, "value1":0.20142008973704545, "value2":4527540178335323769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.934744, "value1":0.08093337102458979, "value2":7857820814429250360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_7":"i","key_4":"d", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.962496, "value1":0.6124995000870105, "value2":6489203228482067372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.160517, "value1":0.5248710269553188, "value2":5026165991297404062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932200, "value1":0.922378624964946, "value2":7930339398930216623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.900245, "value1":0.1182228551739615, "value2":1373671224306855417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_9":"d","key_1":"j","key_8":"c", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.790920, "value1":0.31880287032595506, "value2":6084131296995295782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.237291, "value1":0.13557228914344616, "value2":4432801334887160283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.914465, "value1":0.02040606137133301, "value2":1808915457173232015, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.941296, "value1":0.9439262294487866, "value2":6574505025862591303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.822459, "value1":0.9942489150539108, "value2":3506354754330173943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.239733, "value1":0.4989511485969244, "value2":7173139334014716167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.444066, "value1":0.7594335589002584, "value2":3216271266780228569, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866271, "value1":0.5931978644398052, "value2":3444180839017636171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_7":"a","key_9":"j","key_1":"a", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.421087, "value1":0.6530151828739431, "value2":19805620992839207, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.771459, "value1":0.8073198535663951, "value2":4810887667402700825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.832482, "value1":0.734748250215754, "value2":1123934068717543450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.450110, "value1":0.15409548849312346, "value2":418445000125934927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.234597, "value1":0.3684594320773471, "value2":3167333011558865991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_6":"k","key_9":"g","key_0":"g", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.403637, "value1":0.891873955682516, "value2":6523321617886226804, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.017496, "value1":0.5380231504237305, "value2":1565703384693902706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.577818, "value1":0.942234653476486, "value2":6854337928979531257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_9":"e","key_8":"g", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.886845, "value1":0.561600684880549, "value2":6700870407358100744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.493299, "value1":0.1842060640300563, "value2":7660717134967306529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_8":"j","key_5":"i","key_6":"b", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.043840, "value1":0.2702623264568812, "value2":7040356564551372493, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.051963, "value1":0.5871637139837996, "value2":4534754735986232109, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_4":"c","key_9":"i","key_1":"d", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.568830, "value1":0.08173891497184316, "value2":408699021018747548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.694578, "value1":0.8818619724910839, "value2":5720963057839829442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.095000, "value1":0.9081401378248263, "value2":8656658880665915661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.977416, "value1":0.9899458292085819, "value2":665964700424941220, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.670945, "value1":0.6733933124715158, "value2":2460152217631000428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.636539, "value1":0.6931186653224365, "value2":7451786524160575556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.398486, "value1":0.8808674761773307, "value2":6001178466503449076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.085292, "value1":0.9258669929119259, "value2":4480969611449139348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.451321, "value1":0.8902429562501767, "value2":2399440975743970404, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.100110, "value1":0.4539977466656097, "value2":4906418767346623646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.725438, "value1":0.8779089763048636, "value2":7986118613837183355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.559619, "value1":0.42482197030409785, "value2":921145061767776729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.885834, "value1":0.32962036175845005, "value2":5608652380611657042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.619049, "value1":0.4697848554943135, "value2":6930715352479440739, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.768456, "value1":0.55002846829011, "value2":6856343919804361537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.052488, "value1":0.9193313692023829, "value2":8075338150518594907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.346748, "value1":0.7969202775153313, "value2":3854682510572365821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.668529, "value1":0.8901520814855493, "value2":3780299539456604836, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.680780, "value1":0.9128377273315654, "value2":655380872616480476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.298499, "value1":0.8901980009691262, "value2":4124662751920860689, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.061822, "value1":0.922218588720657, "value2":6935461410888938736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.220215, "value1":0.8707922044422206, "value2":6415853705476229931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.825872, "value1":0.6145357419122679, "value2":3712577567679334743, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.287572, "value1":0.7153750220855114, "value2":6766565708182723447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.258723, "value1":0.8623529521546892, "value2":3906675258214843555, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.417686, "value1":0.6071420867010977, "value2":8210404273898330224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.939674, "value1":0.018125841559591738, "value2":2686797765813242922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.062829, "value1":0.19220806032635843, "value2":4009115073519468001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.761409, "value1":0.282044445283748, "value2":2659364730071661456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.340677, "value1":0.7867304449409817, "value2":4726626950116268051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.848210, "value1":0.7451256645145572, "value2":6540697575244319830, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_7":"b","key_2":"c","key_6":"d", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.082209, "value1":0.08477415388397415, "value2":5473839595744351618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562389, "value1":0.08808192976810916, "value2":1578232710018420800, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.445878, "value1":0.7573342205320885, "value2":3400644244684606362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.603775, "value1":0.183625034999939, "value2":7793796792648526034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.036048, "value1":0.8359424936170327, "value2":5865912253243965224, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.866722, "value1":0.3949434009829113, "value2":3290170127727456353, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.697995, "value1":0.8423729542207233, "value2":4861550929219709907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.211200, "value1":0.36512924472577757, "value2":2972205895387325421, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227557, "value1":0.22341834017020207, "value2":4806018666421767946, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.574952, "value1":0.6007129713703638, "value2":3849253119017333239, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.741721, "value1":0.5702349573898813, "value2":897876839681174700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.379915, "value1":0.6564162460066056, "value2":3131015984288737246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.229369, "value1":0.456646560158155, "value2":4778476677749084799, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.242697, "value1":0.4142421381714439, "value2":7101584169529051655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.212755, "value1":0.39066964505760526, "value2":4253776777664568253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_7":"h","key_9":"k","key_5":"b", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.837272, "value1":0.15824518985564762, "value2":6812701504471384455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.104360, "value1":0.9940983043766688, "value2":2203777894935310413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_4":"j","key_6":"d","key_2":"g", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.391080, "value1":0.5579038729371453, "value2":6131596744931664143, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.265096, "value1":0.7511430046090084, "value2":4283281105409640096, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.881500, "value1":0.8197410181139487, "value2":2167116001911138380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964911, "value1":0.6839484284574476, "value2":2392821775915043629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_4":"f","key_9":"c","key_0":"j", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.906639, "value1":0.12232760875324074, "value2":9162174110871211714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_8":"h","key_9":"i","key_4":"i", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.562264, "value1":0.9910780181559037, "value2":1543791401715243110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.016345, "value1":0.3277590263870748, "value2":3851358761838506924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.963516, "value1":0.13686033235694836, "value2":2786944428195217311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.446494, "value1":0.051498079678560585, "value2":1973784887996536653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350210, "value1":0.13860925037657423, "value2":1354426546825223222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.358378, "value1":0.28813722807183184, "value2":4138306657967174744, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.455527, "value1":0.9787455514136848, "value2":6829598111029855692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_2":"a","key_7":"e","key_1":"c", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.208210, "value1":0.4291478612877578, "value2":749843861604187954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.330867, "value1":0.6408478280255169, "value2":8395361043273182125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.176999, "value1":0.9117058417477248, "value2":8428888804090519109, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_2":"h","key_7":"f","key_1":"d", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.091618, "value1":0.8122911740955318, "value2":1478086552909570392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.243584, "value1":0.06727976096288107, "value2":4178958373440436949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.127002, "value1":0.8012772426876036, "value2":2149729620077994596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_5":"j","key_6":"e","key_3":"h", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.461395, "value1":0.21653382210662625, "value2":2558734709934645897, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.932379, "value1":0.43744123497894677, "value2":5779172669858974850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.180562, "value1":0.6091002318007619, "value2":2233652952119987641, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_3":"h","key_9":"c","key_0":"e", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.738334, "value1":0.8147115963605253, "value2":1038206878786066414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.943138, "value1":0.1805265868912326, "value2":8231024461542686979, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_7":"g","key_0":"j", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.007238, "value1":0.30166077327328183, "value2":5211541986455490237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.926840, "value1":0.7197646392974351, "value2":6170887305282807365, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.976392, "value1":0.02561261869316712, "value2":8003552327575750503, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.665296, "value1":0.38463860662667215, "value2":702575998172013860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.957627, "value1":0.8365108478779814, "value2":3172439569974822855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.779959, "value1":0.005457784803842251, "value2":1217373846044030405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.814440, "value1":0.03586677782979768, "value2":8668868565483927092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_9":"g","key_0":"k","key_1":"f", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.590538, "value1":0.7998407978822464, "value2":5953384113011158992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.381146, "value1":0.6861901618066985, "value2":806723358919392729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.324953, "value1":0.09051509245266014, "value2":930835793453842933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.309761, "value1":0.44326721766264904, "value2":887788347593465760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.722069, "value1":0.08849038862609183, "value2":7352758650578649855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.558827, "value1":0.3337734956738152, "value2":8685182570627965041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.715649, "value1":0.23109099835442082, "value2":1083379478056182665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_9":"h","key_7":"b","key_8":"b", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.964540, "value1":0.6330159342037405, "value2":1167652807667781397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.795126, "value1":0.5778697926785736, "value2":9140994998625427103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.236481, "value1":0.5802899488963301, "value2":2895083535971839167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_1":"e","key_5":"b","key_7":"b", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.996259, "value1":0.2785697344481934, "value2":4913023416457428502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.477938, "value1":0.484568023326054, "value2":3495500670092878690, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.261065, "value1":0.11116858735231369, "value2":5869730879322934574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_2":"f","key_1":"a", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.185846, "value1":0.7126003929744608, "value2":3434450920636329132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.841676, "value1":0.3736127841617722, "value2":7802733756976416976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.350997, "value1":0.582318689502649, "value2":372190874054145543, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.555394, "value1":0.12831996488184216, "value2":4582628444562419674, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.155140, "value1":0.3906336986690914, "value2":8237405956495459279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.367910, "value1":0.741845875474543, "value2":8082126523839106878, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.950768, "value1":0.6652471104213182, "value2":8268690822588065094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.433269, "value1":0.7980991574588373, "value2":8513183309513339766, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.874369, "value1":0.5464494520072651, "value2":5885072115794893058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.537599, "value1":0.7405427933790399, "value2":6635799236257663587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_2":"a","key_7":"g","key_1":"k", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.955763, "value1":0.8778283555530513, "value2":6592488434360400176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.582015, "value1":0.879034635958457, "value2":7219576376735925925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.388984, "value1":0.53843340060884, "value2":2254745170299712210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.646131, "value1":0.5101339977981793, "value2":3292039546456501555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.033442, "value1":0.6456192738711094, "value2":4906576027749942202, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_9":"d","key_0":"j", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.836645, "value1":0.48635849410277077, "value2":1404791026873124424, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.464295, "value1":0.5617144285499786, "value2":7990958033367681704, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.010923, "value1":0.9807071857311617, "value2":5683359887238098850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.438005, "value1":0.5935141575872156, "value2":4865067817966075444, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227310, "value1":0.6681961275740043, "value2":8551103550465421536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.186394, "value1":0.25048622317521113, "value2":2888181774636358576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.053227, "value1":0.10049698552895067, "value2":4557545020451680363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.621258, "value1":0.9388516977567649, "value2":4055003743201350923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.227442, "value1":0.7022685194913532, "value2":3740668486633940756, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.640300, "value1":0.06434066744840804, "value2":1646848528935407928, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_5":"b","key_9":"h","key_3":"c", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.162609, "value1":0.71410216780028, "value2":7370378990581242781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.512792, "value1":0.19345496541059937, "value2":4424425935560475985, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.203787, "value1":0.9476247108925144, "value2":264480497401916863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.076408, "value1":0.13927961349104323, "value2":1149972870879073163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.806676, "value1":0.6172364818942355, "value2":4996985075367830737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.313484, "value1":0.6715999817727264, "value2":5237271908928184120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:06+08:00", "value":0.947617, "value1":0.1400224644800571, "value2":4212016629330280660, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.695969, "value1":0.13461968177274677, "value2":8209146459721882476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466156, "value1":0.7851823238051198, "value2":4609596535178738402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047192, "value1":0.6615735622970099, "value2":3688971736191236071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179244, "value1":0.5525116533205023, "value2":6958696123355987767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.836535, "value1":0.8752088579621973, "value2":1772355477524999885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_8":"k","key_5":"f","key_7":"a", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.385661, "value1":0.6669392703121872, "value2":5026611864593335422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.122798, "value1":0.5624470316374681, "value2":973613507611825241, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663323, "value1":0.2836760055083927, "value2":8274369123408107345, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988003, "value1":0.07587743066420682, "value2":4947294410032829574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.240844, "value1":0.4161174869019418, "value2":4077015391953706312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.945230, "value1":0.14944194749229664, "value2":7660491006096000021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890111, "value1":0.20296484407777726, "value2":1824051875710962915, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522740, "value1":0.021855326277136235, "value2":3218235817889269142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_3":"c","key_1":"j", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.679843, "value1":0.5055217891288131, "value2":4822706751533920431, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191340, "value1":0.9959364873278129, "value2":3932069232230299078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771462, "value1":0.4557053136591391, "value2":184726250007256859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581655, "value1":0.689124829253459, "value2":1817463261736430161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.894089, "value1":0.9069576891707435, "value2":7318133204837024663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.307941, "value1":0.4227283465946924, "value2":356860997794924081, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085219, "value1":0.7548908197263309, "value2":2736908468355530081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510864, "value1":0.6045243894505045, "value2":3203823484629938988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194038, "value1":0.2945791412290687, "value2":8542013145729544847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.822784, "value1":0.7046664475635507, "value2":6531211714773873879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958034, "value1":0.6210073688384752, "value2":5171125204919187937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245514, "value1":0.07787807249020082, "value2":4796876093300411441, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922409, "value1":0.026305095785565266, "value2":4204774996273721582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618949, "value1":0.8021374519845728, "value2":7055530836155104119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_2":"c","key_7":"d","key_1":"f", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.842155, "value1":0.9078167657864898, "value2":659406508816846003, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.429827, "value1":0.6861191669827441, "value2":3716781946259346478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_5":"e","key_6":"j","key_3":"c", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.721875, "value1":0.7721427864001064, "value2":1559707715553693565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.667085, "value1":0.41573209008012846, "value2":770484632734171477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567897, "value1":0.2688265887173671, "value2":8731672977339492909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716882, "value1":0.20533155732399708, "value2":8158121924487423466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.450278, "value1":0.45425248453866135, "value2":7472174068145252016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700766, "value1":0.38900719316872584, "value2":9070338695929499459, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_5":"e","key_0":"c","key_4":"f", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296131, "value1":0.7588298197955197, "value2":1078200536047373074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_1":"c","key_4":"d","key_0":"j", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644620, "value1":0.22063287558750297, "value2":1455208095653529502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826663, "value1":0.9059127191998474, "value2":163331002989197798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.375920, "value1":0.888489428778294, "value2":3478637008435710026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795211, "value1":0.7169431218180555, "value2":5240161460250645540, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686886, "value1":0.1615738416501897, "value2":4522663398730909152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.641311, "value1":0.5919216049468233, "value2":4264524771451405531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_8":"i","key_4":"j", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415500, "value1":0.8013193758641248, "value2":4655946330615841665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310579, "value1":0.03464731335525151, "value2":2648033740620709930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.135452, "value1":0.663217728481436, "value2":6656249157262608643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036458, "value1":0.7592021704654797, "value2":4708550885998497867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_7":"j","key_3":"k","key_6":"j", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196106, "value1":0.9109556832409251, "value2":1182178953302502536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897060, "value1":0.30582754840738535, "value2":5110613128180106106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_7":"h","key_9":"d", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.386690, "value1":0.17482144522900334, "value2":208181032234291542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770888, "value1":0.5775727956988672, "value2":7308314003124624706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710417, "value1":0.9182102368115501, "value2":8001324021984156096, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_6":"c","key_7":"j","key_0":"d", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.133639, "value1":0.8132459830268748, "value2":6362172937464044603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_8":"j","key_3":"b", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313132, "value1":0.6425828291871425, "value2":3812023207458774027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816183, "value1":0.12876194798023052, "value2":353578462778793882, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.537076, "value1":0.3677093772206428, "value2":3410314947529692873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_6":"f","key_8":"f","key_1":"c", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.785801, "value1":0.5650444568188182, "value2":2518267372616726657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793640, "value1":0.36840754337253134, "value2":6490561754060041624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277029, "value1":0.5742325627742796, "value2":4576556191237422408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_5":"b","key_8":"g","key_9":"k", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086053, "value1":0.635715694469195, "value2":3276323627400814300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622335, "value1":0.6664883465636151, "value2":6846298203429885729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_9":"c","key_6":"b","key_7":"e", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784935, "value1":0.8345083778637773, "value2":2989860444276517490, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394440, "value1":0.3881444024090245, "value2":211938452740442471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.150923, "value1":0.2049623118713732, "value2":7830583450265904275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334186, "value1":0.8877162218809062, "value2":5530254516648658024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919131, "value1":0.645089444938841, "value2":2752404297334193415, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.550415, "value1":0.8672407553696915, "value2":7296354502501329771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.251810, "value1":0.12582294112213796, "value2":3850461942759838134, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.768322, "value1":0.5089660327127362, "value2":4637000702345102317, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.5790505163445078, "value2":2746113057580625956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310041, "value1":0.7419516660136578, "value2":8573283812679768621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567732, "value1":0.20310332375311568, "value2":1684484347666739261, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.149277, "value1":0.8537753954365711, "value2":3792414057901721222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019813, "value1":0.3193924681887993, "value2":4416687297621574996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_3":"g","key_5":"b","key_0":"f", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555536, "value1":0.2678716255116121, "value2":1978875629658477083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.676000, "value1":0.268194488154906, "value2":9208242650214533822, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603232, "value1":0.7475788122275303, "value2":6003796300446172845, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272526, "value1":0.7633068823447601, "value2":5387009257178750855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288629, "value1":0.0999015536587548, "value2":5230082044659723490, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198708, "value1":0.5411261720205245, "value2":1324824529248190153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754319, "value1":0.5181119376899533, "value2":7586548988147520888, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_6":"c","key_0":"d","key_3":"d", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.974535, "value1":0.6605961675334583, "value2":4149492144268522705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620393, "value1":0.0400929610839578, "value2":2611761419308325853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489379, "value1":0.20432994152077563, "value2":4859071163254701004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.497240, "value1":0.21587046902788842, "value2":1674028941320053082, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227611, "value1":0.31207060679536985, "value2":3782150896328417443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.813479, "value1":0.9346689305684607, "value2":4170814668135359561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832568, "value1":0.40374034609384823, "value2":4890680963042405726, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.868761, "value1":0.7456764740945415, "value2":7985095078472385264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_5":"k","key_2":"f","key_3":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566095, "value1":0.11155632539641938, "value2":1650668369012843098, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516140, "value1":0.2921171335805605, "value2":699561345288206492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.009998, "value1":0.0698415110112407, "value2":2175710360877139700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472788, "value1":0.6894980351369026, "value2":5185256568181045963, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013027, "value1":0.47916067130802836, "value2":8339740169449446777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947875, "value1":0.15641692100908322, "value2":1874863435098609183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834625, "value1":0.4725529822941945, "value2":3313442976129641776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032027, "value1":0.11526706346113103, "value2":4474958127457336496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.643426, "value1":0.2365209546681749, "value2":2452872778362311452, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399009, "value1":0.10587590188915917, "value2":5343568589706680846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_2":"j","key_6":"c","key_1":"a", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464026, "value1":0.23089518212237822, "value2":6365432144590215713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.573860, "value1":0.41263343273878106, "value2":7100420790565954895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575295, "value1":0.15635829479176244, "value2":972701597090717176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_6":"f","key_8":"j","key_5":"b", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685505, "value1":0.3500577640159365, "value2":5296267831822852754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_3":"f","key_1":"e","key_2":"a", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382859, "value1":0.8962437052233099, "value2":5000561412494049933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.793198, "value1":0.03594880833972008, "value2":5558834996866286026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.296341, "value1":0.6801030998229524, "value2":9104706563374398078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.044529, "value1":0.43868239396683095, "value2":1119681077532090572, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977458, "value1":0.7636375641015349, "value2":3738349085487316798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_1":"c","key_3":"j","key_0":"j", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.769690, "value1":0.38941161495083887, "value2":4119290182206919942, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604117, "value1":0.06703731437988915, "value2":2776279989058320233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.107061, "value1":0.9783306976372779, "value2":6724603539014036116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.243947, "value1":0.3058624284281911, "value2":8607064800338196111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784300, "value1":0.47413357934186534, "value2":6062775812546347462, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053929, "value1":0.8033894774680731, "value2":3798712131176287721, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997633, "value1":0.3495480156862711, "value2":6358034341550557079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.399888, "value1":0.5577617871085867, "value2":7606548109917229902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_4":"k","key_2":"k","key_3":"i", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.343972, "value1":0.37033984748970067, "value2":3823813661350967339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741859, "value1":0.15725835002679114, "value2":7931796757818180793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118790, "value1":0.8710092300018802, "value2":8432660813300591545, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_6":"k","key_8":"f","key_5":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.514050, "value1":0.3589657867466283, "value2":2807064518724813388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105213, "value1":0.48313119234822866, "value2":9012983220702483909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.289587, "value1":0.28297188108556603, "value2":8293007746329622374, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.682523, "value1":0.2601723288341452, "value2":5132563496112960923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.521609, "value1":0.17923659255236565, "value2":9080059564709771845, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.866470, "value1":0.8472875420381546, "value2":7662519070278706881, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139688, "value1":0.701824998288288, "value2":8626519461512263205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_4":"g","key_5":"k","key_2":"g", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693655, "value1":0.3931946923075373, "value2":4088184802361327943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.733184, "value1":0.33562267701400733, "value2":5057125078949459009, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530556, "value1":0.8546701921585885, "value2":7172802034424433024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_2":"b","key_0":"h","key_1":"g", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890826, "value1":0.9156630138494082, "value2":1106383994017980379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736946, "value1":0.28531565085928856, "value2":7492657224478970683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184271, "value1":0.66321078827276, "value2":6482907214355252145, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.771765, "value1":0.8506763394943703, "value2":2997501176379232751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621435, "value1":0.4219907917118007, "value2":2371448704479482567, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522551, "value1":0.4559374479237847, "value2":8705737330976472895, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579601, "value1":0.3374291561460269, "value2":6118047213487455086, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.911838, "value1":0.24595613407282738, "value2":8543292459890755038, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.461163, "value1":0.20032563894255778, "value2":8271655059958021803, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_4":"k","key_0":"c","key_1":"g", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480016, "value1":0.537301255549281, "value2":131039455325957900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264510, "value1":0.8207403430077567, "value2":448751349067313394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.908904, "value1":0.45150207048162533, "value2":1484572129461087565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279527, "value1":0.48678886025337537, "value2":4675520724179222107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.983786, "value1":0.5074627713755523, "value2":6761505393797376999, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407769, "value1":0.8974462165990488, "value2":4427875231986566903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_4":"e","key_7":"e","key_3":"c", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906297, "value1":0.32841731679412833, "value2":4853751460801403413, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.910866, "value1":0.22248621009204866, "value2":6153217292256934376, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481193, "value1":0.07256346779714405, "value2":8357563464914791440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819751, "value1":0.7821155606881222, "value2":6350148722815418199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925016, "value1":0.08017277463129999, "value2":8127038680866850197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_4":"b","key_9":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380866, "value1":0.4236203787530011, "value2":6102770882422449664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.687520, "value1":0.18833602677282602, "value2":2416669714696967955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_7":"j","key_3":"j","key_4":"i", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392864, "value1":0.12162108913858492, "value2":1037740633966701101, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555618, "value1":0.5726263308688928, "value2":5340970336126619965, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041150, "value1":0.1866697214008993, "value2":6089316958837860364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.848709, "value1":0.5912728689790879, "value2":8374928084508664533, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.633435, "value1":0.34567925121735993, "value2":6074382604796755677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615342, "value1":0.43884173283368255, "value2":2552875653507489786, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.420473, "value1":0.024593195105800023, "value2":6998883295968298787, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.493473, "value1":0.517385578986543, "value2":7898967292563966468, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398409, "value1":0.9494831815326669, "value2":1043777551847385399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_0":"f","key_5":"i","key_6":"i", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.261653, "value1":0.605095923821751, "value2":5798307460445919671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734130, "value1":0.17339556093076888, "value2":3888875155355529135, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.234024, "value1":0.07147953681129654, "value2":2220989894337620932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.552276, "value1":0.46428310479897317, "value2":5429095800661644988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.547014, "value1":0.9160534111766008, "value2":4258800689197649539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084253, "value1":0.06651726425775369, "value2":8202506388672279723, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137868, "value1":0.6230337836484641, "value2":8788058834810483014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.168246, "value1":0.1571304566786965, "value2":2933470821094626337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043250, "value1":0.3445304512204906, "value2":8908780418296865224, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542269, "value1":0.007638770771913262, "value2":975662304048103149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016306, "value1":0.44304617075801217, "value2":4700290207925657878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967184, "value1":0.9943193022594318, "value2":7141028805491571993, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_9":"d","key_5":"i","key_8":"j", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522277, "value1":0.8578463560518583, "value2":6372322792910116627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_7":"a","key_3":"b","key_6":"d", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541009, "value1":0.5200416543540609, "value2":5687465452183940942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_3":"c","key_1":"h", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742322, "value1":0.4918651213341637, "value2":7530197332852248825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.478705, "value1":0.02234257118225825, "value2":381364684161431163, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.401319, "value1":0.36716744081601327, "value2":3165286506094547515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_1":"d","key_5":"g","key_6":"c", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.154587, "value1":0.8570612240935895, "value2":440607090836829314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268457, "value1":0.19708890772544582, "value2":716490672588175731, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_1":"k","key_4":"g","key_7":"h", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976461, "value1":0.37921805973698547, "value2":5187015251060220885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.291361, "value1":0.0347133343718353, "value2":4081610772727292868, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.831910, "value1":0.20329085836851984, "value2":1212790782212732680, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_3":"j","key_9":"k","key_2":"g", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045270, "value1":0.742873126094037, "value2":1857680286082818546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719581, "value1":0.8427487279865438, "value2":1584262554399141329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.878973, "value1":0.7217975560465744, "value2":6936890459625848218, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_8":"f","key_0":"f","key_6":"a", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.786254, "value1":0.7388730574867226, "value2":3591226182562159088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719189, "value1":0.6138101273145392, "value2":4767024689432495577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335758, "value1":0.6941935445321573, "value2":1310712364678635982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333397, "value1":0.7084933562313812, "value2":2679977702860384198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.816868, "value1":0.5704940444863804, "value2":3982836666436033897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.806625, "value1":0.7550148139027509, "value2":2819861429300922010, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155981, "value1":0.3978060292285169, "value2":2375324886880679840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.172764, "value1":0.8807292451628046, "value2":1585412394695366401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.127700, "value1":0.8691901922846396, "value2":4006452105841009084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_1":"c","key_7":"h", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.937137, "value1":0.6079865671377581, "value2":9014517461395944209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601939, "value1":0.5587486969852118, "value2":896670201927694097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147910, "value1":0.7860716932621791, "value2":8828747393971238771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271640, "value1":0.8634829980251869, "value2":7534202338930685599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_7":"c","key_8":"d","key_4":"i", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510609, "value1":0.273735584071857, "value2":7995135221543970576, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_4":"c","key_5":"h","key_6":"k", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.424702, "value1":0.6570665232819806, "value2":7130224512963405605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.120070, "value1":0.4366525687584962, "value2":8321790572084844233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_7":"i","key_9":"g","key_4":"e", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.355286, "value1":0.9376756075719435, "value2":7612697762460955574, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_2":"a","key_3":"h","key_0":"d", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057269, "value1":0.9603884122588556, "value2":5675558649074747456, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_9":"b","key_3":"a","key_7":"f", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.396116, "value1":0.3241173308406254, "value2":2149479689060994101, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.271843, "value1":0.5095285278231975, "value2":5564561021464735924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389082, "value1":0.7189491284948987, "value2":8539354123799965742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_8":"k","key_7":"h", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780310, "value1":0.7959419420056967, "value2":791327040768245285, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496100, "value1":0.7137676161743793, "value2":7249423453544847575, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.320840, "value1":0.7088322490051288, "value2":2259284238716415015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.588238, "value1":0.2616266339940352, "value2":7616111814012126210, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.510231, "value1":0.34548313055527086, "value2":9146415948654460714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725142, "value1":0.832268560021817, "value2":8449163586612650016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722343, "value1":0.5127452250176201, "value2":8372677466999750797, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455239, "value1":0.7274771017691084, "value2":8558710521155328448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070961, "value1":0.5108403289338641, "value2":6822737602377250548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.799042, "value1":0.6579145757389928, "value2":6367849243718047663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.997752, "value1":0.29826795257630173, "value2":7963803897292403489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.019959, "value1":0.5329624946388745, "value2":6166072735161874859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183220, "value1":0.1192005017394279, "value2":1414525889342507585, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.216504, "value1":0.10069780302425961, "value2":4147229267124117622, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184537, "value1":0.9674252449470503, "value2":7196918589322334021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.982196, "value1":0.5121645885657403, "value2":1331574421408029626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.807710, "value1":0.6453473320699041, "value2":5880434997959490048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_8":"h","key_7":"d", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.139645, "value1":0.45151332154483154, "value2":5206983362834731524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202627, "value1":0.5085399708189182, "value2":229347926887383465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703675, "value1":0.1410911985436997, "value2":1995545208705825638, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.916538, "value1":0.9588604837943727, "value2":3014447442999476863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174096, "value1":0.8442896590468587, "value2":1669527921851537286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.268618, "value1":0.05460739672623512, "value2":6145595254763691216, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.184210, "value1":0.08111723273128417, "value2":2941741438359406510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_5":"h","key_7":"f","key_0":"d", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419569, "value1":0.11197378055505244, "value2":7190302664501414682, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.080457, "value1":0.46189706497698463, "value2":3739595486194011632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_4":"c","key_7":"e","key_9":"d", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685402, "value1":0.5911751311740201, "value2":375795342652216999, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584152, "value1":0.842391293090085, "value2":3305999218832830126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_9":"d","key_4":"e","key_8":"a", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.719567, "value1":0.7117389665869482, "value2":4436375112558311543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.599224, "value1":0.7507823724841362, "value2":5289861475224212810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298087, "value1":0.03464563720261393, "value2":5487979858306400729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.084762, "value1":0.4408096997046924, "value2":8782838705036394284, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153226, "value1":0.8396685378218853, "value2":2305238685769866566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875795, "value1":0.6283427563062458, "value2":1424824369894963651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.841870, "value1":0.2951891910717211, "value2":1229301210242052079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_9":"c","key_0":"f","key_2":"i", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036891, "value1":0.10272064811801765, "value2":8629352909535519019, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.475886, "value1":0.47884350384173996, "value2":2002152937943396093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_8":"j","key_9":"j","key_0":"g", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286578, "value1":0.5511122934004854, "value2":6456656081230025536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180709, "value1":0.591154696871619, "value2":1085648256858149575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_3":"g","key_5":"g","key_6":"a", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.549035, "value1":0.9073632737596287, "value2":441521694946529713, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.644644, "value1":0.42900060724414874, "value2":7252379057595536961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.185531, "value1":0.28679213225919725, "value2":3666625025829964686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858171, "value1":0.07899073830408582, "value2":8847544646507443860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.9086649445849005, "value2":7300771702176978507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.217129, "value1":0.8286760370555245, "value2":437314161092307860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_7":"d","key_8":"i","key_3":"h", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.092457, "value1":0.3257453112147972, "value2":1241192378749271122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_4":"a","key_2":"e", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.318459, "value1":0.6218522470963161, "value2":2601033448014127071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_1":"a","key_4":"j","key_6":"i", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543632, "value1":0.47194572326020234, "value2":330858668685674119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.494889, "value1":0.42021023869128465, "value2":7698787812508472546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173188, "value1":0.6333586665476976, "value2":1031119021359267335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392225, "value1":0.13669516747392368, "value2":2722787133031533732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.418321, "value1":0.8852713566769926, "value2":1145580014369872247, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499498, "value1":0.37652715283253, "value2":6023088039348052644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049271, "value1":0.7428393913560761, "value2":4074778390717770530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344096, "value1":0.618674195595606, "value2":8432036346015765748, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.504726, "value1":0.8381168181634259, "value2":7490362504268179930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130009, "value1":0.04787900547347569, "value2":8930977988943996511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517353, "value1":0.9123390757414803, "value2":5406757776090744464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.159780, "value1":0.2711186767871672, "value2":6650537343113966916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.601454, "value1":0.036596921749510625, "value2":1801819188066798655, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740877, "value1":0.9080583731663923, "value2":6623419932438799475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.743094, "value1":0.9900165531318826, "value2":383393257357043043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.939900, "value1":0.8769075090910828, "value2":487693007619851718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.102543, "value1":0.4399163118191967, "value2":5888144134235313914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917706, "value1":0.6339047693941521, "value2":6760479052672151531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108610, "value1":0.06502295112846754, "value2":8902773220268340372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.756394, "value1":0.7104082109611765, "value2":3854497302576556911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914430, "value1":0.39800381262255646, "value2":1413572596816022205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_6":"a","key_7":"i","key_8":"b", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.046273, "value1":0.9197740465700138, "value2":5994606503835738604, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.042126, "value1":0.35565383574544296, "value2":206374188969693152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.753926, "value1":0.14736292648599036, "value2":5720098218146494963, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138925, "value1":0.14592833793919333, "value2":5845534364661717380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_7":"e","key_0":"b","key_3":"b", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966713, "value1":0.5691036450918006, "value2":5571037157834825483, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128296, "value1":0.522798351805377, "value2":7845314774564626095, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.374869, "value1":0.16619436230032464, "value2":2683834913042811436, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349899, "value1":0.0806742622823487, "value2":5654997107573600799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.621137, "value1":0.6389883746416457, "value2":6718840923740903865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505989, "value1":0.5736885226925782, "value2":7660274947298694696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245218, "value1":0.06067210631905806, "value2":7481373150924938933, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.961402, "value1":0.40476409107709094, "value2":1656152426641438063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_4":"h","key_6":"d","key_1":"i", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064546, "value1":0.7710994608894077, "value2":5484900997853498427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686047, "value1":0.46617534097616353, "value2":1998746207850635494, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195207, "value1":0.8036664106551048, "value2":21823515460647768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890318, "value1":0.49246668518608533, "value2":4290226524505777397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.662549, "value1":0.0648629447687713, "value2":2600958480076392029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867346, "value1":0.6103284095540984, "value2":959086414558873258, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_6":"f","key_2":"k","key_5":"f", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036038, "value1":0.22211264871806694, "value2":2970074654495356013, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772742, "value1":0.47254264158251036, "value2":5967291460715844391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.765254, "value1":0.59761349012731, "value2":5045361763860538939, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684477, "value1":0.3727521458297325, "value2":7120290254833886255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_3":"i","key_0":"d","key_2":"a", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782498, "value1":0.5592140587339981, "value2":1560278611367944657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.472665, "value1":0.4451202433666956, "value2":7528293771469761549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957028, "value1":0.6722101190310112, "value2":3256843532996983591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_4":"j","key_9":"b","key_3":"g", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.558411, "value1":0.33974258843823785, "value2":9219292136379640073, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.686456, "value1":0.662252940687323, "value2":2872679473486076312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.664702, "value1":0.13252769744708778, "value2":3782836160183678190, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340522, "value1":0.321610730663863, "value2":1993266611063130503, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_7":"i","key_9":"k","key_4":"k", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738508, "value1":0.03455404630849952, "value2":5265512962120119813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722717, "value1":0.039393801449424636, "value2":8411289366884053387, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_4":"a","key_6":"a","key_1":"d", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738985, "value1":0.4374781921483304, "value2":4532229627185520929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.959594, "value1":0.06784833824743756, "value2":2235113542571170507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512001, "value1":0.45518592645372824, "value2":1219663694779720192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699152, "value1":0.9199280757111055, "value2":920010063978439254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870981, "value1":0.7153328294012858, "value2":2241596700674662533, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_4":"f","key_6":"h","key_0":"b", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713651, "value1":0.1871397130196525, "value2":5525165593287246022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608421, "value1":0.2007302921644417, "value2":7082345286303554347, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.898979, "value1":0.4326799832087361, "value2":4510789740667402710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542323, "value1":0.5916017742256197, "value2":7802920355385118767, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245771, "value1":0.3115277557229622, "value2":6710572841494695733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.772755, "value1":0.17023256633885028, "value2":4693930877366777744, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.247931, "value1":0.4740059660910414, "value2":1391972395280563550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394287, "value1":0.02366479635408087, "value2":7477007108410228541, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108172, "value1":0.971283160522281, "value2":6224196629196175206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932465, "value1":0.6431474969443486, "value2":6996302573266425119, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070979, "value1":0.4324217725560448, "value2":9202735673442875059, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_6":"h","key_7":"h","key_8":"d", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862734, "value1":0.5425012681866345, "value2":8705521243115097026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207036, "value1":0.7438270358882513, "value2":1482607831466689197, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_8":"k","key_7":"d", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345448, "value1":0.3875837956213489, "value2":2480267181897079015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553288, "value1":0.3343119844132966, "value2":7890867607759034801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559986, "value1":0.2851751818045875, "value2":3231450807250993029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.008285, "value1":0.11082046459395005, "value2":5670330804396842703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.620145, "value1":0.27733408115115116, "value2":4537754734930200229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_5":"e","key_0":"k", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.861443, "value1":0.7565577041788646, "value2":216193021440411568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.584594, "value1":0.7717911844132257, "value2":7036437829052540741, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_0":"b","key_1":"c","key_4":"b", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.066975, "value1":0.07703464444888349, "value2":8332259918814404840, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_5":"e","key_2":"j", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272133, "value1":0.2064168890828292, "value2":1598424380451243838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.725661, "value1":0.5107106855646196, "value2":2095325894892271000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.992271, "value1":0.04606905650348332, "value2":3458383290691046043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988671, "value1":0.9012802494548691, "value2":3265213363984040214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344354, "value1":0.6315603617727772, "value2":1819854718616345426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198753, "value1":0.2443944990728996, "value2":3740319967339712109, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294599, "value1":0.7030675431685813, "value2":5821390349436487736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.233177, "value1":0.5520329333372563, "value2":2208761416514593046, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.369614, "value1":0.2829108305094591, "value2":2978362585536403835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.484777, "value1":0.3351273859753424, "value2":1348038406704883991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511345, "value1":0.6897076794113861, "value2":3140482719657458044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_3":"j","key_4":"b","key_2":"i", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.780083, "value1":0.056509953898858746, "value2":4983435501891849142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.023345, "value1":0.08633016397064507, "value2":439606973167008387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013043, "value1":0.8750924151560185, "value2":2563435998029443275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929738, "value1":0.6942458871164995, "value2":6742189975038174280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348725, "value1":0.6477096282239411, "value2":614033611645126767, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.148404, "value1":0.3122137082601719, "value2":5921845943066980452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_9":"e","key_2":"d","key_3":"e", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.892407, "value1":0.0024857877019035433, "value2":78557787569325314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.770110, "value1":0.4854945232954409, "value2":4441111957237018619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969650, "value1":0.028054363655575613, "value2":6620021166465815802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096277, "value1":0.7642864471977222, "value2":7079078098912113925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104248, "value1":0.17068081288522707, "value2":2914828831179785715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.033230, "value1":0.6691862134057828, "value2":5879037419816229961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442293, "value1":0.918687680755694, "value2":3522143514446827859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.340520, "value1":0.3316554635970253, "value2":3298849581857165314, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.005701, "value1":0.22902534413133244, "value2":4040792449962896607, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.101103, "value1":0.4459003073823555, "value2":8364403783112020861, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_4":"f","key_1":"g", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.269361, "value1":0.9829136286924077, "value2":305552246857799140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002507, "value1":0.10735284014448321, "value2":2548523454772398525, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516501, "value1":0.3420030652884641, "value2":449036192165412633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.914771, "value1":0.6948526439267081, "value2":3274351745805138467, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151382, "value1":0.9010351248418295, "value2":6225882894853282790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_7":"d","key_2":"c", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.797287, "value1":0.450517653615541, "value2":6406142276072623664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575226, "value1":0.6947969923557978, "value2":4732050542115525893, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.256780, "value1":0.29401472322197164, "value2":8790173514967991405, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_0":"a","key_5":"b","key_9":"a", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791599, "value1":0.6204111480891676, "value2":1162220986630177027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.615289, "value1":0.17893814852804246, "value2":3187373261649831292, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738034, "value1":0.9979754782079555, "value2":6006267840987043849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735591, "value1":0.011768322537597629, "value2":7974963296156276842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.751213, "value1":0.43700619365470833, "value2":6294325004534159296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781793, "value1":0.6112369782572161, "value2":2263959578091534198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118524, "value1":0.4979095180792778, "value2":5142183829715514199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.574787, "value1":0.8076799963798312, "value2":1975720601465651847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814750, "value1":0.45891950977715895, "value2":4074714357288466936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_9":"d","key_2":"c","key_6":"k", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.579803, "value1":0.25446438731696874, "value2":7191932871100542216, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722158, "value1":0.19082711905600802, "value2":1422914021938617784, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342798, "value1":0.5097927561705997, "value2":2281093093995561768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_2":"a","key_9":"f","key_0":"j", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164991, "value1":0.06555881689103157, "value2":7152427537795186892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_0":"b","key_2":"b","key_7":"h", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.316516, "value1":0.1580165967231405, "value2":8769573796829946768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_2":"c","key_4":"g","key_8":"b", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548667, "value1":0.8164945933097875, "value2":1223332369807607717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_8":"c","key_0":"d","key_6":"c", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174435, "value1":0.5629514369055348, "value2":2359802095953741925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_7":"i","key_9":"i","key_6":"g", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.561139, "value1":0.9847446485189013, "value2":8748242508292199474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.604206, "value1":0.7084164989246987, "value2":5662112818525056068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.069641, "value1":0.9392432577384211, "value2":7977134301111651930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_7":"b","key_0":"j","key_6":"c", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977024, "value1":0.8770253339233329, "value2":114419076017958040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.981104, "value1":0.0262761349528985, "value2":3404389245833544901, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.309389, "value1":0.6700115572733264, "value2":1746800498285488694, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907890, "value1":0.3847355476155844, "value2":1062772964889350348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_2":"k","key_7":"c","key_9":"a", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.557849, "value1":0.8423014863985597, "value2":8693557035023467587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_8":"d","key_1":"b", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.174807, "value1":0.14044078429336324, "value2":2899359413194652810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.907265, "value1":0.7213413890977288, "value2":711482054709798828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730322, "value1":0.010569544047203483, "value2":2272852869521847556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047968, "value1":0.8749328675131659, "value2":3313417752475725526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.548996, "value1":0.13476734433713453, "value2":5686844894034380593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090199, "value1":0.47038291207998856, "value2":6737581800965808102, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_2":"i","key_6":"i","key_1":"k", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372121, "value1":0.4982647921591242, "value2":2960213658998005236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.732328, "value1":0.6794810773912932, "value2":5614573251611720393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.195822, "value1":0.7817234737401128, "value2":6738900276029001186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_9":"g","key_4":"h", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.173099, "value1":0.7620323539766012, "value2":6195639668903126539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739085, "value1":0.8805144351999263, "value2":6990390850908433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_3":"k","key_4":"k","key_6":"j", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851839, "value1":0.19515034480983603, "value2":5022120185556407326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.764811, "value1":0.07516380443788656, "value2":4953415333432850814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.954192, "value1":0.6068771093686028, "value2":1921210459347583907, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.501412, "value1":0.20889031917804166, "value2":7266160086492287738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798358, "value1":0.44429041626860627, "value2":7311639409429341020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_3":"i","key_5":"f","key_2":"i", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200103, "value1":0.5998566082715615, "value2":3968426630272414916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443243, "value1":0.7996897987041263, "value2":2955385724626539485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808129, "value1":0.12934958318030565, "value2":8640009016586784249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.890136, "value1":0.395437996374284, "value2":1375026123479284573, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.726245, "value1":0.8383327509398333, "value2":1069495836938595505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.464465, "value1":0.7522623057444222, "value2":3745372652823842076, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_7":"g","key_4":"f","key_5":"c", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.958198, "value1":0.6062226980863115, "value2":692634818001600691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336639, "value1":0.3569530703633769, "value2":5114963101620697516, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.153507, "value1":0.29997973310668347, "value2":35895264299031809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.496206, "value1":0.16796883917469935, "value2":2457448015461697922, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699773, "value1":0.1320495753882598, "value2":2684330380511330426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_6":"i","key_9":"b","key_3":"h", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086489, "value1":0.01686874394925538, "value2":7266008503964487472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.755161, "value1":0.12039999679109185, "value2":7441080332932356910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481861, "value1":0.21023924595248417, "value2":7215904118134021296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141970, "value1":0.6959505249259283, "value2":5593000831636710131, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473262, "value1":0.20484780325876686, "value2":1515840223895186260, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.610674, "value1":0.37794171859625975, "value2":7210398121561053631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.784669, "value1":0.8314547953224727, "value2":6889739271959092821, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.348279, "value1":0.9806070629868892, "value2":6665537531662525499, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_7":"h","key_2":"i","key_5":"h", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342228, "value1":0.8292222808866697, "value2":8140150785342027171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049991, "value1":0.20259901011770426, "value2":860224417352173376, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442752, "value1":0.9271842959401567, "value2":8958731671751036609, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.288552, "value1":0.19606129029896613, "value2":3803062421310719257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_4":"c","key_5":"c","key_2":"h", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851767, "value1":0.8040030606139774, "value2":5022190100385685639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258187, "value1":0.06995124309795013, "value2":3290080872961653036, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.967676, "value1":0.08273649902951419, "value2":5115999717780317436, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.972416, "value1":0.3332024948916442, "value2":3195696687672274736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273980, "value1":0.9801180643314533, "value2":7124826086691987600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962051, "value1":0.7078535280449856, "value2":2450804349449720576, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_2":"e","key_6":"f","key_1":"i", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.342560, "value1":0.21100802753268208, "value2":5026066029722000095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582068, "value1":0.9724895772631098, "value2":9045330876857967867, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716662, "value1":0.5239131397726235, "value2":6699936731993788174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.630988, "value1":0.8757044551570439, "value2":6009726873690078427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.617422, "value1":0.029531083291305976, "value2":7932542565063790762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.389736, "value1":0.40122875818931175, "value2":7350974819717892233, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.470743, "value1":0.286118127257554, "value2":2903264335442728018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_6":"j","key_1":"i","key_4":"i", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.118668, "value1":0.42900363662380264, "value2":2786819190490835132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_9":"g","key_3":"h","key_8":"k", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819243, "value1":0.7832376096990535, "value2":6768946688624968401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519924, "value1":0.1784273794737174, "value2":606279348173747859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575499, "value1":0.8580592683181654, "value2":6065349113607731318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.952434, "value1":0.8198422458926743, "value2":6520729939047239531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.180605, "value1":0.504162836348111, "value2":8107251100494910096, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.680620, "value1":0.9192990596381249, "value2":7916466116307364506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421505, "value1":0.6014385702769156, "value2":6172295401410316312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581350, "value1":0.2695750525680962, "value2":6289934971199391350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.244031, "value1":0.7392367672457897, "value2":7438364572557990950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801331, "value1":0.014860008784846704, "value2":6873217646463864251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.875352, "value1":0.3975554355639885, "value2":4390005070727485119, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929440, "value1":0.40393323123767977, "value2":1148796038957803851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.817616, "value1":0.5647949954501486, "value2":1024684537861148034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.990318, "value1":0.25287778647718107, "value2":7093365367237549470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741736, "value1":0.023580263258422104, "value2":1518420105714291210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463012, "value1":0.7888469551975262, "value2":6786560784050350679, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449551, "value1":0.2636704318808076, "value2":7884141870151198099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419454, "value1":0.6010995928923429, "value2":224905435491328214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_8":"k","key_1":"j","key_5":"i", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.322105, "value1":0.1331880379524159, "value2":7369022064377940075, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581063, "value1":0.603596835974457, "value2":4386876721702326915, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.857080, "value1":0.5842046650044923, "value2":1066043362420191253, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.191051, "value1":0.4639324153416964, "value2":8874234446483833192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298751, "value1":0.20423387563647288, "value2":8774956872647827067, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924593, "value1":0.0659156088635184, "value2":3634318708597769338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404558, "value1":0.7799254034556268, "value2":4510574800615418821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.962467, "value1":0.8168896478129019, "value2":7986325005390816690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622579, "value1":0.0904343638172895, "value2":8283142617455012372, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.710038, "value1":0.6909066964841821, "value2":6203472414968382073, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032660, "value1":0.21417322011190834, "value2":1054486238769744890, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.801225, "value1":0.2765954328390815, "value2":442385272245544502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116566, "value1":0.6031158754932586, "value2":3770751187916956934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.284086, "value1":0.055471202806315845, "value2":7266763851735180811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_8":"j","key_9":"d","key_5":"b", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559271, "value1":0.28967037620921054, "value2":6147439427772857051, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_3":"k","key_4":"c","key_8":"f", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224654, "value1":0.5513198770455077, "value2":7761756752100166184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.575799, "value1":0.07079456843554317, "value2":4847399039818291257, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.819125, "value1":0.4806719127445899, "value2":1129446367168224920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.730720, "value1":0.6013248235298089, "value2":1610990221268346838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.593103, "value1":0.8196760148078414, "value2":6644540894797627889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_9":"e","key_2":"c","key_3":"b", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.218509, "value1":0.5227005302649587, "value2":2525326877519094680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.292767, "value1":0.09698193120132978, "value2":8960471678202836654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_9":"j","key_3":"i","key_7":"i", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858612, "value1":0.675005592095046, "value2":6028830951911645471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736558, "value1":0.24026583264924886, "value2":1360237166500303266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.826970, "value1":0.3893255587955024, "value2":2292740904231475335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.128577, "value1":0.3146179907253604, "value2":164469213437964556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_3":"i","key_4":"a","key_0":"g", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003953, "value1":0.33225441021352176, "value2":5713576701192138110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352438, "value1":0.12247814806052679, "value2":976940527664803795, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.434520, "value1":0.6138506637398143, "value2":1920264902732481602, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.731843, "value1":0.8305061213621728, "value2":4607426852903927950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_7":"h","key_0":"c","key_5":"a", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.032970, "value1":0.8273657553543087, "value2":3103821801573462769, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833737, "value1":0.3431117940438562, "value2":5529881494203381864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.206561, "value1":0.03180903277463739, "value2":1534192978519074976, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598146, "value1":0.7943659660871121, "value2":2512296846803666990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.830318, "value1":0.9077939825338383, "value2":1289520021940183737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409047, "value1":0.694416059529231, "value2":520453313543014186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.949791, "value1":0.6369046983298985, "value2":371741091848582403, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_2":"k","key_1":"a", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354931, "value1":0.7633140731862355, "value2":8981576909845465787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_4":"i","key_5":"k","key_0":"e", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.061767, "value1":0.7699262325323719, "value2":79439898840104021, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.941133, "value1":0.26448399729496197, "value2":6448749789711445941, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.339821, "value1":0.5246395493337294, "value2":44122001398655312, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359654, "value1":0.5954751506379091, "value2":1668054016792098588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.304719, "value1":0.17854657722051143, "value2":5240836642486639588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_5":"d","key_1":"g","key_2":"a", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533350, "value1":0.9102023121053504, "value2":5223044570382986189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.495583, "value1":0.8380393694454433, "value2":2826946501048985900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_6":"c","key_0":"f","key_2":"a", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925864, "value1":0.06057194714527472, "value2":4827880370625631466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200061, "value1":0.3196222334980913, "value2":5536801387533970020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.635607, "value1":0.5926951079597541, "value2":4317404702469315811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041722, "value1":0.2038750977962134, "value2":8603762180622472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.565080, "value1":0.5851130292640377, "value2":8612148593533709646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090894, "value1":0.12207888531940794, "value2":435323305893709131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_9":"e","key_4":"c","key_8":"j", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.071849, "value1":0.7914473708034223, "value2":4833048612300779383, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.202326, "value1":0.6798241827808086, "value2":5321585738808934000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116141, "value1":0.8646385559973755, "value2":2431723757311861306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.074497, "value1":0.2771958266692314, "value2":2948732814035791763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.453427, "value1":0.9750934919782419, "value2":7598695164797163349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.517105, "value1":0.6447380734003394, "value2":5945582855956594632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_9":"j","key_4":"e", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969670, "value1":0.8792534113647463, "value2":9118825598855667812, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064653, "value1":0.7641915057577142, "value2":376649068007551056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.299983, "value1":0.5328541897726483, "value2":7848443219053578824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814787, "value1":0.5822532477986045, "value2":3517444920674306021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_4":"c","key_0":"b", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645109, "value1":0.6788316811493683, "value2":2399412759017043209, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_5":"d","key_6":"f","key_3":"h", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.104970, "value1":0.5150104887448738, "value2":39162375392832614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124629, "value1":0.9415126690810315, "value2":2487867259401435813, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847494, "value1":0.8236014719422879, "value2":8522949906604300533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.694932, "value1":0.5232173520376328, "value2":2865775556504465271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.824405, "value1":0.33768880558190767, "value2":1449467820646971158, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263091, "value1":0.800562879218074, "value2":4640064437824793404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.545676, "value1":0.6991321506153548, "value2":504604265126996830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_2":"k","key_6":"h","key_1":"k", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.685750, "value1":0.256315076419413, "value2":7805803213074768353, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.955572, "value1":0.12777100392825705, "value2":4697380262379050863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.052725, "value1":0.24897441126326256, "value2":2806761823519892967, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832231, "value1":0.7123036010256201, "value2":5558652604185663810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_7":"c","key_3":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.779475, "value1":0.42379979395746137, "value2":5877566589967590752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_1":"a","key_2":"j","key_0":"c", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.056823, "value1":0.7557073972848881, "value2":1462009193456216531, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_3":"h","key_6":"a","key_2":"k", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.179551, "value1":0.33562095061974395, "value2":2905800790209406415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.946943, "value1":0.7895726694044407, "value2":4556871779015432465, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.792955, "value1":0.9478490399693646, "value2":6469796413879524879, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_9":"b","key_0":"b","key_8":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302294, "value1":0.40539394874343054, "value2":1479528610985409190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.859086, "value1":0.8533535931566633, "value2":8328258195824345992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.912801, "value1":0.4080605346123484, "value2":921699075685228177, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026911, "value1":0.5160934551500268, "value2":4192638354189389027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.145578, "value1":0.4697942761887252, "value2":1007860316702817500, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_3":"b","key_0":"i","key_2":"c", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.224036, "value1":0.21446416617045144, "value2":2612501104515894392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.083418, "value1":0.3800566396023636, "value2":1387864613630168710, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.483137, "value1":0.9606563644656662, "value2":2761802265245174240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.855910, "value1":0.606558104786761, "value2":842530041019109994, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332745, "value1":0.01260394609809363, "value2":2781673523626108476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.198691, "value1":0.0952542911286906, "value2":4639530496495690336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.207605, "value1":0.29598976404510996, "value2":149815254996856780, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.219100, "value1":0.7173110134058144, "value2":207519709019999249, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.065034, "value1":0.7683750910989782, "value2":6262446867113607884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362737, "value1":0.5652820007764047, "value2":4138085582551770816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.790498, "value1":0.45469936210178996, "value2":3874789041764013688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.581214, "value1":0.8566672554079884, "value2":5701861631669426000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.741360, "value1":0.7962411718913905, "value2":3357161583638308077, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_9":"k","key_1":"a","key_3":"a", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.934536, "value1":0.07212743130410183, "value2":6319820377917318485, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.258971, "value1":0.09357358589212993, "value2":5081900353713330302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_4":"k","key_1":"i", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.436226, "value1":0.5553015007542973, "value2":6124312980620622172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566508, "value1":0.7132604990750537, "value2":8485961090508027563, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298396, "value1":0.4460940570451377, "value2":5709626152776289179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596697, "value1":0.3774598732286396, "value2":1819820653928002535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.758549, "value1":0.49680425119422045, "value2":1371518397741898898, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.151965, "value1":0.9897517149153644, "value2":3974511699577597667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.223787, "value1":0.9859897177824236, "value2":6368178524733411270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.699805, "value1":0.9240609624872184, "value2":4091684677817576336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.640771, "value1":0.3605713604653968, "value2":2196974552050802581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917234, "value1":0.3769625716544593, "value2":7094294232648752669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541154, "value1":0.07721056009394252, "value2":2450341303960944649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188174, "value1":0.511305577711157, "value2":4677738772360990938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_5":"h","key_2":"a","key_4":"b", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.277439, "value1":0.006756936522771483, "value2":2736398221812059244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.847832, "value1":0.23362336838695744, "value2":6292362019324286334, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.840677, "value1":0.6137390254843741, "value2":6506004284907452986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.397331, "value1":0.5291020730916409, "value2":766558029100390419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_1":"i","key_5":"e","key_8":"i", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.668711, "value1":0.6677953904066025, "value2":2813624786174976455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.728127, "value1":0.9455687108064199, "value2":2598623203001035108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.260429, "value1":0.5485107722176485, "value2":8238880717060307339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_0":"b","key_2":"g","key_8":"g", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.407939, "value1":0.7503222721986553, "value2":8088756690439902034, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.016860, "value1":0.6281421773683825, "value2":4463756144226416929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.592662, "value1":0.7215969122220701, "value2":2957728488172560734, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596916, "value1":0.6365196560861767, "value2":3939747841173152393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.650813, "value1":0.33343709461482135, "value2":8822031185214434530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_8":"c","key_9":"f","key_3":"a", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543088, "value1":0.5660730417813042, "value2":6651394201039088708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.416422, "value1":0.2907903630149851, "value2":5811650096165674198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.505086, "value1":0.44660978158387593, "value2":6270587282710629926, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.055558, "value1":0.394254430161103, "value2":9153429600073084016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.578079, "value1":0.6160395423413576, "value2":7388869261196091596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.347142, "value1":0.28357580684654937, "value2":7651284402887473938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089369, "value1":0.24785364470389606, "value2":2827626134616355884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_2":"f","key_5":"a","key_0":"f", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.922968, "value1":0.9908293125287131, "value2":5006327563319926460, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489701, "value1":0.9764768678976257, "value2":3780590525108581330, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.700790, "value1":0.8524762188330061, "value2":2298426148247437198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.884155, "value1":0.17838161460811935, "value2":707461345329966226, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_1":"e","key_7":"b","key_0":"k", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.877319, "value1":0.597182710013635, "value2":66780576193296735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_2":"a","key_8":"d","key_0":"f", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.906533, "value1":0.44570088765160515, "value2":1159099942400157753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.555879, "value1":0.6225620185948739, "value2":7381136800707660104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.498993, "value1":0.9016213008618695, "value2":8432726482609273880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.349195, "value1":0.16850895992044404, "value2":6332393888509943818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.530337, "value1":0.8558953203814225, "value2":76710422944574508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345065, "value1":0.8230787557550705, "value2":4201898391845439571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947222, "value1":0.9688974381514411, "value2":2441269560109987314, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.287425, "value1":0.20362983597374928, "value2":5349125652009205097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.519644, "value1":0.3548258293385043, "value2":1594647307470342039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_5":"i","key_0":"j","key_1":"f", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.027626, "value1":0.9120926865897546, "value2":356446699135295995, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.054561, "value1":0.22396147232559485, "value2":3930608769790828524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_3":"k","key_6":"e","key_2":"c", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.388931, "value1":0.5101134733621842, "value2":6568513065035568570, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567065, "value1":0.79529931520039, "value2":6002480040724485306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.449050, "value1":0.9291008790205846, "value2":741995070657009132, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.798788, "value1":0.9907520697579564, "value2":2343010721096789225, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.927945, "value1":0.8814555609090619, "value2":6729196069806990782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.917877, "value1":0.15366203128840322, "value2":2435285873402812217, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.301622, "value1":0.20876721045801763, "value2":1403445590939959710, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_8":"c","key_3":"k", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.745719, "value1":0.6060177757203955, "value2":2370675136825151538, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156169, "value1":0.40315211694606023, "value2":5606845635294528121, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.651978, "value1":0.02572219217737101, "value2":6216284509197748039, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.395139, "value1":0.4352070266196555, "value2":2437710317158637331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.132661, "value1":0.3845242564807311, "value2":1179550652058134811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_7":"j","key_1":"k","key_5":"d", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.088075, "value1":0.27124710684397163, "value2":5954367646517477044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_3":"g","key_0":"e","key_1":"j", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213664, "value1":0.9844242730115798, "value2":1430390936289238362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070775, "value1":0.6998833636894136, "value2":8373393772820367148, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.213733, "value1":0.44329246485521806, "value2":6509431828213171835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.608249, "value1":0.15493446331886743, "value2":4384868943457502322, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.956693, "value1":0.9523530599130996, "value2":365504939691815078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.432058, "value1":0.16692183631193722, "value2":4864357898977097624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.303592, "value1":0.934106512625592, "value2":7515104000227729499, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_2":"e","key_7":"a","key_1":"j", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.333787, "value1":0.6736717480454248, "value2":5830960931165083327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.359385, "value1":0.922695683809537, "value2":6703804740319670530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.211845, "value1":0.13745320161077518, "value2":8688931964841266452, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_3":"b","key_6":"f","key_0":"g", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600304, "value1":0.365028686112319, "value2":4172510140835977605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_5":"k","key_6":"f","key_4":"e", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.045527, "value1":0.1293058694163136, "value2":4396801118892738860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.124929, "value1":0.2927998769649342, "value2":7840410733128704268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.708707, "value1":0.42917010198000677, "value2":190549078586280269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.543921, "value1":0.45395915446447055, "value2":4038857743678185292, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001958, "value1":0.8799882402246043, "value2":8959405737069039234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.834357, "value1":0.9787241750393116, "value2":4713255732772978232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.398402, "value1":0.3356690698861162, "value2":8600435717396946203, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_4":"j","key_8":"f","key_3":"h", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.532477, "value1":0.5790086501753019, "value2":585720405314800565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995284, "value1":0.019519382747907175, "value2":561302586535494029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.492003, "value1":0.22729751026433825, "value2":2059445045999328988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_3":"g","key_6":"j","key_8":"k", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.357595, "value1":0.2653305071097244, "value2":7325567281409701416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423155, "value1":0.8492553718976581, "value2":5551377313853459727, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_1":"i","key_8":"i","key_0":"c", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.582438, "value1":0.9616765075659243, "value2":1026983801153932760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409214, "value1":0.1250667617190597, "value2":5921255804215527024, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744368, "value1":0.8513042177093529, "value2":6603153879156409175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970658, "value1":0.2306199750722779, "value2":2906238010168281574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_6":"c","key_1":"e","key_4":"d", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520199, "value1":0.9665587748785213, "value2":2458054282507819610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_7":"a","key_1":"j","key_6":"k", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.098618, "value1":0.12617188969368395, "value2":2230070381051915192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279517, "value1":0.4244894688924968, "value2":7931154210918628962, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.394850, "value1":0.011258876506077731, "value2":3335326787820059838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.672924, "value1":0.7327755721792956, "value2":3880440053684337972, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.431109, "value1":0.6741923566632704, "value2":2327920765233612965, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.659350, "value1":0.11708731624685002, "value2":276841347975235584, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.115617, "value1":0.43754612855223973, "value2":3365515217954889202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.533611, "value1":0.8399047779975791, "value2":2365721152492621420, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265333, "value1":0.7911288127976842, "value2":5955361860868770714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.966474, "value1":0.3536325699896996, "value2":7032196306810796442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_3":"j","key_0":"e", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.684964, "value1":0.8626527282208771, "value2":7197488766581698381, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_6":"k","key_8":"g","key_3":"k", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.559000, "value1":0.010659679933132453, "value2":4115322864019466158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.455110, "value1":0.6204915686658947, "value2":6428809732964624602, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625937, "value1":0.9727096027334052, "value2":6229477409536583105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.113032, "value1":0.5903167177404812, "value2":4156394072465021166, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.567689, "value1":0.10316863931938577, "value2":8418749122094896416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.987774, "value1":0.7873883846984865, "value2":2030919041886568670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_5":"j","key_1":"c","key_2":"a", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.811061, "value1":0.4992724231992665, "value2":3986295112752729017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.194651, "value1":0.2743659705657923, "value2":583251715501918289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_9":"i","key_4":"f","key_5":"a", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950390, "value1":0.5427606927000886, "value2":2986321212639280881, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598120, "value1":0.20329762636615106, "value2":5612645284463886443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950214, "value1":0.4515603514606786, "value2":259527570321604761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.645723, "value1":0.13300625888317788, "value2":9108039921133288290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.001569, "value1":0.01777574006312956, "value2":7180543411595530153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036902, "value1":0.19075245150411826, "value2":5889621655868191557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.929644, "value1":0.8218270153578531, "value2":6338216490369302155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_8":"d","key_5":"k","key_6":"i", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.036799, "value1":0.8121072864023562, "value2":4644698086978403677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099804, "value1":0.41836860978181317, "value2":2663944418369780756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.607018, "value1":0.04603283781595935, "value2":7178332091114199020, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075733, "value1":0.9728379954214806, "value2":4310309256078899557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.795422, "value1":0.6466237769859894, "value2":344630441945165516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.924050, "value1":0.5005809462467858, "value2":4504593684309524057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.352892, "value1":0.9539405805618134, "value2":996952878654521350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.141322, "value1":0.01789552453746491, "value2":5938050187519951412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.783153, "value1":0.1436758434821978, "value2":6441131001960903605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.473891, "value1":0.3495022243472323, "value2":4306013569645375103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.814977, "value1":0.38372259399528474, "value2":436380091488564183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716628, "value1":0.6699570981955947, "value2":3523603923463352771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_6":"f","key_7":"a","key_0":"f", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.735953, "value1":0.6776358805955892, "value2":5856380565120662945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_5":"e","key_8":"b","key_0":"b", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265185, "value1":0.11607380521583967, "value2":5582686822797693437, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_9":"h","key_0":"g","key_5":"i", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516162, "value1":0.2994616004522488, "value2":2261467086953311820, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334873, "value1":0.14715552224004888, "value2":8821110853892469057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.400930, "value1":0.8143295386608986, "value2":1312967392198736440, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.196057, "value1":0.5352481987346313, "value2":3859881997560184862, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.925763, "value1":0.823842231642209, "value2":7076092787875459652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.012976, "value1":0.08033889071311322, "value2":2194666423113168625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656711, "value1":0.979250223824098, "value2":295513105593707440, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542943, "value1":0.34216024465522854, "value2":8482137672426222632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.738294, "value1":0.6164314293888586, "value2":7863948057381466211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_8":"b","key_9":"c","key_6":"i", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.957221, "value1":0.4133406835193543, "value2":3905154346081616319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.047765, "value1":0.9114235810126055, "value2":8926222646986865918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423884, "value1":0.30572384015704446, "value2":2339988500116190675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209371, "value1":0.07849561730590998, "value2":3882974907897625962, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310209, "value1":0.2535587414218041, "value2":7470349035360557369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_2":"g","key_6":"g","key_0":"a", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988088, "value1":0.2879292667105676, "value2":5363951038000132107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.947720, "value1":0.958399617505033, "value2":5218176225849015970, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.674130, "value1":0.9617762098247892, "value2":5270132257081817867, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.279827, "value1":0.8686925044217091, "value2":7240549587345744089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.257178, "value1":0.28758053646406956, "value2":7938343410176527229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.870350, "value1":0.16917076304978226, "value2":9111760754875505635, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.763299, "value1":0.8896924750809221, "value2":6658856857804173307, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_8":"k","key_1":"g","key_5":"h", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734200, "value1":0.44507702947189576, "value2":4594583173060659683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155510, "value1":0.8897408774019222, "value2":3124062956444214018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.363722, "value1":0.07362703906856644, "value2":6124606823090260320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.336581, "value1":0.7586482338064128, "value2":2691495825421343596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.075863, "value1":0.7294045132994561, "value2":8264827508243049447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_8":"i","key_5":"a","key_7":"f", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.108381, "value1":0.45788844677640256, "value2":2616045646790504337, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901873, "value1":0.08770240902249962, "value2":6640573405482547401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.438350, "value1":0.25439050102218247, "value2":7286046839968613612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.314379, "value1":0.33872622756746645, "value2":2585280362910555462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736079, "value1":0.12615475607591273, "value2":8133018761204018534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.130003, "value1":0.7963354396181769, "value2":6450859077912673686, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.485449, "value1":0.39497609142163287, "value2":170013329412671465, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049341, "value1":0.06350907823991539, "value2":18163724507539892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.897568, "value1":0.6759106102023759, "value2":8613373636446320004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.522262, "value1":0.9415564313481927, "value2":183076459384497822, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.456571, "value1":0.3755600914749928, "value2":1317951286539480179, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713142, "value1":0.3684027597862527, "value2":6666246661014977211, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.332747, "value1":0.2712525974797622, "value2":6685862283475642897, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.622289, "value1":0.8099425380336615, "value2":6696254947892698280, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099077, "value1":0.4218807485485893, "value2":8151022600373825319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.833739, "value1":0.471092674976307, "value2":4241908968763096086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.512079, "value1":0.6891384899122228, "value2":2796503546861546091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.405406, "value1":0.9060026283921703, "value2":4773855974745798571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.742427, "value1":0.6129480541080172, "value2":3917404693410650609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_5":"k","key_2":"d","key_3":"e", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.613376, "value1":0.3584257069399756, "value2":6068105032813229930, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.759402, "value1":0.29210317562134064, "value2":3082388413784210214, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.090526, "value1":0.4678159387104084, "value2":1605658432958093758, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380672, "value1":0.5086572266239265, "value2":2924449619880684423, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.249203, "value1":0.6593022058411281, "value2":727243123088870599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.142921, "value1":0.5973096622193752, "value2":2056855353114653381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.041929, "value1":0.03439471294074195, "value2":8729944365498889284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.970035, "value1":0.629094069595545, "value2":954705184774295714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.656521, "value1":0.4935396027418025, "value2":1513112375773530412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.286143, "value1":0.590796213295746, "value2":3085171514913296735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.611048, "value1":0.3503905353305475, "value2":5805006250107454516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_6":"f","key_3":"e","key_4":"h", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265940, "value1":0.8719050400063297, "value2":2334709836967925044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.231945, "value1":0.9482208594424564, "value2":7605532380274260726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.099422, "value1":0.6387093062831974, "value2":6416034503475989540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.862301, "value1":0.0844793706976894, "value2":1724212296853776891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.882958, "value1":0.45080094639121554, "value2":6596820877226155907, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.851802, "value1":0.784896015737414, "value2":2267862818127250105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_1":"h","key_5":"i","key_8":"h", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.871761, "value1":0.9219727566018483, "value2":9044274636085577036, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704109, "value1":0.6618033501935818, "value2":8585487696070011938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.863670, "value1":0.6882846261541673, "value2":5478374167595454238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_6":"j","key_3":"e","key_5":"b", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.183651, "value1":0.21762989261712523, "value2":2054870959404109040, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157246, "value1":0.1072057187986972, "value2":2474773268716838947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.263088, "value1":0.6424740570966222, "value2":4311694979316615362, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.227492, "value1":0.6060012408935248, "value2":8262744174339677603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.919259, "value1":0.26635737492611994, "value2":146832998544852566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_2":"e","key_0":"d", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.810229, "value1":0.6433893313883111, "value2":6020303671131449049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.021406, "value1":0.5663729785218883, "value2":6847532330100114835, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.331141, "value1":0.03123867674281506, "value2":2370510435982901478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.313836, "value1":0.7842831796528092, "value2":1464971588517507800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_6":"i","key_0":"j","key_4":"h", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.147234, "value1":0.232833406436683, "value2":5302769737190544079, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423280, "value1":0.5646235246180139, "value2":3807301011411244203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.872535, "value1":0.3508318181055984, "value2":8035632162257252295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516267, "value1":0.7376190499237748, "value2":1779371132519351560, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.020480, "value1":0.5129193623789473, "value2":6378565762108475442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.421514, "value1":0.72712434334131, "value2":3908844968753938444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.026297, "value1":0.8255820903801332, "value2":5180017205009891488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.138610, "value1":0.5467990020300735, "value2":8134249053330675938, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.754861, "value1":0.5832263769294902, "value2":8216858329154999597, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655676, "value1":0.45804041306603793, "value2":8403704610668639637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.091295, "value1":0.7313452902241809, "value2":6274232113944917429, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_6":"k","key_8":"g","key_4":"b", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370397, "value1":0.7082796943852117, "value2":16646752364177844, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.072185, "value1":0.8009986319191017, "value2":6123195610574092125, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068380, "value1":0.3124676060355689, "value2":2569282749677030904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.013897, "value1":0.9321729707819904, "value2":2281258981052686264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.963470, "value1":0.9927608945020866, "value2":6050801587231871156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.481229, "value1":0.3689051456876514, "value2":1924733693360792633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.295891, "value1":0.44913357839758283, "value2":8069037361024844952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_7":"i","key_1":"h", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.409485, "value1":0.18423629567656025, "value2":5066355149165531813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.744404, "value1":0.9737949013567385, "value2":5174295290317701561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.739701, "value1":0.4976173273573324, "value2":5658855721642744182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.186283, "value1":0.419361633943128, "value2":2053614790636840527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.566841, "value1":0.7094596520198607, "value2":2148895338889314160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.215921, "value1":0.34276587375112927, "value2":1645566772308625552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.701909, "value1":0.22008909623955855, "value2":2295629303145372991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_2":"b","key_7":"f","key_1":"i", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.002697, "value1":0.1765088617877308, "value2":4201797060324467420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354799, "value1":0.6538892310533279, "value2":4577607391337663337, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345385, "value1":0.3843950816957309, "value2":8800458811056296695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.988457, "value1":0.5588868714021769, "value2":8433496452370061074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.415009, "value1":0.1746188760147135, "value2":6912110645446824396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655363, "value1":0.9949350156565988, "value2":8110347995943103660, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.242069, "value1":0.48932951486152193, "value2":8286237027514157124, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.085368, "value1":0.0689091306398061, "value2":5859648999386501040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_6":"e","key_3":"j","key_4":"d", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.661208, "value1":0.7864155291083853, "value2":7826276349217880531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.803222, "value1":0.6015539667825235, "value2":1102772758151131198, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.458180, "value1":0.3809696927779604, "value2":4439664400270353254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.334077, "value1":0.4097523501895673, "value2":7454062761319263955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.885572, "value1":0.548188359160327, "value2":3020054439494862697, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_7":"h","key_8":"b","key_0":"i", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489119, "value1":0.6763563343582067, "value2":4242764599158368849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_4":"d","key_1":"e","key_3":"a", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.187849, "value1":0.7091709802238197, "value2":6867639316908364367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.663732, "value1":0.42631474518610635, "value2":9001795643794270349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.585010, "value1":0.6237338911978378, "value2":4365340422086338410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.379644, "value1":0.15387674044669916, "value2":818900165755081001, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.791337, "value1":0.32588514831554516, "value2":1691735796263334884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.082385, "value1":0.5745615587827129, "value2":772277205802820637, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.086473, "value1":0.8230348349086334, "value2":8156169851471463456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.808463, "value1":0.47620124507643474, "value2":2357967788477712632, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.380686, "value1":0.011323528011005758, "value2":2028993392634762324, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.371579, "value1":0.5028955605922396, "value2":7910467950096619453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.883127, "value1":0.20067026957438594, "value2":3625887507362062386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_4":"g","key_6":"f","key_3":"c", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.305224, "value1":0.7123715236402623, "value2":7522375891893442954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.443461, "value1":0.7555458976714571, "value2":655917858760911221, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.901204, "value1":0.7387538487064921, "value2":7525092307824794219, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.376917, "value1":0.7993175855480273, "value2":2807412127532185201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_6":"h","key_5":"c", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.740075, "value1":0.3001865006906782, "value2":7190874681083752315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.209665, "value1":0.89276344594564, "value2":4778320938543530902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.341730, "value1":0.20200750378616714, "value2":1664606777941171586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_6":"e","key_8":"d","key_1":"f", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.629099, "value1":0.017583528638071864, "value2":2707650519412422348, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.057519, "value1":0.04492925365315839, "value2":2578667952703739482, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.723798, "value1":0.8454916194785576, "value2":1835422305385826626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.034767, "value1":0.201322613886975, "value2":7356183312830760350, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_5":"h","key_8":"a","key_1":"b", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.520001, "value1":0.5251237247182675, "value2":134900877343320532, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.489083, "value1":0.8513994038065534, "value2":2159556869883772494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.344496, "value1":0.16316310680106225, "value2":5518614284082110727, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.724230, "value1":0.6875155435793379, "value2":35737216326475935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568987, "value1":0.5138501785175705, "value2":4338888431721237426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_2":"k","key_4":"h","key_0":"a", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.598387, "value1":0.4390074014236514, "value2":7041842749721887181, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.626270, "value1":0.2391917526821926, "value2":3091537573368363672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_7":"c","key_9":"e","key_2":"b", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.297931, "value1":0.11690748126426959, "value2":4039852836857346610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.175994, "value1":0.6789300486469183, "value2":1246840956922367253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_6":"a","key_3":"i","key_4":"k", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.976992, "value1":0.43367432666147043, "value2":564959136229688173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_7":"g","key_1":"i","key_3":"b", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.703741, "value1":0.47707265609489413, "value2":4068705543834170412, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.235944, "value1":0.8012892224294691, "value2":1988300147401692461, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.499512, "value1":0.9990473134151273, "value2":8394845441054007311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_8":"k","key_9":"h","key_1":"k", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404130, "value1":0.6068400138809874, "value2":1935348003856793271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542560, "value1":0.12776285353397152, "value2":6795264619125251375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.693012, "value1":0.2569652780369043, "value2":1475809083419006811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.068970, "value1":0.3881338979751516, "value2":1218503134871756052, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_9":"i","key_5":"b", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264926, "value1":0.19791850227382649, "value2":5513222623463327397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.408769, "value1":0.39494174800117104, "value2":1576673230492191692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_5":"a","key_2":"f", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.081892, "value1":0.9402614771700961, "value2":1589947956431908818, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.058425, "value1":0.5327215264224379, "value2":445308327885517649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_8":"d","key_3":"b","key_4":"f", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.838442, "value1":0.9519819634366921, "value2":745087205152711718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.064037, "value1":0.23777264651361862, "value2":4675095342249511184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.518857, "value1":0.8856046396694535, "value2":1109051615753148759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077463, "value1":0.8624901705258308, "value2":3793672263940517136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.921342, "value1":0.9049145350717305, "value2":3570805638374912588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.157723, "value1":0.6223760989748452, "value2":2996869515277202402, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.155659, "value1":0.13854963250628805, "value2":6512544911629959850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.977987, "value1":0.036771554508911904, "value2":5994549505435027834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.265064, "value1":0.3403489462496419, "value2":2814929460502496771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404153, "value1":0.3208642154389825, "value2":5256172233350798960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.312852, "value1":0.011902750850203012, "value2":5526100237606282605, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_6":"j","key_5":"g", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.491995, "value1":0.46992302986974577, "value2":157557322951942167, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.704212, "value1":0.9425835794283676, "value2":915577115687321604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.576745, "value1":0.7854877804515764, "value2":6382074091209031167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070556, "value1":0.18030779510661127, "value2":6308540602135606131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_6":"h","key_9":"b","key_5":"j", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.542532, "value1":0.3384327861708296, "value2":1613795669513593265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.511484, "value1":0.8996232864271185, "value2":7405365074827637694, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_3":"i","key_7":"c","key_2":"b", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.562131, "value1":0.10596222820549657, "value2":3306812514398053557, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.391786, "value1":0.20647295311593183, "value2":5486022828508870343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.466143, "value1":0.6224930113197604, "value2":8535178573473477072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423074, "value1":0.3088852292542975, "value2":3166186037825297943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.762488, "value1":0.33490363470863166, "value2":2912130613463378266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.713117, "value1":0.2993000272696047, "value2":494302122350368825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028636, "value1":0.6503466890929638, "value2":4759183611197798725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.404461, "value1":0.043678508683084524, "value2":6408629579798949638, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.077439, "value1":0.2418209601880771, "value2":6820734383602297695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.624831, "value1":0.4718269105814766, "value2":4622812711240963076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.053083, "value1":0.6875169297990352, "value2":6119677537083039891, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.345042, "value1":0.7324917812164005, "value2":5511749504782560024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.623531, "value1":0.9320016848414698, "value2":744369719732623498, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_1":"d","key_4":"c","key_9":"i", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.182404, "value1":0.9674131847138503, "value2":3838480249361897923, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_9":"f","key_2":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.696213, "value1":0.7881956638078709, "value2":4371203118864420550, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.628037, "value1":0.17572736986644139, "value2":3461303046815504013, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.273728, "value1":0.6916742591847842, "value2":2382468702478522273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_9":"h","key_0":"i","key_3":"a", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.335163, "value1":0.9758093684084915, "value2":1680436716344950339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.782020, "value1":0.1537134689497552, "value2":4414043374214499577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.134363, "value1":0.5083067111414208, "value2":4982306711291417821, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.865817, "value1":0.9406393737656967, "value2":5857776137176686892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_2":"f","key_4":"i","key_6":"k", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.716601, "value1":0.1916346431757554, "value2":5910562859970195422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.427366, "value1":0.03546691845658599, "value2":7315776376647279090, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950000, "value1":0.5369396780416313, "value2":7524727304557507728, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.325387, "value1":0.06313877213741768, "value2":7980198837673375086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.950927, "value1":0.32484655923390965, "value2":8297079571694626407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.832701, "value1":0.8101399444767259, "value2":6277528961017233439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.969582, "value1":0.5893386387714608, "value2":8824793605581596113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.302334, "value1":0.03653755659557745, "value2":2394591637437372554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_5":"i","key_1":"b", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.463188, "value1":0.2645292258357236, "value2":7750296583775050063, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.774073, "value1":0.7854839946508317, "value2":4045742754979149401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.729676, "value1":0.4706720879040935, "value2":4750321291295032700, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618814, "value1":0.2230204325484985, "value2":9003120111243218251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.603138, "value1":0.22960374502253963, "value2":6338685193000832935, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.412324, "value1":0.4957126042457222, "value2":8735002444024394358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657244, "value1":0.08184942404758783, "value2":7470578294751016176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.442850, "value1":0.7229538622773333, "value2":4724836800934702763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.156841, "value1":0.7233638365572691, "value2":7858912978560139100, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.734329, "value1":0.3404208114361277, "value2":3050306358924697552, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.465656, "value1":0.38391986849101767, "value2":307361261718394449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.445647, "value1":0.990731352209279, "value2":1207071782335047705, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.298620, "value1":0.11180836767709064, "value2":3870338901819904571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.095445, "value1":0.2870001965302557, "value2":2415945267551238695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.918448, "value1":0.7277791066957175, "value2":2821193899111524763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.541594, "value1":0.09005789780178877, "value2":2925035824598662472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.631726, "value1":0.9959155652844405, "value2":3833701945431571315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.049800, "value1":0.8110556334200572, "value2":2218497283809170432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.070530, "value1":0.65401432282683, "value2":498838541277181702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_1":"i","key_8":"i","key_0":"e", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736247, "value1":0.7394025858411815, "value2":7148599556072452144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.736608, "value1":0.08218047603491815, "value2":3776524585186258439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.003840, "value1":0.6757338653122574, "value2":379207741924258587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.362539, "value1":0.11651117191245357, "value2":3906526956584011608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.539295, "value1":0.44782322310256395, "value2":2595894391836664475, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.392973, "value1":0.8520294340743481, "value2":7384112654505361476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.078027, "value1":0.3547792528217613, "value2":8155526662947982478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.776543, "value1":0.6604220360650918, "value2":4011373177447838470, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.691748, "value1":0.4862664301051333, "value2":7696312329542980810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964567, "value1":0.11169294328511947, "value2":4727405940623359973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_2":"c","key_8":"k","key_9":"g", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.722155, "value1":0.3780569227165093, "value2":3316392130699376882, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457388, "value1":0.9243183084724984, "value2":2432925818286345186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.040558, "value1":0.7743604477646876, "value2":627834147679968207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272066, "value1":0.5917633264655242, "value2":5918933193525430349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_7":"k","key_3":"e","key_5":"k", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.200729, "value1":0.7327561219912169, "value2":1393602674371677220, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.905147, "value1":0.8568183299448732, "value2":9034660617242414820, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.310139, "value1":0.802088771431449, "value2":3471155770712582770, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.821462, "value1":0.5273591260032361, "value2":5140674777677119228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.384953, "value1":0.4025027387592999, "value2":5097429126997347384, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.829568, "value1":0.8045993326905381, "value2":2809186620050229002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.294923, "value1":0.5644401567133956, "value2":3718732987675906525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.556732, "value1":0.48820899714237304, "value2":4078290062854715892, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_0":"j","key_4":"f","key_9":"c", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.176213, "value1":0.2510405463103239, "value2":635371208287508106, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_7":"b","key_9":"k","key_3":"e", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.267004, "value1":0.22999772540025595, "value2":1871342923142731692, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_9":"g","key_6":"i", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.506694, "value1":0.4581021751822573, "value2":8794558811813763792, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_3":"k","key_4":"a","key_1":"c", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.354208, "value1":0.6088901271960526, "value2":132806552362853695, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.516024, "value1":0.7664701528892452, "value2":107006237498756274, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_6":"e","key_3":"h","key_5":"j", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.750598, "value1":0.3217472443950159, "value2":8378919480372645762, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_6":"h","key_2":"j","key_3":"g", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.419841, "value1":0.6663125192971521, "value2":2423610833256438065, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_9":"c","key_0":"e","key_3":"h", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.164890, "value1":0.2006090031725153, "value2":1844725035830445264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.264867, "value1":0.42671643117928926, "value2":3438912836085300466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.430421, "value1":0.07257562390395779, "value2":6088518193551139346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_8":"c","key_1":"k","key_6":"g", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.673454, "value1":0.6691702132206785, "value2":8625848517577296022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.995936, "value1":0.702118704788376, "value2":4086471613439852656, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_5":"i","key_9":"g","key_4":"i", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.964584, "value1":0.7747761523528802, "value2":1036051996047955870, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.948289, "value1":0.25140012327548283, "value2":788619002191666922, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_4":"i","key_2":"a","key_3":"d", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372555, "value1":0.9999769483157629, "value2":7184076213289982875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.920761, "value1":0.868179497430529, "value2":1697965828697918238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.372531, "value1":0.5396817815353727, "value2":7506498166097940651, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.278206, "value1":0.17061135686969672, "value2":713580384695452010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_8":"h","key_3":"f","key_5":"c", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.600654, "value1":0.6616469498902554, "value2":774519242775197957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.188862, "value1":0.5531871959368098, "value2":7422109554403592667, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.457643, "value1":0.7840548969774126, "value2":7827846462514659006, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.655234, "value1":0.76526166835666, "value2":2375166258510616618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.370401, "value1":0.021438154414782817, "value2":2771089523399378961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.137411, "value1":0.5646878830454839, "value2":5968018682136017050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_8":"b","key_9":"h","key_7":"b", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.423235, "value1":0.36529823070220574, "value2":7162022842889925076, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_3":"d","key_7":"f","key_9":"d", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.272788, "value1":0.219359460529438, "value2":181241653539382719, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.707284, "value1":0.3923075865949308, "value2":2782827649523891195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.858145, "value1":0.688162185571026, "value2":5086226934464427172, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.867650, "value1":0.6841047492247676, "value2":8098586901309655486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.902390, "value1":0.706932118893774, "value2":9086709640294623772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_1":"h","key_5":"d","key_9":"f", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131998, "value1":0.8578147943558058, "value2":41651666626493230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_4":"k","key_7":"h","key_0":"h", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.932017, "value1":0.24496475938485004, "value2":7307212968529901562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.469902, "value1":0.3291978179419431, "value2":6674355534843744608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_1":"c","key_7":"d","key_8":"e", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.715852, "value1":0.5949781040197312, "value2":6585582445453110248, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.439874, "value1":0.25098256408304576, "value2":9009893614730547675, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_4":"d","key_6":"k","key_2":"f", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.775831, "value1":0.888090262607404, "value2":7565201070136116477, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.480681, "value1":0.47062171981528383, "value2":4853013107516729574, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.654578, "value1":0.4096044979675714, "value2":1606198625715352901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.881884, "value1":0.31666628836121824, "value2":5391861566395998267, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.028180, "value1":0.2934267405568024, "value2":1598172822680744133, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.245149, "value1":0.09374074237151239, "value2":6863637452711576117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.105508, "value1":0.7783021910779061, "value2":2669034341302440615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474623, "value1":0.5494532131881231, "value2":6109855504680847451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.625728, "value1":0.471409564376847, "value2":5293699949847690348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_5":"g","key_0":"c","key_1":"i", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.382751, "value1":0.7835774962833667, "value2":5926458098948297057, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_3":"e","key_2":"c", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.527286, "value1":0.6322552498331545, "value2":437257708157331646, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.166237, "value1":0.6386670258285629, "value2":2053216930342060527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.942234, "value1":0.7212756328576085, "value2":8527112750674030055, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.880290, "value1":0.19477642153523786, "value2":8136028599409004140, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.477164, "value1":0.7162729706892131, "value2":3933534960467417574, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_7":"e","key_8":"h","key_9":"a", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.089992, "value1":0.47761006384043037, "value2":4843140910052787407, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_2":"a","key_7":"i","key_8":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.204970, "value1":0.4917659716382411, "value2":8639172605312608175, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.428786, "value1":0.5707960425927678, "value2":757157967003444236, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.131753, "value1":0.32321264423119905, "value2":1903644540800881525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.540623, "value1":0.528958932538034, "value2":8999603816652982043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.991648, "value1":0.7098945581193735, "value2":8146175572944228409, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.523699, "value1":0.499825459180699, "value2":1724724639775860908, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116240, "value1":0.6561999665129753, "value2":5656111967421346554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.618968, "value1":0.2848979660832576, "value2":2049018652807530472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.116765, "value1":0.8252075257277172, "value2":4775243296364010632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.410327, "value1":0.06668100857135961, "value2":2183721339943212726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.688645, "value1":0.41653256081146506, "value2":5969120448550176776, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.657571, "value1":0.9568582636724965, "value2":730216901987394123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.043883, "value1":0.40826500145612177, "value2":6407138186851676502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.474102, "value1":0.6914723313082367, "value2":469628036375832328, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.596154, "value1":0.050104900538538846, "value2":1354999585856659679, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_5":"h","key_9":"f","key_0":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.096028, "value1":0.6282483124845963, "value2":116602313358973711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.568083, "value1":0.07619222667132636, "value2":6984709326920626754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.984466, "value1":0.3323790308005344, "value2":675504136493878123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.553260, "value1":0.45304427452622936, "value2":8153841508229333988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:07+08:00", "value":0.781038, "value1":0.2512350719527022, "value2":1697664403294613401, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230875, "value1":0.8575694561008984, "value2":1340681872774498770, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_4":"g","key_5":"g","key_6":"a", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180011, "value1":0.7159258322890252, "value2":7263307563948982502, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_7":"e","key_1":"d", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318368, "value1":0.7865256606198752, "value2":7176709659967207104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.594668, "value1":0.9818689490949409, "value2":4816922436179228865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.178515, "value1":0.9620215494327841, "value2":7544435408144975228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596427, "value1":0.7405089300246521, "value2":6613451887211746808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_3":"g","key_5":"e","key_7":"g", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317317, "value1":0.8954816734384349, "value2":4023989770168105300, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_9":"a","key_1":"h","key_7":"b", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.624103, "value1":0.1565943994591959, "value2":5829676556068380878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.259139, "value1":0.08662232425939413, "value2":2137874058817765285, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.495345, "value1":0.7444165320555235, "value2":1512339128303545319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.873798, "value1":0.10364343021646215, "value2":497864294044982023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_8":"b","key_0":"h","key_3":"k", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077492, "value1":0.7825627514339433, "value2":497480348120915725, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.887058, "value1":0.6684575895211121, "value2":8461019606631895989, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_2":"e","key_7":"j", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374572, "value1":0.30885911828548585, "value2":6810974248136318973, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.381097, "value1":0.062395135046400096, "value2":1505084432772472894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_1":"i","key_3":"j", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796962, "value1":0.4796896579785329, "value2":5095295117314039911, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_6":"k","key_8":"k","key_0":"a", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.625725, "value1":0.5385278812414747, "value2":7715803409892624904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_0":"j","key_1":"d","key_8":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430762, "value1":0.42060637147418184, "value2":7474778452563228226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886443, "value1":0.6738972816294144, "value2":582629852137142515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_7":"e","key_0":"e","key_5":"d", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467512, "value1":0.5597247611878908, "value2":8349056677712047690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301942, "value1":0.5324107668792922, "value2":2601407711369777027, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.959260, "value1":0.3409466272260335, "value2":495480232494613355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_5":"i","key_7":"g","key_8":"e", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824942, "value1":0.690980654161434, "value2":73040149987568706, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647608, "value1":0.9936863999178829, "value2":2595521887523034584, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206529, "value1":0.7042253871626942, "value2":6786989253298041563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654900, "value1":0.04874287974341856, "value2":6080516591012621533, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.192783, "value1":0.636728117953383, "value2":7620028626322794241, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.757400, "value1":0.3932217938599448, "value2":4695382142084123513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578308, "value1":0.7154241327471093, "value2":8875337832548314386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_2":"g","key_8":"c", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025767, "value1":0.4896117284329862, "value2":7827833960384552450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695043, "value1":0.4735636518198123, "value2":5776645863491585306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_0":"f","key_5":"c","key_7":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067392, "value1":0.2568532613491557, "value2":4347058259737260198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145851, "value1":0.867279626694445, "value2":7767138234344286903, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.491824, "value1":0.43935171533716344, "value2":2117420540525504913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413343, "value1":0.7815374349474034, "value2":7674175325619501370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_3":"d","key_8":"h", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408821, "value1":0.6468318207569199, "value2":3382854605353241895, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864749, "value1":0.5018653748831062, "value2":3163263403613237522, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_5":"e","key_7":"h","key_9":"g", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811002, "value1":0.1264068238414509, "value2":8051527093894333669, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922550, "value1":0.797889997967527, "value2":4244570533853409024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_0":"c","key_2":"j","key_6":"i", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045039, "value1":0.6136304375492249, "value2":7079529098273717788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_4":"i","key_0":"d","key_1":"g", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914685, "value1":0.6884426743061036, "value2":3608727831529868858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_3":"k","key_4":"f","key_9":"g", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.429556, "value1":0.37532682108769366, "value2":1177251454911069616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_0":"b","key_1":"b","key_6":"f", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779878, "value1":0.5585578516583662, "value2":5769602877819666427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_0":"c","key_4":"f","key_5":"e", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.711503, "value1":0.3057843361151979, "value2":2044462261279390237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_9":"e","key_0":"d", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142192, "value1":0.0045630377024844515, "value2":4953964644129769622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.542755, "value1":0.236125279127913, "value2":5065525208816588827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578196, "value1":0.8156211595242715, "value2":1914083289002989297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_2":"k","key_4":"e","key_6":"h", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110649, "value1":0.26711648319193754, "value2":2012127151944954874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_1":"e","key_9":"b", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041387, "value1":0.3776450082432283, "value2":6801275370589593726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_7":"h","key_2":"g","key_5":"i", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808716, "value1":0.04169212753399746, "value2":9054979711039988827, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062171, "value1":0.7218616273097948, "value2":740018633881444642, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146630, "value1":0.5003843971030013, "value2":1655397867016219461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.309989, "value1":0.11293107623655986, "value2":3732853355411608797, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.333206, "value1":0.17591880468368, "value2":8123233873446258834, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145427, "value1":0.43545559910718556, "value2":4862315899801656213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_3":"k","key_6":"j","key_7":"j", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.644901, "value1":0.9950957928656415, "value2":4691155465042626084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_5":"a","key_6":"c","key_9":"i", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568884, "value1":0.6163869833398097, "value2":9217755948904745777, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_0":"k","key_5":"k","key_9":"c", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930812, "value1":0.3180893474971966, "value2":8238800974641951308, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.148564, "value1":0.7643083780125881, "value2":8977913426229482254, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369460, "value1":0.37462703037399614, "value2":2532444456287017290, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134410, "value1":0.7407330348790686, "value2":2290478450455253969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.623624, "value1":0.23159512407081487, "value2":39715061290046297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497315, "value1":0.9059175925838082, "value2":9042312906596816530, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.591581, "value1":0.9196881162354394, "value2":3577426841534493512, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_1":"j","key_4":"a","key_7":"k", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.229214, "value1":0.677998841646216, "value2":4155628140480575006, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_8":"f","key_1":"c","key_6":"f", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419533, "value1":0.14115394562090194, "value2":4482838294646117015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_4":"j","key_7":"f","key_0":"g", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352495, "value1":0.614805207196906, "value2":6313920398652894644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158009, "value1":0.13136879968484017, "value2":1444276749086955965, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_4":"d","key_9":"a","key_0":"c", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394523, "value1":0.390863430258994, "value2":1479817062186057883, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084220, "value1":0.8798339682629527, "value2":5374208931949760111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.604039, "value1":0.6734519068343852, "value2":6936119518718549917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.407184, "value1":0.883448918218706, "value2":2783824499389725952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_7":"f","key_2":"k","key_5":"c", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.987554, "value1":0.04159873743578623, "value2":6347778463017159524, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_0":"g","key_3":"g","key_9":"k", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015322, "value1":0.9633747370317121, "value2":886249751055268990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903730, "value1":0.2538648955540415, "value2":5257877330086190982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109241, "value1":0.3895615184638186, "value2":2398924749380096990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_0":"d","key_3":"d","key_6":"c", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946726, "value1":0.8408201754901692, "value2":5956645672957756887, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.126734, "value1":0.8247534822394186, "value2":8919709316239049787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_6":"f","key_2":"k","key_4":"b", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.419044, "value1":0.20899742823905534, "value2":9208355419350776806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354221, "value1":0.38324817137377604, "value2":7243408993217101630, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.620584, "value1":0.25505110416085736, "value2":2000013463174956675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306280, "value1":0.24053747539237172, "value2":104074303789974504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.694994, "value1":0.14347374003976163, "value2":2424092953659426107, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399682, "value1":0.5974894049526064, "value2":6226843809112311375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.244602, "value1":0.4472550716634916, "value2":6661706815157471152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154170, "value1":0.15894047878423925, "value2":1910278084951118791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.338581, "value1":0.8000650933715514, "value2":932928130299819996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385782, "value1":0.2346060605687393, "value2":1779773261165382974, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.925102, "value1":0.31969179251919677, "value2":2628705870105871940, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783196, "value1":0.47412332370344556, "value2":770542511728920904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_3":"e","key_4":"f","key_9":"j", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.779626, "value1":0.6092963005254489, "value2":916209608160555146, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_9":"g","key_3":"g","key_6":"h", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194805, "value1":0.5004447430389768, "value2":4252088119108151132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_3":"j","key_5":"i","key_9":"f", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.758680, "value1":0.486921957595336, "value2":989531111026827083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477327, "value1":0.1584089004594205, "value2":8696347531856094472, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162006, "value1":0.537289345346911, "value2":2077753746787895762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613845, "value1":0.3033370157767779, "value2":7793623951886359033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.373236, "value1":0.3796012074348545, "value2":6115758866481998084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_9":"b","key_4":"h","key_5":"e", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080535, "value1":0.47198866208030016, "value2":6615627174770526167, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.401469, "value1":0.5986326083333331, "value2":2667038551831095428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605284, "value1":0.7390739400172244, "value2":361713221830612946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.543913, "value1":0.568574574470533, "value2":8152708069238595670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262409, "value1":0.058861415000961634, "value2":5954875590777239919, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_3":"i","key_7":"c","key_1":"g", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056599, "value1":0.20694947175492623, "value2":3664736464652868067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_3":"j","key_4":"g","key_1":"c", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.557868, "value1":0.04995052697654973, "value2":2073326631524402431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613501, "value1":0.31045156583158495, "value2":8926511362033688194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_1":"f","key_5":"e","key_6":"h", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.047167, "value1":0.7509456855802353, "value2":846763672875326934, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362486, "value1":0.30051878680197913, "value2":8317722150065418470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.669592, "value1":0.6148907714481391, "value2":3070072928838789844, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_8":"c","key_0":"b", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061749, "value1":0.4026807862923137, "value2":2054834671958607537, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_7":"c","key_8":"e","key_1":"h", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077647, "value1":0.02456732222252374, "value2":4163774252593358275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439333, "value1":0.49417199138729123, "value2":8947661323810810369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.592816, "value1":0.25731799708888187, "value2":5220216189538554324, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.344893, "value1":0.0022390888763635323, "value2":6178831795736016910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_5":"i","key_8":"h","key_4":"e", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350091, "value1":0.39159828262162255, "value2":3395049172404740462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417133, "value1":0.19779968591074149, "value2":3982459725160448681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313282, "value1":0.5954778061697951, "value2":2609201495243589975, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416054, "value1":0.6479875646988947, "value2":6379744126737729174, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416195, "value1":0.9737347842892922, "value2":3437414765355850053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_0":"k","key_1":"k","key_9":"f", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.351132, "value1":0.5274165691272555, "value2":5569886633354264071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_8":"e","key_0":"a","key_6":"h", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273952, "value1":0.4616579953521536, "value2":4410692243343844144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948309, "value1":0.21495634970202532, "value2":1337568920394438603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.120015, "value1":0.6827231094645871, "value2":4910087454372528211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.220281, "value1":0.5451953602332654, "value2":5295885271016141901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222991, "value1":0.38652141757520153, "value2":611291833871236608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432311, "value1":0.4952506069055437, "value2":6235358954173265246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_5":"k","key_2":"g","key_4":"g", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.305907, "value1":0.7399971630668104, "value2":7853724941674753946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.701437, "value1":0.07448132935373361, "value2":1676692481709109964, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233939, "value1":0.9922833997352541, "value2":8485630674384935881, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.196307, "value1":0.3811609484351306, "value2":6762437704329339708, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.598701, "value1":0.7218662020742757, "value2":5746405219362751126, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535319, "value1":0.470769974287709, "value2":5160211896753607127, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455357, "value1":0.41061237500976755, "value2":3731616592511902245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_4":"h","key_6":"c","key_1":"i", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682037, "value1":0.02412728786773881, "value2":8461839080730070447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_4":"i","key_7":"j","key_3":"j", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618426, "value1":0.6208202835196076, "value2":3667062185468357188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094952, "value1":0.9317381860791135, "value2":8865888160428380876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295576, "value1":0.034878221959591026, "value2":4553331116235737600, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605252, "value1":0.5687954362360221, "value2":4628895813773640059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_0":"i","key_3":"a","key_9":"i", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.167973, "value1":0.8144348491443895, "value2":3884298273747196831, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_4":"a","key_1":"k","key_2":"h", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803115, "value1":0.14234455011711838, "value2":7333273581911274613, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809534, "value1":0.31251923098668316, "value2":5000940696241059800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.504196, "value1":0.37041029395255115, "value2":7518926478801994928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886399, "value1":0.2510698129904212, "value2":1686947485208902024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322861, "value1":0.6398715658295313, "value2":5260107554586637670, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_4":"g","key_6":"d","key_7":"a", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.575248, "value1":0.8821568194363127, "value2":2676120924049029771, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.924938, "value1":0.6642600755588765, "value2":9154692137110805453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_7":"f","key_9":"j","key_6":"e", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534144, "value1":0.2220263150896839, "value2":7296067315122241408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680757, "value1":0.36579467004129973, "value2":5336171210655722369, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.361476, "value1":0.42049076121425283, "value2":3694543289414649508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.423620, "value1":0.9226817255397154, "value2":5312418306728579961, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_4":"b","key_7":"h","key_1":"a", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.398746, "value1":0.4448382205324241, "value2":5199222544329766041, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.492975, "value1":0.6490408315820692, "value2":3620354939805403786, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.732787, "value1":0.26280435119702705, "value2":7186318610805553949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385151, "value1":0.7991171293820531, "value2":1271579952216532684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.067778, "value1":0.2653370821341721, "value2":8977665444967590904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_1":"d","key_9":"d","key_0":"a", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941069, "value1":0.7995772580327314, "value2":7272267767125923064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.547209, "value1":0.7653567065104109, "value2":6328078604409399184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_3":"b","key_6":"d","key_7":"a", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649757, "value1":0.49061605503159605, "value2":5991812000082844933, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.017745, "value1":0.8857176370602151, "value2":8890483656354164816, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478915, "value1":0.49344955581785127, "value2":6898479319738793663, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_1":"k","key_2":"d","key_9":"g", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.600610, "value1":0.1145004603393322, "value2":7934326719175266154, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_3":"c","key_9":"j","key_0":"e", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752045, "value1":0.7917363851798865, "value2":6655960268552880542, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.198446, "value1":0.8915785759457425, "value2":6424163196395537238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_2":"g","key_3":"e","key_8":"b", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415881, "value1":0.6309493378355412, "value2":623919229201632317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_6":"c","key_1":"d","key_5":"g", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.652344, "value1":0.6344286572245156, "value2":9189348363675107233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774566, "value1":0.042498471702267726, "value2":6073051427209338110, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.918626, "value1":0.4530903546971259, "value2":5667043007795158116, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_3":"g","key_4":"d","key_2":"c", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.340057, "value1":0.13032389853235377, "value2":2621616876191250318, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_4":"i","key_1":"k","key_3":"d", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363370, "value1":0.25548611447808117, "value2":7869413584380375849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_5":"i","key_6":"i","key_0":"f", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316114, "value1":0.7913570973054609, "value2":714341173131110534, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.726210, "value1":0.6340822577426605, "value2":7078070670524096081, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809728, "value1":0.31189335942918994, "value2":2593764393145326151, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_3":"i","key_0":"e","key_2":"b", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828156, "value1":0.2513167249865656, "value2":4199156936630630581, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074063, "value1":0.6299866633999338, "value2":7375082593010468605, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551341, "value1":0.5634049844701874, "value2":553811382543796103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395240, "value1":0.9163098471877911, "value2":2102676899677978041, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812285, "value1":0.5658853008127968, "value2":860920050673825541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.699745, "value1":0.6686081106918065, "value2":4484995258456828647, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862811, "value1":0.006750603964471356, "value2":8966268587906792618, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649656, "value1":0.33455087075476775, "value2":3638358249838833627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_7":"h","key_2":"b","key_6":"i", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978094, "value1":0.9765011127569251, "value2":5363905042672157366, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824326, "value1":0.5852292268881885, "value2":8525521016560267029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972834, "value1":0.2713220476194647, "value2":1730194033904802699, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.513347, "value1":0.20240183224033417, "value2":3586323241046235606, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_7":"f","key_9":"b","key_3":"a", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165801, "value1":0.25754092027876374, "value2":3897618512711030520, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245034, "value1":0.276430555426299, "value2":7235576154009687664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209920, "value1":0.33218423022426385, "value2":6956800146565179953, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_7":"h","key_1":"k","key_4":"g", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802797, "value1":0.9498551679797749, "value2":4204819498986608868, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.002994, "value1":0.33304452518131644, "value2":299998551436331074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.136076, "value1":0.58569890261748, "value2":8821512503402703105, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605724, "value1":0.3779582962124763, "value2":4381826070482821493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_2":"e","key_4":"c", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.664393, "value1":0.7508643438704989, "value2":9133405472026903918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236332, "value1":0.4456796524317141, "value2":3982698031313790805, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_0":"f","key_6":"a","key_8":"f", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320127, "value1":0.5073225371595068, "value2":2378617268397591284, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_2":"h","key_4":"a","key_9":"e", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865511, "value1":0.36097655561074876, "value2":2650611720712784810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777541, "value1":0.27613383935735863, "value2":3133546266043557517, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647086, "value1":0.09971158325173911, "value2":7192486223907690883, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_9":"j","key_2":"f","key_5":"j", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.172420, "value1":0.6050664492556429, "value2":3816871624777837698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639560, "value1":0.40402358999541305, "value2":7105433557941999346, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_3":"j","key_5":"b","key_2":"k", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.945205, "value1":0.38392410498955126, "value2":1178256579293465746, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_0":"k","key_1":"e","key_3":"g", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277675, "value1":0.3660884472168037, "value2":6046887408166693464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219346, "value1":0.5871741254566942, "value2":2616399527910270141, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116739, "value1":0.7713481330592996, "value2":7718152730102741547, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.951685, "value1":0.6692969608681443, "value2":6198709686728043657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.880602, "value1":0.3938915734742536, "value2":5600053552458054507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.831337, "value1":0.47065388482990184, "value2":5918647530397871720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865810, "value1":0.9557189505427555, "value2":1533596579704888254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_5":"a","key_9":"j","key_3":"b", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.578483, "value1":0.3164962822621824, "value2":4784731554506893709, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.158191, "value1":0.36054084773482775, "value2":6343389237742355198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_0":"f","key_1":"h","key_2":"f", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761167, "value1":0.9149570761492709, "value2":6140964050711145447, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_1":"h","key_4":"a","key_9":"h", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.955044, "value1":0.34237203961558577, "value2":7321019711278117639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137258, "value1":0.2734770926499478, "value2":5312006546786938240, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876289, "value1":0.584796985987568, "value2":2146521016660183394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_5":"e","key_7":"i","key_3":"a", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.345388, "value1":0.7891673161299368, "value2":9211813724293662180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.291358, "value1":0.5443723430932246, "value2":1305196521853642686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107992, "value1":0.33317636728859645, "value2":4303995107486054548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.065010, "value1":0.792101797297302, "value2":5624493167148885444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.512622, "value1":0.026747806334027496, "value2":31336873860845918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796113, "value1":0.3713464074931213, "value2":4041884710949108537, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_5":"b","key_9":"d","key_4":"k", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.794333, "value1":0.5998279188033826, "value2":2263989663686243457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930716, "value1":0.7150630462147562, "value2":5925307983835648806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_1":"a","key_5":"k","key_7":"h", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.631145, "value1":0.12175047903656237, "value2":2424100894887965716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889832, "value1":0.33399829473514187, "value2":3091557798470852457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.036666, "value1":0.8679594907099448, "value2":6691229273816200596, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_1":"h","key_2":"f","key_7":"f", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.853608, "value1":0.5774160801209688, "value2":980449146620020072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.356874, "value1":0.6962074391730027, "value2":2178872746047471215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162362, "value1":0.6573318971529195, "value2":8364158949095374627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.110201, "value1":0.41287549892040376, "value2":7444688864434046520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_5":"f","key_8":"e","key_1":"h", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.441462, "value1":0.960708146788197, "value2":7272057802651917640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.020362, "value1":0.14970758198915565, "value2":5476649529178658344, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434430, "value1":0.7114147585195696, "value2":6784565480587871276, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795723, "value1":0.9222061229277446, "value2":5415396163282444341, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.092199, "value1":0.0594949540077623, "value2":733783646287392653, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.277448, "value1":0.36348546035264123, "value2":2527022975440439707, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607130, "value1":0.21203009019347513, "value2":3167160934083348057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424117, "value1":0.42561548619630274, "value2":439532732667459958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365073, "value1":0.8492201961187499, "value2":1130712017318335183, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248310, "value1":0.6476029521632828, "value2":5228653953193583715, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_2":"g","key_4":"e","key_7":"a", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744619, "value1":0.5713572042259071, "value2":1575714640192742698, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.026559, "value1":0.543673726494076, "value2":3134877395946459112, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972610, "value1":0.09611280545479016, "value2":4984229756632609213, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_7":"e","key_9":"d","key_4":"c", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763906, "value1":0.8517215040951974, "value2":6804317567810698432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_4":"d","key_8":"d","key_9":"c", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105838, "value1":0.8805288612544102, "value2":7653928454276555905, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383435, "value1":0.3683543847634455, "value2":2147384927542025505, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027377, "value1":0.4869984514514779, "value2":6620303412327477269, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_3":"b","key_5":"i","key_2":"h", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.417188, "value1":0.24989866283446016, "value2":7011787964949226310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084529, "value1":0.9844535394576189, "value2":6876673068454117053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032097, "value1":0.6712678103298776, "value2":3320740930339209104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.394244, "value1":0.7796841432655915, "value2":4446257418849643255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_4":"b","key_9":"c","key_3":"g", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.828297, "value1":0.2648442239286722, "value2":25756618593200917, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_6":"f","key_9":"g","key_2":"g", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731687, "value1":0.07632667820503707, "value2":5775744844960229233, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_3":"a","key_5":"e","key_7":"f", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.348738, "value1":0.4462284223326741, "value2":7761375931867531240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817431, "value1":0.9846258074003919, "value2":5377182091927671364, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603703, "value1":0.1579124346010748, "value2":7829825462713051842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743604, "value1":0.012350717590309025, "value2":3299133246704731924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_7":"a","key_4":"i","key_5":"b", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.941914, "value1":0.37406772316207776, "value2":3581449411399106092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_5":"a","key_6":"d","key_3":"a", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.323291, "value1":0.2629524147159168, "value2":2560363092546342475, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_9":"j","key_0":"g","key_8":"j", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.206164, "value1":0.1605470780394023, "value2":1476326866755963525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376246, "value1":0.8410819090377242, "value2":6697199584394109087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_6":"c","key_0":"d", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.438458, "value1":0.05246539756995679, "value2":2689405601803288451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_2":"b","key_3":"e","key_5":"c", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.807203, "value1":0.730536792123947, "value2":7996980666084265973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.842861, "value1":0.2085319390948787, "value2":750997239048285659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.097539, "value1":0.943951867959508, "value2":8362598506758468383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.363032, "value1":0.5870242519277287, "value2":6264967362494680685, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_2":"c","key_4":"g", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212907, "value1":0.4603451527715893, "value2":4171196630516497508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_1":"d","key_8":"d","key_0":"a", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177573, "value1":0.8948006671577682, "value2":5720852449593066754, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_5":"k","key_1":"e","key_3":"j", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286908, "value1":0.7373750433446898, "value2":2889582543857330611, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.780354, "value1":0.11954278924179859, "value2":206386290438050068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_0":"g","key_2":"g","key_7":"i", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744582, "value1":0.5414546611545269, "value2":1573274748881736123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.262127, "value1":0.8114751808169933, "value2":1156734310814584450, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_6":"h","key_9":"f","key_0":"k", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.627688, "value1":0.05870166177597653, "value2":8802667963431750791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_3":"i","key_5":"i","key_0":"k", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432936, "value1":0.5733149806402271, "value2":8971566642401975801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252007, "value1":0.6150552921263627, "value2":6524226679068869341, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.230350, "value1":0.8727510444388746, "value2":5913218581367894399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_4":"c","key_2":"j", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.989663, "value1":0.8376111097555229, "value2":2874987014333615633, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_0":"j","key_2":"i","key_6":"f", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.435356, "value1":0.7022017770410863, "value2":6778124246365180945, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_6":"a","key_2":"b","key_3":"a", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673069, "value1":0.2749380629969126, "value2":1514980079784151790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328789, "value1":0.865778242569301, "value2":384106145681984426, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.986292, "value1":0.9273192971072537, "value2":6526516407617182264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.240749, "value1":0.7234109649728769, "value2":6790075567204476237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.774780, "value1":0.5871689076780541, "value2":5434961575452625626, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.406076, "value1":0.7420142270194664, "value2":4969819996381131142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_0":"b","key_3":"b","key_7":"e", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.146123, "value1":0.2540849065850774, "value2":4942775185551508816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.928078, "value1":0.7980654446229606, "value2":2117433885577041648, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933008, "value1":0.05348899498806063, "value2":7864335047299894108, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655622, "value1":0.7668514308351416, "value2":5108933691323452489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_4":"b","key_7":"e","key_9":"k", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799422, "value1":0.8658511508098131, "value2":1584220631636831527, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153780, "value1":0.5100425920946189, "value2":6860613724470124155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.832729, "value1":0.6352815444925497, "value2":6914920278709539881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_3":"a","key_4":"g", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.862121, "value1":0.40978666741053005, "value2":8297372411728279444, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_2":"i","key_6":"f","key_0":"a", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.046814, "value1":0.6860769857954689, "value2":7207194751565200226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.106175, "value1":0.9141202615431735, "value2":8093659818506693875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.568924, "value1":0.2818708416167068, "value2":4682897128638978687, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.743387, "value1":0.19055511730870822, "value2":6005851706828394010, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.131634, "value1":0.786038669470468, "value2":2052265664401672629, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.864417, "value1":0.6509022777920431, "value2":6405639625996257390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769553, "value1":0.3306779516460208, "value2":63644576329067046, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705994, "value1":0.39211177275150355, "value2":6019747875631249901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.085415, "value1":0.989053918126363, "value2":8753375624734757247, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.966642, "value1":0.146948164341443, "value2":7640844254824954510, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_9":"k","key_5":"b","key_8":"b", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.548525, "value1":0.7240074879750936, "value2":8956195213201860326, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073937, "value1":0.7314322966939647, "value2":254693443211564548, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347726, "value1":0.8209138159763902, "value2":6620059103742183058, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.378046, "value1":0.5808416882480918, "value2":6139629334504131344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_6":"a","key_1":"d","key_4":"a", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797601, "value1":0.7784388478906132, "value2":354987022131004184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_0":"c","key_2":"j","key_6":"k", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411013, "value1":0.45158233474379805, "value2":6507365193047028596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.152084, "value1":0.34791281429560394, "value2":2152822969747742943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808545, "value1":0.6451074006535574, "value2":4535693696526329424, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735815, "value1":0.5617099301887051, "value2":4545722791278282166, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_5":"d","key_8":"f","key_9":"i", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170237, "value1":0.7594425432841077, "value2":2121502996639945525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_8":"d","key_4":"j","key_6":"b", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839462, "value1":0.42088416942799856, "value2":4668152333347445855, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_9":"i","key_0":"a","key_4":"c", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977985, "value1":0.9874529650666419, "value2":7279887982544559299, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_0":"e","key_4":"f","key_9":"f", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777993, "value1":0.32638782808208466, "value2":7945394269696237457, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_1":"e","key_4":"d","key_7":"f", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931446, "value1":0.5421003602914947, "value2":7796969745729042644, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746112, "value1":0.6059100687669938, "value2":8599287190529748569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.950726, "value1":0.8607618900249634, "value2":2432626342683975270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_2":"f","key_4":"c","key_9":"g", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.647237, "value1":0.7468406151868747, "value2":7200054131729412480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.460709, "value1":0.4549015015070196, "value2":1628296475817521290, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_5":"d","key_0":"b","key_2":"f", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007652, "value1":0.3132830085813196, "value2":1943034905431272670, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_2":"b","key_3":"e", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.103043, "value1":0.029190956951399328, "value2":5218539584338803865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506730, "value1":0.4547431011729936, "value2":3644260335220928016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_3":"h","key_7":"c","key_2":"d", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564064, "value1":0.2217922532972611, "value2":3866244575842378014, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.308096, "value1":0.4735939830858083, "value2":6374663223385153398, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415080, "value1":0.9760882060049622, "value2":1970837347852222146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_0":"b","key_6":"d","key_7":"k", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909084, "value1":0.6294752667120725, "value2":9199477369979291397, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.969935, "value1":0.10788341374154452, "value2":7002111391558217482, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_0":"j","key_1":"k","key_5":"f", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.622182, "value1":0.21938176298467404, "value2":4706131254461339381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_9":"c","key_4":"g","key_7":"b", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.722928, "value1":0.21266609523973992, "value2":5957309419507426661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.062556, "value1":0.1304962392242022, "value2":4535950458577175850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_7":"h","key_8":"d","key_6":"h", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.056693, "value1":0.6559280448597679, "value2":6798153078557023165, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_1":"j","key_9":"j","key_0":"a", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.275009, "value1":0.8708722343930831, "value2":2992233127991751043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027324, "value1":0.8089539697327325, "value2":6582915861289307476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769519, "value1":0.6866136235615642, "value2":3188730142010270506, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_1":"h","key_3":"g","key_9":"b", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471216, "value1":0.036922660768241786, "value2":69160799499838231, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_7":"d","key_1":"j", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.289160, "value1":0.9707556984503585, "value2":3226749743394249266, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719377, "value1":0.6150053305666618, "value2":8057451250881285379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_3":"b","key_5":"a","key_8":"g", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884188, "value1":0.32970677783871427, "value2":6299179245210453660, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_0":"d","key_5":"e","key_8":"i", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.128583, "value1":0.46621614630474323, "value2":3810272679777334373, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_1":"j","key_8":"k","key_9":"a", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.016304, "value1":0.07070759725845735, "value2":3860287779798372481, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556253, "value1":0.3162226930220112, "value2":811964996210240316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.662220, "value1":0.04332778395279835, "value2":1557538607744070859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713709, "value1":0.6625325269051697, "value2":4541028658322459718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297119, "value1":0.15013885945438676, "value2":2149756203418913335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685110, "value1":0.9036182355788681, "value2":4377728436719035164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846262, "value1":0.34678741602681534, "value2":2162478118726349508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410862, "value1":0.11378608528656459, "value2":614293064254453793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.060980, "value1":0.22871224178742908, "value2":4099107042293178536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960058, "value1":0.5820948808821428, "value2":1201783870513585432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.214885, "value1":0.03550184669484978, "value2":2051901639161172531, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696549, "value1":0.44298333983599963, "value2":8432193022117819833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368813, "value1":0.6629289022292644, "value2":6617784174875177478, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_3":"h","key_4":"k","key_5":"e", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313857, "value1":0.15526487291119703, "value2":95708758682513207, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744683, "value1":0.6423949710674116, "value2":3611912809247643043, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_0":"f","key_7":"k","key_9":"d", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.786335, "value1":0.589184376420666, "value2":5802753188533704826, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_9":"a","key_0":"a","key_5":"b", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930246, "value1":0.20872893267238513, "value2":2224049910905425698, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_1":"j","key_2":"e", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515204, "value1":0.20960234541456155, "value2":7647445186122035759, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.800072, "value1":0.20249832362898804, "value2":1073544363493262212, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_0":"i","key_8":"d","key_9":"b", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.420652, "value1":0.651081573013809, "value2":355187443325178523, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760554, "value1":0.28915917130829083, "value2":5965361551796616223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.852984, "value1":0.45938574348035566, "value2":2528322122553170113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_6":"b","key_8":"a","key_9":"g", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352738, "value1":0.900978009547613, "value2":3975764535045467018, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066294, "value1":0.08305245567845902, "value2":2394249365413597097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.654208, "value1":0.08608535610142205, "value2":2036163695122831466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_7":"j","key_9":"c","key_3":"h", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.988436, "value1":0.04790022290628863, "value2":6418792057544120186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.884316, "value1":0.5495267159618948, "value2":4438533595454630352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.154168, "value1":0.4348957861418331, "value2":8838135202914157189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874389, "value1":0.42584041610286777, "value2":2273157634255144393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_8":"j","key_9":"a","key_5":"k", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431882, "value1":0.06486477270973905, "value2":3345519389285150964, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.685508, "value1":0.5499323474722744, "value2":5054596449098633988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_3":"d","key_5":"e","key_0":"j", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.660995, "value1":0.17858690508054326, "value2":8567568075077444285, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271767, "value1":0.8266242576408339, "value2":8856688759394626086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755581, "value1":0.7494066282765741, "value2":4893716534306029427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171573, "value1":0.6765014228252123, "value2":5524113092747902135, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269557, "value1":0.24446627014298006, "value2":2739808324008179094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.967939, "value1":0.5539434802495194, "value2":3700007071044991493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.656891, "value1":0.9116275072171186, "value2":7153356187047849001, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.248386, "value1":0.09058562459659583, "value2":7526211020899452764, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_6":"e","key_0":"h","key_3":"f", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.948164, "value1":0.6887494090196015, "value2":4600657675678245799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_2":"e","key_7":"k","key_8":"c", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228675, "value1":0.8494478798473957, "value2":317720425357796205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101265, "value1":0.8190567291465818, "value2":4663927178372352248, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820988, "value1":0.775820207166787, "value2":6038722642101305434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922087, "value1":0.22553959289224842, "value2":779713635671076382, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.888330, "value1":0.020416312330820526, "value2":7204412129555533981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.290446, "value1":0.9852816320447644, "value2":4592385145364652482, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.679824, "value1":0.17818186899277277, "value2":252209780074899647, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686778, "value1":0.8339304122124636, "value2":3658007614464097080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257632, "value1":0.614764442927346, "value2":4642146906724062657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_8":"b","key_2":"c","key_4":"g", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482850, "value1":0.18190770107834264, "value2":8954437725061891309, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998993, "value1":0.7044002667070637, "value2":431084763192513731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.875898, "value1":0.7812743007979214, "value2":2705148521595588507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084211, "value1":0.8180424469178956, "value2":7499444983471620842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053317, "value1":0.3379873516583276, "value2":5874640652872975301, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_8":"i","key_1":"k","key_2":"b", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746275, "value1":0.11387537552973875, "value2":4778407492588259899, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744142, "value1":0.945713466310749, "value2":3761848793498692794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.274112, "value1":0.14967892390238677, "value2":3777191830426872225, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.680316, "value1":0.5765506588965501, "value2":3913164426928614622, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.582474, "value1":0.13820731268553305, "value2":1359306396719437577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.572068, "value1":0.9236046727786195, "value2":731327351641596838, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.838138, "value1":0.5915185036534663, "value2":4354316161210605047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755088, "value1":0.18889070665371863, "value2":4180101489297988878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_7":"j","key_2":"h","key_4":"k", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.362663, "value1":0.4708250834755644, "value2":8678758888644206492, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_0":"e","key_5":"d","key_8":"i", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.464079, "value1":0.13171451384673036, "value2":8187780706682520691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.839674, "value1":0.16354088623961133, "value2":8461213608449952033, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777782, "value1":0.005150900343416017, "value2":9058250336565790520, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.520149, "value1":0.6583211185838453, "value2":179671992454725443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_1":"b","key_3":"b","key_6":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.354725, "value1":0.26618356833133044, "value2":4502556297298851018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.588275, "value1":0.40556503705171876, "value2":1831243556070013492, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.991768, "value1":0.5964588343272011, "value2":7216754835822865383, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.829272, "value1":0.8185283045838991, "value2":7054142095046490626, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028671, "value1":0.9570557166393036, "value2":5667948933240210255, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.171455, "value1":0.3984068015975904, "value2":6709012624962450094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112456, "value1":0.8897199020551853, "value2":619757189037583586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_1":"c","key_9":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.366328, "value1":0.9348857036272434, "value2":4207327317390530443, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.787818, "value1":0.06046424642679486, "value2":3800391541062933624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_9":"f","key_0":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431436, "value1":0.5742437522880309, "value2":3338797017937456724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_0":"g","key_3":"b","key_8":"a", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.130697, "value1":0.8564226707318178, "value2":2563408398646331614, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_4":"k","key_6":"j","key_3":"k", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531522, "value1":0.21916844982697795, "value2":6377428321672110838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799330, "value1":0.2117471404567077, "value2":6841854064863394582, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.415853, "value1":0.7458907419184496, "value2":3858225758706811596, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769460, "value1":0.9292446985209156, "value2":4294237114969210404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.053422, "value1":0.19156586881852822, "value2":8401149164501095807, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.634910, "value1":0.13000039932274055, "value2":4131562742009700028, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703579, "value1":0.6862332441253757, "value2":7454437023717068913, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327549, "value1":0.40704854485658054, "value2":4932202555006075846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.314755, "value1":0.5681370276361754, "value2":3506285901156371358, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.466686, "value1":0.42167473519309434, "value2":3558723778128133952, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160258, "value1":0.4575847781643395, "value2":6023651621402119686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_2":"f","key_5":"e","key_1":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.228508, "value1":0.5111004373311687, "value2":8895687587282710909, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.111511, "value1":0.09506770522467269, "value2":2655655566098996088, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.784900, "value1":0.5435091084418109, "value2":5760921829736802814, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210512, "value1":0.25861533775417506, "value2":6914577108338158111, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_7":"c","key_9":"g","key_2":"c", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.663100, "value1":0.8688427076237359, "value2":9173486087944898117, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.066304, "value1":0.7818969735770589, "value2":4913317894269750379, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_1":"j","key_3":"h","key_4":"h", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824798, "value1":0.9535971366603737, "value2":8366422359718269509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.090470, "value1":0.7644445079507416, "value2":6086882240277966349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994994, "value1":0.046461773505700306, "value2":1146509415122743778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409080, "value1":0.028382316546416403, "value2":2068020782874554558, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_1":"b","key_4":"e", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.815596, "value1":0.08796047569888332, "value2":8378736063122737266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683447, "value1":0.24029660357741323, "value2":8797505888002523812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.611937, "value1":0.2489739097238809, "value2":7210056341279458271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_0":"b","key_4":"j","key_9":"a", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.347662, "value1":0.7553826852514858, "value2":8429002700088766063, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_8":"i","key_0":"e","key_5":"g", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318289, "value1":0.5406931125323163, "value2":1745967906139345488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.900516, "value1":0.7025144762707813, "value2":456260840471493385, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.153699, "value1":0.20283897976623247, "value2":7209402260011396171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_0":"i","key_1":"a","key_5":"c", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.813264, "value1":0.23005364054850463, "value2":4244429874559011543, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_0":"b","key_3":"d","key_8":"a", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.174420, "value1":0.6527990870752745, "value2":5344843801392876512, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.142219, "value1":0.1627182050289727, "value2":114175501496781641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_9":"d","key_5":"e","key_6":"c", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.481454, "value1":0.37374721367966557, "value2":1638215468267960931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.523380, "value1":0.48736316265440816, "value2":4581537275255860768, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_1":"j","key_4":"c","key_9":"j", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317190, "value1":0.0039195876566355085, "value2":5341281001441055802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619251, "value1":0.5258022392764804, "value2":4434264026251453750, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.641615, "value1":0.33039615993236077, "value2":6928858155661176736, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961004, "value1":0.5399790911878092, "value2":7046629202789685841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_2":"a","key_3":"f","key_6":"g", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025798, "value1":0.5984426914038641, "value2":6528145482673170360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.901084, "value1":0.06259887743500082, "value2":3236298745887395724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.489095, "value1":0.21376661515997028, "value2":7928954356959433352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655274, "value1":0.5385351363841719, "value2":7272624008760795422, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.082369, "value1":0.4014980361714972, "value2":7005111860489403789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519418, "value1":0.19683342336444604, "value2":1435942937723913885, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226945, "value1":0.9922423305002035, "value2":867760431487143669, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_7":"d","key_8":"h","key_3":"g", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752765, "value1":0.9916422768378336, "value2":3430696359309587884, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_3":"c","key_5":"f","key_1":"j", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.078412, "value1":0.6964630260101603, "value2":2102364069319122633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.328609, "value1":0.19123451863699675, "value2":8170554763374290069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_8":"j","key_9":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.470499, "value1":0.03919799070813115, "value2":3366292364134765186, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_8":"g","key_9":"d","key_6":"d", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740670, "value1":0.08849655325518448, "value2":1691579140130982717, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_1":"j","key_3":"c","key_0":"a", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.393865, "value1":0.061408201218711954, "value2":174448511358385380, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.200654, "value1":0.2836024427807901, "value2":4326318289877991961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970400, "value1":0.7730199010023195, "value2":8574178501642024588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_6":"f","key_1":"f","key_3":"h", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.465169, "value1":0.542725080263732, "value2":9196833379545093375, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_2":"d","key_4":"i","key_8":"a", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.563410, "value1":0.7958042531287333, "value2":7393645101828557391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_5":"i","key_8":"k","key_1":"j", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213281, "value1":0.7928629505338592, "value2":1752466947098105566, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954474, "value1":0.3032546869220648, "value2":1748556721646671927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.847068, "value1":0.7688745080204612, "value2":5187070635183292150, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251152, "value1":0.5682862581346041, "value2":2363792865069284658, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_6":"d","key_5":"f", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360666, "value1":0.08726468762579924, "value2":643126609584411238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619231, "value1":0.7434795567208692, "value2":4822313794436821661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.603123, "value1":0.10532993201555661, "value2":2544972440464847934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.607459, "value1":0.20036031311934685, "value2":1168760905394403690, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447800, "value1":0.7195092335832372, "value2":5642489879930119928, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.850819, "value1":0.6761002637886042, "value2":833307298139301862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_5":"b","key_8":"j","key_9":"d", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539724, "value1":0.5294892272320822, "value2":2667359803805326763, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_0":"h","key_3":"g","key_7":"k", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.208291, "value1":0.8393697010611292, "value2":683397536019101988, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550311, "value1":0.9963724696480404, "value2":5468577449184757808, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.023114, "value1":0.32353069050613953, "value2":9187169003922087415, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286063, "value1":0.9137524095402076, "value2":6445901279543298486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_9":"k","key_0":"k","key_3":"j", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.204901, "value1":0.4076696093660646, "value2":6677345639081163017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_6":"k","key_0":"b","key_3":"f", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.107007, "value1":0.5776778447129313, "value2":8463235586423448216, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.271841, "value1":0.6930651904253268, "value2":6787444131360802180, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.069717, "value1":0.7690475248125641, "value2":1837461702894857026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.368696, "value1":0.5454486705451115, "value2":3041446083114186683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_5":"c","key_3":"g", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.395274, "value1":0.24368851515268314, "value2":4392956019271269002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_0":"g","key_4":"b","key_6":"h", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.352289, "value1":0.22988542503215345, "value2":6550852329237611667, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.703725, "value1":0.026480349904897907, "value2":1885153320482283414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_0":"k","key_5":"j","key_9":"c", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.256528, "value1":0.26111767999722507, "value2":6808025827523544315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.252909, "value1":0.20711336158884097, "value2":3024039988582180695, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_9":"h","key_1":"d","key_8":"d", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.633970, "value1":0.07845906332760666, "value2":6511716740509653564, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_2":"i","key_6":"k","key_9":"g", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605465, "value1":0.5950957840062573, "value2":2806262404335314036, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.081867, "value1":0.08398189819376621, "value2":5728413114044088113, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_2":"k","key_8":"k","key_9":"f", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.074111, "value1":0.6218602698989528, "value2":1011556855791832544, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675019, "value1":0.11095025598530854, "value2":5207952003333828816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.738057, "value1":0.5410628771237361, "value2":2540645323761255412, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.317014, "value1":0.5259626957217648, "value2":3505614768801064603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776152, "value1":0.302147585405568, "value2":7518755412283013617, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094787, "value1":0.5814132866660779, "value2":4822671923080781538, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.517582, "value1":0.10766355589362117, "value2":8453589798026915731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_1":"k","key_4":"d","key_7":"e", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882088, "value1":0.8040313368583433, "value2":8495228225039478737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_9":"c","key_2":"f","key_6":"b", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.803773, "value1":0.22925549140141077, "value2":8938270200397652857, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403205, "value1":0.07946210415899989, "value2":7443253750975595381, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211183, "value1":0.27303249192461276, "value2":3758490705819764753, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_5":"h","key_6":"e","key_8":"g", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684989, "value1":0.19635657331096867, "value2":3768344870136296803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_6":"d","key_8":"e", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.596986, "value1":0.9569957010370201, "value2":5843413270558977752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.796083, "value1":0.4870882675033077, "value2":8040326882339558395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.674664, "value1":0.05017490820454691, "value2":1512965384940722080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.682218, "value1":0.05038298218752238, "value2":3666135732590503511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871616, "value1":0.007362621326077148, "value2":3382155619299232500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.479139, "value1":0.48744354938142825, "value2":2536302313422526956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408860, "value1":0.5030243670352939, "value2":3709214658604138842, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.079849, "value1":0.4036712273225728, "value2":2906136993668691733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.735107, "value1":0.11572752191146224, "value2":9135232014249483896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.579000, "value1":0.832018905121829, "value2":4291485121105439680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.297326, "value1":0.3053519839954546, "value2":7762295205550727215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_2":"d","key_4":"c","key_9":"c", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.077473, "value1":0.5244918634463549, "value2":3651586678818959294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211495, "value1":0.3162790532742153, "value2":8307153509301118839, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_0":"d","key_2":"i","key_5":"b", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.018730, "value1":0.4578847558537525, "value2":3121380712594178301, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.254109, "value1":0.0022335788974661955, "value2":1113680598327977122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.327840, "value1":0.38572838732788006, "value2":2539441632284031099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_2":"b","key_3":"d","key_9":"f", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.802473, "value1":0.6047951359580281, "value2":1001558191118170380, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299508, "value1":0.05862512584912177, "value2":5514618695776223541, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_5":"c","key_9":"f","key_3":"i", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.011661, "value1":0.31240922806678983, "value2":6137561733651595636, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_3":"d","key_7":"f","key_8":"e", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728655, "value1":0.1278483939203575, "value2":9035381570106713199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.667349, "value1":0.5613783650549519, "value2":596862476082819858, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.253799, "value1":0.021910250550882106, "value2":6533597878636060832, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_9":"k","key_3":"a","key_5":"b", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.288444, "value1":0.30057474196214407, "value2":3169010106041213703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.798940, "value1":0.9681683301198919, "value2":3021363800209965251, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.981962, "value1":0.9612929543752854, "value2":9171352851679704895, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_0":"c","key_1":"a","key_2":"j", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149939, "value1":0.09981385175599812, "value2":2281824057904455016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943557, "value1":0.6955975747592801, "value2":2634327756448196264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459593, "value1":0.6446822058280443, "value2":6342926358496707008, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.734204, "value1":0.6639025089026711, "value2":2476897145958348918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_0":"b","key_8":"b","key_9":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.506327, "value1":0.10053599414522371, "value2":2221394931414407943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.410931, "value1":0.8059701840141592, "value2":3838253097677401347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.964398, "value1":0.603279381255061, "value2":5782827613318848456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_3":"f","key_0":"f","key_2":"d", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.278202, "value1":0.7115230775467353, "value2":8371900549612489852, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408547, "value1":0.21860759782537603, "value2":8039563012338877282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.261349, "value1":0.11227782989658946, "value2":7783215654484926349, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059288, "value1":0.07895743947800146, "value2":5667050368393356509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.601079, "value1":0.6482240475140798, "value2":4251195673123293243, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_1":"e","key_5":"g","key_9":"g", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.225112, "value1":0.7216063470142905, "value2":148937314023613788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.977412, "value1":0.6598247780794319, "value2":5965401930753401354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194225, "value1":0.6141698639690517, "value2":8817766861356540801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.597081, "value1":0.07693962991598453, "value2":5029342104400232103, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.335617, "value1":0.5583831809146028, "value2":1406686773640908295, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_1":"f","key_2":"j","key_4":"d", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916403, "value1":0.7018505752435668, "value2":664450605422502237, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_2":"k","key_6":"i","key_1":"c", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497447, "value1":0.8277122605282176, "value2":1219079943087879547, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716326, "value1":0.7509816698301687, "value2":3921665603218422131, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_3":"h","key_7":"c", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716585, "value1":0.2920975208652007, "value2":3756232698934542100, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301567, "value1":0.8942632920542656, "value2":2291395595568091863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.599725, "value1":0.5659814539312558, "value2":7320359369690410226, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_1":"e","key_6":"d","key_7":"e", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.614192, "value1":0.7911920709201787, "value2":8781166815844312813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.195843, "value1":0.3963219582946417, "value2":4716889546078718843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555996, "value1":0.602486402959913, "value2":4687524095847830496, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.917870, "value1":0.15392369008366974, "value2":7045583703847585198, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_1":"h","key_4":"k","key_9":"a", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.616039, "value1":0.6919323182152138, "value2":511455339796298853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930107, "value1":0.12610440100742193, "value2":6997765950733061577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194345, "value1":0.7209468362854391, "value2":556025580881809360, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_1":"k","key_7":"b","key_9":"b", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.797831, "value1":0.2266151348199214, "value2":2004107184327193155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_7":"i","key_1":"d","key_6":"b", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949681, "value1":0.8884059641939672, "value2":7011478951165849176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.213425, "value1":0.9929181204525178, "value2":8631476778841936239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_0":"k","key_7":"c","key_8":"g", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821834, "value1":0.3090901919909019, "value2":5522454281472043559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.181027, "value1":0.06720359916226379, "value2":4456725170730825432, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.762490, "value1":0.474581541255306, "value2":7005342387987070026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.759790, "value1":0.5398913013405661, "value2":2663889184009973941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.112196, "value1":0.8025326007392437, "value2":883355344681130779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374886, "value1":0.9718695344446878, "value2":1646239120227544644, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.729457, "value1":0.9682037233981026, "value2":6151398746046248850, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048424, "value1":0.7273914683080029, "value2":8000884246159354431, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_2":"f","key_3":"k", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.937043, "value1":0.1526646293780592, "value2":266299265780041426, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.752328, "value1":0.9655606802627454, "value2":5596664612933771240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666200, "value1":0.10048257972248233, "value2":4289061240642605106, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.922103, "value1":0.8963130727289086, "value2":8836663020903305631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105202, "value1":0.8314839649252564, "value2":3551543011617183084, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886891, "value1":0.66836051888972, "value2":72366475317978094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_2":"a","key_3":"c","key_7":"k", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.432056, "value1":0.745064543848614, "value2":2189880642993257536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_2":"g","key_8":"g","key_0":"b", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042716, "value1":0.3956017159748215, "value2":3605211356176936944, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.984359, "value1":0.2911243881226883, "value2":7856628067522273320, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.295731, "value1":0.4520766897124352, "value2":610878216891620607, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.650753, "value1":0.03928359938969399, "value2":2072987167295070336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211564, "value1":0.7625006233645502, "value2":4401190374987004409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792725, "value1":0.47390601969195995, "value2":4329714295913254034, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.444277, "value1":0.5857721636217157, "value2":3580156964556605157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_6":"j","key_2":"k","key_3":"e", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.535990, "value1":0.9433719155892893, "value2":5643233427116703402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_4":"a","key_5":"c","key_0":"a", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.688127, "value1":0.7720856573472716, "value2":5631377472070167195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.697966, "value1":0.9263837993798216, "value2":4093734777340860370, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_5":"b","key_8":"e","key_9":"c", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.996439, "value1":0.3910900735196092, "value2":8375564545525062485, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872727, "value1":0.8324218421388521, "value2":7152928233109153205, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.246110, "value1":0.2551941160691682, "value2":5602869232123697435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_2":"c","key_3":"k","key_6":"e", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.931727, "value1":0.5795265646922866, "value2":7403691202626500132, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_6":"i","key_9":"e","key_2":"h", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.673141, "value1":0.06896155961831285, "value2":4415828287435091542, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_8":"a","key_9":"h","key_7":"d", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684610, "value1":0.16872221976263158, "value2":505317720854796636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559417, "value1":0.12924994486885147, "value2":4787886921303451969, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.094359, "value1":0.5164211297919008, "value2":1315713014945297109, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007976, "value1":0.7735634457421625, "value2":8579409036573615608, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286892, "value1":0.43624877878480095, "value2":6924175474906299981, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_7":"c","key_1":"h","key_2":"e", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.902926, "value1":0.6780220008663965, "value2":4822375571733927097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.486540, "value1":0.23333524037004374, "value2":6314520721238807162, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.007170, "value1":0.9857337079596743, "value2":4448133028853613455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.030552, "value1":0.24355749578071725, "value2":8327413108187969392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.379606, "value1":0.4729923914146341, "value2":914941226083135930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_5":"e","key_8":"i","key_1":"i", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835232, "value1":0.9602105867588695, "value2":3810448989361248064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433684, "value1":0.9595084872985944, "value2":455874222103117607, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411653, "value1":0.037665634780994134, "value2":6833101830522402640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_4":"a","key_5":"j","key_2":"e", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.713632, "value1":0.5664925972399699, "value2":7794307235376671149, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391266, "value1":0.6608939962095463, "value2":5860330325168511608, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.212303, "value1":0.2089060448996, "value2":5651881484326487830, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.180078, "value1":0.7876093968011537, "value2":2862206598597477486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.175199, "value1":0.14395244434800714, "value2":2376804168343245120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.809965, "value1":0.12003398504952464, "value2":5187708593154594566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_7":"a","key_1":"j","key_2":"e", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.851381, "value1":0.607956049195976, "value2":7288813225458184427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_7":"d","key_5":"g", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933118, "value1":0.7246239590339857, "value2":1234716149527350625, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.939251, "value1":0.34949617152726564, "value2":1847804623713516854, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665651, "value1":0.7331985679295692, "value2":1370393562180863722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.827386, "value1":0.6896205618409067, "value2":1887650763331073033, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.637520, "value1":0.13783942741851515, "value2":7482561145284406008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_0":"c","key_5":"f","key_6":"g", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657534, "value1":0.26275939644072077, "value2":6263897972267140208, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_3":"f","key_6":"a","key_9":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.740684, "value1":0.9117133919549965, "value2":6149129956357538684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.032295, "value1":0.9865289627211871, "value2":5555291464918052387, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545212, "value1":0.5648289415381519, "value2":6789070914120574869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_1":"j","key_3":"g","key_0":"e", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.321143, "value1":0.6917810996904441, "value2":4171532082462131046, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_3":"e","key_4":"e","key_2":"d", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.693006, "value1":0.9813132387635279, "value2":6633038595296515364, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_2":"j","key_7":"f","key_9":"b", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.895453, "value1":0.7852256444188942, "value2":143518625251882031, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_1":"d","key_6":"b","key_9":"e", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165192, "value1":0.9962691534924663, "value2":1006093794801043666, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.353290, "value1":0.5341309097782416, "value2":1258165174425707067, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_2":"k","key_7":"a", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.257167, "value1":0.7039937676297155, "value2":1145181626509784790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.413647, "value1":0.8270957619940229, "value2":2929580620701623843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.817697, "value1":0.5617132216229666, "value2":9160744958657565723, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.117851, "value1":0.20816644440142626, "value2":5357495560720255737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_1":"j","key_5":"j","key_8":"b", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545126, "value1":0.5346084869679877, "value2":7401215357009964293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.824766, "value1":0.44946104541199977, "value2":4136655445147421363, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.292920, "value1":0.8837887841085235, "value2":7386066379061583540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_3":"h","key_4":"j","key_8":"f", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.949095, "value1":0.3445432872175305, "value2":7354448880530731294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705243, "value1":0.4595023646368128, "value2":1229720604223933971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_5":"c","key_6":"a","key_1":"j", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610093, "value1":0.4977300016550588, "value2":6055289293785959372, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.710138, "value1":0.8494303412007083, "value2":3284734735098008783, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.503055, "value1":0.7306081415151926, "value2":7335475436979640441, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194198, "value1":0.4080577435044893, "value2":1817328694183447312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409059, "value1":0.8174198346097175, "value2":1028006350387911539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696870, "value1":0.3114167302109578, "value2":5761149591452007692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_1":"c","key_9":"j","key_0":"a", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483258, "value1":0.6191704897662682, "value2":6062885989254649767, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.534924, "value1":0.6117230725465559, "value2":193279975656637243, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639609, "value1":0.47495483196675986, "value2":5738372947955229915, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532964, "value1":0.01280767557324011, "value2":3610414855244726673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.741210, "value1":0.5736887790720044, "value2":9118221200383527859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_8":"i","key_1":"k","key_3":"c", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.459701, "value1":0.09043619123087489, "value2":4263857141830808661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.064891, "value1":0.8303048678510601, "value2":1417745368253260336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.193982, "value1":0.5954818624712542, "value2":1866813575466479927, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.728941, "value1":0.42773413448912906, "value2":5011276511494346486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.978643, "value1":0.9720431088123873, "value2":5835438838603942392, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.376007, "value1":0.5894070491287171, "value2":1536141728264317947, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_8":"g","key_3":"k","key_6":"k", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.226693, "value1":0.26901914665508003, "value2":1625748998624229948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.963696, "value1":0.5769249508695262, "value2":9054438071997375931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.418644, "value1":0.766497146060167, "value2":5911142909752098984, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_9":"g","key_3":"e","key_8":"f", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.564856, "value1":0.11883244261524029, "value2":1743904390192603929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055140, "value1":0.52087787431266, "value2":402388317326273396, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_5":"k","key_6":"i","key_8":"d", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.165833, "value1":0.8230062512281381, "value2":3737986001509812244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.537380, "value1":0.3719641116241002, "value2":7620785809655703628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_1":"b","key_3":"i","key_4":"h", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059138, "value1":0.3359364898323379, "value2":2258599254367397283, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_7":"g","key_8":"k","key_4":"g", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532103, "value1":0.07168825170242699, "value2":8419082906568750462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_7":"f","key_1":"a","key_6":"c", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.286202, "value1":0.8607616531331588, "value2":5301488987302943339, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477954, "value1":0.7462715469038064, "value2":8695685405375237694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_2":"e","key_9":"h","key_1":"e", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.377516, "value1":0.1251480000451169, "value2":5693123483571141635, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799701, "value1":0.377835354984664, "value2":7635656423964308029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_1":"c","key_2":"a","key_5":"j", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947251, "value1":0.7193521006934637, "value2":3495998112139060094, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_9":"c","key_3":"g","key_8":"h", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.630158, "value1":0.27928250023508844, "value2":8779396817967517012, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.886093, "value1":0.6334013505397469, "value2":5098375435747464833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.942245, "value1":0.3390011070632797, "value2":1737381722392040741, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_7":"k","key_4":"f","key_6":"k", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.009762, "value1":0.2515798026660567, "value2":2465310044177783178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.896699, "value1":0.584030830613032, "value2":5977614913522628306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820378, "value1":0.7811580321417821, "value2":1709536028386531684, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653683, "value1":0.7005991085562125, "value2":7295655368561133786, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522361, "value1":0.7296163959268779, "value2":5916302487026661508, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_0":"e","key_4":"i","key_5":"e", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.127496, "value1":0.24876472584091772, "value2":8637191483262110011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.091531, "value1":0.9069922237355105, "value2":3364698751790785415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.137633, "value1":0.9475238029065713, "value2":6717128691639834645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550582, "value1":0.1420769771765079, "value2":1644264737205720610, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.059092, "value1":0.38378121050522995, "value2":4775737880122110580, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_0":"f","key_6":"f","key_7":"a", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236099, "value1":0.8215350110894001, "value2":1784075564711617189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_8":"b","key_0":"b","key_5":"e", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.636343, "value1":0.4453072911799391, "value2":3315454218709810887, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_0":"g","key_5":"i","key_9":"h", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.799575, "value1":0.17197278191202653, "value2":5903905346180673218, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.584133, "value1":0.4235206584790561, "value2":1691406943949519298, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.015622, "value1":0.34252290670938007, "value2":5505899012400247300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_2":"e","key_4":"d","key_6":"h", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.021661, "value1":0.11232931832514273, "value2":6468715321661687664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_5":"h","key_1":"d", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.593330, "value1":0.8735006589361365, "value2":831831146444791684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_3":"j","key_9":"e","key_2":"c", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.273986, "value1":0.25953904800502825, "value2":5320773233674225264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357026, "value1":0.11876193765893824, "value2":8703386441096799470, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.903715, "value1":0.0317054368098292, "value2":2551886074115870652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.905359, "value1":0.9780528237638186, "value2":3462176768443711504, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.272907, "value1":0.5077414966128044, "value2":5351891928699763264, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_2":"e","key_4":"i","key_5":"d", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.610476, "value1":0.5253243304839273, "value2":7118905075072261164, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.684508, "value1":0.3558646139061106, "value2":5157018975048237545, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_4":"j","key_5":"b","key_9":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748383, "value1":0.7510255655970447, "value2":4180265211771950319, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.801038, "value1":0.2649030050080282, "value2":509475464752529120, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.657950, "value1":0.2544231826829063, "value2":4768343360725290987, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.149149, "value1":0.24278734421658493, "value2":2310223808628977449, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.558331, "value1":0.3866811766896264, "value2":3120828999446479419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.076776, "value1":0.8191629913767823, "value2":531265332159947696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881067, "value1":0.16630603149873838, "value2":8244214645789653431, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.385842, "value1":0.7521248304618617, "value2":4350867045350111094, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.914596, "value1":0.12306660567858013, "value2":5899167982504356408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.159683, "value1":0.24808395371130693, "value2":3321017992196518857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_9":"b","key_7":"d", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.058378, "value1":0.4115012836332614, "value2":5551372136464658518, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_6":"g","key_0":"a","key_2":"g", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819794, "value1":0.9548946315713374, "value2":2661379806705881657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_9":"g","key_1":"c","key_8":"a", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.293628, "value1":0.30579616916280866, "value2":2771189581765698489, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.904159, "value1":0.33983467932998995, "value2":3076540259059103682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_1":"c","key_2":"f","key_8":"b", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.946959, "value1":0.14422457278803882, "value2":619548712042120909, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.383295, "value1":0.23130372874617672, "value2":6179693520548172628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_6":"k","key_7":"c", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482434, "value1":0.8720643520387502, "value2":6593209094509926896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613596, "value1":0.9574764532675973, "value2":8706916487148773835, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_3":"i","key_4":"d","key_6":"g", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.551130, "value1":0.9901941419895447, "value2":5559435899121181473, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954673, "value1":0.7022122624865335, "value2":6926608561209244320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_5":"b","key_6":"g","key_9":"e", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781834, "value1":0.1129731019718482, "value2":3356276176028256450, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_4":"f","key_9":"e","key_0":"b", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.746610, "value1":0.5148787087784333, "value2":2343142454537691934, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.409647, "value1":0.3828843970677094, "value2":4346661084777988652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.427543, "value1":0.6220594320308941, "value2":653819026690948218, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_9":"e","key_1":"d", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.467056, "value1":0.23591956688182053, "value2":5924417417665977435, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_0":"h","key_1":"b","key_9":"d", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.245070, "value1":0.32465613049106207, "value2":8745282484552741587, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.719344, "value1":0.6149579128373653, "value2":7721284234183306230, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_0":"j","key_9":"d", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095318, "value1":0.7783448314627689, "value2":7496490910707147022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_3":"k","key_4":"d","key_8":"h", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696761, "value1":0.23959350451136963, "value2":7667698021180616606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.170976, "value1":0.2107634553446009, "value2":1067019224393043111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177955, "value1":0.15559883601716398, "value2":570151235008619291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_2":"h","key_8":"i", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.334876, "value1":0.05250568880425774, "value2":313709473500400875, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_5":"h","key_4":"b", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.645622, "value1":0.5874495193049836, "value2":4998753194711581798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.403716, "value1":0.681377437500311, "value2":6083653541111639303, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.511374, "value1":0.3173656336054262, "value2":3280525464397494846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_2":"g","key_3":"g","key_6":"f", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.218153, "value1":0.6649425163171951, "value2":984695368325234886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_5":"g","key_6":"j","key_4":"a", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424332, "value1":0.13915015029096536, "value2":177058255857848790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.882802, "value1":0.9604830357087591, "value2":3462432728417263080, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_9":"f","key_0":"a","key_5":"d", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.933587, "value1":0.05218287981155221, "value2":9126870231879692354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426076, "value1":0.7165203610998703, "value2":4392854854028036091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.302645, "value1":0.34553849779237905, "value2":2907279110591723700, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.269877, "value1":0.8711177705575849, "value2":8366161311313542515, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_5":"j","key_7":"c","key_9":"a", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.995800, "value1":0.6906708270307872, "value2":2179499360020827214, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_1":"a","key_5":"h","key_0":"f", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.433217, "value1":0.612549221250184, "value2":3062667501076675896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.550193, "value1":0.8112010714019721, "value2":3511775054758615153, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.947917, "value1":0.4309103807271251, "value2":5927887337207634099, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.874180, "value1":0.8528260534422161, "value2":6693253358548032140, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_8":"i","key_9":"j","key_1":"c", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.714464, "value1":0.7660112509524851, "value2":4667923922359952270, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_5":"j","key_6":"j","key_7":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.088061, "value1":0.2662708805339307, "value2":2956847029720911185, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960166, "value1":0.6603171866146907, "value2":4062360360972662646, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.771991, "value1":0.06535928533875911, "value2":3275575695469134507, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.907738, "value1":0.4729789504578976, "value2":6665898542309531788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_6":"c","key_9":"e","key_5":"f", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.446527, "value1":0.054360638009933175, "value2":8447197342747029062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.871013, "value1":0.9325815658063547, "value2":3295816637987069156, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.322468, "value1":0.7896334184172146, "value2":2062716273940484317, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.145073, "value1":0.15908044288327283, "value2":31202777072956548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.450280, "value1":0.20707514852594208, "value2":3916360901630366923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.834638, "value1":0.6993008706531769, "value2":5937180712489291188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.116654, "value1":0.23637601643034672, "value2":4626933419743360623, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_5":"h","key_9":"b","key_3":"a", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.265735, "value1":0.37825344249371357, "value2":1259576346841819002, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.683748, "value1":0.8969615470937745, "value2":3878280364099631128, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_7":"b","key_9":"f", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.219470, "value1":0.2735521439606725, "value2":2004029025316087157, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_0":"a","key_2":"e","key_7":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476407, "value1":0.3060298444596037, "value2":2752192683000499763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_2":"e","key_3":"i","key_5":"j", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973916, "value1":0.6236548692215601, "value2":1218902262886276264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.777866, "value1":0.4605467848114755, "value2":1483419366258255115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.045114, "value1":0.6713933571025698, "value2":7389349750224761078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.570321, "value1":0.6477885368549638, "value2":952734344221236, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_2":"h","key_5":"e", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.821043, "value1":0.6987507461356852, "value2":7961454653125032378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.586904, "value1":0.6114733442241732, "value2":3313794499633786815, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.705481, "value1":0.05021439091843144, "value2":1692908847970065889, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_0":"k","key_2":"k","key_9":"i", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.497544, "value1":0.7911397038857368, "value2":7192760038857907146, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.932189, "value1":0.9148929522533059, "value2":9058585624863979355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.769986, "value1":0.08834899410804685, "value2":1734900992084257549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695397, "value1":0.9679316547671841, "value2":7641677535001923086, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.084900, "value1":0.7534718430023655, "value2":5595569809726413737, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.562250, "value1":0.19691330948497934, "value2":5079334807276275801, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.843384, "value1":0.13037385236459426, "value2":1781212766913480310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_2":"c","key_0":"d", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.102533, "value1":0.8945673902737089, "value2":4413715911002881, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350432, "value1":0.9789410844857529, "value2":8061203484333522616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_7":"b","key_2":"j","key_6":"i", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857648, "value1":0.17405749140176002, "value2":9069075917121269047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_5":"a","key_7":"b","key_4":"g", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.559262, "value1":0.2097671258596821, "value2":613585525667388168, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.712886, "value1":0.7017574550139256, "value2":807895944720896464, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.936984, "value1":0.8849159113766228, "value2":6360061394425352910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.545107, "value1":0.7492436378397768, "value2":1930556756703562417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_7":"b","key_8":"e","key_9":"b", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.681593, "value1":0.044080654493611975, "value2":4672445330328029132, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388330, "value1":0.4856178458787539, "value2":6153421365310799825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_2":"f","key_9":"b", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.879197, "value1":0.16996161009540478, "value2":260583885651182711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.770981, "value1":0.7283458398178451, "value2":6187029331538785539, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_7":"f","key_1":"i","key_2":"b", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.101721, "value1":0.5814924092013966, "value2":3246021123474652609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665903, "value1":0.2698824852319885, "value2":1593984595070302417, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_3":"b","key_6":"h","key_9":"i", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.014945, "value1":0.41108393803888965, "value2":849125531814643716, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_7":"g","key_3":"h","key_6":"j", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.455276, "value1":0.875114673636132, "value2":5155846223745306265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.044985, "value1":0.9997830302808556, "value2":2517744318699317082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.236657, "value1":0.10301472150509888, "value2":3916175947622962913, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.595845, "value1":0.681705831836613, "value2":4919968063926505445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.909129, "value1":0.10658700130911442, "value2":9017518189462648176, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994913, "value1":0.5742718119176038, "value2":2926652004213539521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.350277, "value1":0.15279738754509417, "value2":5229335911692726266, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.022399, "value1":0.3330863087425972, "value2":4953311412687090451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268930, "value1":0.7193122390898454, "value2":4256538916269794836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_3":"d","key_5":"k","key_7":"e", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.983431, "value1":0.0510995090351011, "value2":1768238660724776680, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672570, "value1":0.255109482234144, "value2":5911656502600207006, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.478340, "value1":0.7629467314333804, "value2":2248461028173940598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954691, "value1":0.780819131964203, "value2":4988575109325316188, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_7":"i","key_3":"f","key_4":"d", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.854956, "value1":0.888056149862958, "value2":8089385147769653445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_3":"j","key_4":"d","key_6":"e", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.028845, "value1":0.6170371731466705, "value2":4427914831964658823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_4":"b","key_1":"i","key_2":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.639652, "value1":0.8230090544258113, "value2":6386454055832759048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.810891, "value1":0.8133492204735011, "value2":6725307713283048525, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_5":"c","key_8":"a","key_9":"b", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.761452, "value1":0.9287463566978789, "value2":1984332858107282238, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.318552, "value1":0.12867977830402738, "value2":4258557511687812956, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_1":"e","key_3":"a","key_4":"d", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.316280, "value1":0.5022437385431537, "value2":995773241737628619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.954122, "value1":0.5240772572203263, "value2":6037304727626529988, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.177671, "value1":0.76376619698564, "value2":2077146903364533171, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.731650, "value1":0.7327573768170443, "value2":3115299633570203304, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.050578, "value1":0.7589539042984136, "value2":8472839195693038957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_7":"e","key_0":"j", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.606514, "value1":0.9364993124842877, "value2":5612839525396767856, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.369368, "value1":0.048054238834199556, "value2":5282267936079713586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_9":"i","key_8":"d", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.619618, "value1":0.3228482622950357, "value2":8295983975137448078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.783482, "value1":0.20802671279789772, "value2":3029896725457460111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_7":"j","key_8":"d","key_5":"k", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332994, "value1":0.9302305981560293, "value2":1560834164036895150, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_9":"e","key_2":"b","key_7":"c", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.320192, "value1":0.22351800921367507, "value2":367222968940142358, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.881266, "value1":0.08414682717481595, "value2":4156359981061160049, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.708929, "value1":0.5859616740437592, "value2":4218658472873761504, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.974600, "value1":0.18236565991254491, "value2":7158153525091699583, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_4":"e","key_6":"d","key_2":"i", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.760005, "value1":0.2499852109211892, "value2":3103407843415565026, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.994395, "value1":0.6346134433025367, "value2":8895656809884955572, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482016, "value1":0.09096777794371665, "value2":6086112719790576068, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442570, "value1":0.8266959930861106, "value2":2275327626493666884, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.416700, "value1":0.36518175488705273, "value2":5793835688591222111, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.488107, "value1":0.8111992750792233, "value2":3432067066553462639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.296157, "value1":0.748182736526456, "value2":5908492840734386304, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.655322, "value1":0.77012733378921, "value2":2328983933411827460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.299380, "value1":0.8617587030482643, "value2":4249851149905507488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.569550, "value1":0.8799767655587674, "value2":114484019272599760, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.737420, "value1":0.660133828675834, "value2":47092260178100634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.961538, "value1":0.5805975021285331, "value2":3098519932913684476, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_0":"c","key_3":"b","key_6":"c", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197731, "value1":0.7355297940768138, "value2":5347385484676578771, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653672, "value1":0.9639932888139914, "value2":4154150822821006863, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.209735, "value1":0.25867188901232296, "value2":7573003393945357593, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.243923, "value1":0.34698853560785725, "value2":4046809595104442791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.055125, "value1":0.9118136199691569, "value2":6132859986145115351, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.129146, "value1":0.737958001899574, "value2":3639000020359385593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.439824, "value1":0.7612995291197047, "value2":9120669622233440416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.222157, "value1":0.2453738047252645, "value2":5040757825498916672, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_4":"f","key_8":"d","key_3":"b", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.916336, "value1":0.7380620101187605, "value2":4223451018219910420, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.365543, "value1":0.035239810434389264, "value2":7558213648465294836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.860762, "value1":0.4102850563256095, "value2":1429493420542448489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_4":"e","key_6":"k","key_9":"j", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.388872, "value1":0.5076851349389522, "value2":5984142931688899418, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.324767, "value1":0.32798077886519317, "value2":7010471358073816774, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605843, "value1":0.9054516940429083, "value2":5196636487460887513, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653438, "value1":0.9546349180436571, "value2":1964265110886490351, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576178, "value1":0.9072096436167739, "value2":4650061523788539671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_1":"h","key_7":"b","key_8":"a", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.304613, "value1":0.07126080366571305, "value2":2739761425628315393, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.399413, "value1":0.9339282528280148, "value2":2738842204765000811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.943959, "value1":0.08784682744168541, "value2":7785212390244201422, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_7":"g","key_8":"h","key_1":"j", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.155440, "value1":0.06596281114416563, "value2":5614935012679054029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_4":"d","key_2":"f", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.585232, "value1":0.38735104883356297, "value2":5357846878220935711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_8":"b","key_1":"j","key_4":"k", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.231739, "value1":0.7284804092280743, "value2":1167665308911410443, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.555892, "value1":0.5915854017921878, "value2":1567147347614329809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_2":"a","key_7":"g","key_9":"k", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.855303, "value1":0.09176207020704319, "value2":3808984845296778232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.938710, "value1":0.4819531123010938, "value2":4697805519732493178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.476902, "value1":0.22082616880251751, "value2":2129509252592428060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.744743, "value1":0.33257150382979367, "value2":7648865469799769588, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.306532, "value1":0.04481922646422544, "value2":5772762433855832524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364825, "value1":0.3587026951251621, "value2":5278057080002614026, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_4":"c","key_3":"j", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.484995, "value1":0.39478883780380347, "value2":3136752353967205980, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.359118, "value1":0.08414989767840077, "value2":1942845542819450866, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095178, "value1":0.16106032134150491, "value2":2585752892335863488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.268896, "value1":0.613902523132248, "value2":687319106737333189, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_0":"g","key_6":"k","key_9":"g", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.716029, "value1":0.4960997765937663, "value2":2263474829486803965, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.748556, "value1":0.30380613001187684, "value2":569759032424044386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_3":"c","key_5":"b","key_1":"c", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.819011, "value1":0.9766133829878352, "value2":4041486474077430360, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_4":"i","key_8":"j","key_3":"i", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.109307, "value1":0.7831795468260019, "value2":6028647505036494731, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_3":"c","key_4":"c","key_6":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.179692, "value1":0.36834450950873376, "value2":5143099232657374567, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_4":"j","key_6":"c","key_7":"j", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.408811, "value1":0.37776607558956526, "value2":6856480231263656565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_5":"i","key_6":"b","key_8":"j", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.006253, "value1":0.5270930124398272, "value2":1411653061914296136, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.776912, "value1":0.4551811310760191, "value2":1536148212309064628, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_0":"i","key_4":"g","key_5":"j", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.447453, "value1":0.8174960750892899, "value2":4331073111578365688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.696722, "value1":0.2434463944121451, "value2":3006348927793308525, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.747552, "value1":0.46652870663413926, "value2":8487775958608976630, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618043, "value1":0.5667967701695303, "value2":445864738400581228, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_6":"g","key_0":"b","key_3":"i", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.162568, "value1":0.6628301293329222, "value2":8197909955604355013, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_7":"h","key_9":"g", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.861425, "value1":0.3485000903134516, "value2":8816338849324478138, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_4":"j","key_5":"a","key_3":"e", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.806516, "value1":0.4953223448201175, "value2":7711648028791565560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.970539, "value1":0.2083396416590356, "value2":5815736098137722214, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_3":"d","key_6":"a","key_9":"c", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108983, "value1":0.42614208367663026, "value2":5403929578330914973, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_0":"i","key_3":"a","key_9":"h", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.095108, "value1":0.4675226715836234, "value2":4103322823365161263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.279008, "value1":0.5189838609534998, "value2":5583248834742899795, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_9":"f","key_3":"k","key_6":"g", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.080151, "value1":0.04025953736590447, "value2":1709967085975216083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_2":"i","key_7":"b","key_9":"c", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.960603, "value1":0.48558007294067956, "value2":147087377780565755, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_9":"k","key_3":"f","key_4":"j", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.522699, "value1":0.8996419465199031, "value2":8383086671138869204, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.812906, "value1":0.42145746065428685, "value2":3807545219581503501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_4":"i","key_6":"k","key_2":"f", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.973610, "value1":0.9578107981118065, "value2":4941505183486571577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.972491, "value1":0.7644375474791567, "value2":5826757011862504756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_2":"i","key_3":"d","key_7":"i", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.048527, "value1":0.2281920875592009, "value2":4718173727183162630, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_0":"f","key_4":"b", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.515627, "value1":0.3746899214010938, "value2":5127946555951269169, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.346004, "value1":0.5308363801498025, "value2":1141799226947758562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_7":"d","key_3":"k","key_5":"c", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.391384, "value1":0.5665502681985652, "value2":5138410786658734048, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.857410, "value1":0.1451852744317182, "value2":6603434642077187944, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.414061, "value1":0.265028330167258, "value2":8861513360159934632, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_1":"j","key_2":"j","key_0":"g", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.653333, "value1":0.28437623266560746, "value2":5007130114350069495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.123846, "value1":0.29659698191393213, "value2":8540905938700351347, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.477802, "value1":0.6141130483574686, "value2":4145514544028354410, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_2":"j","key_3":"i","key_0":"d", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.531351, "value1":0.3974291747311908, "value2":7891249160701823099, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_5":"a","key_8":"c","key_9":"e", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.197949, "value1":0.9490013407410278, "value2":8218574898456721251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_4":"h","key_9":"a","key_3":"k", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.411857, "value1":0.8458181580037749, "value2":3063835093954499296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_0":"k","key_6":"c","key_9":"a", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.877510, "value1":0.828501312919876, "value2":5054318282608028643, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.665063, "value1":0.5885292293078199, "value2":7952781418541268636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.041521, "value1":0.9878113457449454, "value2":7450867973872472368, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_2":"h","key_8":"i","key_0":"d", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.073061, "value1":0.939649958460976, "value2":7875175737198226822, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_5":"k","key_9":"d","key_4":"i", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.781888, "value1":0.5031750456506159, "value2":7116386269513023773, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_3":"f","key_5":"f","key_0":"b", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.160017, "value1":0.23130907810117704, "value2":2438724480357189127, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_6":"d","key_7":"b","key_2":"c", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.835359, "value1":0.4415459106308202, "value2":8077656610946018544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.251312, "value1":0.10148089002006272, "value2":1896400908090076616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.632483, "value1":0.7757884759608441, "value2":4168124167987668028, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_1":"a","key_4":"i","key_7":"f", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.772638, "value1":0.14296917471420942, "value2":5892426599914094550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.426554, "value1":0.4152913794309927, "value2":3491478566670263783, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.672520, "value1":0.5730477814369939, "value2":7734177692857714726, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_0":"e","key_1":"b","key_8":"e", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.025475, "value1":0.8422765595744146, "value2":7729206509934976065, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384117, "value1":0.370521152586741, "value2":5964592513649221172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.666556, "value1":0.5944594406776452, "value2":6664609919274167291, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_9":"g","key_2":"c","key_8":"k", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.424484, "value1":0.6898557357450742, "value2":7620342586495006501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.313298, "value1":0.7769705418438511, "value2":5904168920374633694, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.556524, "value1":0.7197157815883978, "value2":1785992040018522529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.360907, "value1":0.5368763819006307, "value2":1221035619431068900, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.576297, "value1":0.2976581980063287, "value2":6069726578064867551, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.425007, "value1":0.05760812531545847, "value2":3218040486200104849, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.442385, "value1":0.7140138290502642, "value2":5982710133241839409, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.364950, "value1":0.7133219094796532, "value2":2621884575274392083, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.211537, "value1":0.9662389413186122, "value2":6837048628615994370, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.161776, "value1":0.9162528731861198, "value2":98225928051258831, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_3":"e","key_5":"k","key_7":"k", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.811493, "value1":0.5668121931771126, "value2":4899385784647025957, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822383, "value1":0.016988952685027904, "value2":7343988970036816255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.865454, "value1":0.48779466867997384, "value2":3996429126028951858, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751902, "value1":0.517057218253837, "value2":4621244619940358234, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.525579, "value1":0.7270638229553396, "value2":4556294068287353316, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_3":"i","key_8":"k", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.822049, "value1":0.8487037862810487, "value2":8759327482669918657, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_2":"a","key_3":"d","key_4":"i", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.483767, "value1":0.795243973695518, "value2":5137269638112922032, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_7":"e","key_1":"c","key_2":"a", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.042738, "value1":0.7385935337493647, "value2":1091563145939490509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.521929, "value1":0.10598411546030329, "value2":1018022188446431653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.792757, "value1":0.4495950673323956, "value2":1856520078086460155, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.695775, "value1":0.8122734246000456, "value2":2520061166650080735, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_5":"h","key_1":"k","key_2":"a", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.105722, "value1":0.9820835252391706, "value2":7455782937968387088, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.820568, "value1":0.22530315350938107, "value2":3276960573086645000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.140320, "value1":0.5664165038340241, "value2":7444501896936670459, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_9":"i","key_1":"g","key_7":"g", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.501828, "value1":0.6731262889242471, "value2":1425113709817927232, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.930172, "value1":0.48625040372483846, "value2":5327468289350467201, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.706858, "value1":0.07914368053314678, "value2":1800402440657604521, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.134725, "value1":0.4678852928796053, "value2":2325913710025246340, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_6":"g","key_1":"j","key_4":"e", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.876578, "value1":0.9424035369996965, "value2":8055626132574707503, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.108826, "value1":0.29002692913973105, "value2":774202804060462158, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.031892, "value1":0.9251224968475468, "value2":6954662396929356191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.539689, "value1":0.24245379006948578, "value2":460788187782305876, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.858085, "value1":0.02056945040393724, "value2":4590009984548317252, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.202540, "value1":0.37546125893661825, "value2":6399435061817101457, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_0":"i","key_2":"b", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.384494, "value1":0.010928897877400335, "value2":2783102565309398205, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_1":"f","key_9":"g","key_0":"k", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.061578, "value1":0.259570974455371, "value2":8569411528487848312, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_8":"h","key_9":"e","key_2":"h", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.027239, "value1":0.09519589010929196, "value2":6611488629400003141, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.210858, "value1":0.6912816460992459, "value2":7725980859343417825, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.825726, "value1":0.48050045655332785, "value2":6843673580007675724, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.430241, "value1":0.8940246106870715, "value2":7204593499636831454, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_5":"d","key_9":"f","key_1":"h", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.332710, "value1":0.05615399939144247, "value2":4573382128185893311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_0":"h","key_4":"k","key_7":"h", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.374418, "value1":0.5152317418659397, "value2":7908425193534726276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_8":"k","key_2":"a", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.519252, "value1":0.2299372870124918, "value2":6913359063794068428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.405529, "value1":0.49571648923268996, "value2":1675726536905469427, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_8":"i","key_2":"j", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.605031, "value1":0.09917638686306547, "value2":3890367351649717445, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_6":"k","key_2":"f","key_4":"d", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.194280, "value1":0.5756913899097403, "value2":6546196099308660172, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.763643, "value1":0.049961431289849915, "value2":2156338695320389229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_6":"j","key_7":"c","key_4":"a", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.755741, "value1":0.4853393193247385, "value2":5380709479032006182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.434015, "value1":0.6730243170500183, "value2":4072172223941247851, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_7":"a","key_9":"a","key_4":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.188499, "value1":0.8036991963254426, "value2":2042780350133201142, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.788778, "value1":0.6590319721348468, "value2":6545712378323399560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.431899, "value1":0.10111044541197162, "value2":3033625496942345343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.846633, "value1":0.4624813135357754, "value2":5961232902625455903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.649911, "value1":0.42335939078880974, "value2":9206873087714105578, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_6":"c","key_7":"i", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.532557, "value1":0.8374513257646443, "value2":4063679098902368460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.993435, "value1":0.026158890681344544, "value2":8561365511354827010, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.808220, "value1":0.6771449672369052, "value2":2392568978099066809, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_2":"h","key_9":"g","key_0":"b", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.357037, "value1":0.7840791560680648, "value2":1293280054604430799, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.686467, "value1":0.009361168431443715, "value2":1158105653011230048, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.971166, "value1":0.5749373357687312, "value2":5056280036307903399, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_7":"f","key_9":"d","key_3":"d", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.544825, "value1":0.6259231852657728, "value2":1572064409805832355, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_9":"c","key_5":"i", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.889816, "value1":0.6691815472587526, "value2":5159786074932669495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_7":"b","key_1":"e","key_5":"b", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.751169, "value1":0.5117344773320296, "value2":2709445168494389044, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.358316, "value1":0.5060937970199147, "value2":4487218678538317276, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_4":"f","key_1":"g","key_2":"h", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.577717, "value1":0.8145066136721544, "value2":3597604992290980548, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.990084, "value1":0.031203549542102884, "value2":6033709675731250939, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.613856, "value1":0.5443798055071071, "value2":1023967738912955990, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.795434, "value1":0.2935457393386216, "value2":1392092134182260244, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.233100, "value1":0.7087138744541079, "value2":6901046651435105011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.215643, "value1":0.301381153698089, "value2":5724617386824483093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_1":"h","key_4":"e","key_0":"d", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.536387, "value1":0.050992650300612716, "value2":1926309233688447762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_9":"c","key_3":"e", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.661031, "value1":0.5259350399353443, "value2":3757516621940142781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.883217, "value1":0.9897587697648693, "value2":166453122026887271, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.482631, "value1":0.7903296358928328, "value2":6917975765021057279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.301142, "value1":0.03214802664084894, "value2":1209824808667363268, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.618630, "value1":0.34174203368906997, "value2":687273686120110296, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_9":"e","key_3":"g","key_4":"a", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.872422, "value1":0.7312694201106051, "value2":1954246364933112681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.675094, "value1":0.02386805295420225, "value2":2582799143372120508, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.397537, "value1":0.7375439234748727, "value2":3761195316124557282, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_6":"a","key_4":"k","key_5":"d", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.998621, "value1":0.44001824362826686, "value2":2298817484668315029, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:08+08:00", "value":0.471460, "value1":0.5040752524973563, "value2":7440124910918057262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303559, "value1":0.34760281561681783, "value2":3856193801537489239, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000000", "key_1":"d","key_7":"e", "__series_id__":-6361865120410491981, "__mgmt_id__":-6361865120410491981} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477238, "value1":0.10358452410298057, "value2":3433455763442399890, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000001", "key_5":"g","key_6":"a","key_4":"g", "__series_id__":-7417794931361433675, "__mgmt_id__":-7417794931361433675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974082, "value1":0.9936842430072798, "value2":7647779573135010712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000002", "key_1":"a","key_2":"g","key_8":"f", "__series_id__":5649892766065403969, "__mgmt_id__":5649892766065403969} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.679212, "value1":0.33597414933520936, "value2":7871330604163867562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000003", "key_5":"f","key_7":"a","key_8":"k", "__series_id__":2491409761442743947, "__mgmt_id__":2491409761442743947} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.420730, "value1":0.4119033435452023, "value2":7398827678901559124, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000004", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.546764, "value1":0.0476713858704466, "value2":112659874965942961, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000005", "key_1":"h","key_7":"b","key_9":"a", "__series_id__":-8265343809411007481, "__mgmt_id__":-8265343809411007481} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730784, "value1":0.2868023177018098, "value2":7913444002207179200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000006", "key_5":"e","key_7":"g","key_3":"g", "__series_id__":-2350915848502491710, "__mgmt_id__":-2350915848502491710} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862440, "value1":0.5312714100525405, "value2":6175515648240114931, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000007", "key_3":"e","key_4":"e","key_8":"a", "__series_id__":2613217386916107323, "__mgmt_id__":2613217386916107323} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769163, "value1":0.9602531490116414, "value2":5345461780217485343, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000008", "key_7":"e","key_9":"f", "__series_id__":-3082451392512806683, "__mgmt_id__":-3082451392512806683} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855124, "value1":0.553411156209368, "value2":4062088865979243439, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000010", "key_1":"e","key_5":"c","key_6":"a", "__series_id__":4593224894946540012, "__mgmt_id__":4593224894946540012} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.210418, "value1":0.8711908622408605, "value2":8107060141858943082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000009", "key_0":"h","key_3":"k","key_8":"b", "__series_id__":-4771670780972268233, "__mgmt_id__":-4771670780972268233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436733, "value1":0.7746479416891845, "value2":3021240871572817126, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000020", "key_1":"c","key_6":"i", "__series_id__":-3867686789927146028, "__mgmt_id__":-3867686789927146028} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.692065, "value1":0.594498398968098, "value2":988266694346712016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000021", "key_2":"i","key_3":"f","key_9":"c", "__series_id__":-7557088021808002998, "__mgmt_id__":-7557088021808002998} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.441926, "value1":0.8776750344471822, "value2":6190746113032965957, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000022", "key_7":"g","key_8":"e","key_5":"i", "__series_id__":-1674287583760082371, "__mgmt_id__":-1674287583760082371} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013512, "value1":0.14091811490255743, "value2":4960449076483043020, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000023", "key_2":"j","key_4":"b", "__series_id__":606159230414197914, "__mgmt_id__":606159230414197914} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548236, "value1":0.32376195419487824, "value2":4889372139161861463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000024", "key_0":"h","key_1":"e","key_6":"h", "__series_id__":1993954834508334303, "__mgmt_id__":1993954834508334303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.828595, "value1":0.6520731819283484, "value2":508152344585097791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000025", "key_0":"a","key_4":"j","key_5":"b", "__series_id__":4180237166358025978, "__mgmt_id__":4180237166358025978} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759392, "value1":0.11332264235829892, "value2":8149117049560006848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000026", "key_1":"f","key_2":"c","key_7":"d", "__series_id__":6877367775299575478, "__mgmt_id__":6877367775299575478} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901588, "value1":0.1665732927247099, "value2":1166517989622232910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000027", "key_8":"c","key_2":"g", "__series_id__":1066752686003512442, "__mgmt_id__":1066752686003512442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035913, "value1":0.07988414283718974, "value2":7894355898795371390, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000028", "key_3":"c","key_5":"e","key_6":"j", "__series_id__":1070579679626379987, "__mgmt_id__":1070579679626379987} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.583982, "value1":0.25757663417932825, "value2":6007061793954380382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000029", "key_1":"f","key_4":"e","key_7":"c", "__series_id__":-1894427622426263214, "__mgmt_id__":-1894427622426263214} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636209, "value1":0.38383005316111324, "value2":4618851528086053316, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000030", "key_0":"j", "__series_id__":1193720041638406067, "__mgmt_id__":1193720041638406067} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672395, "value1":0.002191687957779578, "value2":1207882083575373188, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000031", "key_5":"c","key_7":"f","key_0":"f", "__series_id__":-3676185689090961166, "__mgmt_id__":-3676185689090961166} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462535, "value1":0.3192805953827191, "value2":365124979437384544, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000032", "key_8":"g","key_9":"k","key_5":"b", "__series_id__":1957099006661774855, "__mgmt_id__":1957099006661774855} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508318, "value1":0.6798889100739353, "value2":255815407107212152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000033", "key_0":"i","key_1":"h","key_9":"f", "__series_id__":-3818704758089256053, "__mgmt_id__":-3818704758089256053} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905933, "value1":0.882876839434679, "value2":1335222091221270902, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000034", "key_6":"b","key_7":"e","key_9":"c", "__series_id__":399363825354316617, "__mgmt_id__":399363825354316617} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.721902, "value1":0.3457876641727297, "value2":7448197074268866329, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000015", "key_0":"a","key_6":"k","key_8":"k", "__series_id__":5494931827757869027, "__mgmt_id__":5494931827757869027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.837057, "value1":0.03599562374649498, "value2":1792244431983896309, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000011", "key_1":"j","key_3":"c", "__series_id__":-8183722192123266908, "__mgmt_id__":-8183722192123266908} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056476, "value1":0.22903239483804314, "value2":5638633766102759583, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000012", "key_7":"j","key_2":"e", "__series_id__":-2041989208784968202, "__mgmt_id__":-2041989208784968202} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.457531, "value1":0.05016525338775747, "value2":2976936137139451754, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000013", "key_3":"f","key_6":"k","key_7":"e", "__series_id__":-6691559199598613340, "__mgmt_id__":-6691559199598613340} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617621, "value1":0.16600638886403743, "value2":3903308221116403976, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000014", "key_3":"j","key_1":"i", "__series_id__":-3560743409529210195, "__mgmt_id__":-3560743409529210195} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415798, "value1":0.5206207545088288, "value2":7426460172854544634, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000018", "key_8":"d","key_0":"j","key_1":"d", "__series_id__":1507879893505985029, "__mgmt_id__":1507879893505985029} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.470076, "value1":0.02758169975135837, "value2":3279193513719457677, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000016", "key_2":"d","key_5":"d", "__series_id__":143205045117664527, "__mgmt_id__":143205045117664527} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.847049, "value1":0.818759672974311, "value2":5946222659493106047, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000017", "key_0":"e","key_5":"d","key_7":"e", "__series_id__":1382081309323095369, "__mgmt_id__":1382081309323095369} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367937, "value1":0.4074258090017152, "value2":6858775210252472573, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000035", "key_8":"h","key_3":"d", "__series_id__":-1504596091368927264, "__mgmt_id__":-1504596091368927264} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973712, "value1":0.13887537149474657, "value2":2800343207358775074, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000019", "key_1":"b","key_4":"h","key_7":"j", "__series_id__":6028509310571555531, "__mgmt_id__":6028509310571555531} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403585, "value1":0.2978164609006992, "value2":6234312472150253679, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000037", "key_0":"c","key_3":"i","key_7":"a", "__series_id__":1849668056084388119, "__mgmt_id__":1849668056084388119} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904490, "value1":0.7280336451114688, "value2":2026079147622689137, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000036", "key_9":"g","key_5":"e","key_7":"h", "__series_id__":-311983220320231940, "__mgmt_id__":-311983220320231940} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.541237, "value1":0.1368259441713387, "value2":6424894922861577254, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000038", "key_6":"i","key_0":"c","key_2":"j", "__series_id__":-2882676588454577349, "__mgmt_id__":-2882676588454577349} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.783065, "value1":0.5704544299311838, "value2":4180282487041693175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000039", "key_0":"d","key_1":"g","key_4":"i", "__series_id__":-3229117909436309362, "__mgmt_id__":-3229117909436309362} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.890800, "value1":0.7757728483786852, "value2":8132981856885227577, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000040", "key_6":"f","key_0":"b","key_1":"b", "__series_id__":-8814450514247281081, "__mgmt_id__":-8814450514247281081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901025, "value1":0.22908718180210688, "value2":4747104448031391939, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000050", "key_4":"j","key_8":"i", "__series_id__":-59315983402160733, "__mgmt_id__":-59315983402160733} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180212, "value1":0.5476402281995001, "value2":8223100966387528196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000051", "key_5":"e","key_6":"h", "__series_id__":-2101950891145961065, "__mgmt_id__":-2101950891145961065} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626159, "value1":0.7996650121856338, "value2":64003761107808118, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000041", "key_9":"g","key_3":"k","key_4":"f", "__series_id__":-8681749725467241001, "__mgmt_id__":-8681749725467241001} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545550, "value1":0.1740742638651991, "value2":5824563041193046282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000042", "key_0":"d","key_9":"e", "__series_id__":-781004633247834938, "__mgmt_id__":-781004633247834938} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026911, "value1":0.9915609719154322, "value2":3839917984332114189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000043", "key_4":"f","key_5":"e","key_0":"c", "__series_id__":6425939792925870980, "__mgmt_id__":6425939792925870980} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970605, "value1":0.049651840615534626, "value2":3663818871674092196, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000044", "key_0":"j","key_1":"c","key_4":"d", "__series_id__":8203812288919961648, "__mgmt_id__":8203812288919961648} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.795861, "value1":0.6046374060545386, "value2":6467693625848837489, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000045", "key_4":"b","key_8":"a", "__series_id__":6111785442276013538, "__mgmt_id__":6111785442276013538} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741291, "value1":0.4689122935380642, "value2":8701125505618349182, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000046", "key_6":"h","key_2":"k","key_4":"e", "__series_id__":-8269425104286464293, "__mgmt_id__":-8269425104286464293} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.877913, "value1":0.596765963129067, "value2":8094571743010403516, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000047", "key_2":"g","key_5":"i","key_7":"h", "__series_id__":7532055644348176501, "__mgmt_id__":7532055644348176501} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.790394, "value1":0.06093826578205693, "value2":733010313976904838, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000048", "key_9":"b","key_1":"e", "__series_id__":8866600531285973006, "__mgmt_id__":8866600531285973006} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603208, "value1":0.8364659181351286, "value2":1088146971265159206, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000049", "key_1":"d","key_3":"h","key_9":"j", "__series_id__":6579635354321950989, "__mgmt_id__":6579635354321950989} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823621, "value1":0.47775163306986124, "value2":7344279199603749996, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000053", "key_1":"f","key_4":"a","key_7":"k", "__series_id__":-8314831493892247699, "__mgmt_id__":-8314831493892247699} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171666, "value1":0.02782471041039711, "value2":1659046247294927834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000052", "key_3":"c","key_4":"a","key_9":"g", "__series_id__":3850395059791465036, "__mgmt_id__":3850395059791465036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.899246, "value1":0.23664326650032677, "value2":764233009249486091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000058", "key_5":"k","key_9":"c","key_0":"k", "__series_id__":7996759019555320515, "__mgmt_id__":7996759019555320515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658885, "value1":0.13203154011115373, "value2":5571262153472749697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000064", "key_4":"a","key_7":"k","key_1":"j", "__series_id__":8816773697446392595, "__mgmt_id__":8816773697446392595} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.711786, "value1":0.6314799048378197, "value2":8955079539774261393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000065", "key_0":"g","key_4":"j","key_7":"f", "__series_id__":-3451026078807038448, "__mgmt_id__":-3451026078807038448} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.358690, "value1":0.4626190815088963, "value2":5349880367436474865, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000066", "key_2":"b","key_3":"b","key_6":"a", "__series_id__":-5636974685330669635, "__mgmt_id__":-5636974685330669635} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.436669, "value1":0.7604788504004301, "value2":2830736413446294733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000067", "key_0":"c","key_4":"d","key_9":"a", "__series_id__":7762095970378497348, "__mgmt_id__":7762095970378497348} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283630, "value1":0.11435547311453685, "value2":5764151651661369748, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000068", "key_3":"f","key_4":"g","key_9":"j", "__series_id__":-7984852869056523373, "__mgmt_id__":-7984852869056523373} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550602, "value1":0.9394300496620045, "value2":3385128630739961936, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000069", "key_5":"e","key_9":"b", "__series_id__":-910590231097844596, "__mgmt_id__":-910590231097844596} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004479, "value1":0.08103496493647538, "value2":1096104398248692524, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000070", "key_1":"g","key_9":"g", "__series_id__":-471143189299142315, "__mgmt_id__":-471143189299142315} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272179, "value1":0.007258870887977933, "value2":2565647835692463774, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000071", "key_2":"k","key_5":"c","key_7":"f", "__series_id__":-6570289835833793858, "__mgmt_id__":-6570289835833793858} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.207218, "value1":0.8139650845703748, "value2":905077995346686402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000072", "key_3":"g","key_9":"k","key_0":"g", "__series_id__":1154074168030394148, "__mgmt_id__":1154074168030394148} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217842, "value1":0.48125835395143424, "value2":6789722720604854599, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000073", "key_4":"k","key_6":"b", "__series_id__":-1497332666716611286, "__mgmt_id__":-1497332666716611286} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.155093, "value1":0.7189778774843847, "value2":6292044296017930854, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000074", "key_0":"f","key_3":"g","key_5":"b", "__series_id__":-1714854460223739254, "__mgmt_id__":-1714854460223739254} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723501, "value1":0.3703438091023322, "value2":3955334986917106318, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000075", "key_9":"f","key_3":"j","key_5":"i", "__series_id__":2282711603191250138, "__mgmt_id__":2282711603191250138} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216869, "value1":0.12053628422917362, "value2":636444633907082482, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000076", "key_4":"e","key_8":"h", "__series_id__":6573048433538125738, "__mgmt_id__":6573048433538125738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.568591, "value1":0.23735656022841886, "value2":4679225743620568386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000077", "key_0":"k","key_7":"h", "__series_id__":6163909700883376820, "__mgmt_id__":6163909700883376820} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.792289, "value1":0.8769118352474913, "value2":8705903369099981004, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000078", "key_0":"b","key_1":"a","key_9":"h", "__series_id__":-9084206948870776280, "__mgmt_id__":-9084206948870776280} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.377367, "value1":0.9028525772471804, "value2":8394233771809162273, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000079", "key_3":"d","key_6":"c","key_0":"d", "__series_id__":-5863629355677328674, "__mgmt_id__":-5863629355677328674} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511370, "value1":0.06603888812647277, "value2":1587071294143636433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000080", "key_3":"g","key_6":"h","key_9":"g", "__series_id__":571477606045741529, "__mgmt_id__":571477606045741529} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060860, "value1":0.3291125358805135, "value2":7312301032222805401, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000081", "key_4":"e","key_6":"e","key_8":"f", "__series_id__":4534444186575327872, "__mgmt_id__":4534444186575327872} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816308, "value1":0.6388914782369252, "value2":1128303197769336870, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000082", "key_4":"h","key_5":"e","key_9":"b", "__series_id__":6974900966400270870, "__mgmt_id__":6974900966400270870} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.068805, "value1":0.319277617656504, "value2":6681946261878539245, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000083", "key_1":"g","key_6":"b","key_9":"a", "__series_id__":-3699381386157183052, "__mgmt_id__":-3699381386157183052} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.805301, "value1":0.009280749666995351, "value2":544578594920738627, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000084", "key_1":"c","key_3":"k","key_9":"j", "__series_id__":-640392872965375373, "__mgmt_id__":-640392872965375373} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493659, "value1":0.6717590147189912, "value2":2097325929268646339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000085", "key_1":"d","key_5":"j", "__series_id__":467337617461513179, "__mgmt_id__":467337617461513179} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.253857, "value1":0.2354585485783823, "value2":5182832084934966477, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000086", "key_2":"k","key_4":"b","key_6":"f", "__series_id__":-186107375404205023, "__mgmt_id__":-186107375404205023} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454134, "value1":0.7059786460183258, "value2":8734869026681606253, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000087", "key_2":"f","key_3":"k","key_5":"k", "__series_id__":3255744113557958985, "__mgmt_id__":3255744113557958985} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.769480, "value1":0.16571806998598893, "value2":1451771174019239273, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000088", "key_3":"f","key_8":"h","key_9":"d", "__series_id__":-5608070008742618972, "__mgmt_id__":-5608070008742618972} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.043302, "value1":0.0805122859716286, "value2":8615098369512434691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000089", "key_2":"d","key_3":"a","key_7":"j", "__series_id__":-5630411415645077903, "__mgmt_id__":-5630411415645077903} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.493432, "value1":0.37562446579716846, "value2":447590635610718249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000090", "key_6":"j","key_7":"b","key_9":"a", "__series_id__":1861220525601045919, "__mgmt_id__":1861220525601045919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.340933, "value1":0.476702778910786, "value2":2962733058848207480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000091", "key_3":"e","key_4":"c", "__series_id__":8215289218858840673, "__mgmt_id__":8215289218858840673} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.351270, "value1":0.04893100925940487, "value2":3216639061913163460, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000092", "key_0":"j","key_3":"i","key_4":"b", "__series_id__":5453110602370885979, "__mgmt_id__":5453110602370885979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819062, "value1":0.43774247002290106, "value2":6810500585509381104, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000093", "key_1":"g","key_4":"d", "__series_id__":3212540495939612883, "__mgmt_id__":3212540495939612883} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.789988, "value1":0.13044652854726224, "value2":6447819044361282673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000094", "key_1":"c","key_2":"e","key_7":"j", "__series_id__":7697276239589840540, "__mgmt_id__":7697276239589840540} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.137973, "value1":0.9280703615023543, "value2":2854967575973809701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000095", "key_1":"c","key_2":"g","key_5":"c", "__series_id__":8069055566639478930, "__mgmt_id__":8069055566639478930} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359816, "value1":0.6661196608352099, "value2":7585154254761447569, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000096", "key_1":"a","key_2":"j","key_6":"c", "__series_id__":-1514690877472430993, "__mgmt_id__":-1514690877472430993} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.374877, "value1":0.2719104405916207, "value2":1097487272974828210, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000097", "key_9":"j","key_3":"e","key_4":"f", "__series_id__":2509711739945206486, "__mgmt_id__":2509711739945206486} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766465, "value1":0.8656426220974218, "value2":2886501368125017740, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000098", "key_0":"c","key_2":"f", "__series_id__":850770171572844857, "__mgmt_id__":850770171572844857} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267362, "value1":0.594401465949902, "value2":1715419869280476267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000099", "key_5":"b","key_6":"f","key_8":"j", "__series_id__":-6714122630243923351, "__mgmt_id__":-6714122630243923351} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.689698, "value1":0.09904556283985433, "value2":361858677456228074, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000100", "key_1":"e","key_2":"a","key_3":"f", "__series_id__":1425425266702944455, "__mgmt_id__":1425425266702944455} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244000, "value1":0.4739214533638735, "value2":6722075717885814318, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000101", "key_1":"g","key_3":"i","key_7":"c", "__series_id__":-1511448512568506759, "__mgmt_id__":-1511448512568506759} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957522, "value1":0.4793380026682715, "value2":8094832594089347058, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000102", "key_1":"i","key_8":"d", "__series_id__":8031847456333582984, "__mgmt_id__":8031847456333582984} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578816, "value1":0.32497702056237604, "value2":8938729328568281395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000103", "key_6":"h","key_1":"f","key_5":"e", "__series_id__":-5978406926652878392, "__mgmt_id__":-5978406926652878392} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999599, "value1":0.703313156867648, "value2":5755903281901537381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000104", "key_1":"c","key_3":"j","key_4":"g", "__series_id__":5179748796340991382, "__mgmt_id__":5179748796340991382} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829223, "value1":0.9601134377422823, "value2":4062518676536818057, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000105", "key_2":"b","key_3":"d","key_4":"d", "__series_id__":-1943556293666606448, "__mgmt_id__":-1943556293666606448} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.145200, "value1":0.7216961923243395, "value2":9217282407920967916, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000106", "key_0":"b","key_8":"c", "__series_id__":-2041801578517922928, "__mgmt_id__":-2041801578517922928} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820803, "value1":0.3093137747462916, "value2":7468155061834676810, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000107", "key_1":"h","key_4":"b","key_9":"k", "__series_id__":6273045326923481571, "__mgmt_id__":6273045326923481571} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682889, "value1":0.04485861766843415, "value2":4270651219964958841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000054", "key_6":"j","key_7":"j","key_3":"k", "__series_id__":1613127414612560725, "__mgmt_id__":1613127414612560725} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549256, "value1":0.7924629761166294, "value2":5462651594092502924, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000055", "key_6":"c","key_9":"i","key_5":"a", "__series_id__":-1984980466248998163, "__mgmt_id__":-1984980466248998163} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515774, "value1":0.1753099089980214, "value2":4245273970497284960, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000056", "key_9":"d","key_7":"h", "__series_id__":-3125920023802002767, "__mgmt_id__":-3125920023802002767} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280472, "value1":0.19807313433421816, "value2":6932983129449398217, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000057", "key_2":"a","key_3":"b","key_5":"j", "__series_id__":6282741732940207638, "__mgmt_id__":6282741732940207638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.183908, "value1":0.8803355871960044, "value2":3795185008893112660, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000062", "key_3":"j","key_7":"f", "__series_id__":8373535740233056743, "__mgmt_id__":8373535740233056743} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.250576, "value1":0.8766744958241532, "value2":8316217539045644702, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000059", "key_0":"d","key_6":"c","key_7":"j", "__series_id__":-3343911352267775498, "__mgmt_id__":-3343911352267775498} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811589, "value1":0.7673136679205708, "value2":1656000207332411296, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000060", "key_3":"b","key_8":"j", "__series_id__":-3921123017101809079, "__mgmt_id__":-3921123017101809079} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.617752, "value1":0.33750626542384204, "value2":2818737959409200195, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000061", "key_3":"f","key_4":"d","key_5":"d", "__series_id__":-1426521396101200607, "__mgmt_id__":-1426521396101200607} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.402465, "value1":0.5975891290632979, "value2":8117852232376404604, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000109", "key_3":"i","key_8":"e", "__series_id__":7638108789977515061, "__mgmt_id__":7638108789977515061} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.779395, "value1":0.6126614135133973, "value2":1972761209604651335, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000063", "key_1":"c","key_6":"f","key_8":"f", "__series_id__":9194958382530241609, "__mgmt_id__":9194958382530241609} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171014, "value1":0.3147679970132471, "value2":3872394099215939892, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000108", "key_0":"j","key_1":"c","key_3":"j", "__series_id__":4793341946183464793, "__mgmt_id__":4793341946183464793} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.602514, "value1":0.9979964274456362, "value2":8280861801559743509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000112", "key_4":"e","key_5":"i","key_8":"h", "__series_id__":9018975067096005230, "__mgmt_id__":9018975067096005230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.870286, "value1":0.1346288239954595, "value2":2167488486464723184, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000110", "key_1":"h","key_7":"c","key_8":"e", "__series_id__":8844840765227221040, "__mgmt_id__":8844840765227221040} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.902696, "value1":0.53464205049288, "value2":8019370943564861440, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000113", "key_2":"k","key_3":"i","key_4":"k", "__series_id__":597549186953823831, "__mgmt_id__":597549186953823831} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050828, "value1":0.7038559591596831, "value2":6864265904757939564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000111", "key_4":"f","key_7":"k", "__series_id__":-3268807719566530897, "__mgmt_id__":-3268807719566530897} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.465826, "value1":0.7686595552419602, "value2":1927886873036321447, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000114", "key_3":"e","key_5":"h", "__series_id__":-7441759397162295151, "__mgmt_id__":-7441759397162295151} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149509, "value1":0.9153544198103084, "value2":7814774915075726621, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000116", "key_9":"f","key_0":"k","key_1":"k", "__series_id__":-1794846826413575242, "__mgmt_id__":-1794846826413575242} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.485222, "value1":0.9188620757546552, "value2":5192187368195488529, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000117", "key_5":"f","key_6":"k","key_8":"f", "__series_id__":-6864550317982743146, "__mgmt_id__":-6864550317982743146} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520165, "value1":0.1037499454739713, "value2":2861179515410191072, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000115", "key_5":"i","key_9":"e", "__series_id__":-8442342819784056585, "__mgmt_id__":-8442342819784056585} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894795, "value1":0.6670064141722539, "value2":8570373623610344103, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000118", "key_1":"i","key_4":"c","key_6":"a", "__series_id__":-362456897096662535, "__mgmt_id__":-362456897096662535} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858375, "value1":0.9690125335533832, "value2":6598305524856472910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000120", "key_2":"d","key_3":"e","key_7":"d", "__series_id__":9126429541977437814, "__mgmt_id__":9126429541977437814} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639161, "value1":0.07029962179108588, "value2":4198012464202851461, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000121", "key_3":"e","key_5":"a","key_7":"h", "__series_id__":5675848291676949887, "__mgmt_id__":5675848291676949887} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.422856, "value1":0.243592288273798, "value2":7060563137853890653, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000119", "key_0":"a","key_6":"h","key_8":"e", "__series_id__":-8033450871750625569, "__mgmt_id__":-8033450871750625569} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356376, "value1":0.12506395613411664, "value2":3055799774165215720, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000124", "key_2":"g","key_4":"g","key_5":"k", "__series_id__":2604548282526698701, "__mgmt_id__":2604548282526698701} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296059, "value1":0.7972570100142576, "value2":2611996617533897408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000122", "key_1":"g","key_8":"i","key_9":"k", "__series_id__":4117833149263619100, "__mgmt_id__":4117833149263619100} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.461343, "value1":0.024502184861611945, "value2":1123093231220631621, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000125", "key_2":"b","key_3":"h","key_8":"f", "__series_id__":9137075327044920409, "__mgmt_id__":9137075327044920409} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529133, "value1":0.12083989402126777, "value2":7219868115171352433, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000123", "key_4":"g","key_6":"b","key_7":"h", "__series_id__":6357857786217316014, "__mgmt_id__":6357857786217316014} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110681, "value1":0.8049351220640796, "value2":4305443631669757518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000126", "key_0":"k","key_1":"a","key_2":"c", "__series_id__":6767355780455098874, "__mgmt_id__":6767355780455098874} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.981566, "value1":0.19931635115078358, "value2":4376781646759017505, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000128", "key_3":"j","key_4":"i","key_7":"j", "__series_id__":4247426367176705919, "__mgmt_id__":4247426367176705919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631835, "value1":0.4428488032269089, "value2":2962840158667509272, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000127", "key_1":"i","key_4":"h","key_6":"c", "__series_id__":2924902692178299433, "__mgmt_id__":2924902692178299433} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596817, "value1":0.01388179851872705, "value2":5526059113621970146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000129", "key_5":"k","key_7":"k", "__series_id__":1154434336808537704, "__mgmt_id__":1154434336808537704} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988044, "value1":0.08966434896908937, "value2":845094337456207269, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000130", "key_2":"e","key_6":"d","key_9":"g", "__series_id__":-7451864092156701946, "__mgmt_id__":-7451864092156701946} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032366, "value1":0.4632294657773622, "value2":2896178627081634846, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000132", "key_9":"i","key_0":"i","key_3":"a", "__series_id__":-2069560104892378999, "__mgmt_id__":-2069560104892378999} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.195349, "value1":0.3544490738754114, "value2":5485050023862879453, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000133", "key_5":"g","key_8":"k", "__series_id__":-8474357488465320783, "__mgmt_id__":-8474357488465320783} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.488827, "value1":0.05815655955779658, "value2":5446494658153199948, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000131", "key_0":"h","key_1":"g","key_2":"b", "__series_id__":1284856134736405307, "__mgmt_id__":1284856134736405307} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723246, "value1":0.6834006388190103, "value2":4274252704044712919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000134", "key_1":"c","key_2":"b", "__series_id__":-3182385159433660007, "__mgmt_id__":-3182385159433660007} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141449, "value1":0.3967110267075465, "value2":2849522299224594762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000136", "key_2":"i","key_4":"d","key_6":"h", "__series_id__":-7985943178384355012, "__mgmt_id__":-7985943178384355012} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.712863, "value1":0.2643526520952433, "value2":6055035031034459665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000137", "key_1":"k","key_4":"a","key_6":"k", "__series_id__":5765688178051837788, "__mgmt_id__":5765688178051837788} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.974542, "value1":0.07826746989883603, "value2":8013938940173093770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000135", "key_9":"b","key_4":"b", "__series_id__":8590391987934982290, "__mgmt_id__":8590391987934982290} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476496, "value1":0.08282820679282239, "value2":8740534065670450787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000138", "key_0":"f","key_5":"f","key_8":"c", "__series_id__":-558308991247212939, "__mgmt_id__":-558308991247212939} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.265568, "value1":0.7141009753996177, "value2":8015024495118709084, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000141", "key_1":"a","key_6":"b","key_8":"h", "__series_id__":-6238834606581445812, "__mgmt_id__":-6238834606581445812} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163000, "value1":0.9787912408769939, "value2":1407010919457389562, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000139", "key_1":"k","key_2":"h","key_4":"a", "__series_id__":2031701996903259529, "__mgmt_id__":2031701996903259529} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560997, "value1":0.22142486556739396, "value2":2439658579639630159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000142", "key_6":"e","key_7":"f","key_9":"j", "__series_id__":-8634907421753499502, "__mgmt_id__":-8634907421753499502} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325355, "value1":0.2775049821475937, "value2":2346830518660166745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000140", "key_1":"c","key_2":"g", "__series_id__":-3116728592375192395, "__mgmt_id__":-3116728592375192395} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491491, "value1":0.3135314864967855, "value2":1129771089143019455, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000143", "key_6":"d","key_7":"a","key_4":"g", "__series_id__":28452354456102768, "__mgmt_id__":28452354456102768} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.459317, "value1":0.2812010467625179, "value2":7830625013744386941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000145", "key_0":"c","key_1":"g","key_4":"k", "__series_id__":6364458956549614348, "__mgmt_id__":6364458956549614348} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.684694, "value1":0.4935589110227349, "value2":2540823529765104488, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000146", "key_1":"c","key_7":"i", "__series_id__":-2332013085895889868, "__mgmt_id__":-2332013085895889868} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.237886, "value1":0.07014579422912577, "value2":7022680572815993337, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000144", "key_0":"g","key_3":"b","key_9":"e", "__series_id__":5563967542164640109, "__mgmt_id__":5563967542164640109} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.414046, "value1":0.06496797245021041, "value2":2711277679569543532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000147", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799639, "value1":0.8106870375099371, "value2":1007449970411192794, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000149", "key_0":"a","key_3":"c", "__series_id__":9223070499161054046, "__mgmt_id__":9223070499161054046} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.264752, "value1":0.0960343886163872, "value2":3312684645465619564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000150", "key_1":"a","key_4":"b","key_7":"h", "__series_id__":-1724818728110458595, "__mgmt_id__":-1724818728110458595} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035469, "value1":0.0008360965935936516, "value2":5539611781321403864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000151", "key_3":"c","key_4":"e","key_7":"e", "__series_id__":8270758825059813284, "__mgmt_id__":8270758825059813284} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.306352, "value1":0.07245144250837937, "value2":3686038376310116060, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000148", "key_3":"d","key_9":"d", "__series_id__":-590968405571452953, "__mgmt_id__":-590968405571452953} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.048629, "value1":0.3482697350949442, "value2":2011496763457379089, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000153", "key_5":"i","key_8":"j","key_9":"d", "__series_id__":-1784521893174042304, "__mgmt_id__":-1784521893174042304} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978673, "value1":0.9982728323868918, "value2":702860377347588565, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000152", "key_0":"d","key_2":"k","key_9":"h", "__series_id__":-2817668530773309530, "__mgmt_id__":-2817668530773309530} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987564, "value1":0.21515438859453273, "value2":2583082073184523816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000154", "key_0":"a","key_1":"d","key_9":"d", "__series_id__":7423366130131613786, "__mgmt_id__":7423366130131613786} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.842602, "value1":0.3444834642501587, "value2":4908475207072135082, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000155", "key_6":"d","key_7":"a","key_3":"b", "__series_id__":3207093433415838288, "__mgmt_id__":3207093433415838288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966769, "value1":0.7858094998520652, "value2":6077992589658011123, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000157", "key_3":"b","key_4":"k","key_8":"j", "__series_id__":3452730991306891257, "__mgmt_id__":3452730991306891257} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.165450, "value1":0.6383111602434741, "value2":7226409376141032655, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000156", "key_1":"h","key_3":"c", "__series_id__":-6643065536824436920, "__mgmt_id__":-6643065536824436920} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.575923, "value1":0.8562665414193692, "value2":2173652645149705334, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000158", "key_2":"d","key_9":"g","key_1":"k", "__series_id__":-8405131525831889671, "__mgmt_id__":-8405131525831889671} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010707, "value1":0.5955696347147411, "value2":5814682382951778842, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000159", "key_0":"e","key_3":"c","key_9":"j", "__series_id__":2961372573105506275, "__mgmt_id__":2961372573105506275} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271135, "value1":0.1229263700863702, "value2":6350918748538162755, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000162", "key_5":"g","key_6":"c","key_1":"d", "__series_id__":-8351460038839747494, "__mgmt_id__":-8351460038839747494} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143782, "value1":0.975371469047403, "value2":8279280670157614673, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000160", "key_2":"c","key_7":"g", "__series_id__":-4057754142371582237, "__mgmt_id__":-4057754142371582237} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536001, "value1":0.2818838543932868, "value2":5548964524114515649, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000161", "key_8":"b","key_2":"g","key_3":"e", "__series_id__":-1798332360868592711, "__mgmt_id__":-1798332360868592711} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.274203, "value1":0.525661345793623, "value2":7632566212944122955, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000163", "key_4":"f","key_8":"b","key_9":"f", "__series_id__":-7847383196004021619, "__mgmt_id__":-7847383196004021619} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394070, "value1":0.7913688238824117, "value2":2991067692464176886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000164", "key_0":"a","key_1":"g", "__series_id__":1804397642211293298, "__mgmt_id__":1804397642211293298} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243187, "value1":0.9989819654882474, "value2":7814314371764134982, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000165", "key_2":"c","key_3":"g","key_4":"d", "__series_id__":3764804326138865786, "__mgmt_id__":3764804326138865786} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490869, "value1":0.30039262696070124, "value2":7927229014613646504, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000166", "key_6":"i","key_0":"f","key_5":"i", "__series_id__":-2684979140479497919, "__mgmt_id__":-2684979140479497919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.850916, "value1":0.8481086988896461, "value2":3502125731621593449, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000167", "key_1":"k","key_3":"d","key_4":"i", "__series_id__":-5840655664568080155, "__mgmt_id__":-5840655664568080155} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113680, "value1":0.32930706694057965, "value2":6445034393695608409, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000168", "key_2":"b","key_3":"j","key_9":"k", "__series_id__":5570741952766792787, "__mgmt_id__":5570741952766792787} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192297, "value1":0.3973423811708676, "value2":8167271156640822516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000170", "key_0":"e","key_2":"b","key_3":"i", "__series_id__":6454348724083965233, "__mgmt_id__":6454348724083965233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646981, "value1":0.6421917320177064, "value2":2441775949474583008, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000169", "key_4":"b","key_6":"d","key_9":"b", "__series_id__":7915149619701574573, "__mgmt_id__":7915149619701574573} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.482724, "value1":0.3152543309412001, "value2":8846147135578854122, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000171", "key_2":"h","key_5":"c", "__series_id__":5504457377504091819, "__mgmt_id__":5504457377504091819} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777161, "value1":0.26611205622992573, "value2":4707069095348547526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000172", "key_1":"c","key_6":"c", "__series_id__":-1599150549254325081, "__mgmt_id__":-1599150549254325081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.529669, "value1":0.25143936257925226, "value2":9113227763618672958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000175", "key_3":"j","key_6":"f", "__series_id__":4231140258106081014, "__mgmt_id__":4231140258106081014} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708875, "value1":0.10515643642729285, "value2":7582828897771165151, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000173", "key_2":"j","key_3":"e","key_5":"c", "__series_id__":-317199018426739656, "__mgmt_id__":-317199018426739656} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041864, "value1":0.7204022074713844, "value2":4505351249313209540, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000174", "key_0":"h","key_4":"c","key_7":"j", "__series_id__":3354108409869678272, "__mgmt_id__":3354108409869678272} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.517118, "value1":0.2778141202519239, "value2":7041089660646534661, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000176", "key_2":"i", "__series_id__":-4244341193881875964, "__mgmt_id__":-4244341193881875964} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255695, "value1":0.5252045968012302, "value2":1609597628350217466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000177", "key_4":"i","key_7":"c","key_8":"d", "__series_id__":4564325618898740878, "__mgmt_id__":4564325618898740878} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.001450, "value1":0.15419422555422022, "value2":6306687575854728640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000179", "key_2":"b","key_6":"i","key_7":"h", "__series_id__":-5304717018001475026, "__mgmt_id__":-5304717018001475026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.230403, "value1":0.7229181740004053, "value2":1306679799234664681, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000178", "key_6":"k","key_4":"c","key_5":"h", "__series_id__":87816280540931590, "__mgmt_id__":87816280540931590} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035841, "value1":0.6895973487797646, "value2":2559143873675894370, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000180", "key_3":"d","key_5":"e", "__series_id__":-404532909115380097, "__mgmt_id__":-404532909115380097} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.449904, "value1":0.7368166767257398, "value2":7052097343090100521, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000183", "key_4":"e","key_7":"i","key_9":"g", "__series_id__":7971366120583252671, "__mgmt_id__":7971366120583252671} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139346, "value1":0.807172748525886, "value2":4308240895194146306, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000181", "key_0":"d","key_2":"a","key_3":"h", "__series_id__":-4147830406690806858, "__mgmt_id__":-4147830406690806858} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977592, "value1":0.7560595694496225, "value2":3367349773561100464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000182", "key_3":"a","key_7":"f","key_9":"b", "__series_id__":-105708042353818101, "__mgmt_id__":-105708042353818101} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.831129, "value1":0.43661553653957313, "value2":1514573148689105923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000184", "key_1":"h","key_4":"d", "__series_id__":-3492803355001090052, "__mgmt_id__":-3492803355001090052} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804508, "value1":0.7956852333376209, "value2":8820887809742814806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000185", "key_1":"g","key_5":"d","key_7":"k", "__series_id__":-5856257105266260668, "__mgmt_id__":-5856257105266260668} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.858280, "value1":0.8493727617212654, "value2":8684654170930913793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000188", "key_2":"c","key_3":"h","key_6":"i", "__series_id__":-7501418065343643474, "__mgmt_id__":-7501418065343643474} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640988, "value1":0.2223414127619492, "value2":7250214226981174458, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000187", "key_1":"b","key_5":"k", "__series_id__":540058938037018937, "__mgmt_id__":540058938037018937} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153838, "value1":0.9864893077863718, "value2":1366599113373589549, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000189", "key_2":"g","key_3":"j","key_9":"k", "__series_id__":-5031137265295521924, "__mgmt_id__":-5031137265295521924} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144317, "value1":0.034179582463187676, "value2":4811674883286124820, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000186", "key_4":"g","key_7":"h","key_1":"k", "__series_id__":6384311434791564160, "__mgmt_id__":6384311434791564160} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.670072, "value1":0.3811265659697653, "value2":4090252794464650298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000191", "key_3":"g","key_5":"f","key_6":"a", "__series_id__":-2628080776869037402, "__mgmt_id__":-2628080776869037402} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486186, "value1":0.9084897972797951, "value2":2292875585752622652, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000190", "key_4":"c","key_2":"e", "__series_id__":-6809092537603749426, "__mgmt_id__":-6809092537603749426} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724703, "value1":0.04582892017411178, "value2":5556549536478579022, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000193", "key_4":"a","key_9":"e","key_2":"h", "__series_id__":358820623790332114, "__mgmt_id__":358820623790332114} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012322, "value1":0.6305358760062275, "value2":1288975268472701530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000192", "key_6":"a","key_8":"f","key_0":"f", "__series_id__":-2847970172981397249, "__mgmt_id__":-2847970172981397249} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.218849, "value1":0.8926861107470235, "value2":7051558642534889749, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000194", "key_1":"b","key_2":"c","key_6":"b", "__series_id__":2557920836494908533, "__mgmt_id__":2557920836494908533} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.577242, "value1":0.96150058991849, "value2":5205402300881087071, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000195", "key_1":"e","key_4":"g","key_5":"a", "__series_id__":7929286596360127546, "__mgmt_id__":7929286596360127546} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525938, "value1":0.010643579624627687, "value2":9179800346032814438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000196", "key_6":"i","key_9":"d", "__series_id__":7118429874597622369, "__mgmt_id__":7118429874597622369} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393624, "value1":0.5282994953423363, "value2":6629726373755372175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000197", "key_1":"d","key_4":"d","key_9":"g", "__series_id__":-7936570873523680981, "__mgmt_id__":-7936570873523680981} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697100, "value1":0.328819156896584, "value2":7890032700245067940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000200", "key_3":"a","key_5":"e","key_7":"i", "__series_id__":3639352674740645135, "__mgmt_id__":3639352674740645135} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.911016, "value1":0.23451404605523937, "value2":1559532623894065416, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000198", "key_0":"h","key_8":"i", "__series_id__":8823283681255314421, "__mgmt_id__":8823283681255314421} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128593, "value1":0.958219222491235, "value2":5238496228333042095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000199", "key_2":"f","key_5":"j","key_9":"j", "__series_id__":-4500545634898797152, "__mgmt_id__":-4500545634898797152} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704394, "value1":0.5513948920447201, "value2":9062299692444161349, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000201", "key_7":"h","key_1":"c", "__series_id__":1531212333300811705, "__mgmt_id__":1531212333300811705} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255075, "value1":0.35898456496590414, "value2":5087989108776558277, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000202", "key_3":"c","key_4":"h","key_6":"c", "__series_id__":-4469691104781668428, "__mgmt_id__":-4469691104781668428} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788201, "value1":0.7466905262312481, "value2":479894845243154201, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000204", "key_4":"k","key_5":"b","key_9":"d", "__series_id__":-2014901588395566836, "__mgmt_id__":-2014901588395566836} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565236, "value1":0.15707319443827472, "value2":93010754190809493, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000203", "key_7":"h","key_8":"k", "__series_id__":6517439227519034897, "__mgmt_id__":6517439227519034897} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.462851, "value1":0.1393959893683005, "value2":1383438862980713878, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000206", "key_2":"k","key_3":"j","key_5":"b", "__series_id__":2739631612785937741, "__mgmt_id__":2739631612785937741} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.587928, "value1":0.9257613742750218, "value2":461452136435423636, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000205", "key_6":"d", "__series_id__":-4056544719737454862, "__mgmt_id__":-4056544719737454862} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.204910, "value1":0.45361150342145223, "value2":8114896084179399798, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000208", "key_2":"a","key_4":"e", "__series_id__":-3440838895860869905, "__mgmt_id__":-3440838895860869905} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970393, "value1":0.8262379198959926, "value2":1410864297783479414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000207", "key_1":"e","key_3":"g","key_0":"k", "__series_id__":-4893960988253975402, "__mgmt_id__":-4893960988253975402} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571697, "value1":0.05523474900350812, "value2":6912470816433257487, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000210", "key_2":"a","key_6":"g", "__series_id__":278539141476854582, "__mgmt_id__":278539141476854582} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900965, "value1":0.3994455375010903, "value2":5910800813383852824, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000209", "key_1":"f","key_7":"g", "__series_id__":-2634149506137991219, "__mgmt_id__":-2634149506137991219} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533983, "value1":0.8439578978089138, "value2":6520871065252687028, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000211", "key_0":"k","key_1":"j","key_6":"k", "__series_id__":-6152235210239582771, "__mgmt_id__":-6152235210239582771} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732970, "value1":0.4096847894313233, "value2":4961560961953820016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000213", "key_2":"f","key_4":"g","key_7":"e", "__series_id__":-9078008273642591081, "__mgmt_id__":-9078008273642591081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.512990, "value1":0.9925070463343235, "value2":4031040144184448394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000212", "key_9":"j","key_3":"b","key_5":"a", "__series_id__":3789075411738053288, "__mgmt_id__":3789075411738053288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.875107, "value1":0.8529856732344743, "value2":8171410486624791720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000214", "key_0":"a","key_2":"c","key_6":"c", "__series_id__":2418661337577043035, "__mgmt_id__":2418661337577043035} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.662542, "value1":0.7969197657043129, "value2":4707105172837538302, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000215", "key_1":"h","key_2":"f","key_0":"f", "__series_id__":-8769274023320427631, "__mgmt_id__":-8769274023320427631} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.132562, "value1":0.5095481874741737, "value2":8469586257322312509, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000216", "key_1":"j","key_5":"j","key_7":"b", "__series_id__":7110746347371624233, "__mgmt_id__":7110746347371624233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171675, "value1":0.7140266335782444, "value2":8998618655550288053, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000217", "key_9":"h","key_1":"h","key_4":"a", "__series_id__":-9044049296600485351, "__mgmt_id__":-9044049296600485351} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829529, "value1":0.8556707112048024, "value2":84360751498479327, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000218", "key_2":"k","key_7":"j","key_8":"a", "__series_id__":-2702487361565044629, "__mgmt_id__":-2702487361565044629} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.383509, "value1":0.6267421890569228, "value2":3548620263321255935, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000219", "key_5":"k","key_7":"h","key_1":"a", "__series_id__":-4467063443052925540, "__mgmt_id__":-4467063443052925540} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725412, "value1":0.6792289410958199, "value2":8743030074795737682, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000221", "key_3":"h","key_7":"d","key_8":"i", "__series_id__":-4360566614594027009, "__mgmt_id__":-4360566614594027009} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.217960, "value1":0.06523873388277363, "value2":4237433473740985929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000220", "key_5":"g","key_6":"h","key_9":"d", "__series_id__":-7365143648791602426, "__mgmt_id__":-7365143648791602426} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.052828, "value1":0.39596522726411565, "value2":5268673916985408378, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000223", "key_6":"i","key_1":"a","key_4":"j", "__series_id__":-5771429335930317158, "__mgmt_id__":-5771429335930317158} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.243583, "value1":0.24855400987123744, "value2":7145331513033105803, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000222", "key_7":"f","key_1":"h","key_2":"f", "__series_id__":-5063514376085494737, "__mgmt_id__":-5063514376085494737} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.643757, "value1":0.4621121441928861, "value2":4607779943623171423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000225", "key_2":"e","key_4":"a", "__series_id__":-384155371210942527, "__mgmt_id__":-384155371210942527} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.777085, "value1":0.9713754435345697, "value2":3872286492266737542, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000224", "key_1":"e","key_7":"g", "__series_id__":9053870091904767989, "__mgmt_id__":9053870091904767989} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741256, "value1":0.33772530129863026, "value2":2720260557973633476, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000227", "key_7":"d","key_9":"a", "__series_id__":-6003259494256310900, "__mgmt_id__":-6003259494256310900} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040901, "value1":0.5366678442503715, "value2":5043767454211598671, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000226", "key_1":"e","key_9":"e", "__series_id__":921644124074663399, "__mgmt_id__":921644124074663399} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.566517, "value1":0.3199753043754913, "value2":1787736737657465265, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000228", "key_2":"g","key_6":"e","key_7":"c", "__series_id__":-7491755130759432770, "__mgmt_id__":-7491755130759432770} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871703, "value1":0.40150570806322944, "value2":7426113550614212915, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000229", "key_0":"d","key_6":"k", "__series_id__":163960823954443450, "__mgmt_id__":163960823954443450} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.063390, "value1":0.8093881999536079, "value2":6062591517984503571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000230", "key_7":"d","key_8":"h", "__series_id__":-1595629483288736026, "__mgmt_id__":-1595629483288736026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630695, "value1":0.5987756911109672, "value2":4564890959914284940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000231", "key_3":"a","key_4":"k","key_7":"g", "__series_id__":-3765777172219922311, "__mgmt_id__":-3765777172219922311} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089316, "value1":0.9756718556017356, "value2":4422554680156925050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000232", "key_7":"a","key_2":"g","key_4":"e", "__series_id__":1085094834466986289, "__mgmt_id__":1085094834466986289} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433679, "value1":0.6504367128725771, "value2":7179863028410040853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000242", "key_0":"i","key_1":"g","key_4":"e", "__series_id__":7762506373054977320, "__mgmt_id__":7762506373054977320} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418340, "value1":0.5167928405286337, "value2":796797443554380087, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000243", "key_2":"h","key_3":"b","key_5":"i", "__series_id__":-1219396340318367916, "__mgmt_id__":-1219396340318367916} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148063, "value1":0.2982285613686749, "value2":3455828331199929793, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000244", "key_3":"g","key_5":"j", "__series_id__":4286369265264583743, "__mgmt_id__":4286369265264583743} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.562273, "value1":0.1744043124258891, "value2":7056453619952557866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000245", "key_3":"g","key_5":"g","key_7":"k", "__series_id__":7176509012015840215, "__mgmt_id__":7176509012015840215} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554680, "value1":0.9985428971035419, "value2":3134607000357482375, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000246", "key_0":"k","key_3":"i","key_5":"i", "__series_id__":3478232514132647486, "__mgmt_id__":3478232514132647486} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.498692, "value1":0.34351042666983833, "value2":9219919937586266788, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000247", "key_2":"j","key_4":"c", "__series_id__":3089314848340334430, "__mgmt_id__":3089314848340334430} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983651, "value1":0.20667520837153053, "value2":8261113453188452260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000248", "key_0":"f","key_2":"i","key_9":"c", "__series_id__":2147688371268242959, "__mgmt_id__":2147688371268242959} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900233, "value1":0.49430245377625404, "value2":1887699130768202184, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000249", "key_0":"d","key_6":"c", "__series_id__":-9171186733886989406, "__mgmt_id__":-9171186733886989406} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.269041, "value1":0.24848195612656065, "value2":27297122896884729, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000250", "key_0":"g","key_8":"j","key_9":"j", "__series_id__":-5350966201698202835, "__mgmt_id__":-5350966201698202835} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.716521, "value1":0.703324321123364, "value2":5138993531610984864, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000251", "key_0":"k","key_6":"h","key_9":"f", "__series_id__":235119588830124008, "__mgmt_id__":235119588830124008} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.331287, "value1":0.6585119514499647, "value2":7805239120031598554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000252", "key_6":"a","key_3":"g","key_5":"g", "__series_id__":6120835208953874959, "__mgmt_id__":6120835208953874959} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.898740, "value1":0.7897434254535518, "value2":8577413153173117857, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000253", "key_2":"b","key_3":"a","key_6":"a", "__series_id__":-3573749945377143552, "__mgmt_id__":-3573749945377143552} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366667, "value1":0.4615000106211479, "value2":4478388423093675456, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000254", "key_1":"f","key_3":"b","key_8":"c", "__series_id__":-4086421139340971808, "__mgmt_id__":-4086421139340971808} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060808, "value1":0.3611416577570406, "value2":1763611736817673526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000255", "key_1":"i","key_7":"e","key_9":"e", "__series_id__":-7974731736398137339, "__mgmt_id__":-7974731736398137339} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987608, "value1":0.6511729153448047, "value2":7895118250604250511, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000256", "key_3":"g","key_4":"b","key_9":"c", "__series_id__":-5418919362836041490, "__mgmt_id__":-5418919362836041490} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.920802, "value1":0.8966340709602193, "value2":1619385988446426949, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000257", "key_2":"g","key_6":"f","key_9":"g", "__series_id__":-2260859392228394720, "__mgmt_id__":-2260859392228394720} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.759468, "value1":0.9772199770736059, "value2":486666067155730811, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000258", "key_5":"e","key_7":"f","key_3":"a", "__series_id__":9030574374130186147, "__mgmt_id__":9030574374130186147} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.697569, "value1":0.2215623962484913, "value2":998498599369301173, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000259", "key_1":"i","key_5":"k","key_9":"g", "__series_id__":712723991923371456, "__mgmt_id__":712723991923371456} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.550984, "value1":0.23501917809435333, "value2":203018852406902455, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000260", "key_0":"i","key_8":"f", "__series_id__":-153500247406501062, "__mgmt_id__":-153500247406501062} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865253, "value1":0.24390812385740993, "value2":2606627451901201346, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000261", "key_0":"k","key_6":"f", "__series_id__":15499690335888556, "__mgmt_id__":15499690335888556} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050134, "value1":0.6480270130439445, "value2":6247279314207653751, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000262", "key_4":"i","key_5":"b","key_7":"a", "__series_id__":5372354603718802516, "__mgmt_id__":5372354603718802516} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614221, "value1":0.7050465104295126, "value2":3377246595104905199, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000263", "key_3":"a","key_5":"a","key_6":"d", "__series_id__":2200789686389610660, "__mgmt_id__":2200789686389610660} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166533, "value1":0.3877660847452995, "value2":7536153185490054501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000264", "key_1":"i","key_2":"j","key_4":"c", "__series_id__":-4182605193058408022, "__mgmt_id__":-4182605193058408022} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644807, "value1":0.7190092729952982, "value2":5813482232408839023, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000265", "key_3":"d","key_4":"b","key_6":"f", "__series_id__":-2588047046090071899, "__mgmt_id__":-2588047046090071899} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739661, "value1":0.7257627180887302, "value2":3214351184373245408, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000266", "key_3":"e","key_5":"c","key_2":"b", "__series_id__":-610758200249505967, "__mgmt_id__":-610758200249505967} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552516, "value1":0.4939513402753451, "value2":465350226895561432, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000267", "key_1":"i","key_4":"h","key_6":"d", "__series_id__":4102647630186908830, "__mgmt_id__":4102647630186908830} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503358, "value1":0.14929982411048387, "value2":2610344885697195594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000268", "key_6":"j","key_8":"c","key_9":"d", "__series_id__":-1094261238021429217, "__mgmt_id__":-1094261238021429217} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.765487, "value1":0.45537690586006585, "value2":4288868163982148480, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000269", "key_1":"d","key_2":"f","key_6":"b", "__series_id__":-8125072424306725518, "__mgmt_id__":-8125072424306725518} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.838240, "value1":0.7605183341936481, "value2":7774940282079152902, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000270", "key_0":"a","key_1":"d","key_8":"d", "__series_id__":7114248156060763562, "__mgmt_id__":7114248156060763562} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646978, "value1":0.988755613370451, "value2":4713140451733570872, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000271", "key_4":"g","key_2":"c", "__series_id__":-581163781821289738, "__mgmt_id__":-581163781821289738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746410, "value1":0.8174040579458727, "value2":7395122770602669759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000272", "key_1":"e","key_3":"j","key_5":"k", "__series_id__":6056165502075962127, "__mgmt_id__":6056165502075962127} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.823505, "value1":0.758453176804426, "value2":7518087779323040688, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000273", "key_2":"g","key_7":"i","key_0":"g", "__series_id__":-5164253557477824984, "__mgmt_id__":-5164253557477824984} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020233, "value1":0.29899993635819666, "value2":5085917800846866623, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000274", "key_3":"k","key_7":"g","key_9":"h", "__series_id__":-3886540797918122503, "__mgmt_id__":-3886540797918122503} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561926, "value1":0.8862149028157266, "value2":3614075635724672575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000275", "key_2":"i","key_6":"f","key_0":"j", "__series_id__":-1628388129285480657, "__mgmt_id__":-1628388129285480657} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.437110, "value1":0.32845862992503655, "value2":2977427326936422575, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000276", "key_2":"c","key_5":"i","key_7":"b", "__series_id__":-2367882984291197965, "__mgmt_id__":-2367882984291197965} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706546, "value1":0.15047181179887134, "value2":2307845458700050829, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000277", "key_1":"g","key_3":"e","key_4":"d", "__series_id__":6636444104004800874, "__mgmt_id__":6636444104004800874} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.619350, "value1":0.1117344044076149, "value2":4988357694886705006, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000278", "key_1":"c","key_4":"h","key_5":"j", "__series_id__":4540625073764757770, "__mgmt_id__":4540625073764757770} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.808461, "value1":0.715214325361221, "value2":7172177171113874947, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000279", "key_9":"d","key_1":"h","key_3":"a", "__series_id__":-8073602608240199567, "__mgmt_id__":-8073602608240199567} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154987, "value1":0.3436009437038216, "value2":35294647035701645, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000280", "key_7":"i","key_8":"b","key_6":"a", "__series_id__":3782764195228049883, "__mgmt_id__":3782764195228049883} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.293185, "value1":0.1982820741163276, "value2":4721034174581841393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000281", "key_4":"g","key_3":"a", "__series_id__":-7196643947589422854, "__mgmt_id__":-7196643947589422854} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.177501, "value1":0.24102187665043326, "value2":5183952097737049563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000282", "key_0":"c","key_2":"c", "__series_id__":-2802937496335850283, "__mgmt_id__":-2802937496335850283} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434009, "value1":0.4074440055222358, "value2":2740366314909969021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000283", "key_1":"f","key_5":"f","key_7":"c", "__series_id__":-7868070420964471300, "__mgmt_id__":-7868070420964471300} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.962158, "value1":0.6945642879060417, "value2":8488893912024069218, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000284", "key_3":"b","key_7":"e","key_0":"b", "__series_id__":-107956853517303026, "__mgmt_id__":-107956853517303026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037976, "value1":0.05977827983958813, "value2":8714018046983615411, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000285", "key_5":"b","key_8":"b","key_9":"k", "__series_id__":8743004723181703205, "__mgmt_id__":8743004723181703205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.491879, "value1":0.5875339799333666, "value2":3507531774420107282, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000286", "key_2":"h","key_6":"i", "__series_id__":-6080411562657950991, "__mgmt_id__":-6080411562657950991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708852, "value1":0.941564068848663, "value2":7441568589430319196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000287", "key_2":"k","key_5":"f","key_6":"f", "__series_id__":-5085436153869299952, "__mgmt_id__":-5085436153869299952} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665427, "value1":0.15650940720497003, "value2":2725018303601038315, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000288", "key_0":"e","key_8":"d", "__series_id__":-3652832040735163825, "__mgmt_id__":-3652832040735163825} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728790, "value1":0.16909603056430217, "value2":7913664106090409634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000289", "key_0":"c","key_6":"f", "__series_id__":-4827582452648415077, "__mgmt_id__":-4827582452648415077} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748451, "value1":0.9309585511638759, "value2":6423846277129286464, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000290", "key_5":"k","key_8":"h", "__series_id__":-8331465659376192273, "__mgmt_id__":-8331465659376192273} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701551, "value1":0.2886585284950581, "value2":1577558950874330311, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000291", "key_0":"d","key_2":"a","key_3":"i", "__series_id__":-7479610835286291676, "__mgmt_id__":-7479610835286291676} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356768, "value1":0.42594704925771354, "value2":3150589955894640969, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000292", "key_3":"b","key_5":"g","key_9":"j", "__series_id__":8712448124612225936, "__mgmt_id__":8712448124612225936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.056847, "value1":0.8050681337140542, "value2":3504760629437797890, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000293", "key_9":"k","key_4":"b","key_7":"e", "__series_id__":-6611753493518302504, "__mgmt_id__":-6611753493518302504} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.797591, "value1":0.9381497767913094, "value2":8069233160965701213, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000294", "key_0":"f","key_5":"a","key_7":"j", "__series_id__":7452684393613271105, "__mgmt_id__":7452684393613271105} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.653188, "value1":0.21115616320062328, "value2":2091006864800018643, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000295", "key_0":"a","key_2":"i","key_6":"f", "__series_id__":8312888208372477794, "__mgmt_id__":8312888208372477794} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489289, "value1":0.6249865113250967, "value2":3636072468848176197, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000296", "key_2":"e","key_7":"e","key_9":"f", "__series_id__":-3794951196896029722, "__mgmt_id__":-3794951196896029722} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.426149, "value1":0.7525465264386705, "value2":2554715790387615481, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000297", "key_4":"k","key_6":"f","key_7":"i", "__series_id__":2203039791051080232, "__mgmt_id__":2203039791051080232} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629922, "value1":0.244780100518832, "value2":2525567513505106874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000298", "key_3":"g","key_4":"j","key_9":"b", "__series_id__":-3485863643022996311, "__mgmt_id__":-3485863643022996311} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.652167, "value1":0.44363110594266053, "value2":2074305704102402281, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000299", "key_0":"g","key_6":"e", "__series_id__":-5266012011623334172, "__mgmt_id__":-5266012011623334172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111575, "value1":0.7142448949298847, "value2":2599619943612027614, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000300", "key_1":"c","key_4":"c","key_8":"g", "__series_id__":-3153928638315052797, "__mgmt_id__":-3153928638315052797} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.720934, "value1":0.46314177190410183, "value2":5473312180675819943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000240", "key_8":"d","key_9":"c","key_4":"d", "__series_id__":-6076759433305586941, "__mgmt_id__":-6076759433305586941} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.542023, "value1":0.17074635857117051, "value2":576994586783200158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000241", "key_4":"e","key_8":"a","key_9":"d", "__series_id__":8834769746480377365, "__mgmt_id__":8834769746480377365} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862606, "value1":0.849600128173387, "value2":4044011490168844428, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000302", "key_1":"d","key_4":"a","key_6":"a", "__series_id__":-8329489313689560671, "__mgmt_id__":-8329489313689560671} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.581818, "value1":0.4717710471390932, "value2":2021524622009984217, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000303", "key_2":"j","key_6":"k","key_0":"c", "__series_id__":-9030241219395401074, "__mgmt_id__":-9030241219395401074} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.634138, "value1":0.3045108371861026, "value2":769588972141174181, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000305", "key_6":"j","key_8":"h","key_9":"d", "__series_id__":5111506451882024399, "__mgmt_id__":5111506451882024399} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.554282, "value1":0.9061849342045651, "value2":5246859609213477778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000304", "key_3":"b","key_6":"f","key_8":"d", "__series_id__":-6099394299024364632, "__mgmt_id__":-6099394299024364632} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.275353, "value1":0.12209379665466558, "value2":3903264969659125855, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000306", "key_8":"f","key_9":"i","key_5":"d", "__series_id__":-2889220439504706291, "__mgmt_id__":-2889220439504706291} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931382, "value1":0.38822648110972485, "value2":4321311553224557998, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000308", "key_1":"f","key_2":"f","key_4":"b", "__series_id__":-77561723918691242, "__mgmt_id__":-77561723918691242} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.703867, "value1":0.6637597201672076, "value2":5683430536850722079, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000307", "key_4":"j","key_6":"b","key_8":"d", "__series_id__":-1773617826702713581, "__mgmt_id__":-1773617826702713581} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958497, "value1":0.011335239730124937, "value2":2491874770397540377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000309", "key_2":"b","key_5":"h","key_6":"k", "__series_id__":5942386536226088787, "__mgmt_id__":5942386536226088787} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569325, "value1":0.8472934144463635, "value2":6850134665353105473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000301", "key_4":"k","key_7":"i","key_9":"k", "__series_id__":1447592414457730764, "__mgmt_id__":1447592414457730764} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564173, "value1":0.28056996036761495, "value2":1661710999114510072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000237", "key_0":"d","key_5":"h","key_7":"f", "__series_id__":4924210048711090822, "__mgmt_id__":4924210048711090822} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.046264, "value1":0.7988719698656381, "value2":80936298017494610, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000233", "key_3":"e","key_4":"j","key_9":"i", "__series_id__":1859691237979991845, "__mgmt_id__":1859691237979991845} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.086050, "value1":0.7269357269059649, "value2":3703081511929708827, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000234", "key_0":"f","key_4":"d","key_9":"a", "__series_id__":6844297030045792921, "__mgmt_id__":6844297030045792921} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.691310, "value1":0.968367674569065, "value2":8685309070712113009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000235", "key_1":"h","key_5":"f","key_8":"e", "__series_id__":-126935382287702459, "__mgmt_id__":-126935382287702459} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.734580, "value1":0.16065685872806734, "value2":8341255006788851479, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000236", "key_0":"d","key_1":"h", "__series_id__":-8355447172459145172, "__mgmt_id__":-8355447172459145172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110471, "value1":0.2857648075220212, "value2":7895547866112316696, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000239", "key_9":"d","key_4":"c","key_7":"e", "__series_id__":8424755961445166330, "__mgmt_id__":8424755961445166330} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.594791, "value1":0.6011580461407761, "value2":8667655502460513802, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000238", "key_4":"a","key_5":"k","key_7":"e", "__series_id__":8975641431294639588, "__mgmt_id__":8975641431294639588} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.074613, "value1":0.7518175733891237, "value2":6808912328433269494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000310", "key_0":"b","key_4":"f","key_6":"h", "__series_id__":-5148903865376086541, "__mgmt_id__":-5148903865376086541} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827510, "value1":0.4635989764886008, "value2":8535571864109924880, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000312", "key_0":"a","key_4":"c","key_9":"i", "__series_id__":1983990905315571262, "__mgmt_id__":1983990905315571262} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020243, "value1":0.9625575743602495, "value2":3255772979573003294, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000311", "key_0":"b","key_2":"f","key_5":"d", "__series_id__":7756220639980209274, "__mgmt_id__":7756220639980209274} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.508001, "value1":0.9246513785407203, "value2":6789019308045406067, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000315", "key_3":"e","key_2":"b", "__series_id__":290238844948171466, "__mgmt_id__":290238844948171466} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.175770, "value1":0.6231844141584031, "value2":5604032128383909386, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000313", "key_9":"g","key_2":"f","key_4":"c", "__series_id__":2212165241620182378, "__mgmt_id__":2212165241620182378} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338187, "value1":0.7085089535016765, "value2":7296894563489025180, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000314", "key_3":"c","key_4":"i","key_6":"g", "__series_id__":-5641349172597598498, "__mgmt_id__":-5641349172597598498} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924177, "value1":0.9072759724228442, "value2":2056681116966727911, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000318", "key_7":"f","key_1":"e","key_4":"d", "__series_id__":4136822437169743683, "__mgmt_id__":4136822437169743683} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474372, "value1":0.09374734312421011, "value2":8184327150731414451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000316", "key_4":"f","key_6":"k","key_8":"i", "__series_id__":5588418121070213379, "__mgmt_id__":5588418121070213379} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970300, "value1":0.9441105421259884, "value2":6689514820107172641, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000317", "key_4":"f","key_9":"f","key_0":"e", "__series_id__":5487859603492872728, "__mgmt_id__":5487859603492872728} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.035348, "value1":0.14347791413195674, "value2":2423771496925044659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000320", "key_2":"d","key_3":"h","key_7":"c", "__series_id__":-7510162853285205458, "__mgmt_id__":-7510162853285205458} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.807051, "value1":0.1533684521364213, "value2":8351770123785179919, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000319", "key_2":"b","key_9":"g", "__series_id__":8960166785941818502, "__mgmt_id__":8960166785941818502} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814282, "value1":0.48165153274600025, "value2":2348575893675627866, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000322", "key_7":"k","key_0":"b","key_6":"d", "__series_id__":1191951441001778119, "__mgmt_id__":1191951441001778119} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438370, "value1":0.4065426353146516, "value2":5566206066472585427, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000321", "key_5":"d","key_7":"e","key_8":"j", "__series_id__":-7226896218622797912, "__mgmt_id__":-7226896218622797912} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.321909, "value1":0.057863583124824335, "value2":2888660799506273189, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000323", "key_2":"c","key_7":"d", "__series_id__":-5283488199064223774, "__mgmt_id__":-5283488199064223774} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.191383, "value1":0.5857228209038977, "value2":343682983842502232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000324", "key_5":"f","key_0":"j","key_1":"k", "__series_id__":-2097373818072827151, "__mgmt_id__":-2097373818072827151} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.030645, "value1":0.054007147421613756, "value2":7408821068820948342, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000325", "key_0":"f","key_3":"g","key_4":"c", "__series_id__":1668656312027665848, "__mgmt_id__":1668656312027665848} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.022320, "value1":0.5082171149433462, "value2":708896583672613900, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000326", "key_4":"g","key_7":"b","key_9":"c", "__series_id__":-3365185338503951486, "__mgmt_id__":-3365185338503951486} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.359147, "value1":0.10928795884420436, "value2":5509040341747881439, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000327", "key_0":"a","key_1":"j","key_9":"j", "__series_id__":-1972359892487806876, "__mgmt_id__":-1972359892487806876} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245107, "value1":0.08990020039433881, "value2":4415685661085482000, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000328", "key_8":"d","key_6":"h","key_7":"h", "__series_id__":-5888979106861644322, "__mgmt_id__":-5888979106861644322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.685276, "value1":0.9327230470765273, "value2":7932626821430255186, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000329", "key_0":"a","key_6":"h", "__series_id__":4350337870315148248, "__mgmt_id__":4350337870315148248} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160518, "value1":0.9697823127450904, "value2":9176665002489756811, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000330", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851145, "value1":0.3835892105704814, "value2":5122340871731266104, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000331", "key_3":"g","key_9":"b","key_1":"h", "__series_id__":407580274955802052, "__mgmt_id__":407580274955802052} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390684, "value1":0.8638095835380812, "value2":2196309567306303445, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000332", "key_1":"j","key_7":"d", "__series_id__":5055564084756188165, "__mgmt_id__":5055564084756188165} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197750, "value1":0.5321904528249127, "value2":7059427737086849843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000333", "key_1":"f","key_4":"b","key_9":"e", "__series_id__":-4657725461050027132, "__mgmt_id__":-4657725461050027132} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.606374, "value1":0.1712913717696049, "value2":6671507460576496016, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000334", "key_3":"h","key_6":"k", "__series_id__":1853720382505804304, "__mgmt_id__":1853720382505804304} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814189, "value1":0.21162810005419255, "value2":6089699189069603789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000335", "key_0":"k","key_5":"e", "__series_id__":-4222087669256355081, "__mgmt_id__":-4222087669256355081} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.565029, "value1":0.651039299686122, "value2":2488735588911638142, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000336", "key_0":"k","key_3":"e","key_7":"e", "__series_id__":9141431587741375337, "__mgmt_id__":9141431587741375337} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310043, "value1":0.3341800696393593, "value2":1886591498798226069, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000337", "key_4":"b","key_0":"b","key_1":"c", "__series_id__":3652241825851980931, "__mgmt_id__":3652241825851980931} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.646773, "value1":0.587523277238801, "value2":3984875805092838666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000338", "key_2":"j","key_5":"e", "__series_id__":8334871318878782618, "__mgmt_id__":8334871318878782618} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487672, "value1":0.4085632878496485, "value2":4006763171436697564, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000339", "key_4":"k","key_5":"b","key_9":"g", "__series_id__":-3854993404396318547, "__mgmt_id__":-3854993404396318547} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180539, "value1":0.614724172390471, "value2":3474361726777387017, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000340", "key_0":"i","key_3":"d","key_8":"i", "__series_id__":3475233701191086871, "__mgmt_id__":3475233701191086871} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413423, "value1":0.8015313942257714, "value2":3699130733479555941, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000341", "key_4":"b","key_5":"j","key_6":"e", "__series_id__":-3248778423373175907, "__mgmt_id__":-3248778423373175907} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375412, "value1":0.6194400337512462, "value2":89560215401334488, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000342", "key_0":"c","key_1":"j","key_9":"h", "__series_id__":-5544260272497947688, "__mgmt_id__":-5544260272497947688} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739736, "value1":0.01565923183582112, "value2":2040823465506133480, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000345", "key_9":"a","key_1":"j","key_8":"k", "__series_id__":6759359888504379937, "__mgmt_id__":6759359888504379937} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164236, "value1":0.05609752284300993, "value2":459144572970603788, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000343", "key_5":"a","key_8":"g","key_3":"b", "__series_id__":2983615759963648182, "__mgmt_id__":2983615759963648182} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857605, "value1":0.3918518578057805, "value2":2660076743726482223, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000344", "key_5":"e","key_8":"i","key_0":"d", "__series_id__":-5636891213483281862, "__mgmt_id__":-5636891213483281862} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182324, "value1":0.19737269797519194, "value2":4090312209234045823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000348", "key_5":"e","key_3":"h","key_4":"k", "__series_id__":5337681060194140560, "__mgmt_id__":5337681060194140560} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.413638, "value1":0.23128603835680528, "value2":2432247666638343859, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000346", "key_4":"e","key_8":"g","key_9":"c", "__series_id__":-8711805126627568168, "__mgmt_id__":-8711805126627568168} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029664, "value1":0.7948810691175306, "value2":1090049786919167701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000347", "key_2":"f","key_5":"f","key_7":"b", "__series_id__":-3947171592563903621, "__mgmt_id__":-3947171592563903621} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953471, "value1":0.7020043042253941, "value2":4064796540960008955, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000350", "key_7":"k","key_9":"d","key_0":"f", "__series_id__":-6388370063319424844, "__mgmt_id__":-6388370063319424844} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925306, "value1":0.8273248639036314, "value2":7383533353614933778, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000349", "key_2":"i","key_3":"j","key_4":"b", "__series_id__":-2115245116045217035, "__mgmt_id__":-2115245116045217035} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.353560, "value1":0.7154582581173291, "value2":7672433526090470392, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000351", "key_2":"e","key_6":"c", "__series_id__":6934366355856667102, "__mgmt_id__":6934366355856667102} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.440133, "value1":0.8443212771497766, "value2":4871633339564000823, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000352", "key_2":"e","key_1":"j", "__series_id__":-620632640643666803, "__mgmt_id__":-620632640643666803} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079309, "value1":0.9841787259191465, "value2":5158435517537975833, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000353", "key_5":"b","key_9":"a","key_0":"a", "__series_id__":-2179974092013090529, "__mgmt_id__":-2179974092013090529} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687146, "value1":0.547850782348478, "value2":107890618456621161, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000354", "key_9":"b","key_0":"i","key_8":"d", "__series_id__":-296190224456934318, "__mgmt_id__":-296190224456934318} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.013887, "value1":0.9829126596506367, "value2":2398801344797864345, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000355", "key_2":"d","key_3":"e","key_9":"e", "__series_id__":8742164021866916569, "__mgmt_id__":8742164021866916569} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142865, "value1":0.17685569327146947, "value2":2003576222991068603, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000356", "key_0":"f", "__series_id__":4071060278153716716, "__mgmt_id__":4071060278153716716} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.503089, "value1":0.2089707954081848, "value2":5329263674364077685, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000357", "key_2":"f","key_5":"d","key_8":"c", "__series_id__":-9118286530436017049, "__mgmt_id__":-9118286530436017049} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055628, "value1":0.9427569700789281, "value2":8376349363379609336, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000358", "key_9":"g","key_6":"b","key_8":"a", "__series_id__":731731928735478358, "__mgmt_id__":731731928735478358} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663522, "value1":0.5447970553906242, "value2":7034169322760923664, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000360", "key_0":"a","key_5":"j","key_9":"f", "__series_id__":7612998775307430333, "__mgmt_id__":7612998775307430333} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.569193, "value1":0.4748210869235274, "value2":5009934188959179459, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000359", "key_3":"k","key_6":"j","key_9":"k", "__series_id__":1213816084730454228, "__mgmt_id__":1213816084730454228} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.964851, "value1":0.1480433642451733, "value2":5780759776802899813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000362", "key_0":"c","key_1":"d", "__series_id__":-3958867498699920655, "__mgmt_id__":-3958867498699920655} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.674266, "value1":0.01926887357642196, "value2":5539462413096535, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000361", "key_3":"h","key_7":"j","key_9":"c", "__series_id__":-8505220651241159003, "__mgmt_id__":-8505220651241159003} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397018, "value1":0.3343175727423989, "value2":8363367362679630761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000364", "key_8":"c","key_2":"e","key_7":"k", "__series_id__":-2954096273482449380, "__mgmt_id__":-2954096273482449380} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.665761, "value1":0.9909509158985629, "value2":598831903509406706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000363", "key_0":"h","key_3":"f","key_6":"e", "__series_id__":8139057134364652777, "__mgmt_id__":8139057134364652777} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225685, "value1":0.9692539977538356, "value2":6753560478082272468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000365", "key_1":"g","key_4":"f", "__series_id__":-5686626809685373979, "__mgmt_id__":-5686626809685373979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.857888, "value1":0.3089073345023022, "value2":469238271712070571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000366", "key_2":"j","key_6":"i","key_8":"b", "__series_id__":3877400967867596203, "__mgmt_id__":3877400967867596203} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296144, "value1":0.8953425439665634, "value2":5808544938153527415, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000367", "key_7":"c","key_9":"a","key_2":"k", "__series_id__":-6807168092950711042, "__mgmt_id__":-6807168092950711042} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641028, "value1":0.26770927578250153, "value2":4804718717106479393, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000368", "key_1":"b","key_8":"d", "__series_id__":5574887167499930380, "__mgmt_id__":5574887167499930380} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.865055, "value1":0.21277992187222125, "value2":7652322888419294762, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000369", "key_5":"i","key_6":"e","key_7":"a", "__series_id__":771413411855505413, "__mgmt_id__":771413411855505413} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.147216, "value1":0.4361812114988848, "value2":5453489060842625813, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000370", "key_1":"b","key_6":"c","key_8":"i", "__series_id__":-6271489171177472422, "__mgmt_id__":-6271489171177472422} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931652, "value1":0.9338772117034952, "value2":7926554101044226499, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000371", "key_8":"b","key_9":"k", "__series_id__":-4846483967205784738, "__mgmt_id__":-4846483967205784738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451647, "value1":0.49187863740688065, "value2":626417169671954650, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000372", "key_3":"e","key_6":"b","key_9":"a", "__series_id__":-8691105454096685296, "__mgmt_id__":-8691105454096685296} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.552836, "value1":0.08117641553234622, "value2":6479037192065714806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000373", "key_2":"j","key_7":"c","key_9":"g", "__series_id__":8557606287926271463, "__mgmt_id__":8557606287926271463} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.644667, "value1":0.8782871355247011, "value2":7618066153253340565, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000374", "key_5":"k","key_8":"j","key_9":"a", "__series_id__":-1732152959738129259, "__mgmt_id__":-1732152959738129259} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.486533, "value1":0.34858818937194813, "value2":8450118240549444684, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000375", "key_1":"j","key_6":"a","key_7":"d", "__series_id__":-7668104505878687309, "__mgmt_id__":-7668104505878687309} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080371, "value1":0.1720187423008763, "value2":7683320479544120606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000376", "key_0":"j","key_3":"d","key_5":"e", "__series_id__":5740007109533794595, "__mgmt_id__":5740007109533794595} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527916, "value1":0.09000309934383972, "value2":5784629514237788639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000377", "key_1":"b","key_9":"h", "__series_id__":-6226140292610819318, "__mgmt_id__":-6226140292610819318} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.682327, "value1":0.9194439212395367, "value2":4269189023261883692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000378", "key_0":"e","key_6":"f","key_8":"d", "__series_id__":2424948940195949107, "__mgmt_id__":2424948940195949107} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.961106, "value1":0.4405062734385544, "value2":1799260749356167755, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000379", "key_3":"f","key_7":"a","key_9":"k", "__series_id__":1112883101154142655, "__mgmt_id__":1112883101154142655} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.640959, "value1":0.04934157301568955, "value2":4101326969656042416, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000382", "key_3":"j","key_6":"d", "__series_id__":3159384072740264381, "__mgmt_id__":3159384072740264381} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827544, "value1":0.6093692966542871, "value2":5408051744545184279, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000380", "key_0":"j","key_1":"f","key_4":"j", "__series_id__":-2354035420748708685, "__mgmt_id__":-2354035420748708685} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.916148, "value1":0.37486434605250213, "value2":8005097965237384093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000381", "key_2":"c","key_6":"k","key_9":"d", "__series_id__":608500456554677241, "__mgmt_id__":608500456554677241} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477104, "value1":0.7823899457952176, "value2":2425125801220437344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000384", "key_0":"j","key_2":"a","key_9":"f", "__series_id__":1099531734875113482, "__mgmt_id__":1099531734875113482} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.631094, "value1":0.38182215652170554, "value2":6885679048992226816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000383", "key_1":"j","key_4":"f", "__series_id__":2312983566483226074, "__mgmt_id__":2312983566483226074} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242126, "value1":0.897478447775271, "value2":5819274494025981203, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000386", "key_4":"g","key_8":"b","key_2":"c", "__series_id__":-381978251896117922, "__mgmt_id__":-381978251896117922} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134175, "value1":0.8364024700095963, "value2":2669419474556636937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000385", "key_2":"b","key_7":"h","key_0":"b", "__series_id__":7921693770137371726, "__mgmt_id__":7921693770137371726} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.785456, "value1":0.47900969825766393, "value2":427990087950056586, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000387", "key_0":"d","key_6":"c","key_8":"c", "__series_id__":-501490243711306648, "__mgmt_id__":-501490243711306648} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.702759, "value1":0.4509768057551493, "value2":6899018348994248896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000388", "key_6":"g","key_7":"i","key_9":"i", "__series_id__":8692326850526983033, "__mgmt_id__":8692326850526983033} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.844804, "value1":0.4506687041971474, "value2":2704465019491437568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000389", "key_0":"j","key_6":"c","key_7":"b", "__series_id__":8901305062217396154, "__mgmt_id__":8901305062217396154} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841097, "value1":0.08259262088200636, "value2":3254311166592288697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000390", "key_1":"k","key_2":"b","key_8":"i", "__series_id__":9008787801683632143, "__mgmt_id__":9008787801683632143} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.579894, "value1":0.634370403808335, "value2":8239103463468673989, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000391", "key_2":"j","key_4":"d","key_9":"c", "__series_id__":-69191581506590721, "__mgmt_id__":-69191581506590721} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.671756, "value1":0.3978266397618443, "value2":2011463990612690211, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000392", "key_2":"a","key_3":"e", "__series_id__":8615557143580011860, "__mgmt_id__":8615557143580011860} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639712, "value1":0.5843126119887666, "value2":8514657269434627024, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000393", "key_3":"h","key_8":"j","key_9":"j", "__series_id__":6714523309571458463, "__mgmt_id__":6714523309571458463} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.090535, "value1":0.47166906237349404, "value2":2986627777164864344, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000394", "key_5":"h","key_6":"h","key_8":"j", "__series_id__":-2436038626322037217, "__mgmt_id__":-2436038626322037217} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.079733, "value1":0.49607189078610653, "value2":7935174201590062011, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000395", "key_2":"e", "__series_id__":-9153846671465256928, "__mgmt_id__":-9153846671465256928} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484442, "value1":0.10087228739324149, "value2":5758223217302730640, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000396", "key_2":"i","key_3":"i","key_5":"f", "__series_id__":-5512922667133642205, "__mgmt_id__":-5512922667133642205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.881297, "value1":0.743802063178793, "value2":7666746241262123009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000399", "key_4":"h","key_9":"g", "__series_id__":-2023745873911087429, "__mgmt_id__":-2023745873911087429} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189244, "value1":0.8471771982112483, "value2":8717600770763450338, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000397", "key_0":"k","key_2":"f","key_7":"d", "__series_id__":5253653569099281848, "__mgmt_id__":5253653569099281848} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.958729, "value1":0.24046081337833752, "value2":2429080107883352722, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000398", "key_6":"j","key_3":"k","key_4":"k", "__series_id__":2768830248655599965, "__mgmt_id__":2768830248655599965} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.904544, "value1":0.5967867401359257, "value2":5915623307556355066, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000402", "key_2":"h","key_4":"k","key_7":"j", "__series_id__":5713557924465070249, "__mgmt_id__":5713557924465070249} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.016356, "value1":0.2664341494092446, "value2":6082384923182522956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000400", "key_0":"f","key_1":"c","key_6":"c", "__series_id__":3213205110735329894, "__mgmt_id__":3213205110735329894} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906737, "value1":0.8646356549293164, "value2":6093331013624350146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000401", "key_2":"j","key_3":"k","key_6":"h", "__series_id__":75456790631548363, "__mgmt_id__":75456790631548363} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.650026, "value1":0.17820070872685148, "value2":2506978860590163255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000404", "key_0":"f","key_3":"b", "__series_id__":7710697653197251649, "__mgmt_id__":7710697653197251649} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.380897, "value1":0.559051486368018, "value2":959051280528283072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000403", "key_1":"k","key_2":"i","key_6":"i", "__series_id__":6985690126792518272, "__mgmt_id__":6985690126792518272} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.645954, "value1":0.44113348983757733, "value2":3640682441395107897, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000406", "key_2":"i","key_6":"j","key_7":"f", "__series_id__":-2773568618767656687, "__mgmt_id__":-2773568618767656687} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.705006, "value1":0.7603563234560595, "value2":185556188647790850, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000405", "key_0":"e","key_1":"b","key_9":"j", "__series_id__":7861793687162385908, "__mgmt_id__":7861793687162385908} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.946294, "value1":0.6649942159160177, "value2":8061815176820463265, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000408", "key_5":"d","key_8":"i","key_0":"e", "__series_id__":45276947053002039, "__mgmt_id__":45276947053002039} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.007871, "value1":0.472486274082402, "value2":8110170220979486982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000407", "key_3":"a","key_6":"e","key_8":"c", "__series_id__":2924521682348973507, "__mgmt_id__":2924521682348973507} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.817193, "value1":0.01062083501470273, "value2":8652411971395108438, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000409", "key_4":"f","key_5":"c","key_7":"g", "__series_id__":-2737327345688605951, "__mgmt_id__":-2737327345688605951} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.203058, "value1":0.2545105965864827, "value2":3692372485982923495, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000410", "key_1":"i","key_5":"h","key_7":"j", "__series_id__":6562077812082021205, "__mgmt_id__":6562077812082021205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724358, "value1":0.49614729928160556, "value2":794673815448737267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000411", "key_4":"d","key_7":"g","key_8":"c", "__series_id__":-2272216867643398248, "__mgmt_id__":-2272216867643398248} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.029363, "value1":0.8128873282396352, "value2":5575910506626889012, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000412", "key_3":"b","key_6":"b","key_1":"b", "__series_id__":5144841073600140923, "__mgmt_id__":5144841073600140923} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.388594, "value1":0.37922666609989497, "value2":5857319265944344227, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000413", "key_2":"f","key_7":"a", "__series_id__":4158991472976645693, "__mgmt_id__":4158991472976645693} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.111761, "value1":0.07280095456744823, "value2":7190870932028618354, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000417", "key_1":"a","key_9":"c", "__series_id__":6965159951419158515, "__mgmt_id__":6965159951419158515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556559, "value1":0.011144718316818122, "value2":5637958779343170502, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000414", "key_4":"i","key_5":"i","key_8":"k", "__series_id__":-3222371660815337538, "__mgmt_id__":-3222371660815337538} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.900852, "value1":0.2436048429102491, "value2":3178328441413468775, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000415", "key_4":"j","key_7":"k","key_9":"e", "__series_id__":-1567565762361025496, "__mgmt_id__":-1567565762361025496} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.746269, "value1":0.44159443384096786, "value2":1064726188435771697, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000416", "key_3":"h","key_6":"i","key_9":"b", "__series_id__":-1238077055518429859, "__mgmt_id__":-1238077055518429859} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.089137, "value1":0.5802300524767475, "value2":1179475447484216893, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000420", "key_4":"g","key_7":"b","key_8":"d", "__series_id__":3771919008646036855, "__mgmt_id__":3771919008646036855} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045149, "value1":0.8412834417368987, "value2":2843566571645769718, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000418", "key_1":"c","key_6":"i","key_7":"e", "__series_id__":-1309345613761574603, "__mgmt_id__":-1309345613761574603} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.032148, "value1":0.9106893921082526, "value2":2375027858901255873, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000419", "key_9":"c","key_1":"c", "__series_id__":1682100904241849578, "__mgmt_id__":1682100904241849578} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.180832, "value1":0.8856494080629095, "value2":2259505020949178568, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000423", "key_6":"k","key_8":"e", "__series_id__":-8447273919193029542, "__mgmt_id__":-8447273919193029542} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571194, "value1":0.2087472777529694, "value2":3787711915274839208, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000421", "key_0":"f","key_9":"f", "__series_id__":7000894493225784097, "__mgmt_id__":7000894493225784097} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.113569, "value1":0.6253289147788459, "value2":2290822587909135561, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000422", "key_8":"a","key_0":"g","key_3":"b", "__series_id__":-7658939295344412745, "__mgmt_id__":-7658939295344412745} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708878, "value1":0.3051231954056551, "value2":536614295748659218, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000425", "key_3":"e","key_8":"d","key_9":"k", "__series_id__":-3816976869357525258, "__mgmt_id__":-3816976869357525258} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397834, "value1":0.004032765819560939, "value2":2143090210108317054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000424", "key_1":"i","key_2":"d","key_8":"b", "__series_id__":-1506749977011375670, "__mgmt_id__":-1506749977011375670} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.279982, "value1":0.6698352738967139, "value2":8265876412027179825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000427", "key_2":"i","key_5":"h","key_7":"h", "__series_id__":-5166232873100135825, "__mgmt_id__":-5166232873100135825} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.117595, "value1":0.36898116171165335, "value2":7917937119372643007, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000426", "key_1":"e","key_2":"f","key_5":"e", "__series_id__":6996402696140969043, "__mgmt_id__":6996402696140969043} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.490128, "value1":0.7119026496334794, "value2":4217226709077888871, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000428", "key_0":"k","key_6":"f", "__series_id__":-3373007512765482806, "__mgmt_id__":-3373007512765482806} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.337373, "value1":0.24700927278867024, "value2":8633933010561287594, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000430", "key_4":"c","key_9":"j","key_1":"j", "__series_id__":135101285191202942, "__mgmt_id__":135101285191202942} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393403, "value1":0.3187858750570018, "value2":7155530342572901954, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000429", "key_5":"e","key_6":"c","key_9":"d", "__series_id__":-7303512666916957767, "__mgmt_id__":-7303512666916957767} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787897, "value1":0.731243787879042, "value2":3162978630167698275, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000435", "key_3":"j","key_5":"a","key_9":"h", "__series_id__":5515312804905743253, "__mgmt_id__":5515312804905743253} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.394636, "value1":0.15744716640466827, "value2":6804804340000012886, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000441", "key_0":"i","key_3":"h","key_6":"f", "__series_id__":-4447699968119962519, "__mgmt_id__":-4447699968119962519} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589125, "value1":0.9206991440936233, "value2":3240974047558242833, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000442", "key_1":"i","key_2":"e","key_6":"f", "__series_id__":-3084091086636934702, "__mgmt_id__":-3084091086636934702} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122199, "value1":0.9027140609185327, "value2":4793574504420000662, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000443", "key_0":"i","key_7":"k", "__series_id__":2885514682724414397, "__mgmt_id__":2885514682724414397} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.739368, "value1":0.0296490298815606, "value2":7450276243203379932, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000444", "key_4":"e","key_1":"b", "__series_id__":-5450738729835841525, "__mgmt_id__":-5450738729835841525} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839046, "value1":0.885358696687765, "value2":2580404634255699486, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000445", "key_2":"i","key_3":"k","key_4":"a", "__series_id__":-4333434873637454832, "__mgmt_id__":-4333434873637454832} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432366, "value1":0.6379661863831838, "value2":4996786498907777190, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000431", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129751, "value1":0.5622916100360914, "value2":8341777138936467511, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000432", "key_2":"k","key_3":"b", "__series_id__":6925990235282897427, "__mgmt_id__":6925990235282897427} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202469, "value1":0.24574613351572672, "value2":7146206121672605367, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000433", "key_2":"c","key_7":"c","key_9":"g", "__series_id__":-5525623792640186251, "__mgmt_id__":-5525623792640186251} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.832142, "value1":0.5178470928944944, "value2":4133801221430187756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000434", "key_4":"d","key_5":"b", "__series_id__":-662733069139439850, "__mgmt_id__":-662733069139439850} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080311, "value1":0.486175008187209, "value2":5039371713095786937, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000439", "key_4":"e","key_5":"i","key_6":"b", "__series_id__":-2265846297529327958, "__mgmt_id__":-2265846297529327958} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723875, "value1":0.552001350718302, "value2":7101322585254858336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000436", "key_2":"h","key_4":"c","key_5":"c", "__series_id__":-2289697776939494418, "__mgmt_id__":-2289697776939494418} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434143, "value1":0.6525700542145968, "value2":2081065923746015157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000437", "key_3":"d","key_8":"a","key_0":"b", "__series_id__":7170156078773313786, "__mgmt_id__":7170156078773313786} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.410329, "value1":0.15418313782350743, "value2":2121964379701341165, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000438", "key_1":"j","key_6":"f", "__series_id__":-1249302972235794145, "__mgmt_id__":-1249302972235794145} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423463, "value1":0.984744888870281, "value2":7148485292923472325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000447", "key_0":"e","key_5":"g","key_8":"i", "__series_id__":-2140177391110748196, "__mgmt_id__":-2140177391110748196} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658931, "value1":0.9086990564903296, "value2":4792038160363904869, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000440", "key_4":"h","key_1":"j","key_3":"h", "__series_id__":6087272256412147288, "__mgmt_id__":6087272256412147288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.891060, "value1":0.6990632896148243, "value2":8537384774472959390, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000446", "key_9":"a","key_0":"b","key_4":"j", "__series_id__":1653825528745674615, "__mgmt_id__":1653825528745674615} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.154122, "value1":0.10983855840529697, "value2":3710113544658117053, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000448", "key_1":"a","key_5":"c","key_0":"i", "__series_id__":-6800341232275211831, "__mgmt_id__":-6800341232275211831} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.543143, "value1":0.3045282199429521, "value2":6184622652371484263, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000449", "key_1":"i","key_4":"i","key_6":"j", "__series_id__":9133274435304278352, "__mgmt_id__":9133274435304278352} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174103, "value1":0.1622180223685842, "value2":1046120514820502820, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000450", "key_3":"h","key_8":"k","key_9":"g", "__series_id__":-8792288547333335206, "__mgmt_id__":-8792288547333335206} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393987, "value1":0.12110279784132964, "value2":5830532384578071588, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000451", "key_1":"a","key_4":"h","key_5":"c", "__series_id__":-5438437909542518597, "__mgmt_id__":-5438437909542518597} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907289, "value1":0.9509521630571898, "value2":4933206466498891083, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000452", "key_0":"i","key_9":"h", "__series_id__":-2968545362379784314, "__mgmt_id__":-2968545362379784314} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.987070, "value1":0.5242278192300958, "value2":7374919878503376521, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000454", "key_6":"g","key_2":"a","key_3":"f", "__series_id__":2472948582313971389, "__mgmt_id__":2472948582313971389} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.641701, "value1":0.3760479049716257, "value2":4296425671360614419, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000453", "key_2":"g","key_3":"e", "__series_id__":-7988236866042391249, "__mgmt_id__":-7988236866042391249} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.460082, "value1":0.7715932085121491, "value2":4216486260610422297, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000456", "key_1":"j","key_3":"c","key_5":"f", "__series_id__":8355473217236408944, "__mgmt_id__":8355473217236408944} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.308698, "value1":0.6733032497636853, "value2":8314310095556752952, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000455", "key_3":"b","key_9":"k", "__series_id__":8293129132311830565, "__mgmt_id__":8293129132311830565} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356337, "value1":0.9180371571261539, "value2":7442497884658250592, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000457", "key_3":"g","key_7":"d","key_8":"h", "__series_id__":-4752720542644485754, "__mgmt_id__":-4752720542644485754} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407046, "value1":0.8826200541831951, "value2":6272206334208867752, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000459", "key_1":"k","key_6":"h","key_7":"g", "__series_id__":-4674174992473955574, "__mgmt_id__":-4674174992473955574} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263547, "value1":0.5493018603625203, "value2":5332141292811821371, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000458", "key_0":"j","key_6":"d","key_7":"b", "__series_id__":-3194670603174080572, "__mgmt_id__":-3194670603174080572} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.690903, "value1":0.6510082736025081, "value2":2282336418703477702, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000460", "key_3":"d","key_5":"k", "__series_id__":6308978262160939848, "__mgmt_id__":6308978262160939848} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599129, "value1":0.21529129751322154, "value2":5758745763053757888, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000461", "key_9":"j","key_8":"j", "__series_id__":1579468215752806140, "__mgmt_id__":1579468215752806140} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.294727, "value1":0.17537508292575937, "value2":1562987372702839556, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000463", "key_0":"g","key_3":"e","key_5":"a", "__series_id__":-7402209760770487886, "__mgmt_id__":-7402209760770487886} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.905165, "value1":0.5372817599107224, "value2":5788710138851546437, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000462", "key_6":"d","key_8":"g","key_9":"d", "__series_id__":-6170144038960690855, "__mgmt_id__":-6170144038960690855} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714618, "value1":0.5147887871360798, "value2":2734992135513585397, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000464", "key_0":"a","key_1":"j","key_3":"c", "__series_id__":-5087344553152355661, "__mgmt_id__":-5087344553152355661} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.845879, "value1":0.03659856815217442, "value2":717088561981717268, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000465", "key_1":"f","key_3":"h","key_6":"f", "__series_id__":5180580127014211139, "__mgmt_id__":5180580127014211139} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.172178, "value1":0.5465356547159791, "value2":6019732615424773946, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000466", "key_0":"b","key_5":"g","key_7":"h", "__series_id__":7455696895619819425, "__mgmt_id__":7455696895619819425} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263201, "value1":0.7472793901983326, "value2":6735352007287694901, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000467", "key_4":"i","key_8":"a","key_2":"d", "__series_id__":-588642916571085263, "__mgmt_id__":-588642916571085263} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418982, "value1":0.029853021112705932, "value2":7962917578850181628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000468", "key_1":"j","key_5":"i","key_8":"k", "__series_id__":8224240495996749932, "__mgmt_id__":8224240495996749932} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.866946, "value1":0.10874512532228968, "value2":4523576734362004451, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000469", "key_1":"c","key_7":"k","key_9":"b", "__series_id__":4454815443328996164, "__mgmt_id__":4454815443328996164} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.556178, "value1":0.3725090999387309, "value2":8591394434164967395, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000470", "key_0":"b","key_2":"i", "__series_id__":6592318257860924169, "__mgmt_id__":6592318257860924169} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827178, "value1":0.564605338181153, "value2":1565791376780985293, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000472", "key_9":"d","key_5":"b","key_8":"j", "__series_id__":-4340249040349917919, "__mgmt_id__":-4340249040349917919} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.427240, "value1":0.5937818761029834, "value2":725738161862814755, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000471", "key_1":"b","key_3":"f","key_8":"b", "__series_id__":-7157525334911084152, "__mgmt_id__":-7157525334911084152} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.153851, "value1":0.6812022447305227, "value2":2627803719401441193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000473", "key_7":"k","key_0":"h","key_3":"g", "__series_id__":-501644526880268050, "__mgmt_id__":-501644526880268050} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982516, "value1":0.5197201488713766, "value2":5245967664424169262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000474", "key_8":"f","key_3":"k","key_4":"c", "__series_id__":-5128277451134372125, "__mgmt_id__":-5128277451134372125} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.922654, "value1":0.8797142069534722, "value2":2538441830233727435, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000475", "key_3":"b","key_5":"a","key_7":"k", "__series_id__":-5102999602639548770, "__mgmt_id__":-5102999602639548770} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036732, "value1":0.8882312994752156, "value2":2860925793915692434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000476", "key_4":"g","key_6":"b","key_9":"f", "__series_id__":341574877665132696, "__mgmt_id__":341574877665132696} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.848915, "value1":0.5650391916990911, "value2":8021099293155564160, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000477", "key_0":"b","key_3":"f","key_6":"k", "__series_id__":7373492341893200542, "__mgmt_id__":7373492341893200542} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.189564, "value1":0.5152547802767392, "value2":1569067174314315770, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000478", "key_2":"k","key_4":"f","key_9":"h", "__series_id__":1901060728526527618, "__mgmt_id__":1901060728526527618} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.802112, "value1":0.9536154419324024, "value2":6565733445920939612, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000480", "key_1":"b","key_6":"k","key_8":"b", "__series_id__":4352772564110544322, "__mgmt_id__":4352772564110544322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.983922, "value1":0.4632188030903474, "value2":3429972812413807249, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000479", "key_3":"i","key_7":"i","key_9":"j", "__series_id__":1055170066665701026, "__mgmt_id__":1055170066665701026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477177, "value1":0.333324845660299, "value2":4203450320031165809, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000482", "key_2":"f","key_3":"f","key_7":"g", "__series_id__":8421383249890408754, "__mgmt_id__":8421383249890408754} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097938, "value1":0.9171714255895188, "value2":2925569760522108797, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000481", "key_5":"f","key_6":"d", "__series_id__":6086829639929716526, "__mgmt_id__":6086829639929716526} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.551984, "value1":0.23531625096597034, "value2":8336676757418711666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000483", "key_2":"c","key_3":"b","key_9":"e", "__series_id__":1075978440552535992, "__mgmt_id__":1075978440552535992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.774461, "value1":0.26483886652245114, "value2":1317358716847210463, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000484", "key_5":"f","key_6":"h", "__series_id__":8178311935413748067, "__mgmt_id__":8178311935413748067} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354793, "value1":0.850445247905183, "value2":3123908181356206943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000485", "key_0":"g","key_3":"i","key_4":"a", "__series_id__":-2493248663566896078, "__mgmt_id__":-2493248663566896078} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.409594, "value1":0.7075083567078372, "value2":7315484295756660732, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000486", "key_1":"d","key_5":"d","key_6":"k", "__series_id__":1438560060133493844, "__mgmt_id__":1438560060133493844} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407267, "value1":0.6525751933510557, "value2":5480727731388217494, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000487", "key_4":"b","key_6":"h","key_0":"g", "__series_id__":-6171460502415307816, "__mgmt_id__":-6171460502415307816} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.572642, "value1":0.5899578957097297, "value2":5245828360681206097, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000488", "key_3":"g","key_5":"c", "__series_id__":1395566022379593096, "__mgmt_id__":1395566022379593096} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.731119, "value1":0.6986058457616005, "value2":4059549341372221062, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000490", "key_9":"c","key_0":"k","key_5":"j", "__series_id__":5547056308479578094, "__mgmt_id__":5547056308479578094} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.627725, "value1":0.5600488168022775, "value2":2564410170230629806, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000489", "key_0":"c","key_5":"a","key_7":"h", "__series_id__":3327209389577407524, "__mgmt_id__":3327209389577407524} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.050520, "value1":0.8920464631997315, "value2":9043850999622300098, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000493", "key_1":"d","key_8":"d","key_9":"h", "__series_id__":2544607124526528771, "__mgmt_id__":2544607124526528771} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.500683, "value1":0.004883654125166552, "value2":8338418914599161440, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000491", "key_2":"g","key_6":"a", "__series_id__":5779250518202732840, "__mgmt_id__":5779250518202732840} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.970722, "value1":0.01152412783847713, "value2":4977793912981679064, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000492", "key_0":"k","key_3":"j","key_9":"k", "__series_id__":-2839670665154701142, "__mgmt_id__":-2839670665154701142} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.860222, "value1":0.796875300130385, "value2":126152372728632215, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000496", "key_0":"h","key_5":"f","key_6":"g", "__series_id__":-1266368252316689299, "__mgmt_id__":-1266368252316689299} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629467, "value1":0.658754492873807, "value2":2189248151613602825, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000494", "key_9":"g","key_2":"i","key_6":"k", "__series_id__":7466907812445173243, "__mgmt_id__":7466907812445173243} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.098126, "value1":0.3838755732008427, "value2":8710340059607681652, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000495", "key_1":"a","key_2":"k", "__series_id__":5689680869548667384, "__mgmt_id__":5689680869548667384} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812187, "value1":0.7527055571509035, "value2":4522503161830090202, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000498", "key_3":"d","key_4":"d","key_8":"b", "__series_id__":-3733803121350110594, "__mgmt_id__":-3733803121350110594} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.303237, "value1":0.4170647409134182, "value2":1359570910802075414, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000497", "key_1":"e","key_2":"f","key_0":"d", "__series_id__":8712710324521618111, "__mgmt_id__":8712710324521618111} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794250, "value1":0.581379594095589, "value2":6842995986442641402, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000499", "key_0":"e","key_4":"i","key_5":"k", "__series_id__":6109391882474182075, "__mgmt_id__":6109391882474182075} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247703, "value1":0.7617287060418184, "value2":8955434883327208405, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000500", "key_4":"b","key_6":"h","key_7":"a", "__series_id__":-7178736237433154402, "__mgmt_id__":-7178736237433154402} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.815768, "value1":0.5618395871174356, "value2":6646583475463132751, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000501", "key_2":"i","key_5":"b","key_0":"d", "__series_id__":4499962670615869243, "__mgmt_id__":4499962670615869243} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.248823, "value1":0.1954800995898102, "value2":1363474452253194433, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000502", "key_4":"c","key_9":"c","key_2":"d", "__series_id__":-1065871006724225368, "__mgmt_id__":-1065871006724225368} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.531845, "value1":0.018792519031917622, "value2":2287281788624224406, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000503", "key_1":"g","key_2":"a","key_5":"d", "__series_id__":-659431953737416175, "__mgmt_id__":-659431953737416175} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.966028, "value1":0.08281088225013297, "value2":1000202598795297845, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000506", "key_3":"e","key_5":"g","key_6":"g", "__series_id__":8189907994910807144, "__mgmt_id__":8189907994910807144} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.935610, "value1":0.3880941622449169, "value2":3231927844022227730, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000504", "key_0":"f","key_2":"a","key_6":"c", "__series_id__":-5767774656667519977, "__mgmt_id__":-5767774656667519977} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.268000, "value1":0.37669283923759284, "value2":1820767283408305591, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000505", "key_9":"f","key_2":"k","key_8":"k", "__series_id__":-7385360351933713516, "__mgmt_id__":-7385360351933713516} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.229141, "value1":0.6757864654636687, "value2":850049194951914043, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000509", "key_4":"j","key_6":"f", "__series_id__":5118302165653502709, "__mgmt_id__":5118302165653502709} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.889660, "value1":0.14317979698283573, "value2":4562037574783866851, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000507", "key_1":"a","key_3":"i", "__series_id__":-3913420601372985247, "__mgmt_id__":-3913420601372985247} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836795, "value1":0.7422552273700601, "value2":1164639099163690154, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000508", "key_2":"j","key_5":"h","key_8":"a", "__series_id__":5489942196046137682, "__mgmt_id__":5489942196046137682} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.311980, "value1":0.6163196714525997, "value2":5427592556050607568, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000511", "key_7":"d","key_8":"k", "__series_id__":3213709674737485646, "__mgmt_id__":3213709674737485646} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584578, "value1":0.38212773421861185, "value2":1414397003551807223, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000510", "key_4":"c","key_8":"j","key_9":"e", "__series_id__":3354432852796179518, "__mgmt_id__":3354432852796179518} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065172, "value1":0.5125177091208547, "value2":2833074963007555228, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000513", "key_2":"f","key_6":"b","key_9":"c", "__series_id__":3033113919287788353, "__mgmt_id__":3033113919287788353} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.564271, "value1":0.5135791893267198, "value2":3074560175179976918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000512", "key_7":"e","key_1":"k","key_4":"d", "__series_id__":7586011388498218156, "__mgmt_id__":7586011388498218156} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.894567, "value1":0.8429539964906779, "value2":3097821505618288757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000514", "key_0":"j","key_3":"c", "__series_id__":-6286863743200497075, "__mgmt_id__":-6286863743200497075} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.733587, "value1":0.9459948105676399, "value2":3812465080486684733, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000516", "key_6":"e","key_8":"g","key_5":"h", "__series_id__":-2440202381808951160, "__mgmt_id__":-2440202381808951160} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.053881, "value1":0.6018467371683773, "value2":5589588667277003729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000515", "key_4":"e","key_9":"j", "__series_id__":-5305523743817782515, "__mgmt_id__":-5305523743817782515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599521, "value1":0.5951991237753406, "value2":4485462521384926953, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000517", "key_8":"e","key_6":"d", "__series_id__":1669966689336778997, "__mgmt_id__":1669966689336778997} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354099, "value1":0.15573795731250434, "value2":8080731261812523696, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000518", "key_2":"h","key_3":"a", "__series_id__":-5859891958230088706, "__mgmt_id__":-5859891958230088706} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969724, "value1":0.21276604577267808, "value2":8483165896229109105, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000519", "key_5":"i","key_7":"a","key_8":"c", "__series_id__":-8614030474790995564, "__mgmt_id__":-8614030474790995564} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263088, "value1":0.8705529546951637, "value2":1868660470516930885, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000520", "key_0":"b","key_4":"c", "__series_id__":-5335613431145046141, "__mgmt_id__":-5335613431145046141} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.212304, "value1":0.28383507730046376, "value2":3678145555670791199, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000521", "key_3":"h","key_5":"d","key_6":"f", "__series_id__":7716255806460584549, "__mgmt_id__":7716255806460584549} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965823, "value1":0.23336598309503237, "value2":1387918772122086546, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000526", "key_0":"k","key_4":"c", "__series_id__":536692980690746972, "__mgmt_id__":536692980690746972} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.536800, "value1":0.6576906678976834, "value2":6988018932226580717, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000532", "key_7":"e","key_9":"k", "__series_id__":-2729388522952487322, "__mgmt_id__":-2729388522952487322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.884180, "value1":0.9109732193854059, "value2":2950864149550098491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000533", "key_2":"i","key_3":"b","key_9":"f", "__series_id__":-2477031115887723911, "__mgmt_id__":-2477031115887723911} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448663, "value1":0.37035460819883087, "value2":6933572641209900634, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000534", "key_1":"a","key_3":"a","key_9":"k", "__series_id__":2278565491101561575, "__mgmt_id__":2278565491101561575} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.965580, "value1":0.9403308475710539, "value2":5776171572361995550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000535", "key_9":"a","key_1":"h","key_4":"k", "__series_id__":1973129972136064399, "__mgmt_id__":1973129972136064399} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.349421, "value1":0.19847751056585392, "value2":3418305989652553355, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000536", "key_7":"f","key_8":"e","key_3":"d", "__series_id__":-6372545903488947811, "__mgmt_id__":-6372545903488947811} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.469447, "value1":0.13572479625794354, "value2":4687020001075041801, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000537", "key_1":"k","key_2":"k","key_6":"h", "__series_id__":-5454642247960722939, "__mgmt_id__":-5454642247960722939} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393705, "value1":0.5764820211787193, "value2":296609812894901072, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000538", "key_0":"a","key_4":"f","key_6":"e", "__series_id__":4278955237678039512, "__mgmt_id__":4278955237678039512} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.862549, "value1":0.3023964248018198, "value2":7566431551834336763, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000539", "key_3":"i","key_6":"e", "__series_id__":-959503629477387471, "__mgmt_id__":-959503629477387471} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.415447, "value1":0.07086082925570908, "value2":6961625726898421777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000540", "key_6":"f","key_7":"e", "__series_id__":-1742054426754355322, "__mgmt_id__":-1742054426754355322} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495354, "value1":0.5778296524778606, "value2":2860533724845138756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000541", "key_3":"c","key_7":"c", "__series_id__":-8666413906498636151, "__mgmt_id__":-8666413906498636151} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.510139, "value1":0.9905369504866598, "value2":7400245650080116394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000542", "key_2":"j","key_0":"c","key_1":"a", "__series_id__":3424389104673400266, "__mgmt_id__":3424389104673400266} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.129326, "value1":0.3322893702950888, "value2":273820673011485924, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000543", "key_2":"k","key_3":"h","key_6":"a", "__series_id__":-58410619722955524, "__mgmt_id__":-58410619722955524} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.793922, "value1":0.5795215776373468, "value2":7565118610448753571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000544", "key_2":"c","key_5":"b","key_7":"i", "__series_id__":-6221064276046606416, "__mgmt_id__":-6221064276046606416} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114305, "value1":0.4965968551084071, "value2":3076343444745895229, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000545", "key_3":"c","key_6":"g", "__series_id__":5954441612890880180, "__mgmt_id__":5954441612890880180} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539515, "value1":0.7424425979048557, "value2":3570552565123324839, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000546", "key_8":"b","key_9":"b","key_0":"b", "__series_id__":118846903220307185, "__mgmt_id__":118846903220307185} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714983, "value1":0.4711244304607692, "value2":5134628196697329789, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000547", "key_0":"k","key_1":"c","key_5":"k", "__series_id__":4553344308252282896, "__mgmt_id__":4553344308252282896} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182520, "value1":0.11684538125954054, "value2":4642866567682796714, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000548", "key_4":"i","key_9":"c", "__series_id__":-4492587331751010254, "__mgmt_id__":-4492587331751010254} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701117, "value1":0.46816724142010296, "value2":8359014347252866849, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000549", "key_0":"f","key_2":"d","key_3":"f", "__series_id__":6888405107466727721, "__mgmt_id__":6888405107466727721} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135967, "value1":0.6405455196141, "value2":6186468831622665956, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000550", "key_5":"f","key_7":"d", "__series_id__":4146396063903913458, "__mgmt_id__":4146396063903913458} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.424454, "value1":0.5144728145894553, "value2":1160618526301999790, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000551", "key_0":"i","key_2":"c","key_3":"b", "__series_id__":-8996471880146267514, "__mgmt_id__":-8996471880146267514} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.476344, "value1":0.8109846569048221, "value2":1049565304185759462, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000552", "key_2":"i","key_8":"d", "__series_id__":7974823066874456629, "__mgmt_id__":7974823066874456629} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.571700, "value1":0.5504531727903846, "value2":133387170146495310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000553", "key_5":"c","key_7":"g","key_8":"g", "__series_id__":-7222273993022799913, "__mgmt_id__":-7222273993022799913} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324604, "value1":0.16411596790106633, "value2":4350527762007764055, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000554", "key_9":"g","key_1":"e","key_5":"g", "__series_id__":8195321219257482319, "__mgmt_id__":8195321219257482319} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.272811, "value1":0.06793234743561845, "value2":3491914082935836765, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000555", "key_2":"h","key_3":"k", "__series_id__":-7823955719877279009, "__mgmt_id__":-7823955719877279009} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496778, "value1":0.2139795243463974, "value2":9108500124467919738, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000556", "key_3":"i","key_4":"e","key_6":"b", "__series_id__":-7533557651239603121, "__mgmt_id__":-7533557651239603121} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.855922, "value1":0.8034347088304469, "value2":1547578413909373466, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000557", "key_7":"g","key_3":"c","key_6":"e", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.516953, "value1":0.46095603855138756, "value2":5334901216429463877, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000558", "key_5":"i","key_6":"g","key_7":"k", "__series_id__":4836771707913559561, "__mgmt_id__":4836771707913559561} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.941610, "value1":0.9546476007247299, "value2":1561805534534904078, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000522", "key_0":"c","key_6":"j","key_9":"a", "__series_id__":-1936761894710132790, "__mgmt_id__":-1936761894710132790} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157907, "value1":0.9790803369585773, "value2":7490355100154440772, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000523", "key_0":"g","key_3":"g","key_5":"k", "__series_id__":773876279162078760, "__mgmt_id__":773876279162078760} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.667071, "value1":0.23582538943103912, "value2":6462516448359960159, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000524", "key_3":"d","key_9":"f","key_2":"b", "__series_id__":1091864146610553369, "__mgmt_id__":1091864146610553369} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.447846, "value1":0.08703603490291748, "value2":6137779903742056640, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000525", "key_3":"c","key_6":"e","key_8":"b", "__series_id__":892072526354293416, "__mgmt_id__":892072526354293416} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.201754, "value1":0.703586761350304, "value2":5895955356127986300, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000529", "key_2":"h","key_6":"g","key_8":"i", "__series_id__":-1093921118198888503, "__mgmt_id__":-1093921118198888503} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.214135, "value1":0.28977641931297293, "value2":2670270804816119209, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000527", "key_3":"a","key_5":"b","key_9":"k", "__series_id__":-211929146564550067, "__mgmt_id__":-211929146564550067} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.171493, "value1":0.09189861336902101, "value2":3835094797774579940, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000528", "key_8":"g","key_0":"k","key_7":"c", "__series_id__":1873826160144342211, "__mgmt_id__":1873826160144342211} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.507098, "value1":0.8469316808229229, "value2":4394084036001197157, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000559", "key_2":"j","key_4":"d","key_1":"f", "__series_id__":2746195208096574288, "__mgmt_id__":2746195208096574288} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.567706, "value1":0.4315878020065951, "value2":3026063068553888025, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000530", "key_9":"f","key_3":"i","key_5":"c", "__series_id__":588182638119919303, "__mgmt_id__":588182638119919303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.369862, "value1":0.25873173058422444, "value2":2694491932093728275, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000531", "key_2":"c","key_3":"h","key_4":"j", "__series_id__":3248509495393935407, "__mgmt_id__":3248509495393935407} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393535, "value1":0.2941316364368656, "value2":3384752395363156914, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000561", "key_4":"k","key_9":"e", "__series_id__":9158941661696733777, "__mgmt_id__":9158941661696733777} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354927, "value1":0.13513861393265603, "value2":8481431502383347556, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000560", "key_1":"c","key_2":"k","key_6":"i", "__series_id__":1735221365148676523, "__mgmt_id__":1735221365148676523} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.468119, "value1":0.5679212228606271, "value2":2713405380262941377, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000563", "key_5":"a","key_0":"f","key_2":"f", "__series_id__":-7133474471903036847, "__mgmt_id__":-7133474471903036847} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.163006, "value1":0.4227063637930681, "value2":6321042013913967311, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000562", "key_7":"c","key_3":"h", "__series_id__":479037977924539995, "__mgmt_id__":479037977924539995} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.656131, "value1":0.5736248371417144, "value2":2472629476600653129, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000565", "key_7":"e","key_1":"e","key_6":"d", "__series_id__":-4396946655183034762, "__mgmt_id__":-4396946655183034762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.511635, "value1":0.7728428788355186, "value2":7276523252477639153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000564", "key_0":"f","key_2":"a","key_8":"d", "__series_id__":7316972110622220521, "__mgmt_id__":7316972110622220521} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.080037, "value1":0.9949587251210614, "value2":1148201472548354758, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000567", "key_2":"i","key_4":"b","key_6":"a", "__series_id__":-5290103708506341880, "__mgmt_id__":-5290103708506341880} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.725064, "value1":0.05030431554866517, "value2":4579601693593819193, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000566", "key_1":"i","key_4":"k", "__series_id__":1480107244684990778, "__mgmt_id__":1480107244684990778} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.342713, "value1":0.6797941637549237, "value2":2282393859292368923, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000569", "key_4":"f","key_9":"g", "__series_id__":-7177607339064466056, "__mgmt_id__":-7177607339064466056} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012227, "value1":0.25217885012179175, "value2":2820047039844714200, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000568", "key_2":"a","key_3":"h", "__series_id__":-5844764843762034587, "__mgmt_id__":-5844764843762034587} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.121409, "value1":0.08353369182495488, "value2":6648793005050735517, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000570", "key_5":"g","key_7":"b", "__series_id__":-6473745524789598047, "__mgmt_id__":-6473745524789598047} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.944843, "value1":0.7935143781883287, "value2":1141746305838051122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000571", "key_7":"b","key_9":"b","key_1":"k", "__series_id__":1150094511583599589, "__mgmt_id__":1150094511583599589} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216784, "value1":0.687358900553268, "value2":5413083464946109711, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000572", "key_1":"d","key_6":"b","key_7":"i", "__series_id__":-7604649366390654955, "__mgmt_id__":-7604649366390654955} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.744704, "value1":0.26762670212162987, "value2":4557617436876999047, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000573", "key_4":"d","key_5":"d","key_8":"k", "__series_id__":8408649499406926918, "__mgmt_id__":8408649499406926918} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.105367, "value1":0.7098778066529884, "value2":2505389065681570054, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000574", "key_3":"k","key_2":"f", "__series_id__":-1770598027909394560, "__mgmt_id__":-1770598027909394560} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164496, "value1":0.375746421410976, "value2":874151975168015904, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000575", "key_1":"k","key_8":"d", "__series_id__":1743799415683896668, "__mgmt_id__":1743799415683896668} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648692, "value1":0.26898807878557085, "value2":8174372151408578377, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000576", "key_2":"k","key_5":"b","key_9":"c", "__series_id__":-1515679063946229924, "__mgmt_id__":-1515679063946229924} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160502, "value1":0.8995428170685682, "value2":3027568952484727683, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000577", "key_1":"g","key_5":"h","key_6":"j", "__series_id__":5401788690197380769, "__mgmt_id__":5401788690197380769} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149122, "value1":0.0907853096527507, "value2":4677811628653537246, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000578", "key_2":"a","key_4":"b","key_5":"h", "__series_id__":2418823558047176150, "__mgmt_id__":2418823558047176150} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.296780, "value1":0.3995862207774202, "value2":3598801053628990992, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000579", "key_3":"k","key_6":"e","key_7":"f", "__series_id__":7326416396107376619, "__mgmt_id__":7326416396107376619} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.464060, "value1":0.9713793366221031, "value2":4865123999270835737, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000580", "key_3":"c","key_7":"k","key_2":"a", "__series_id__":-7152958117457316606, "__mgmt_id__":-7152958117457316606} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288334, "value1":0.6512177032146146, "value2":1792245924965210603, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000581", "key_2":"f","key_3":"e","key_9":"h", "__series_id__":279512434362576559, "__mgmt_id__":279512434362576559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.869907, "value1":0.007513793036623767, "value2":7995353402339020473, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000582", "key_8":"i","key_1":"i","key_5":"e", "__series_id__":5960810744832862505, "__mgmt_id__":5960810744832862505} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527381, "value1":0.5920004855495694, "value2":1386169961861528891, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000583", "key_0":"k","key_4":"e","key_9":"k", "__series_id__":4012996769448248803, "__mgmt_id__":4012996769448248803} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.928125, "value1":0.7418903709677874, "value2":5060751330275548305, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000584", "key_2":"e","key_6":"b", "__series_id__":-3053901563656550752, "__mgmt_id__":-3053901563656550752} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131800, "value1":0.9351703924333601, "value2":827562310794270736, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000585", "key_8":"g","key_0":"b","key_2":"g", "__series_id__":69745064320595303, "__mgmt_id__":69745064320595303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.367161, "value1":0.9314616031794654, "value2":8545717112597189310, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000586", "key_1":"e","key_5":"k","key_7":"f", "__series_id__":-4390751437491974261, "__mgmt_id__":-4390751437491974261} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.704807, "value1":0.7062626357522712, "value2":1882734585108600742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000588", "key_3":"a","key_8":"c","key_9":"f", "__series_id__":-1861794853839231064, "__mgmt_id__":-1861794853839231064} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270485, "value1":0.942646876486616, "value2":4665280302852454779, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000587", "key_2":"e","key_4":"a","key_5":"j", "__series_id__":-212718667757998205, "__mgmt_id__":-212718667757998205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.521647, "value1":0.7127769947534454, "value2":7002036018179718342, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000591", "key_0":"a","key_1":"b","key_6":"f", "__series_id__":-573574065846351162, "__mgmt_id__":-573574065846351162} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.011614, "value1":0.2150115012001749, "value2":2368106908969391947, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000589", "key_1":"a","key_4":"a","key_8":"f", "__series_id__":3087654112644698738, "__mgmt_id__":3087654112644698738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.160161, "value1":0.5544338078106065, "value2":6068552071959300958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000590", "key_0":"k","key_9":"g", "__series_id__":-8037889140845157220, "__mgmt_id__":-8037889140845157220} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.545350, "value1":0.5865458699587862, "value2":1122097202466314910, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000594", "key_1":"g","key_3":"a","key_9":"e", "__series_id__":9079902230263459230, "__mgmt_id__":9079902230263459230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.948757, "value1":0.9353310253197936, "value2":9083948862293457994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000592", "key_0":"k","key_1":"e","key_7":"b", "__series_id__":8051016939451460244, "__mgmt_id__":8051016939451460244} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.003970, "value1":0.13057333820161732, "value2":4887307186667923394, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000593", "key_3":"e","key_5":"k","key_6":"b", "__series_id__":3503362438270569084, "__mgmt_id__":3503362438270569084} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.811016, "value1":0.0597992472783255, "value2":7198325554706277943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000596", "key_0":"a","key_4":"b","key_8":"b", "__series_id__":7050945119323337512, "__mgmt_id__":7050945119323337512} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687715, "value1":0.029450254550187717, "value2":7347375517848816816, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000595", "key_1":"h","key_2":"j","key_6":"e", "__series_id__":3553226356531159335, "__mgmt_id__":3553226356531159335} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892901, "value1":0.5081883055870532, "value2":3289845879859033841, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000598", "key_0":"a","key_4":"a","key_5":"c", "__series_id__":-9155171941742690136, "__mgmt_id__":-9155171941742690136} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006320, "value1":0.35956731353734483, "value2":6174658822769262086, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000597", "key_2":"k","key_3":"e","key_6":"j", "__series_id__":1068182841660501054, "__mgmt_id__":1068182841660501054} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.088057, "value1":0.5893474112602632, "value2":2264301146979603153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000599", "key_1":"g","key_2":"e","key_6":"d", "__series_id__":-6900891616086871180, "__mgmt_id__":-6900891616086871180} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366406, "value1":0.09531376481437555, "value2":8460504532640850958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000600", "key_8":"e","key_9":"c","key_5":"b", "__series_id__":2476461678528820161, "__mgmt_id__":2476461678528820161} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.742879, "value1":0.21104814940031102, "value2":8239563761654323619, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000601", "key_2":"h","key_6":"e","key_9":"d", "__series_id__":-3049533378195154637, "__mgmt_id__":-3049533378195154637} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014364, "value1":0.27433866046434513, "value2":1123249940897490575, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000602", "key_0":"j","key_1":"f","key_5":"i", "__series_id__":2279437384480879496, "__mgmt_id__":2279437384480879496} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442350, "value1":0.5779165272183289, "value2":7076967122005445282, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000603", "key_6":"e","key_2":"c","key_3":"k", "__series_id__":-4803521805997940763, "__mgmt_id__":-4803521805997940763} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.363242, "value1":0.7223710776768617, "value2":4947532477450255339, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000604", "key_2":"h","key_6":"i","key_9":"e", "__series_id__":-3252709821186360534, "__mgmt_id__":-3252709821186360534} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.794231, "value1":0.9595692700611697, "value2":7086029131344930712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000605", "key_7":"d","key_8":"a","key_9":"h", "__series_id__":-3485520211684722157, "__mgmt_id__":-3485520211684722157} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.696096, "value1":0.2902282025796308, "value2":2826023125040762860, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000606", "key_2":"e","key_4":"d","key_7":"e", "__series_id__":4549675943707384462, "__mgmt_id__":4549675943707384462} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.366795, "value1":0.8338186933189231, "value2":7469909471986494672, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000608", "key_1":"e","key_4":"g","key_9":"i", "__series_id__":-2171418575767265313, "__mgmt_id__":-2171418575767265313} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.642061, "value1":0.025310231945638022, "value2":5571346287819468860, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000607", "key_5":"d","key_8":"h", "__series_id__":-9129308575221288808, "__mgmt_id__":-9129308575221288808} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673449, "value1":0.9385869454324781, "value2":1722013371242991173, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000611", "key_3":"k","key_8":"c", "__series_id__":5101814561270847054, "__mgmt_id__":5101814561270847054} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.375120, "value1":0.09609963357456859, "value2":5591579449982830960, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000609", "key_6":"k","key_7":"e", "__series_id__":8885896453973036591, "__mgmt_id__":8885896453973036591} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391308, "value1":0.5308398826498623, "value2":2744980541531859716, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000610", "key_1":"h","key_2":"e","key_7":"c", "__series_id__":1198284593719567346, "__mgmt_id__":1198284593719567346} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.658264, "value1":0.08305585159828374, "value2":3868051792086152227, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000613", "key_1":"j","key_7":"d","key_9":"k", "__series_id__":-7940551193719921457, "__mgmt_id__":-7940551193719921457} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.540598, "value1":0.9729141879466757, "value2":2886891615332464723, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000612", "key_4":"c","key_9":"b", "__series_id__":8092412906218116635, "__mgmt_id__":8092412906218116635} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.077420, "value1":0.756799108978319, "value2":3709648144634270824, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000615", "key_9":"a","key_3":"f","key_6":"a", "__series_id__":4176822743017399294, "__mgmt_id__":4176822743017399294} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.762914, "value1":0.7697479099606849, "value2":3475167187400734782, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000614", "key_5":"f","key_6":"g","key_0":"c", "__series_id__":478848127830000932, "__mgmt_id__":478848127830000932} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907718, "value1":0.44276263620911915, "value2":8007229889762742191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000616", "key_1":"k","key_5":"d","key_7":"j", "__series_id__":8101237851747291018, "__mgmt_id__":8101237851747291018} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.677735, "value1":0.7114610811603074, "value2":420499484804274745, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000617", "key_4":"c","key_5":"j","key_6":"a", "__series_id__":-2776938222678525934, "__mgmt_id__":-2776938222678525934} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097065, "value1":0.9614980550005483, "value2":401687894704730769, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000618", "key_0":"e","key_1":"j","key_3":"g", "__series_id__":-4653576099777458808, "__mgmt_id__":-4653576099777458808} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107273, "value1":0.7588043489806351, "value2":5258772379410637950, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000619", "key_0":"h","key_1":"i","key_5":"h", "__series_id__":-1382504029843268487, "__mgmt_id__":-1382504029843268487} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.907641, "value1":0.16046492238486323, "value2":5755077471601284208, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000620", "key_7":"f","key_9":"b","key_2":"j", "__series_id__":8725412239193059199, "__mgmt_id__":8725412239193059199} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407605, "value1":0.5733175508809538, "value2":2858689191661453591, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000621", "key_4":"e","key_2":"d","key_3":"e", "__series_id__":4735366876457776088, "__mgmt_id__":4735366876457776088} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412001, "value1":0.6163004444656547, "value2":3938161718625625501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000624", "key_1":"j","key_2":"e","key_7":"a", "__series_id__":-2247559342224111326, "__mgmt_id__":-2247559342224111326} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174438, "value1":0.7741718765670191, "value2":8151281186076102637, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000622", "key_4":"d","key_6":"b","key_8":"i", "__series_id__":5375659145983011030, "__mgmt_id__":5375659145983011030} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433895, "value1":0.6190720762613371, "value2":4753613664907841729, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000623", "key_0":"j","key_5":"h","key_8":"d", "__series_id__":-5274555278497901604, "__mgmt_id__":-5274555278497901604} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.754894, "value1":0.6861936508507025, "value2":4982660479879807579, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000626", "key_0":"g","key_3":"b","key_6":"f", "__series_id__":5847739109028520326, "__mgmt_id__":5847739109028520326} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.135440, "value1":0.8484346139801648, "value2":4916410761830427120, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000625", "key_5":"g","key_7":"d", "__series_id__":4275286754468204823, "__mgmt_id__":4275286754468204823} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.901526, "value1":0.7029579471688302, "value2":7091187814115860335, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000628", "key_1":"g","key_4":"i","key_7":"b", "__series_id__":-1771117005440832452, "__mgmt_id__":-1771117005440832452} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969298, "value1":0.9971487141151381, "value2":4678178370126159004, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000627", "key_4":"e","key_5":"k","key_6":"f", "__series_id__":9167379236828970956, "__mgmt_id__":9167379236828970956} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.302462, "value1":0.5451450337065363, "value2":4647862155949132966, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000629", "key_9":"e","key_1":"d","key_6":"b", "__series_id__":3813234962899481725, "__mgmt_id__":3813234962899481725} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.174504, "value1":0.8364678736703143, "value2":4662336943284880044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000630", "key_7":"a","key_2":"k", "__series_id__":-7333165572846986991, "__mgmt_id__":-7333165572846986991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724429, "value1":0.7605319213856574, "value2":5775369825299529065, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000631", "key_1":"i","key_8":"g","key_9":"d", "__series_id__":-6728759103923072294, "__mgmt_id__":-6728759103923072294} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196992, "value1":0.485221307178674, "value2":8161727305469743810, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000632", "key_2":"j","key_3":"i","key_7":"j", "__series_id__":-1482359697266060568, "__mgmt_id__":-1482359697266060568} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673298, "value1":0.12884478288346873, "value2":3433606731713363675, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000633", "key_1":"f","key_4":"j","key_6":"j", "__series_id__":1728706682623658501, "__mgmt_id__":1728706682623658501} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.008175, "value1":0.6854875474834051, "value2":7177608068823530117, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000634", "key_8":"b","key_1":"j","key_5":"j", "__series_id__":300660524563048433, "__mgmt_id__":300660524563048433} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.152906, "value1":0.937989165728203, "value2":2388297772896467042, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000635", "key_2":"b","key_5":"k", "__series_id__":-9079998154472042434, "__mgmt_id__":-9079998154472042434} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.354518, "value1":0.5251867146486633, "value2":2890885067302311396, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000637", "key_0":"j","key_3":"b","key_6":"a", "__series_id__":-1667478984335911846, "__mgmt_id__":-1667478984335911846} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.872640, "value1":0.7546063425064781, "value2":8461706694506205274, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000636", "key_8":"f","key_3":"h","key_4":"j", "__series_id__":-7580463150974888537, "__mgmt_id__":-7580463150974888537} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403113, "value1":0.7711761108505557, "value2":8776285961045082566, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000638", "key_8":"k","key_3":"g","key_6":"j", "__series_id__":-2535477892348838006, "__mgmt_id__":-2535477892348838006} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.713712, "value1":0.3686652392717231, "value2":3561820735837135599, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000639", "key_0":"c","key_1":"i","key_8":"i", "__series_id__":-4203576198063720420, "__mgmt_id__":-4203576198063720420} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.115073, "value1":0.21496854825964173, "value2":1666306538045744009, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000640", "key_3":"j","key_7":"d","key_8":"j", "__series_id__":8723049886767350264, "__mgmt_id__":8723049886767350264} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.127362, "value1":0.0348593634697097, "value2":3277092601539134615, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000641", "key_1":"j","key_5":"c","key_6":"a", "__series_id__":-3836128937730354935, "__mgmt_id__":-3836128937730354935} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.042887, "value1":0.9391085748107487, "value2":3037900421018347064, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000642", "key_5":"h","key_7":"h","key_8":"c", "__series_id__":960597803634566310, "__mgmt_id__":960597803634566310} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.957170, "value1":0.2742109117364031, "value2":7538881467268645993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000644", "key_9":"d", "__series_id__":1621069224688140711, "__mgmt_id__":1621069224688140711} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.284083, "value1":0.6552095129564431, "value2":8324596282478983070, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000643", "key_1":"e","key_4":"d","key_6":"c", "__series_id__":-6270497400672793233, "__mgmt_id__":-6270497400672793233} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.139776, "value1":0.32060963733799414, "value2":4387891090637798442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000645", "key_0":"a","key_1":"c","key_9":"j", "__series_id__":8806806789681853791, "__mgmt_id__":8806806789681853791} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.446847, "value1":0.7514268853453937, "value2":1042496829752491158, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000646", "key_1":"j","key_6":"k","key_7":"a", "__series_id__":-5236981910974541822, "__mgmt_id__":-5236981910974541822} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.429327, "value1":0.7027895753887083, "value2":4302894157835600757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000647", "key_2":"f","key_7":"e", "__series_id__":7563784551155215837, "__mgmt_id__":7563784551155215837} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.496272, "value1":0.2459555820489922, "value2":4743527397302686896, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000648", "key_1":"e","key_2":"e","key_9":"h", "__series_id__":-43719334813854435, "__mgmt_id__":-43719334813854435} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599745, "value1":0.3085143386003063, "value2":8510884550594616394, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000649", "key_3":"a","key_4":"b","key_9":"f", "__series_id__":3514458506748617193, "__mgmt_id__":3514458506748617193} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233545, "value1":0.4081732430516885, "value2":4780887317944713264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000650", "key_2":"a","key_5":"j","key_1":"c", "__series_id__":4505204653313518164, "__mgmt_id__":4505204653313518164} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.396869, "value1":0.3834820243913789, "value2":2867737400343386587, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000651", "key_2":"i","key_4":"j", "__series_id__":-232361702857049638, "__mgmt_id__":-232361702857049638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.000965, "value1":0.7645272489414043, "value2":8292218247137907442, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000652", "key_4":"f","key_5":"a","key_9":"i", "__series_id__":5720057473891543633, "__mgmt_id__":5720057473891543633} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.390310, "value1":0.9922190988066435, "value2":6520801004207332848, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000653", "key_0":"b","key_4":"c","key_6":"a", "__series_id__":1284561962336905172, "__mgmt_id__":1284561962336905172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.408746, "value1":0.11967416242419873, "value2":1289404280671317844, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000656", "key_4":"f","key_6":"k","key_7":"k", "__series_id__":8363757466721457609, "__mgmt_id__":8363757466721457609} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.513154, "value1":0.43853228030860203, "value2":3816263914269322442, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000654", "key_3":"g","key_8":"h","key_9":"c", "__series_id__":8721730122668119771, "__mgmt_id__":8721730122668119771} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.425564, "value1":0.9721541652281699, "value2":1199225910826614784, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000655", "key_0":"b","key_6":"e", "__series_id__":-1988713756160061772, "__mgmt_id__":-1988713756160061772} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.896976, "value1":0.5841175850919846, "value2":6380661791453585128, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000659", "key_1":"k","key_3":"c","key_8":"i", "__series_id__":6964189443680817110, "__mgmt_id__":6964189443680817110} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.385017, "value1":0.3642021068005677, "value2":7460310647019127617, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000657", "key_3":"a","key_6":"f","key_7":"h", "__series_id__":-2570923717357710934, "__mgmt_id__":-2570923717357710934} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.861063, "value1":0.18830600982566692, "value2":6167598179189147828, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000658", "key_4":"j","key_8":"i", "__series_id__":5508198311483266554, "__mgmt_id__":5508198311483266554} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.942353, "value1":0.4661402168565476, "value2":8018642185562026949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000661", "key_1":"d","key_5":"k","key_6":"e", "__series_id__":6489846711316186670, "__mgmt_id__":6489846711316186670} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.750347, "value1":0.3217686952651615, "value2":830941682724984711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000660", "key_4":"h","key_8":"i", "__series_id__":-8142533296407611789, "__mgmt_id__":-8142533296407611789} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.926420, "value1":0.0451768298368594, "value2":3858422103614938711, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000663", "key_0":"j","key_2":"i","key_4":"g", "__series_id__":-4185260234803399992, "__mgmt_id__":-4185260234803399992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451255, "value1":0.012250475428480134, "value2":468924453611892408, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000662", "key_2":"j","key_3":"d","key_8":"h", "__series_id__":4321156897846092336, "__mgmt_id__":4321156897846092336} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.708013, "value1":0.5360411826543746, "value2":7399971881816481719, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000665", "key_3":"k","key_6":"k","key_8":"g", "__series_id__":-3779813639604971340, "__mgmt_id__":-3779813639604971340} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.661279, "value1":0.5535815660872848, "value2":4401269288089200510, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000664", "key_0":"e","key_3":"j", "__series_id__":-924271420436849046, "__mgmt_id__":-924271420436849046} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.710860, "value1":0.6074402471823324, "value2":7073880640711856002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000666", "key_1":"b","key_3":"b", "__series_id__":-6286400394770701784, "__mgmt_id__":-6286400394770701784} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245087, "value1":0.7738521153135302, "value2":5302937747460405991, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000667", "key_4":"i","key_7":"h", "__series_id__":4370944294749235713, "__mgmt_id__":4370944294749235713} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.724822, "value1":0.06719994230964553, "value2":1984993846520154478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000668", "key_3":"e","key_8":"f","key_9":"g", "__series_id__":6563059919034350414, "__mgmt_id__":6563059919034350414} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841255, "value1":0.7199212526961583, "value2":3580548125031735289, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000669", "key_4":"c","key_5":"e","key_9":"d", "__series_id__":-2565464395137357469, "__mgmt_id__":-2565464395137357469} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130289, "value1":0.14169983508615547, "value2":7980800511434824689, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000670", "key_6":"i","key_8":"d","key_5":"k", "__series_id__":7709323868101614601, "__mgmt_id__":7709323868101614601} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.215185, "value1":0.05456621441388223, "value2":3313875019669857059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000671", "key_1":"c","key_5":"d","key_7":"b", "__series_id__":1006029062071662005, "__mgmt_id__":1006029062071662005} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.730585, "value1":0.1938148933748891, "value2":2133259010192691032, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000672", "key_4":"h","key_1":"b","key_3":"i", "__series_id__":-8935610045063183738, "__mgmt_id__":-8935610045063183738} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.603048, "value1":0.27374938601691284, "value2":6401354620294279338, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000674", "key_1":"a","key_6":"c","key_7":"f", "__series_id__":8769961756187444147, "__mgmt_id__":8769961756187444147} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.148949, "value1":0.5947828484485513, "value2":8147515290794947255, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000673", "key_4":"g","key_7":"g","key_8":"k", "__series_id__":5226949060967918762, "__mgmt_id__":5226949060967918762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.610152, "value1":0.4085609201069522, "value2":9002523177156729302, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000677", "key_2":"e","key_4":"h","key_7":"i", "__series_id__":463887977784803555, "__mgmt_id__":463887977784803555} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.607808, "value1":0.9266705539974761, "value2":4953852228607795114, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000675", "key_1":"c","key_3":"d","key_6":"j", "__series_id__":-4239837266494773189, "__mgmt_id__":-4239837266494773189} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.197811, "value1":0.7637421466978226, "value2":7727001922124797631, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000676", "key_0":"k","key_2":"k","key_5":"k", "__series_id__":5973456801068675752, "__mgmt_id__":5973456801068675752} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.672455, "value1":0.2140320622344972, "value2":8504587633123200194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000679", "key_2":"c","key_3":"j","key_9":"e", "__series_id__":-7318834342806204719, "__mgmt_id__":-7318834342806204719} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781600, "value1":0.18861407062595717, "value2":2041192855707568703, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000678", "key_4":"i","key_5":"e","key_0":"e", "__series_id__":4358356389076501559, "__mgmt_id__":4358356389076501559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.648320, "value1":0.9980339865005591, "value2":6740770170971470746, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000681", "key_1":"e","key_3":"f","key_9":"f", "__series_id__":4592924626643072689, "__mgmt_id__":4592924626643072689} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481193, "value1":0.5203406695297639, "value2":6425306083284764527, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000680", "key_0":"a","key_8":"i", "__series_id__":7094885463570430101, "__mgmt_id__":7094885463570430101} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954444, "value1":0.5949050635321682, "value2":2690790583476351571, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000683", "key_1":"k","key_6":"a","key_9":"c", "__series_id__":6146792295053586949, "__mgmt_id__":6146792295053586949} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971678, "value1":0.3007304152123297, "value2":4562717443975915242, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000682", "key_0":"f","key_3":"b","key_6":"k", "__series_id__":-2103117075427447025, "__mgmt_id__":-2103117075427447025} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026021, "value1":0.27575882598297025, "value2":978861683515042668, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000684", "key_5":"d","key_2":"e","key_4":"i", "__series_id__":-2256474065290328854, "__mgmt_id__":-2256474065290328854} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.840073, "value1":0.15154871188435406, "value2":7561212279807822666, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000685", "key_0":"j","key_3":"k", "__series_id__":5011990492559181611, "__mgmt_id__":5011990492559181611} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.393063, "value1":0.5006737130316841, "value2":835200053720218776, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000686", "key_1":"d","key_4":"a","key_8":"g", "__series_id__":107727703190771119, "__mgmt_id__":107727703190771119} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.589637, "value1":0.29002140408035565, "value2":5208166554872726624, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000687", "key_9":"b","key_4":"j","key_5":"b", "__series_id__":648754459071852131, "__mgmt_id__":648754459071852131} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525315, "value1":0.7084919237025886, "value2":4888712288784480920, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000688", "key_2":"b","key_7":"b","key_8":"c", "__series_id__":4070672682488929531, "__mgmt_id__":4070672682488929531} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026489, "value1":0.9459638977891762, "value2":6799511668473867834, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000689", "key_3":"i","key_5":"h","key_8":"a", "__series_id__":4263193799253961991, "__mgmt_id__":4263193799253961991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.356460, "value1":0.8482487483215856, "value2":4902836930964609800, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000690", "key_1":"g","key_5":"h","key_8":"k", "__series_id__":174857968371242321, "__mgmt_id__":174857968371242321} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.871117, "value1":0.35037006883754834, "value2":6550768382747712274, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000692", "key_7":"a","key_0":"f","key_6":"f", "__series_id__":-7804957703108199728, "__mgmt_id__":-7804957703108199728} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.062992, "value1":0.3329675649141359, "value2":8563631384946292163, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000691", "key_0":"b","key_5":"h","key_6":"h", "__series_id__":4819173149125300851, "__mgmt_id__":4819173149125300851} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.751101, "value1":0.9811341531679872, "value2":621345745999214417, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000694", "key_5":"i","key_9":"h","key_0":"g", "__series_id__":-519381987822359762, "__mgmt_id__":-519381987822359762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.535295, "value1":0.922840944798324, "value2":586063777073635225, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000693", "key_0":"b","key_5":"e","key_8":"b", "__series_id__":-3336015353703936523, "__mgmt_id__":-3336015353703936523} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.258523, "value1":0.7865906481221376, "value2":1141638202189461929, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000695", "key_6":"a","key_8":"d","key_9":"g", "__series_id__":-205391342820866960, "__mgmt_id__":-205391342820866960} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.10833711533264617, "value2":5739011556983431812, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000696", "key_1":"d","key_2":"d","key_4":"h", "__series_id__":-2006641466119522129, "__mgmt_id__":-2006641466119522129} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.407251, "value1":0.4266309513517482, "value2":4708724256538796193, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000697", "key_6":"h","key_2":"e","key_4":"d", "__series_id__":9026717280601479145, "__mgmt_id__":9026717280601479145} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.931023, "value1":0.6949078276162761, "value2":6914579995867144592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000698", "key_1":"d","key_5":"h", "__series_id__":-8286050823772945646, "__mgmt_id__":-8286050823772945646} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.882968, "value1":0.27938573922269233, "value2":2134628658651324357, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000700", "key_3":"g","key_4":"b","key_7":"e", "__series_id__":-2956385702922923027, "__mgmt_id__":-2956385702922923027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.599828, "value1":0.31593636777024753, "value2":2026576607314660982, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000699", "key_0":"c","key_2":"a","key_7":"c", "__series_id__":9222865247780531255, "__mgmt_id__":9222865247780531255} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.166118, "value1":0.30221394298164866, "value2":7042763501275182896, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000702", "key_0":"e","key_3":"f","key_9":"d", "__series_id__":-4059907291499193197, "__mgmt_id__":-4059907291499193197} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114195, "value1":0.16993347985800208, "value2":356239511173356550, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000701", "key_2":"h","key_5":"i","key_7":"a", "__series_id__":-1270585470594030161, "__mgmt_id__":-1270585470594030161} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.676940, "value1":0.8142744662164688, "value2":1615963097558644501, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000704", "key_5":"e","key_7":"g", "__series_id__":-7518756723037496675, "__mgmt_id__":-7518756723037496675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.525949, "value1":0.9287689609270591, "value2":8144174091016595622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000703", "key_6":"i","key_8":"b","key_9":"c", "__series_id__":-6000742232332707948, "__mgmt_id__":-6000742232332707948} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852923, "value1":0.6682421979821775, "value2":1660916359233146706, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000705", "key_1":"b","key_6":"d", "__series_id__":4486095316201482072, "__mgmt_id__":4486095316201482072} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714742, "value1":0.6940562364820247, "value2":8279633113622088778, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000706", "key_5":"e","key_6":"h","key_9":"d", "__series_id__":-4604912175677665552, "__mgmt_id__":-4604912175677665552} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.069728, "value1":0.023054188899827854, "value2":7432273276574908949, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000707", "key_7":"d","key_9":"b", "__series_id__":6934979284740660675, "__mgmt_id__":6934979284740660675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.397060, "value1":0.6029753734289958, "value2":5859678381429923175, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000708", "key_0":"a","key_2":"g","key_6":"g", "__series_id__":-7255426429556711853, "__mgmt_id__":-7255426429556711853} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.788868, "value1":0.4080940744104717, "value2":2775616148615876742, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000711", "key_0":"j","key_4":"h","key_6":"i", "__series_id__":2945323205155494561, "__mgmt_id__":2945323205155494561} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.497778, "value1":0.0933297131976096, "value2":2710674316640074263, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000709", "key_1":"c","key_8":"a","key_9":"g", "__series_id__":5757871568571719806, "__mgmt_id__":5757871568571719806} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.251386, "value1":0.5835060543515784, "value2":1694819851220806994, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000710", "key_8":"b","key_1":"c","key_2":"f", "__series_id__":4674231085769924073, "__mgmt_id__":4674231085769924073} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.144280, "value1":0.7762012742074322, "value2":6551654579563921110, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000714", "key_4":"d","key_6":"g","key_3":"i", "__series_id__":-6413509023368384468, "__mgmt_id__":-6413509023368384468} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778177, "value1":0.8529872059895475, "value2":4843965544345174856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000712", "key_7":"c","key_6":"k", "__series_id__":4513696354870901992, "__mgmt_id__":4513696354870901992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495383, "value1":0.8821308188431486, "value2":7181474101563747015, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000713", "key_2":"a","key_3":"b","key_4":"c", "__series_id__":1385361195095616515, "__mgmt_id__":1385361195095616515} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421122, "value1":0.04021723515347381, "value2":8758814764030728391, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000716", "key_0":"e","key_6":"c", "__series_id__":4929797706112254955, "__mgmt_id__":4929797706112254955} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.055252, "value1":0.5894200751471586, "value2":3745371455267104958, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000715", "key_2":"i","key_5":"a", "__series_id__":1992091351558433546, "__mgmt_id__":1992091351558433546} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343067, "value1":0.2157486959489704, "value2":3077439566978720910, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000718", "key_9":"e","key_5":"b","key_6":"g", "__series_id__":-5772222342437624470, "__mgmt_id__":-5772222342437624470} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.489306, "value1":0.4542119549735926, "value2":5846254702440735178, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000717", "key_0":"b","key_4":"f","key_9":"e", "__series_id__":6445180479932093175, "__mgmt_id__":6445180479932093175} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.233627, "value1":0.2946889464915457, "value2":2465181057584653557, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000719", "key_1":"d","key_4":"b","key_9":"g", "__series_id__":-5864279158098419036, "__mgmt_id__":-5864279158098419036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.523076, "value1":0.579630183513931, "value2":4633358909899504280, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000720", "key_5":"a","key_7":"f","key_8":"i", "__series_id__":-7458229987167107139, "__mgmt_id__":-7458229987167107139} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.317577, "value1":0.49633251300853154, "value2":7243885459219565500, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000722", "key_1":"b","key_9":"d","key_0":"h", "__series_id__":4346321792630722699, "__mgmt_id__":4346321792630722699} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.142016, "value1":0.41458124596836643, "value2":87263970443989795, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000721", "key_1":"d","key_9":"e", "__series_id__":-5354114461131055911, "__mgmt_id__":-5354114461131055911} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.014459, "value1":0.3889314073670658, "value2":6642542110052864260, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000724", "key_0":"a","key_5":"d","key_9":"f", "__series_id__":-2698660337334151799, "__mgmt_id__":-2698660337334151799} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.249899, "value1":0.5981564529561014, "value2":4725189280029367657, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000723", "key_3":"a","key_4":"f","key_8":"k", "__series_id__":-7342214837885234023, "__mgmt_id__":-7342214837885234023} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.087078, "value1":0.48311123248467286, "value2":7915582091365577059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000726", "key_5":"k", "__series_id__":-2988860553018354593, "__mgmt_id__":-2988860553018354593} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.681336, "value1":0.3569069050336655, "value2":7513851204641167627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000725", "key_3":"a","key_6":"g","key_9":"d", "__series_id__":9196263055514333027, "__mgmt_id__":9196263055514333027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266570, "value1":0.6724875939535222, "value2":4615308903415712327, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000728", "key_7":"c","key_9":"a","key_5":"j", "__series_id__":2242518671299929316, "__mgmt_id__":2242518671299929316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.474729, "value1":0.3876423090885996, "value2":319900338975917805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000727", "key_2":"i","key_8":"e", "__series_id__":-1694708898393520778, "__mgmt_id__":-1694708898393520778} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.328471, "value1":0.061053039369872907, "value2":9085903408798527191, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000729", "key_0":"f","key_1":"a","key_5":"h", "__series_id__":-4903270819894625628, "__mgmt_id__":-4903270819894625628} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.010016, "value1":0.7472091713600674, "value2":8976037153524824381, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000730", "key_2":"g","key_3":"e","key_6":"b", "__series_id__":-870546761351951316, "__mgmt_id__":-870546761351951316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.412206, "value1":0.7796477061197584, "value2":6842085213034340664, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000731", "key_0":"k","key_6":"e", "__series_id__":-322672579238854109, "__mgmt_id__":-322672579238854109} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.288276, "value1":0.26992787569132787, "value2":1806846857113271486, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000732", "key_4":"i","key_8":"k", "__series_id__":6392136191524438462, "__mgmt_id__":6392136191524438462} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.501201, "value1":0.781398815935773, "value2":4251812051340782659, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000733", "key_9":"d","key_0":"j", "__series_id__":-2261204010007076042, "__mgmt_id__":-2261204010007076042} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.270221, "value1":0.051986823543989535, "value2":4993776637838984787, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000734", "key_4":"d","key_8":"h","key_3":"k", "__series_id__":-5219068177349312395, "__mgmt_id__":-5219068177349312395} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.986643, "value1":0.7021309080168643, "value2":703842817955022555, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000735", "key_6":"c","key_7":"k","key_9":"c", "__series_id__":-274426148915748697, "__mgmt_id__":-274426148915748697} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.453898, "value1":0.4599169268098947, "value2":7963554429950594113, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000736", "key_0":"i","key_4":"k","key_6":"e", "__series_id__":-5109918428716040627, "__mgmt_id__":-5109918428716040627} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.202118, "value1":0.94451324780601, "value2":7780362140982384262, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000737", "key_8":"i","key_2":"h", "__series_id__":5199655525790884573, "__mgmt_id__":5199655525790884573} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.801389, "value1":0.7867483731478175, "value2":7192213512007631986, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000738", "key_4":"b","key_5":"h", "__series_id__":8135807143640598550, "__mgmt_id__":8135807143640598550} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.228749, "value1":0.4182819773434862, "value2":3027632485243175966, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000739", "key_2":"d","key_3":"e","key_5":"k", "__series_id__":-3767462971529671998, "__mgmt_id__":-3767462971529671998} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.184080, "value1":0.9567901938339193, "value2":5313717148514876661, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000740", "key_0":"f","key_2":"b","key_3":"j", "__series_id__":-7292257139050233576, "__mgmt_id__":-7292257139050233576} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.707330, "value1":0.826196713820751, "value2":8078395604816162581, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000741", "key_3":"g","key_6":"f","key_2":"g", "__series_id__":4040887025298036951, "__mgmt_id__":4040887025298036951} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130983, "value1":0.31932740597678144, "value2":8325619009408095997, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000745", "key_6":"j","key_7":"j","key_5":"j", "__series_id__":-8575913527796938168, "__mgmt_id__":-8575913527796938168} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477577, "value1":0.8037112800341328, "value2":4641052188020943712, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000742", "key_4":"a","key_5":"g","key_6":"j", "__series_id__":242486419215872474, "__mgmt_id__":242486419215872474} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.835784, "value1":0.981119547136869, "value2":5830864768532244942, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000743", "key_1":"g","key_4":"d","key_9":"h", "__series_id__":6502989350228972750, "__mgmt_id__":6502989350228972750} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.951325, "value1":0.9017984159519533, "value2":7172321278356171509, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000744", "key_1":"c","key_8":"i","key_9":"j", "__series_id__":5180505041240508376, "__mgmt_id__":5180505041240508376} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949416, "value1":0.9735936936041008, "value2":2397210877296618753, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000748", "key_3":"e","key_4":"h","key_6":"f", "__series_id__":-4268724067754570381, "__mgmt_id__":-4268724067754570381} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.227491, "value1":0.056278467666299646, "value2":3619861078542842990, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000746", "key_1":"i","key_3":"d","key_8":"c", "__series_id__":-8558923509815976303, "__mgmt_id__":-8558923509815976303} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.614325, "value1":0.9338179733958348, "value2":1856588427373456649, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000747", "key_2":"h","key_3":"j","key_9":"c", "__series_id__":-7411561607080856762, "__mgmt_id__":-7411561607080856762} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.167780, "value1":0.6264999203061654, "value2":1887390160426943194, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000750", "key_1":"b","key_8":"h", "__series_id__":446092245233888064, "__mgmt_id__":446092245233888064} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.421682, "value1":0.3034767704367279, "value2":1117719742198046331, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000749", "key_5":"f","key_6":"c","key_9":"e", "__series_id__":-5792100618530611790, "__mgmt_id__":-5792100618530611790} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.131869, "value1":0.7226359930389739, "value2":6009067442340606148, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000751", "key_0":"d","key_2":"f", "__series_id__":-1414104685122802275, "__mgmt_id__":-1414104685122802275} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.576451, "value1":0.4077319058968275, "value2":5262129379842658951, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000752", "key_0":"e","key_1":"g","key_8":"i", "__series_id__":6008408224877168924, "__mgmt_id__":6008408224877168924} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.487400, "value1":0.49878540971656044, "value2":378267425259112843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000753", "key_1":"e","key_7":"c", "__series_id__":-8137066097479334829, "__mgmt_id__":-8137066097479334829} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417401, "value1":0.7695532802557139, "value2":6353721086899765325, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000754", "key_8":"h","key_1":"h","key_5":"i", "__series_id__":-3785821557868987000, "__mgmt_id__":-3785821557868987000} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.597343, "value1":0.9334570369057625, "value2":8106460523980946423, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000756", "key_0":"d","key_2":"c", "__series_id__":-1328640254547822589, "__mgmt_id__":-1328640254547822589} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947755, "value1":0.599677921450711, "value2":1518293404194401856, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000755", "key_2":"f","key_8":"e", "__series_id__":-6495179908941869036, "__mgmt_id__":-6495179908941869036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.533432, "value1":0.4134611723516883, "value2":4789921078575497056, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000757", "key_3":"e","key_5":"b","key_6":"j", "__series_id__":9138369831619846992, "__mgmt_id__":9138369831619846992} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.271038, "value1":0.7693055378733823, "value2":2557684070989427251, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000759", "key_2":"h","key_9":"g", "__series_id__":2446733477236612230, "__mgmt_id__":2446733477236612230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.209928, "value1":0.5501502486470365, "value2":6523836811607831530, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000758", "key_2":"j","key_6":"i","key_7":"b", "__series_id__":-6200607058083360641, "__mgmt_id__":-6200607058083360641} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.514479, "value1":0.742552198141705, "value2":5448311945921729044, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000760", "key_2":"d","key_4":"i","key_5":"k", "__series_id__":4358771423920240400, "__mgmt_id__":4358771423920240400} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.097323, "value1":0.342716151747015, "value2":5333351730397550620, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000762", "key_0":"d","key_2":"e", "__series_id__":5138476994793802870, "__mgmt_id__":5138476994793802870} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.036400, "value1":0.9364465322438232, "value2":3515155010826544091, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000761", "key_4":"g","key_5":"a","key_7":"b", "__series_id__":-4249457985570074994, "__mgmt_id__":-4249457985570074994} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.161709, "value1":0.4801914817306215, "value2":4167056653387877843, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000763", "key_4":"b","key_6":"k","key_8":"g", "__series_id__":6935428815854533176, "__mgmt_id__":6935428815854533176} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.128367, "value1":0.42861309279829746, "value2":6367124429829008192, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000764", "key_0":"a","key_5":"h", "__series_id__":-5504675752837572439, "__mgmt_id__":-5504675752837572439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.104233, "value1":0.5604661779053219, "value2":8908744031459681115, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000765", "key_3":"a","key_5":"h","key_9":"b", "__series_id__":1985563680930509217, "__mgmt_id__":1985563680930509217} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.814224, "value1":0.8505352643130756, "value2":3611537515395296790, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000766", "key_7":"a","key_9":"e", "__series_id__":-4328016376789558277, "__mgmt_id__":-4328016376789558277} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.257817, "value1":0.9534356262275627, "value2":6358735114691060805, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000769", "key_3":"i","key_5":"j","key_2":"e", "__series_id__":9179346307913990022, "__mgmt_id__":9179346307913990022} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590971, "value1":0.8532839122834444, "value2":4683754442672481085, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000767", "key_9":"f","key_7":"b", "__series_id__":-5432215965398723066, "__mgmt_id__":-5432215965398723066} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.110492, "value1":0.563891691575366, "value2":4631985245890497320, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000768", "key_2":"e","key_7":"a","key_0":"a", "__series_id__":-136250900044526365, "__mgmt_id__":-136250900044526365} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.953652, "value1":0.20700443841383587, "value2":1925175802270628257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000771", "key_4":"f","key_8":"k", "__series_id__":4875237959865447451, "__mgmt_id__":4875237959865447451} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012273, "value1":0.5960529408710756, "value2":6611884946690949474, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000770", "key_2":"d","key_4":"f","key_5":"j", "__series_id__":1497740143071633371, "__mgmt_id__":1497740143071633371} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107781, "value1":0.5528403788424425, "value2":2332669698469083563, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000773", "key_5":"e","key_2":"h", "__series_id__":6481359940216246477, "__mgmt_id__":6481359940216246477} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289974, "value1":0.4420135407607212, "value2":3690892585669795676, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000772", "key_0":"j","key_1":"f","key_6":"c", "__series_id__":3152074120145678841, "__mgmt_id__":3152074120145678841} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.193764, "value1":0.5159643736419933, "value2":6218735166584063092, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000775", "key_0":"e","key_6":"a", "__series_id__":3438830744121252320, "__mgmt_id__":3438830744121252320} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.386095, "value1":0.777297131904327, "value2":636830504959980122, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000774", "key_2":"g","key_4":"c","key_8":"e", "__series_id__":-3553038228034037944, "__mgmt_id__":-3553038228034037944} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.977928, "value1":0.01676315692753733, "value2":4060533084263704194, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000776", "key_2":"k","key_9":"i","key_0":"k", "__series_id__":-988677622257336616, "__mgmt_id__":-988677622257336616} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.066993, "value1":0.001865435996499948, "value2":4739969679441502931, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000777", "key_0":"j","key_4":"c","key_8":"i", "__series_id__":-6528423083423872684, "__mgmt_id__":-6528423083423872684} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.910616, "value1":0.057836079222670464, "value2":5276870038639917478, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000778", "key_1":"b","key_3":"g","key_7":"g", "__series_id__":-8717617664237859896, "__mgmt_id__":-8717617664237859896} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.971825, "value1":0.18153964718314727, "value2":2807508624852997267, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000779", "key_1":"h","key_7":"i", "__series_id__":6851652104392799931, "__mgmt_id__":6851652104392799931} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.782626, "value1":0.41404427259634824, "value2":6643422113997639050, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000780", "key_1":"j","key_2":"b","key_3":"i", "__series_id__":-3260329422376183072, "__mgmt_id__":-3260329422376183072} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.892943, "value1":0.5830967595104455, "value2":499806168567753593, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000783", "key_9":"b","key_7":"b","key_8":"e", "__series_id__":5207812472034615754, "__mgmt_id__":5207812472034615754} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.423402, "value1":0.9103545774481498, "value2":911868702803131336, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000781", "key_2":"g","key_7":"h","key_9":"d", "__series_id__":-6610823537308815374, "__mgmt_id__":-6610823537308815374} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006069, "value1":0.7748758341609289, "value2":6539224747903805609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000785", "key_5":"j","key_6":"d","key_9":"k", "__series_id__":9158185736224292675, "__mgmt_id__":9158185736224292675} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.143311, "value1":0.3741630815676959, "value2":375500557355639144, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000782", "key_9":"b","key_2":"f", "__series_id__":-7560317748452054958, "__mgmt_id__":-7560317748452054958} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267012, "value1":0.06234671061027635, "value2":5595306219658860658, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000784", "key_1":"i","key_2":"b","key_7":"f", "__series_id__":999269332951330346, "__mgmt_id__":999269332951330346} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.548719, "value1":0.7992113362076428, "value2":7214856807865350002, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000786", "key_1":"c","key_4":"e", "__series_id__":732108570561027442, "__mgmt_id__":732108570561027442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.919970, "value1":0.973745629473067, "value2":4169123266267227326, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000787", "key_3":"h","key_6":"j","key_7":"g", "__series_id__":-8959368755671248306, "__mgmt_id__":-8959368755671248306} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.071680, "value1":0.08778281877161515, "value2":7668684104220358875, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000788", "key_6":"h","key_9":"i","key_3":"b", "__series_id__":489626333821893626, "__mgmt_id__":489626333821893626} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.222513, "value1":0.4425030874471083, "value2":5614432156299415760, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000789", "key_5":"a","key_6":"d", "__series_id__":-5982588030568173473, "__mgmt_id__":-5982588030568173473} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.031556, "value1":0.6971180191067721, "value2":6128893201277772027, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000790", "key_1":"f","key_6":"e","key_8":"d", "__series_id__":-2565229845181866432, "__mgmt_id__":-2565229845181866432} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.839957, "value1":0.03151274581462018, "value2":8360563655204493448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000792", "key_6":"b","key_9":"h", "__series_id__":4084306027943312742, "__mgmt_id__":4084306027943312742} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.924623, "value1":0.5790628849747803, "value2":6190166511034220622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000791", "key_1":"h","key_2":"e", "__series_id__":-5467297294721627584, "__mgmt_id__":-5467297294721627584} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.647381, "value1":0.18478113887851288, "value2":2971279323948578904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000794", "key_1":"b","key_5":"h","key_8":"a", "__series_id__":3522844273960320717, "__mgmt_id__":3522844273960320717} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.818067, "value1":0.36190113937740354, "value2":1714448127957842853, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000793", "key_2":"f","key_6":"d","key_7":"k", "__series_id__":4737984741355726102, "__mgmt_id__":4737984741355726102} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.715974, "value1":0.23843892303460118, "value2":2439937704889068215, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000796", "key_0":"h", "__series_id__":-1605771282004434260, "__mgmt_id__":-1605771282004434260} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.192672, "value1":0.6080907905788115, "value2":976012584384444918, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000795", "key_1":"f","key_8":"i","key_9":"d", "__series_id__":6086398502324277057, "__mgmt_id__":6086398502324277057} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.157561, "value1":0.6894676876528992, "value2":7135259052981755980, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000798", "key_5":"a","key_8":"h", "__series_id__":-5140571792209393544, "__mgmt_id__":-5140571792209393544} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.134773, "value1":0.09678670230466922, "value2":8180183833096777286, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000797", "key_0":"j","key_5":"i","key_8":"b", "__series_id__":-9156936360867657099, "__mgmt_id__":-9156936360867657099} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.616057, "value1":0.8520049469696344, "value2":2950033021270268791, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000799", "key_6":"i","key_8":"d", "__series_id__":3968087823541282551, "__mgmt_id__":3968087823541282551} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.255317, "value1":0.7477685612049465, "value2":179441442795111864, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000800", "key_7":"e","key_3":"d","key_5":"k", "__series_id__":1018474149027954404, "__mgmt_id__":1018474149027954404} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.906502, "value1":0.8256982980631076, "value2":4367074024600682775, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000801", "key_4":"d","key_6":"e","key_3":"j", "__series_id__":2939099060116576031, "__mgmt_id__":2939099060116576031} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.116793, "value1":0.8891425230837835, "value2":1719946264236570624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000802", "key_3":"f","key_4":"d","key_7":"i", "__series_id__":6511586288648013387, "__mgmt_id__":6511586288648013387} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.324866, "value1":0.5386590615308351, "value2":8011171897264590756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000803", "key_3":"k","key_7":"c","key_9":"f", "__series_id__":7054595087473761218, "__mgmt_id__":7054595087473761218} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.352403, "value1":0.7153316706866707, "value2":4379546221964794474, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000804", "key_1":"i","key_2":"b","key_4":"b", "__series_id__":-1949065403756251767, "__mgmt_id__":-1949065403756251767} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.836874, "value1":0.14443484962198383, "value2":1482104610634689373, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000805", "key_9":"b","key_5":"c","key_8":"a", "__series_id__":-6012620175802667643, "__mgmt_id__":-6012620175802667643} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.403735, "value1":0.8924124150178864, "value2":9156970764441343023, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000806", "key_0":"i","key_7":"h","key_8":"b", "__series_id__":-8852023680179592672, "__mgmt_id__":-8852023680179592672} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.141142, "value1":0.434266569307909, "value2":4978801873133265222, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000809", "key_2":"k","key_4":"f", "__series_id__":-5796750192728400794, "__mgmt_id__":-5796750192728400794} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.234292, "value1":0.7164352385365066, "value2":8038096300931135389, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000807", "key_3":"a","key_4":"d","key_1":"e", "__series_id__":-5308657687944985141, "__mgmt_id__":-5308657687944985141} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.973287, "value1":0.6600392290118359, "value2":2638933878817864633, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000808", "key_1":"e","key_3":"e","key_6":"b", "__series_id__":-6764254286344273733, "__mgmt_id__":-6764254286344273733} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.728882, "value1":0.476991229582368, "value2":6510855349407866452, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000812", "key_0":"b","key_1":"h","key_5":"k", "__series_id__":2941893197325067644, "__mgmt_id__":2941893197325067644} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787371, "value1":0.7330600795491425, "value2":6570294519251889382, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000810", "key_3":"c","key_6":"e","key_7":"g", "__series_id__":-1019576594847263341, "__mgmt_id__":-1019576594847263341} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903661, "value1":0.5944117402324142, "value2":6524268081747181442, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000811", "key_2":"a","key_3":"c", "__series_id__":-8832669958039901063, "__mgmt_id__":-8832669958039901063} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.659983, "value1":0.26592399871964695, "value2":1212972365053449050, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000815", "key_1":"j","key_4":"k","key_8":"b", "__series_id__":8212472004484910391, "__mgmt_id__":8212472004484910391} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.723204, "value1":0.31589384939280846, "value2":2041435063818693781, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000813", "key_1":"j","key_7":"g","key_8":"h", "__series_id__":-587356467785105936, "__mgmt_id__":-587356467785105936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.701487, "value1":0.5974401671491981, "value2":625551200860992484, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000814", "key_2":"f","key_4":"d", "__series_id__":3248867546045330458, "__mgmt_id__":3248867546045330458} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.893866, "value1":0.38643143057076507, "value2":6815713871810191176, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000817", "key_3":"c","key_5":"f","key_7":"d", "__series_id__":5999814608189423062, "__mgmt_id__":5999814608189423062} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.561678, "value1":0.4057036812724311, "value2":4919580315129580518, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000816", "key_4":"e","key_6":"d", "__series_id__":3265694429872982019, "__mgmt_id__":3265694429872982019} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.004391, "value1":0.4801368737089002, "value2":7547608457928538573, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000819", "key_2":"c","key_4":"h", "__series_id__":4395099723947130617, "__mgmt_id__":4395099723947130617} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.923757, "value1":0.7739736243035474, "value2":6832671633674661606, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000818", "key_3":"c","key_4":"g","key_6":"f", "__series_id__":-391108988132571423, "__mgmt_id__":-391108988132571423} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.819284, "value1":0.7572793995611461, "value2":68290908865890434, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000820", "key_4":"d","key_5":"f", "__series_id__":-3512838032523373226, "__mgmt_id__":-3512838032523373226} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.524308, "value1":0.44777317605800043, "value2":4272485517844220240, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000821", "key_0":"j","key_7":"e", "__series_id__":-2019541078929997410, "__mgmt_id__":-2019541078929997410} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.804762, "value1":0.20222965946470728, "value2":2164767398150388054, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000822", "key_5":"c","key_6":"h", "__series_id__":-5919712371960744559, "__mgmt_id__":-5919712371960744559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.060732, "value1":0.060291087703427844, "value2":8373529279962935628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000823", "key_8":"d","key_9":"i", "__series_id__":4237925648967851541, "__mgmt_id__":4237925648967851541} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325849, "value1":0.2785168531780935, "value2":8895250356037639155, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000825", "key_5":"k","key_7":"j","key_8":"d", "__series_id__":-770834305924250230, "__mgmt_id__":-770834305924250230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.378439, "value1":0.43808690997464833, "value2":7750734163790876517, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000824", "key_0":"f","key_3":"f","key_5":"a", "__series_id__":6485264920164065997, "__mgmt_id__":6485264920164065997} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.341809, "value1":0.21201598129686366, "value2":5439024219774015639, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000827", "key_0":"k","key_4":"g", "__series_id__":2073654399592069134, "__mgmt_id__":2073654399592069134} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.027383, "value1":0.519384455440142, "value2":2830105304694072087, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000826", "key_2":"b","key_7":"c","key_9":"e", "__series_id__":2882295507697788229, "__mgmt_id__":2882295507697788229} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.578370, "value1":0.4103102602537048, "value2":3491530677431875174, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000828", "key_1":"i","key_3":"b","key_7":"g", "__series_id__":-7220759306975072706, "__mgmt_id__":-7220759306975072706} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.588914, "value1":0.6688147984910581, "value2":7129525479125432993, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000829", "key_3":"i","key_4":"k","key_6":"a", "__series_id__":7896555977684467219, "__mgmt_id__":7896555977684467219} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.781407, "value1":0.7254015801455519, "value2":2135519716242173777, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000830", "key_2":"i","key_4":"e","key_6":"d", "__series_id__":6393502484828816639, "__mgmt_id__":6393502484828816639} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.852755, "value1":0.6592390034228756, "value2":5378675889994915559, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000831", "key_1":"k","key_8":"k","key_9":"h", "__series_id__":-1817488686594165705, "__mgmt_id__":-1817488686594165705} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.996821, "value1":0.5491613669334785, "value2":3858296464786926451, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000832", "key_0":"e","key_3":"h", "__series_id__":-5891578966361414907, "__mgmt_id__":-5891578966361414907} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.364027, "value1":0.6056949341817615, "value2":2956076236179631886, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000833", "key_1":"k","key_4":"c","key_9":"k", "__series_id__":-5759786475451562505, "__mgmt_id__":-5759786475451562505} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.605233, "value1":0.2433469771258824, "value2":8502972839119365612, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000834", "key_0":"a","key_2":"k","key_4":"h", "__series_id__":2288700931631591137, "__mgmt_id__":2288700931631591137} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.092828, "value1":0.29453399849858286, "value2":5192582279143186491, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000835", "key_9":"k","key_2":"a","key_7":"g", "__series_id__":-5779278934174818645, "__mgmt_id__":-5779278934174818645} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.179799, "value1":0.19121582692459024, "value2":9103159579935481171, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000836", "key_1":"k","key_3":"h","key_9":"g", "__series_id__":-1876395674046103849, "__mgmt_id__":-1876395674046103849} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.054748, "value1":0.7255577017707996, "value2":8476394128272498701, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000837", "key_5":"b","key_9":"i", "__series_id__":-7012885648223233057, "__mgmt_id__":-7012885648223233057} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.959235, "value1":0.5345272240165994, "value2":8621793756396494146, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000838", "key_5":"j","key_6":"h","key_9":"b", "__series_id__":5804665130094856991, "__mgmt_id__":5804665130094856991} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.735109, "value1":0.7362204691534435, "value2":942332422397834734, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000839", "key_0":"f","key_4":"h", "__series_id__":-7344671437631279298, "__mgmt_id__":-7344671437631279298} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.778088, "value1":0.8485308761635748, "value2":8355286652172671018, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000842", "key_3":"b","key_6":"c","key_0":"c", "__series_id__":-6621083798343228179, "__mgmt_id__":-6621083798343228179} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.026529, "value1":0.058403258262061046, "value2":3881865567315099972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000840", "key_2":"b","key_3":"i","key_7":"c", "__series_id__":-1607750710100122463, "__mgmt_id__":-1607750710100122463} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.405396, "value1":0.78515016074467, "value2":3124806945169635921, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000841", "key_1":"i","key_2":"d","key_6":"b", "__series_id__":-8161171256724117796, "__mgmt_id__":-8161171256724117796} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.325780, "value1":0.8484383369933163, "value2":5639577297430380059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000844", "key_2":"a","key_7":"b", "__series_id__":5946130633726276979, "__mgmt_id__":5946130633726276979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.125526, "value1":0.9437452957692816, "value2":1600583134494091930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000843", "key_0":"f","key_9":"i", "__series_id__":-8946617436119621199, "__mgmt_id__":-8946617436119621199} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.040638, "value1":0.8412837985326597, "value2":1891897645252751874, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000846", "key_1":"h","key_4":"k","key_8":"e", "__series_id__":8845218157639111760, "__mgmt_id__":8845218157639111760} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.225539, "value1":0.7607755326390468, "value2":4494740517836964298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000845", "key_8":"b","key_0":"c","key_3":"d", "__series_id__":2951654281967391321, "__mgmt_id__":2951654281967391321} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.417989, "value1":0.8351179906534547, "value2":6640751019261660918, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000848", "key_2":"f","key_5":"a", "__series_id__":8257293525391999222, "__mgmt_id__":8257293525391999222} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.930391, "value1":0.7797566384384467, "value2":5344540873017186839, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000847", "key_0":"k","key_5":"g","key_9":"d", "__series_id__":-3884817009449312047, "__mgmt_id__":-3884817009449312047} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.130784, "value1":0.9081520679550602, "value2":2673655563019876645, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000849", "key_3":"e","key_4":"e", "__series_id__":2341952400790155478, "__mgmt_id__":2341952400790155478} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.065288, "value1":0.37878147094860193, "value2":40939122026803562, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000850", "key_3":"b","key_4":"f","key_8":"d", "__series_id__":6875804869664046307, "__mgmt_id__":6875804869664046307} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.023954, "value1":0.7421706325870447, "value2":3936838989809781841, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000851", "key_4":"f","key_5":"g","key_8":"c", "__series_id__":-593944646664066983, "__mgmt_id__":-593944646664066983} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.639378, "value1":0.04809332390342072, "value2":138564553177528595, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000852", "key_4":"d","key_7":"i", "__series_id__":-3088435001982835586, "__mgmt_id__":-3088435001982835586} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.968036, "value1":0.4229941922357839, "value2":3252691734036720501, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000853", "key_6":"k","key_9":"j","key_4":"e", "__series_id__":3654818614521571007, "__mgmt_id__":3654818614521571007} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.925601, "value1":0.5862382194484029, "value2":8809171247244862983, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000854", "key_1":"d","key_2":"f","key_4":"g", "__series_id__":-8643978816140336726, "__mgmt_id__":-8643978816140336726} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.615163, "value1":0.41538444706998723, "value2":7806072407872661917, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000855", "key_1":"j","key_8":"c","key_9":"d", "__series_id__":1969862486326467304, "__mgmt_id__":1969862486326467304} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.432657, "value1":0.2054174608330251, "value2":523721991362366859, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000856", "key_3":"i","key_5":"a","key_6":"d", "__series_id__":9047687757698452618, "__mgmt_id__":9047687757698452618} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.584157, "value1":0.38115353295991117, "value2":5695252506949414616, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000860", "key_4":"j","key_5":"c", "__series_id__":1895373427612509316, "__mgmt_id__":1895373427612509316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.969537, "value1":0.31115941644555445, "value2":3813252754578465682, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000867", "key_9":"g","key_0":"g","key_6":"k", "__series_id__":3557796764686593576, "__mgmt_id__":3557796764686593576} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.755713, "value1":0.6078558755538082, "value2":4745658402471707078, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000868", "key_3":"i","key_4":"i","key_8":"j", "__series_id__":7934961687481133421, "__mgmt_id__":7934961687481133421} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.299770, "value1":0.39494561763395397, "value2":7454472376009363943, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000869", "key_1":"c","key_3":"c","key_5":"b", "__series_id__":7028579199313468984, "__mgmt_id__":7028579199313468984} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.549446, "value1":0.9276674467108077, "value2":7450051253648573468, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000870", "key_4":"c","key_6":"c","key_3":"c", "__series_id__":-5487303349427062046, "__mgmt_id__":-5487303349427062046} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.339276, "value1":0.16417640235809516, "value2":376449946500069421, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000871", "key_7":"j","key_4":"j","key_6":"c", "__series_id__":6074226575508628248, "__mgmt_id__":6074226575508628248} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.044679, "value1":0.022401336029968563, "value2":7584563374980649526, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000872", "key_6":"b","key_8":"j","key_5":"i", "__series_id__":-7977482960971959221, "__mgmt_id__":-7977482960971959221} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.978082, "value1":0.634805138120068, "value2":2509593268797839298, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000873", "key_8":"g","key_9":"e", "__series_id__":5541777429936042172, "__mgmt_id__":5541777429936042172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.020832, "value1":0.2523976489068679, "value2":5330781529709694724, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000874", "key_1":"e","key_9":"f", "__series_id__":-2446119420668167024, "__mgmt_id__":-2446119420668167024} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.874340, "value1":0.5447943369952332, "value2":753638192956179932, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000875", "key_9":"i","key_1":"d","key_4":"c", "__series_id__":-6264079928411176214, "__mgmt_id__":-6264079928411176214} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.532731, "value1":0.3415364500896806, "value2":4617908501249248943, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000857", "key_3":"j","key_4":"i", "__series_id__":5259931197981260673, "__mgmt_id__":5259931197981260673} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.888667, "value1":0.03025871907918381, "value2":389467035709739331, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000858", "key_8":"a","key_1":"h","key_7":"b", "__series_id__":6649881170970133250, "__mgmt_id__":6649881170970133250} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.630653, "value1":0.3815158272162314, "value2":1365697000438851836, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000859", "key_4":"h","key_5":"h","key_6":"a", "__series_id__":-3642784698751404809, "__mgmt_id__":-3642784698751404809} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.947344, "value1":0.018503516339994433, "value2":6964047457370572921, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000863", "key_3":"j","key_4":"c", "__series_id__":-8290345855141384414, "__mgmt_id__":-8290345855141384414} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.635335, "value1":0.7735503313212675, "value2":7863214250826862536, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000861", "key_5":"g","key_6":"j", "__series_id__":4177659989436222538, "__mgmt_id__":4177659989436222538} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.323808, "value1":0.9277969228346735, "value2":7104799245414939756, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000862", "key_1":"b","key_4":"b","key_8":"d", "__series_id__":5224088293720667434, "__mgmt_id__":5224088293720667434} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.527571, "value1":0.19552679967510578, "value2":2150832405165980506, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000864", "key_1":"a","key_7":"a","key_9":"j", "__series_id__":5463684432285000465, "__mgmt_id__":5463684432285000465} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.076891, "value1":0.45024886897327476, "value2":7461894110864047862, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000865", "key_0":"c","key_3":"h", "__series_id__":8013027916162000847, "__mgmt_id__":8013027916162000847} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.433797, "value1":0.03730319472343055, "value2":8041320080812623904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000866", "key_4":"c","key_8":"d","key_9":"a", "__series_id__":-3273450908649443185, "__mgmt_id__":-3273450908649443185} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745634, "value1":0.6889162665973937, "value2":7455824590286680395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000876", "key_4":"g","key_5":"j","key_0":"i", "__series_id__":5397678799753812971, "__mgmt_id__":5397678799753812971} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.310508, "value1":0.4992657909321023, "value2":5571385578668134720, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000877", "key_0":"b","key_3":"i","key_6":"g", "__series_id__":-2152602439220104890, "__mgmt_id__":-2152602439220104890} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.343053, "value1":0.21251046051646363, "value2":6470523853548900348, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000878", "key_5":"d","key_8":"e","key_9":"d", "__series_id__":9073385261045683729, "__mgmt_id__":9073385261045683729} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.903927, "value1":0.9720310949900105, "value2":7380705280316406577, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000879", "key_3":"e","key_4":"j","key_5":"a", "__series_id__":7510078098956056860, "__mgmt_id__":7510078098956056860} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.200927, "value1":0.42664784751514406, "value2":5730113255528003277, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000880", "key_9":"g","key_7":"h", "__series_id__":3753170390494216769, "__mgmt_id__":3753170390494216769} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.563845, "value1":0.8055339411145567, "value2":7403476714487686904, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000882", "key_6":"a","key_9":"c","key_3":"d", "__series_id__":5554465061930605789, "__mgmt_id__":5554465061930605789} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.329254, "value1":0.8021973639133464, "value2":3080745955617166362, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000881", "key_2":"f","key_9":"f", "__series_id__":-2011474303426561839, "__mgmt_id__":-2011474303426561839} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.879352, "value1":0.4277546544111605, "value2":1478811657161573091, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000884", "key_3":"a","key_9":"h","key_0":"i", "__series_id__":-7410343999162991368, "__mgmt_id__":-7410343999162991368} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.442846, "value1":0.006346529763383217, "value2":5543697338259725446, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000883", "key_7":"b","key_9":"c","key_2":"i", "__series_id__":-3993102668143572443, "__mgmt_id__":-3993102668143572443} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.094781, "value1":0.40728088001797547, "value2":6613387129832293245, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000885", "key_3":"k","key_6":"g","key_9":"f", "__series_id__":6443612413446657716, "__mgmt_id__":6443612413446657716} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.975150, "value1":0.3661351435962595, "value2":830819320558121686, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000886", "key_4":"j","key_9":"k","key_3":"f", "__series_id__":3230940730205655533, "__mgmt_id__":3230940730205655533} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.245141, "value1":0.02569157141735858, "value2":2603499185616374516, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000887", "key_3":"h","key_6":"f","key_9":"d", "__series_id__":4336773822987251236, "__mgmt_id__":4336773822987251236} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.949515, "value1":0.3643114110362523, "value2":832829152625075749, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000888", "key_6":"k","key_2":"f","key_4":"i", "__series_id__":-3917471179921730230, "__mgmt_id__":-3917471179921730230} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.741852, "value1":0.28794172682449687, "value2":453442634675086185, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000889", "key_3":"f","key_5":"c","key_8":"k", "__series_id__":-8475829672726930205, "__mgmt_id__":-8475829672726930205} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.738181, "value1":0.6087693219046798, "value2":7047508567290984350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000890", "key_0":"a","key_8":"h", "__series_id__":-2835267016325291387, "__mgmt_id__":-2835267016325291387} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.492510, "value1":0.5382269802655012, "value2":2689541564459522343, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000891", "key_3":"d","key_7":"i","key_2":"i", "__series_id__":2755156865704901559, "__mgmt_id__":2755156865704901559} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.988702, "value1":0.0728816330863758, "value2":5556532985784164765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000892", "key_4":"b","key_0":"f", "__series_id__":-3684675206016205580, "__mgmt_id__":-3684675206016205580} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.655057, "value1":0.7734197867503856, "value2":5244006855203800595, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000893", "key_1":"h","key_2":"h", "__series_id__":-5902123974422169223, "__mgmt_id__":-5902123974422169223} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.520001, "value1":0.4376795320695461, "value2":1470774311764721388, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000894", "key_3":"k","key_5":"c","key_7":"d", "__series_id__":6986853351187564285, "__mgmt_id__":6986853351187564285} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.164473, "value1":0.6572062696131487, "value2":1949710197196336093, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000895", "key_0":"b","key_7":"a","key_9":"d", "__series_id__":-8229195773865957634, "__mgmt_id__":-8229195773865957634} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.190159, "value1":0.9525604598487357, "value2":5478966602860717950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000899", "key_0":"b","key_3":"f","key_5":"f", "__series_id__":-5149502230124017537, "__mgmt_id__":-5149502230124017537} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.829372, "value1":0.6923157559412111, "value2":5023489111088251295, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000896", "key_1":"b","key_5":"i", "__series_id__":-4421406399737878814, "__mgmt_id__":-4421406399737878814} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.391391, "value1":0.6340217327197775, "value2":8000248111604879168, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000897", "key_0":"d","key_2":"h","key_8":"i", "__series_id__":-389408712295111002, "__mgmt_id__":-389408712295111002} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883302, "value1":0.8070620002939071, "value2":653446263691796732, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000898", "key_4":"i","key_5":"k","key_9":"d", "__series_id__":6306506282210639638, "__mgmt_id__":6306506282210639638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.745426, "value1":0.45672801954182946, "value2":6893619651580083925, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000902", "key_1":"e","key_9":"a", "__series_id__":6312106658869392745, "__mgmt_id__":6312106658869392745} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.982512, "value1":0.5060051247468136, "value2":7419982333827237040, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000900", "key_0":"j","key_4":"c","key_7":"j", "__series_id__":5247126023882251421, "__mgmt_id__":5247126023882251421} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.827729, "value1":0.15692262245568137, "value2":3804375123078218665, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000901", "key_0":"b","key_4":"h","key_5":"f", "__series_id__":2218975377452306524, "__mgmt_id__":2218975377452306524} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.244618, "value1":0.6655062510702947, "value2":1607640639377335930, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000905", "key_0":"k","key_3":"j","key_4":"d", "__series_id__":5113417579211753098, "__mgmt_id__":5113417579211753098} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.945643, "value1":0.3793044572114196, "value2":8429298239346478761, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000903", "key_2":"c","key_6":"d","key_7":"b", "__series_id__":-4936732084670057584, "__mgmt_id__":-4936732084670057584} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.626649, "value1":0.22068614209697351, "value2":8899816776188911022, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000904", "key_1":"i","key_5":"b","key_6":"f", "__series_id__":-5163554538407216849, "__mgmt_id__":-5163554538407216849} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.045981, "value1":0.34313373362766186, "value2":1556434949234629007, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000906", "key_0":"g","key_1":"j","key_2":"j", "__series_id__":8148489552467310439, "__mgmt_id__":8148489552467310439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.381111, "value1":0.4930245395823471, "value2":3486502897378084847, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000907", "key_0":"e","key_1":"i","key_8":"i", "__series_id__":-3944050620113208172, "__mgmt_id__":-3944050620113208172} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.596053, "value1":0.23068160326995202, "value2":1094706285002463668, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000908", "key_6":"i","key_7":"e", "__series_id__":-4730499102096054036, "__mgmt_id__":-4730499102096054036} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.454071, "value1":0.4016206555788349, "value2":7766949472696777624, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000911", "key_3":"k","key_4":"h","key_9":"a", "__series_id__":-2283650812855623557, "__mgmt_id__":-2283650812855623557} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.122082, "value1":0.12814082114630934, "value2":6860595913103836152, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000909", "key_0":"d","key_2":"j","key_3":"i", "__series_id__":8383629273121902238, "__mgmt_id__":8383629273121902238} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.267563, "value1":0.915043661363235, "value2":7117774485449523197, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000910", "key_8":"c","key_9":"e","key_5":"a", "__series_id__":8192386169222148683, "__mgmt_id__":8192386169222148683} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.883324, "value1":0.48609131055046695, "value2":168158041910482971, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000914", "key_7":"g","key_8":"h", "__series_id__":-6271717978981701453, "__mgmt_id__":-6271717978981701453} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.798512, "value1":0.9160093333280128, "value2":5957101416277255647, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000912", "key_6":"c","key_9":"a","key_0":"k", "__series_id__":-6816225054378181173, "__mgmt_id__":-6816225054378181173} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.956973, "value1":0.08410692933360829, "value2":3670111299929079308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000913", "key_4":"i","key_7":"f","key_1":"a", "__series_id__":-2352461444413661306, "__mgmt_id__":-2352461444413661306} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.799611, "value1":0.006983554187354543, "value2":560914434528784616, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000916", "key_3":"e","key_7":"b","key_9":"k", "__series_id__":-3385512603376303519, "__mgmt_id__":-3385512603376303519} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.539372, "value1":0.1527069613506525, "value2":5396674816694339651, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000915", "key_8":"e","key_0":"e","key_1":"b", "__series_id__":-6851409186770428825, "__mgmt_id__":-6851409186770428825} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.538750, "value1":0.21348159925068225, "value2":4409731296615206554, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000918", "key_2":"j","key_3":"g","key_6":"h", "__series_id__":1189528832259185027, "__mgmt_id__":1189528832259185027} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.854320, "value1":0.921941847389472, "value2":4199458567691684987, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000917", "key_4":"d","key_6":"a", "__series_id__":-4493020240885746936, "__mgmt_id__":-4493020240885746936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.451148, "value1":0.5527973855585238, "value2":5090625521103681352, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000919", "key_6":"i","key_9":"g", "__series_id__":8661813703504300442, "__mgmt_id__":8661813703504300442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.585722, "value1":0.08217397453744285, "value2":5891174882356923897, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000920", "key_8":"k","key_9":"g","key_2":"c", "__series_id__":-8613423020305213465, "__mgmt_id__":-8613423020305213465} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.280675, "value1":0.17159417706244065, "value2":1155748852941674950, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000921", "key_3":"e","key_4":"e","key_7":"b", "__series_id__":-5237232801702246555, "__mgmt_id__":-5237232801702246555} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673546, "value1":0.9064641866629127, "value2":5826431744512876414, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000922", "key_5":"b","key_7":"h","key_9":"k", "__series_id__":-2788154845987041856, "__mgmt_id__":-2788154845987041856} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.748734, "value1":0.7184282675490825, "value2":465504921239629350, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000923", "key_3":"h","key_5":"j","key_6":"e", "__series_id__":-9060373555969354210, "__mgmt_id__":-9060373555969354210} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448890, "value1":0.21947167931430242, "value2":97944850538911159, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000925", "key_0":"e","key_3":"h","key_9":"c", "__series_id__":5845352045669007793, "__mgmt_id__":5845352045669007793} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.034158, "value1":0.6471204512621925, "value2":6508843210808028097, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000924", "key_1":"c","key_3":"k","key_4":"a", "__series_id__":4773367547171091406, "__mgmt_id__":4773367547171091406} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.495253, "value1":0.7273452290068572, "value2":2698451345131648958, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000927", "key_1":"g","key_4":"g","key_9":"d", "__series_id__":2681296371108068246, "__mgmt_id__":2681296371108068246} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.362458, "value1":0.6109728168729301, "value2":8601029350565666815, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000926", "key_1":"k","key_3":"b","key_5":"g", "__series_id__":-1620750445123524491, "__mgmt_id__":-1620750445123524491} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.263467, "value1":0.5089131780165215, "value2":7125458167541328714, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000929", "key_1":"g","key_5":"c","key_8":"i", "__series_id__":1541348361758017758, "__mgmt_id__":1541348361758017758} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373946, "value1":0.4318385725650386, "value2":2714678475704530055, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000928", "key_5":"k","key_7":"k","key_3":"e", "__series_id__":7246491393905121980, "__mgmt_id__":7246491393905121980} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.714267, "value1":0.9575832102733836, "value2":6392945512202004757, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000931", "key_0":"j","key_7":"g", "__series_id__":-3841678152688368567, "__mgmt_id__":-3841678152688368567} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.448342, "value1":0.22459353529927165, "value2":3260297440141888471, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000930", "key_3":"b","key_7":"b", "__series_id__":-3322655866165129051, "__mgmt_id__":-3322655866165129051} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.636333, "value1":0.5831279192445404, "value2":4643961286576988019, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000932", "key_1":"j","key_4":"e","key_6":"g", "__series_id__":-8920885921549084638, "__mgmt_id__":-8920885921549084638} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.544083, "value1":0.5292455018046274, "value2":2947716990317042035, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000933", "key_2":"g","key_4":"j","key_6":"d", "__series_id__":-2133694680914957018, "__mgmt_id__":-2133694680914957018} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.834093, "value1":0.03492084160563978, "value2":317252294366234894, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000934", "key_7":"h","key_8":"i", "__series_id__":6445232353634246963, "__mgmt_id__":6445232353634246963} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266819, "value1":0.3363934052516993, "value2":7398703592096323308, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000935", "key_4":"c","key_6":"f","key_7":"i", "__series_id__":-8058668717700101835, "__mgmt_id__":-8058668717700101835} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.006504, "value1":0.7543858008870775, "value2":1775651744055633759, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000936", "key_9":"c","key_0":"j","key_4":"f", "__series_id__":8609030237689560074, "__mgmt_id__":8609030237689560074} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.114763, "value1":0.5343311798406293, "value2":3440557676505536535, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000937", "key_4":"i","key_8":"h","key_9":"i", "__series_id__":126931402558459640, "__mgmt_id__":126931402558459640} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.338231, "value1":0.46269924741830853, "value2":2414529329564577894, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000938", "key_5":"b","key_6":"k","key_7":"h", "__series_id__":7103485997179993993, "__mgmt_id__":7103485997179993993} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.613570, "value1":0.5208853551038555, "value2":1476820686457140000, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000939", "key_1":"k","key_6":"g","key_8":"c", "__series_id__":2618446898923694438, "__mgmt_id__":2618446898923694438} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.954830, "value1":0.41362302765893333, "value2":5510842372132552038, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000940", "key_5":"h","key_6":"e", "__series_id__":5745116829380616979, "__mgmt_id__":5745116829380616979} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.009033, "value1":0.39228566619392624, "value2":4427054644662596059, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000941", "key_4":"i","key_5":"i","key_9":"g", "__series_id__":-8638927876938940545, "__mgmt_id__":-8638927876938940545} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.182689, "value1":0.6549868089884497, "value2":4035004692027824929, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000943", "key_3":"d","key_4":"i","key_2":"a", "__series_id__":509974957040433669, "__mgmt_id__":509974957040433669} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.037729, "value1":0.7486872272646202, "value2":4847504507990849152, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000942", "key_8":"k","key_3":"i", "__series_id__":1971862127581653694, "__mgmt_id__":1971862127581653694} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.025112, "value1":0.34907292952301494, "value2":6027470898165077200, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000945", "key_3":"f","key_5":"c","key_8":"h", "__series_id__":-1804379138188887773, "__mgmt_id__":-1804379138188887773} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.438904, "value1":0.6318755547819239, "value2":7111135668532014691, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000944", "key_1":"c","key_2":"a","key_7":"e", "__series_id__":-2151408237051366225, "__mgmt_id__":-2151408237051366225} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.812145, "value1":0.3675412312936663, "value2":5692745323903444622, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000947", "key_1":"d","key_2":"h","key_7":"f", "__series_id__":6639686090018856376, "__mgmt_id__":6639686090018856376} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.445020, "value1":0.3643115537301237, "value2":1216854539448528229, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000946", "key_1":"j","key_2":"d", "__series_id__":-3178230256095965210, "__mgmt_id__":-3178230256095965210} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.149138, "value1":0.5909969143971484, "value2":7044155605397820069, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000949", "key_1":"b","key_2":"c","key_7":"h", "__series_id__":2114766337578869876, "__mgmt_id__":2114766337578869876} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.841137, "value1":0.5031802947436413, "value2":4756712241095891395, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000948", "key_1":"k","key_2":"a","key_5":"h", "__series_id__":8896905802866397624, "__mgmt_id__":8896905802866397624} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.766360, "value1":0.42396402951055434, "value2":5375665261877407507, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000951", "key_1":"g","key_7":"g","key_9":"i", "__series_id__":7871636101908571807, "__mgmt_id__":7871636101908571807} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.663263, "value1":0.21109789310310256, "value2":7791325581861318665, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000950", "key_0":"i","key_9":"c", "__series_id__":5714461649218477173, "__mgmt_id__":5714461649218477173} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.623457, "value1":0.10997466706442727, "value2":4022405196702857232, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000953", "key_0":"k","key_1":"f","key_9":"g", "__series_id__":-2237621028132536337, "__mgmt_id__":-2237621028132536337} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.687642, "value1":0.5890580006230834, "value2":4798397386267850627, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000952", "key_2":"b","key_0":"i", "__series_id__":227860907906626026, "__mgmt_id__":227860907906626026} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.041534, "value1":0.7259891148801754, "value2":7530272514921895889, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000956", "key_7":"i","key_6":"c", "__series_id__":-8040910452447775473, "__mgmt_id__":-8040910452447775473} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.999424, "value1":0.8243889839817137, "value2":2465746906164654846, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000954", "key_2":"h","key_8":"h","key_9":"e", "__series_id__":606884971259349760, "__mgmt_id__":606884971259349760} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.282429, "value1":0.6519583818560468, "value2":6092402468810251742, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000955", "key_1":"e","key_6":"j","key_7":"d", "__series_id__":-5478061519059366611, "__mgmt_id__":-5478061519059366611} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.816484, "value1":0.18010854645368016, "value2":8191892113724148654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000958", "key_2":"i","key_6":"f", "__series_id__":-124525408228495085, "__mgmt_id__":-124525408228495085} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.851506, "value1":0.5899554257608193, "value2":4617440579117228711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000957", "key_0":"c","key_1":"i","key_5":"g", "__series_id__":-7770945742068148908, "__mgmt_id__":-7770945742068148908} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.382630, "value1":0.5412707148385829, "value2":3548532358946210519, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000960", "key_2":"c","key_3":"e", "__series_id__":980402580434312102, "__mgmt_id__":980402580434312102} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.473179, "value1":0.6228095400098033, "value2":5572845251799335628, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000959", "key_0":"b","key_2":"h","key_9":"g", "__series_id__":-4104284881803245064, "__mgmt_id__":-4104284881803245064} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.266666, "value1":0.5868701622069099, "value2":6991916315373855609, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000962", "key_9":"d","key_3":"d","key_7":"f", "__series_id__":-8580448352974816854, "__mgmt_id__":-8580448352974816854} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.952001, "value1":0.6189910604042005, "value2":6928874617046934238, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000961", "key_7":"b","key_8":"b","key_9":"h", "__series_id__":5093385098667663207, "__mgmt_id__":5093385098667663207} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.107532, "value1":0.28068061790544596, "value2":7672880301382817972, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000964", "key_5":"b","key_7":"b","key_1":"e", "__series_id__":-7638387450661547132, "__mgmt_id__":-7638387450661547132} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.688840, "value1":0.6035662636797529, "value2":5891970996329552554, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000963", "key_4":"h","key_5":"a","key_8":"b", "__series_id__":3013184525747786739, "__mgmt_id__":3013184525747786739} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.283889, "value1":0.9151251791608558, "value2":2952612587882160016, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000966", "key_1":"a","key_2":"f", "__series_id__":-3208973475909870211, "__mgmt_id__":-3208973475909870211} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.629939, "value1":0.5930273861647344, "value2":7387132779647308781, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000965", "key_5":"i","key_9":"c", "__series_id__":-5881314218080136649, "__mgmt_id__":-5881314218080136649} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.418325, "value1":0.9507448442764749, "value2":167010402424657095, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000967", "key_9":"f","key_1":"h","key_5":"d", "__series_id__":-1869366054771332234, "__mgmt_id__":-1869366054771332234} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.373327, "value1":0.6218852126400313, "value2":8371256546452205404, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000968", "key_7":"h","key_0":"h","key_4":"k", "__series_id__":853455729293892, "__mgmt_id__":853455729293892} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.515481, "value1":0.0854866845303678, "value2":6987828829468502321, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000969", "key_2":"a","key_8":"k", "__series_id__":-1880586229668866123, "__mgmt_id__":-1880586229668866123} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.820817, "value1":0.698882043745769, "value2":8235587889784697376, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000970", "key_7":"d","key_8":"e","key_1":"c", "__series_id__":-4611408819127013586, "__mgmt_id__":-4611408819127013586} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.365626, "value1":0.4531560215251981, "value2":4722270617283077560, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000971", "key_2":"j","key_8":"i", "__series_id__":8076734271383159573, "__mgmt_id__":8076734271383159573} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484276, "value1":0.6659326817404172, "value2":8721021931100680532, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000973", "key_0":"d","key_1":"f","key_6":"c", "__series_id__":660960526644120465, "__mgmt_id__":660960526644120465} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.528708, "value1":0.03487880924213162, "value2":6041728758379789938, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000972", "key_2":"f","key_4":"d","key_6":"k", "__series_id__":7454847514743186106, "__mgmt_id__":7454847514743186106} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.590377, "value1":0.34239911489608105, "value2":7295038165138773264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000975", "key_1":"k","key_2":"a","key_7":"g", "__series_id__":-4087977803898185094, "__mgmt_id__":-4087977803898185094} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.481162, "value1":0.8346535177821227, "value2":4835378003160835691, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000974", "key_4":"a","key_6":"j","key_7":"c", "__series_id__":7365105254973640807, "__mgmt_id__":7365105254973640807} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.664721, "value1":0.9670197543347829, "value2":7077761062907993448, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000977", "key_1":"h","key_7":"f", "__series_id__":3179964060483693959, "__mgmt_id__":3179964060483693959} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.242914, "value1":0.7732131934363016, "value2":6013582492758534765, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000976", "key_4":"a","key_7":"a","key_9":"a", "__series_id__":4943495658983356925, "__mgmt_id__":4943495658983356925} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.185221, "value1":0.10448347281111905, "value2":4626218721873500711, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000979", "key_8":"i","key_2":"a","key_7":"i", "__series_id__":5476702965391324178, "__mgmt_id__":5476702965391324178} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.126356, "value1":0.6396318028070207, "value2":7845764314900334021, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000978", "key_9":"a","key_7":"e","key_8":"h", "__series_id__":-7659960989015255316, "__mgmt_id__":-7659960989015255316} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.673777, "value1":0.7918565155988029, "value2":7808431007480445571, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000980", "key_0":"d","key_4":"d","key_7":"f", "__series_id__":1217326367461456504, "__mgmt_id__":1217326367461456504} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.706238, "value1":0.49404656958089044, "value2":8867399074074176905, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000981", "key_1":"g","key_2":"h","key_4":"f", "__series_id__":3600556749683819879, "__mgmt_id__":3600556749683819879} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.216638, "value1":0.2610211982854062, "value2":5851177857276860153, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000982", "key_0":"f","key_5":"h","key_9":"f", "__series_id__":6176476763516588600, "__mgmt_id__":6176476763516588600} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247760, "value1":0.720377370743505, "value2":284915355108242656, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000983", "key_0":"b","key_7":"c","key_9":"e", "__series_id__":5570056772565202366, "__mgmt_id__":5570056772565202366} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.484098, "value1":0.20488798005491424, "value2":8924950640215344491, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000984", "key_0":"k","key_3":"f","key_9":"g", "__series_id__":8119351734948487376, "__mgmt_id__":8119351734948487376} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.345583, "value1":0.26657891861913446, "value2":1138178968371346598, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000985", "key_0":"a","key_7":"j", "__series_id__":6742140614929127838, "__mgmt_id__":6742140614929127838} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.315052, "value1":0.7472751608991773, "value2":5324337837076333029, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000988", "key_3":"g","key_4":"a","key_9":"e", "__series_id__":-1525457297647045442, "__mgmt_id__":-1525457297647045442} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.247789, "value1":0.35415032152422526, "value2":3640906326280302592, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000986", "key_2":"b","key_3":"a","key_5":"e", "__series_id__":-3761616453811965441, "__mgmt_id__":-3761616453811965441} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.787688, "value1":0.6162646188121896, "value2":8883014095767101323, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000987", "key_2":"a","key_5":"b","key_7":"e", "__series_id__":-1779822276119301297, "__mgmt_id__":-1779822276119301297} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.477237, "value1":0.04016123924628, "value2":9093318721099578903, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000990", "key_0":"j","key_9":"d", "__series_id__":8284198859322921936, "__mgmt_id__":8284198859322921936} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.560140, "value1":0.30318020971980014, "value2":2686556711005804654, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000989", "key_0":"c","key_2":"h","key_7":"f", "__series_id__":-4372668294353943556, "__mgmt_id__":-4372668294353943556} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.002924, "value1":0.5022585667052755, "value2":3145515076422963133, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000992", "key_4":"k","key_5":"d","key_6":"a", "__series_id__":-7954753156742574681, "__mgmt_id__":-7954753156742574681} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.732614, "value1":0.3545238139152862, "value2":1059232874175794196, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000991", "key_5":"i","key_8":"h", "__series_id__":-7563389249691286962, "__mgmt_id__":-7563389249691286962} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.332613, "value1":0.32230993437491456, "value2":2724091454501972237, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000994", "key_1":"f","key_3":"a","key_7":"a", "__series_id__":-1843525876921281439, "__mgmt_id__":-1843525876921281439} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.012206, "value1":0.931677320101252, "value2":5370261932149351692, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000993", "key_1":"c","key_7":"h","key_8":"a", "__series_id__":1826472716586726705, "__mgmt_id__":1826472716586726705} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.196655, "value1":0.8224901135235602, "value2":9208484728253262649, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000995", "key_1":"k","key_5":"h","key_8":"k", "__series_id__":-6562641159861820695, "__mgmt_id__":-6562641159861820695} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.326865, "value1":0.5922171068495821, "value2":7224402625385076257, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000996", "key_2":"d","key_5":"j", "__series_id__":2665602518452103625, "__mgmt_id__":2665602518452103625} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.289542, "value1":0.9512116686992638, "value2":7523857296190024264, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000997", "key_0":"d","key_1":"h","key_4":"e", "__series_id__":-481551255976768557, "__mgmt_id__":-481551255976768557} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.170535, "value1":0.5872728280225836, "value2":6413447856713598093, "value3":true, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000999", "key_3":"c","key_5":"b","key_9":"h", "__series_id__":-636081195137433630, "__mgmt_id__":-636081195137433630} +{"timestamp":"2023-01-20T00:00:09+08:00", "value":0.434673, "value1":0.18109149352889556, "value2":1218373397895276791, "value3":false, "value4": 0.4964787564538794, "value5": true, "__name__":"metric_00000998", "key_3":"e","key_9":"c", "__series_id__":-6832493598228589478, "__mgmt_id__":-6832493598228589478} diff --git a/docker/test_prom_metric.hjson b/docker/test_prom_metric.hjson index ce9c7650..6a4cb47e 100644 --- a/docker/test_prom_metric.hjson +++ b/docker/test_prom_metric.hjson @@ -29,7 +29,7 @@ PrometheusSchema: true DynamicSchema: { Enable: true - whiteList: ^(key_[0-9]?|value[1-4]?|__mgmt_id|__name__|timestamp|__series_id)$ + whiteList: ^(key_[0-9]?|value[1-4]?|__mgmt_id__|__name__|timestamp|__series_id__)$ blackList: ^(key_3|value4)$ } } diff --git a/go.metrictest.sh b/go.metrictest.sh index 558ee160..c7759c22 100755 --- a/go.metrictest.sh +++ b/go.metrictest.sh @@ -7,12 +7,12 @@ echo "create metric test tables" curl "localhost:8123" -d "DROP TABLE IF EXISTS test_prom_metric ON CLUSTER abc SYNC" curl "localhost:8123" -d "CREATE TABLE test_prom_metric ON CLUSTER abc ( - __series_id Int64, + __series_id__ Int64, timestamp DateTime CODEC(DoubleDelta, LZ4), value Float32 CODEC(ZSTD(15)) ) ENGINE=ReplicatedReplacingMergeTree() PARTITION BY toYYYYMMDD(timestamp) -ORDER BY (__series_id, timestamp);" +ORDER BY (__series_id__, timestamp);" curl "localhost:8123" -d "DROP TABLE IF EXISTS dist_test_prom_metric ON CLUSTER abc SYNC" curl "localhost:8123" -d "CREATE TABLE dist_test_prom_metric ON CLUSTER abc AS test_prom_metric ENGINE = Distributed(abc, default, test_prom_metric);" @@ -20,12 +20,12 @@ curl "localhost:8123" -d "CREATE TABLE dist_test_prom_metric ON CLUSTER abc AS t curl "localhost:8123" -d "DROP TABLE IF EXISTS test_prom_series ON CLUSTER abc SYNC" curl "localhost:8123" -d "CREATE TABLE test_prom_series ON CLUSTER abc ( - __series_id Int64, - __mgmt_id Int64, + __series_id__ Int64, + __mgmt_id__ Int64, labels String, __name__ String ) ENGINE=ReplicatedReplacingMergeTree() -ORDER BY (__name__, __series_id);" +ORDER BY (__name__, __series_id__);" curl "localhost:8123" -d "DROP TABLE IF EXISTS dist_test_prom_series ON CLUSTER abc SYNC" curl "localhost:8123" -d "CREATE TABLE dist_test_prom_series ON CLUSTER abc AS test_prom_series ENGINE = Distributed(abc, default, test_prom_series);" @@ -43,12 +43,12 @@ timeout 30 ./bin/clickhouse_sinker --local-cfg-file docker/test_prom_metric.hjso schema=`curl "localhost:8123" -d 'DESC test_prom_metric' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` echo "Got test_prom_metric schema => $schema" -[ $schema = "__series_idInt64,timestampDateTimeDoubleDelta,LZ4,value1Nullable(Float64),value2Nullable(Int64),value3Nullable(Bool),valueFloat32ZSTD(15)," ] || exit 1 +[ $schema = "__series_id__Int64,timestampDateTimeDoubleDelta,LZ4,value1Nullable(Float64),value2Nullable(Int64),value3Nullable(Bool),valueFloat32ZSTD(15)," ] || exit 1 schema=`curl "localhost:8123" -d 'DESC test_prom_series' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` echo "Got test_prom_series schema => $schema" -[ $schema = "key_0Nullable(String),key_1Nullable(String),key_2Nullable(String),key_4Nullable(String),key_5Nullable(String),key_6Nullable(String),key_7Nullable(String),key_8Nullable(String),key_9Nullable(String),labelsString,__mgmt_idInt64,__name__String,__series_idInt64," ] || exit 1 +[ $schema = "key_0Nullable(String),key_1Nullable(String),key_2Nullable(String),key_4Nullable(String),key_5Nullable(String),key_6Nullable(String),key_7Nullable(String),key_8Nullable(String),key_9Nullable(String),labelsString,__mgmt_idInt64,__name__String,__series_id__Int64," ] || exit 1 echo "check result 1" count=`curl "localhost:8123" -d 'select count() from dist_test_prom_metric'` diff --git a/output/clickhouse.go b/output/clickhouse.go index fd208bf3..2bb0653b 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -279,11 +279,11 @@ func (c *ClickHouse) initSeriesSchema(conn *pool.Conn) (err error) { c.IdxSerID = -1 return } - // Move column "__series_id" to the last. + // Move column "__series_id__" to the last. var dimSerID *model.ColumnWithType for i := 0; i < len(c.Dims); { dim := c.Dims[i] - if dim.Name == "__series_id" && dim.Type.Type == model.Int64 { + if dim.Name == "__series_id__" && dim.Type.Type == model.Int64 { dimSerID = dim c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) break @@ -292,7 +292,7 @@ func (c *ClickHouse) initSeriesSchema(conn *pool.Conn) (err error) { } } if dimSerID == nil { - err = errors.Newf("Metric table %s.%s shall have column `__series_id UInt64`.", c.dbName, c.TableName) + err = errors.Newf("Metric table %s.%s shall have column `__series_id__ UInt64`.", c.dbName, c.TableName) return } c.IdxSerID = len(c.Dims) @@ -303,8 +303,8 @@ func (c *ClickHouse) initSeriesSchema(conn *pool.Conn) (err error) { c.seriesTbl = c.TableName + "_series" } expSeriesDims := []*model.ColumnWithType{ - {Name: "__series_id", Type: &model.TypeInfo{Type: model.Int64}}, - {Name: "__mgmt_id", Type: &model.TypeInfo{Type: model.Int64}}, + {Name: "__series_id__", Type: &model.TypeInfo{Type: model.Int64}}, + {Name: "__mgmt_id__", Type: &model.TypeInfo{Type: model.Int64}}, {Name: "labels", Type: &model.TypeInfo{Type: model.String}}, } var seriesDims []*model.ColumnWithType @@ -328,7 +328,7 @@ func (c *ClickHouse) initSeriesSchema(conn *pool.Conn) (err error) { } } if badFirst { - err = errors.Newf(`First columns of %s are expect to be "__series_id Int64, __mgmt_id Int64, labels String".`, c.seriesTbl) + err = errors.Newf(`First columns of %s are expect to be "__series_id__ Int64, __mgmt_id__ Int64, labels String".`, c.seriesTbl) return } c.NameKey = "__name__" // prometheus uses internal "__name__" label for metric name diff --git a/task/sinker.go b/task/sinker.go index cc24f349..11eedec8 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -41,11 +41,11 @@ var ( createTableSQL = `CREATE TABLE IF NOT EXISTS %s AS %s.%s ENGINE=Merge('%s', '%s')` dropTableSQL = `DROP TABLE IF EXISTS %s ` countSeriesSQL = `WITH (SELECT max(timestamp) FROM %s) AS m - SELECT count() FROM %s FINAL WHERE __series_id GLOBAL IN ( - SELECT DISTINCT __series_id FROM %s WHERE timestamp >= addSeconds(m, -%d));` + SELECT count() FROM %s FINAL WHERE __series_id__ GLOBAL IN ( + SELECT DISTINCT __series_id__ FROM %s WHERE timestamp >= addSeconds(m, -%d));` loadSeriesSQL = `WITH (SELECT max(timestamp) FROM %s) AS m - SELECT toInt64(__series_id) AS sid, toInt64(__mgmt_id) AS mid FROM %s FINAL WHERE sid GLOBAL IN ( - SELECT DISTINCT toInt64(__series_id) FROM %s WHERE timestamp >= addSeconds(m, -%d) + SELECT toInt64(__series_id__) AS sid, toInt64(__mgmt_id__) AS mid FROM %s FINAL WHERE sid GLOBAL IN ( + SELECT DISTINCT toInt64(__series_id__) FROM %s WHERE timestamp >= addSeconds(m, -%d) ) ORDER BY sid;` ) diff --git a/task/task.go b/task/task.go index 9848ecd4..51f55180 100644 --- a/task/task.go +++ b/task/task.go @@ -219,13 +219,13 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) (r *model.Row) { if service.idxSerID >= 0 { - // If some labels are not Prometheus native, ETL shall calculate and pass "__series_id" and "__mgmt_id". - val := metric.GetInt64("__series_id", false) + // If some labels are not Prometheus native, ETL shall calculate and pass "__series_id__" and "__mgmt_id__". + val := metric.GetInt64("__series_id__", false) seriesID := val.(int64) - val = metric.GetInt64("__mgmt_id", false) + val = metric.GetInt64("__mgmt_id__", false) mgmtID := val.(int64) newSeries := service.clickhouse.AllowWriteSeries(seriesID, mgmtID) - rowcount := service.idxSerID + 1 // including __series_id + rowcount := service.idxSerID + 1 // including __series_id__ if newSeries { rowcount += (service.numDims - service.idxSerID + 3) } @@ -234,10 +234,10 @@ func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) for i := 0; i < service.idxSerID; i++ { row = append(row, model.GetValueByType(metric, service.dims[i])) } - row = append(row, seriesID) // __series_id + row = append(row, seriesID) // __series_id__ if newSeries { var labels []string - row = append(row, mgmtID, nil) // __mgmt_id, labels + row = append(row, mgmtID, nil) // __mgmt_id__, labels for i := service.idxSerID + 3; i < service.numDims; i++ { dim := service.dims[i] val := model.GetValueByType(metric, dim) From 8e756c21b9cf7ac99d72d1b6dab763d5edc5628f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=91=A8=E7=BB=8F=E7=BA=AC?= Date: Mon, 18 Dec 2023 19:08:53 +0800 Subject: [PATCH 378/404] feat: support live probe for k8s --- cmd/clickhouse_sinker/main.go | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index a4524638..e95e321e 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -168,6 +168,26 @@ func main() { } } }) + health.Health.AddLivenessCheck("task", func() error { + var err error + if runner != nil && runner.GetCurrentConfig() != nil { + var stateLags map[string]cm.StateLag + var count int + if stateLags, err = cm.GetTaskStateAndLags(runner.GetCurrentConfig()); err == nil { + for _, value := range stateLags { + if value.State == "Dead" { + count++ + } + } + if count == len(stateLags) { + return fmt.Errorf("All task is Dead.") + } + } else { + return err + } + } + return nil + }) mux.Handle("/metrics", httpMetrics) mux.HandleFunc("/ready", health.Health.ReadyEndpoint) // GET /ready?full=1 mux.HandleFunc("/live", health.Health.LiveEndpoint) // GET /live?full=1 From d41609cde4b9e642f9b2f36c4823647e074142ef Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 10 Dec 2023 17:22:03 +0800 Subject: [PATCH 379/404] Bump golang.org/x/net from 0.9.0 to 0.17.0 (#181) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.9.0 to 0.17.0. - [Commits](https://github.com/golang/net/compare/v0.9.0...v0.17.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Zhichang Yu From 38a43709afb4d8dc8670f2d4e88bfea929cf86e3 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Thu, 18 Jan 2024 17:15:39 +0800 Subject: [PATCH 380/404] fix: get __series_id__ and __mgmt_id__ column name from table --- go.sum | 12 ++++++++-- output/clickhouse.go | 54 ++++++++++++++++++++++++++++++-------------- task/sinker.go | 18 ++++++++++----- task/task.go | 4 ++-- 4 files changed, 61 insertions(+), 27 deletions(-) diff --git a/go.sum b/go.sum index 5d664e49..0f957bae 100644 --- a/go.sum +++ b/go.sum @@ -292,8 +292,16 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= -golang.org/x/exp v0.0.0-20230810033253-352e893a4cad h1:g0bG7Z4uG+OgH2QDODnjp6ggkk1bJDsINcuWmJN1iJU= -golang.org/x/exp v0.0.0-20230810033253-352e893a4cad/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= +golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20231226003508-02704c960a9b h1:kLiC65FbiHWFAOu+lxwNPujcsl8VYyTYYEZnsOO1WK4= +golang.org/x/exp v0.0.0-20231226003508-02704c960a9b/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= +golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= diff --git a/output/clickhouse.go b/output/clickhouse.go index 2bb0653b..f459c0d7 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -71,6 +71,8 @@ type ClickHouse struct { distMetricTbls []string distSeriesTbls []string + DimSerID string + DimMgmtID string seriesQuota *model.SeriesQuota @@ -274,16 +276,44 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, sc *pool.ShardConn) { } } +func (c *ClickHouse) getSeriesDims(dims []*model.ColumnWithType, conn *pool.Conn) { + for _, dim := range dims { + if strings.Contains(dim.Name, "series_id") { + c.DimSerID = dim.Name + } + if strings.Contains(dim.Name, "mgmt_id") { + c.DimMgmtID = dim.Name + } + } +} + func (c *ClickHouse) initSeriesSchema(conn *pool.Conn) (err error) { if !c.taskCfg.PrometheusSchema { c.IdxSerID = -1 return } + + // Add string columns from series table + if c.seriesTbl == "" { + c.seriesTbl = c.TableName + "_series" + } + + var seriesDims []*model.ColumnWithType + if seriesDims, err = getDims(c.dbName, c.seriesTbl, nil, c.taskCfg.Parser, conn); err != nil { + if errors.Is(err, ErrTblNotExist) { + err = errors.Wrapf(err, "Please create series table for %s.%s", c.dbName, c.TableName) + return + } + return + } + + c.getSeriesDims(seriesDims, conn) + // Move column "__series_id__" to the last. var dimSerID *model.ColumnWithType for i := 0; i < len(c.Dims); { dim := c.Dims[i] - if dim.Name == "__series_id__" && dim.Type.Type == model.Int64 { + if dim.Name == c.DimSerID && dim.Type.Type == model.Int64 { dimSerID = dim c.Dims = append(c.Dims[:i], c.Dims[i+1:]...) break @@ -292,29 +322,18 @@ func (c *ClickHouse) initSeriesSchema(conn *pool.Conn) (err error) { } } if dimSerID == nil { - err = errors.Newf("Metric table %s.%s shall have column `__series_id__ UInt64`.", c.dbName, c.TableName) + err = errors.Newf("Metric table %s.%s shall have column `%s UInt64`.", c.dbName, c.TableName, c.DimSerID) return } c.IdxSerID = len(c.Dims) c.Dims = append(c.Dims, dimSerID) - // Add string columns from series table - if c.seriesTbl == "" { - c.seriesTbl = c.TableName + "_series" - } expSeriesDims := []*model.ColumnWithType{ - {Name: "__series_id__", Type: &model.TypeInfo{Type: model.Int64}}, - {Name: "__mgmt_id__", Type: &model.TypeInfo{Type: model.Int64}}, + {Name: c.DimSerID, Type: &model.TypeInfo{Type: model.Int64}}, + {Name: c.DimMgmtID, Type: &model.TypeInfo{Type: model.Int64}}, {Name: "labels", Type: &model.TypeInfo{Type: model.String}}, } - var seriesDims []*model.ColumnWithType - if seriesDims, err = getDims(c.dbName, c.seriesTbl, nil, c.taskCfg.Parser, conn); err != nil { - if errors.Is(err, ErrTblNotExist) { - err = errors.Wrapf(err, "Please create series table for %s.%s", c.dbName, c.TableName) - return - } - return - } + var badFirst bool if len(seriesDims) < len(expSeriesDims) { badFirst = true @@ -328,7 +347,7 @@ func (c *ClickHouse) initSeriesSchema(conn *pool.Conn) (err error) { } } if badFirst { - err = errors.Newf(`First columns of %s are expect to be "__series_id__ Int64, __mgmt_id__ Int64, labels String".`, c.seriesTbl) + err = errors.Newf(`First columns of %s are expect to be %s Int64, %s Int64, labels String".`, c.seriesTbl, c.DimSerID, c.DimMgmtID) return } c.NameKey = "__name__" // prometheus uses internal "__name__" label for metric name @@ -423,6 +442,7 @@ func (c *ClickHouse) initSchema() (err error) { }) } } + if err = c.initSeriesSchema(conn); err != nil { return } diff --git a/task/sinker.go b/task/sinker.go index 11eedec8..578c5d34 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -41,11 +41,11 @@ var ( createTableSQL = `CREATE TABLE IF NOT EXISTS %s AS %s.%s ENGINE=Merge('%s', '%s')` dropTableSQL = `DROP TABLE IF EXISTS %s ` countSeriesSQL = `WITH (SELECT max(timestamp) FROM %s) AS m - SELECT count() FROM %s FINAL WHERE __series_id__ GLOBAL IN ( - SELECT DISTINCT __series_id__ FROM %s WHERE timestamp >= addSeconds(m, -%d));` + SELECT count() FROM %s FINAL WHERE %s GLOBAL IN ( + SELECT DISTINCT %s FROM %s WHERE timestamp >= addSeconds(m, -%d));` loadSeriesSQL = `WITH (SELECT max(timestamp) FROM %s) AS m - SELECT toInt64(__series_id__) AS sid, toInt64(__mgmt_id__) AS mid FROM %s FINAL WHERE sid GLOBAL IN ( - SELECT DISTINCT toInt64(__series_id__) FROM %s WHERE timestamp >= addSeconds(m, -%d) + SELECT toInt64(%s) AS sid, toInt64(%s) AS mid FROM %s FINAL WHERE sid GLOBAL IN ( + SELECT DISTINCT toInt64(%s) FROM %s WHERE timestamp >= addSeconds(m, -%d) ) ORDER BY sid;` ) @@ -608,8 +608,14 @@ func loadBmSeries(conn *pool.Conn, sqKey string, tasks []*Service, activeSeriesR // merge all metric tables to get the latest timestamp // old bmseries record won't be loaded into memory to avoid OOM var reg string + var dimSerID, dimMgmtID string + + /* FIXME: We can't assume that the series_id and mgmt_id of all tasks are the same, + because some tasks may be old and some are newly created + */ for _, svc := range tasks { r := svc.clickhouse.GetMetricTable() + dimSerID, dimMgmtID = svc.clickhouse.DimSerID, svc.clickhouse.DimMgmtID if r != "" { reg += ("^" + r + "$|") } @@ -623,14 +629,14 @@ func loadBmSeries(conn *pool.Conn, sqKey string, tasks []*Service, activeSeriesR } var count uint64 - query = fmt.Sprintf(countSeriesSQL, mergetable, sqKey, mergetable, activeSeriesRange) + query = fmt.Sprintf(countSeriesSQL, mergetable, sqKey, dimSerID, dimSerID, mergetable, activeSeriesRange) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) if err = conn.QueryRow(query).Scan(&count); err != nil { return } seriesMap := make(map[int64]int64, count) - query = fmt.Sprintf(loadSeriesSQL, mergetable, sqKey, mergetable, activeSeriesRange) + query = fmt.Sprintf(loadSeriesSQL, mergetable, dimSerID, dimMgmtID, sqKey, dimSerID, mergetable, activeSeriesRange) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", tasks[0].taskCfg.Name)) rs, err := conn.Query(query) if err != nil { diff --git a/task/task.go b/task/task.go index 51f55180..252af1a5 100644 --- a/task/task.go +++ b/task/task.go @@ -220,9 +220,9 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { func (service *Service) metric2Row(metric model.Metric, msg *model.InputMessage) (r *model.Row) { if service.idxSerID >= 0 { // If some labels are not Prometheus native, ETL shall calculate and pass "__series_id__" and "__mgmt_id__". - val := metric.GetInt64("__series_id__", false) + val := metric.GetInt64(service.clickhouse.DimSerID, false) seriesID := val.(int64) - val = metric.GetInt64("__mgmt_id__", false) + val = metric.GetInt64(service.clickhouse.DimMgmtID, false) mgmtID := val.(int64) newSeries := service.clickhouse.AllowWriteSeries(seriesID, mgmtID) rowcount := service.idxSerID + 1 // including __series_id__ From 09b6a3eae8a668e3199000262589b7f4c79170c8 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Fri, 19 Jan 2024 13:35:26 +0800 Subject: [PATCH 381/404] make systest issue --- docker-compose.yml | 8 +++--- docker/test_auto_schema.hjson | 2 +- docker/test_dynamic_schema.hjson | 2 +- docker/test_fixed_schema.hjson | 2 +- docker/test_prom_metric.hjson | 2 +- go.metrictest.sh | 28 +++++++++---------- go.test.sh | 46 ++++++++++++++++---------------- 7 files changed, 45 insertions(+), 45 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index 6357ab4a..450f4679 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -7,7 +7,7 @@ services: restart: always hostname: zookeeper ports: - - "2181:2181" + - "52181:2181" environment: ALLOW_ANONYMOUS_LOGIN: 1 ZOO_4LW_COMMANDS_WHITELIST: "*" @@ -37,8 +37,8 @@ services: image: clickhouse/clickhouse-server:23.8 restart: always ports: - - "8123:8123" - - "9000:9000" + - "58123:8123" + - "59000:9000" ulimits: nofile: soft: 262144 @@ -55,4 +55,4 @@ services: - PREFER_HOST_MODE=hostname - MODE=standalone ports: - - "8848:8848" + - "58848:8848" diff --git a/docker/test_auto_schema.hjson b/docker/test_auto_schema.hjson index efc3c7c8..2b56b817 100644 --- a/docker/test_auto_schema.hjson +++ b/docker/test_auto_schema.hjson @@ -5,7 +5,7 @@ 127.0.0.1 ] ] - port: 9000 + port: 59000 db: default username: "" password: "" diff --git a/docker/test_dynamic_schema.hjson b/docker/test_dynamic_schema.hjson index 7840d82d..9db101c5 100644 --- a/docker/test_dynamic_schema.hjson +++ b/docker/test_dynamic_schema.hjson @@ -6,7 +6,7 @@ 127.0.0.1 ] ] - port: 9000 + port: 59000 db: default username: "" password: "" diff --git a/docker/test_fixed_schema.hjson b/docker/test_fixed_schema.hjson index aa5f2d6f..2ec7656c 100644 --- a/docker/test_fixed_schema.hjson +++ b/docker/test_fixed_schema.hjson @@ -5,7 +5,7 @@ 127.0.0.1 ] ] - port: 9000 + port: 59000 db: default username: "" password: "" diff --git a/docker/test_prom_metric.hjson b/docker/test_prom_metric.hjson index 6a4cb47e..ede6f87c 100644 --- a/docker/test_prom_metric.hjson +++ b/docker/test_prom_metric.hjson @@ -6,7 +6,7 @@ 127.0.0.1 ] ] - port: 9000 + port: 59000 db: gauge username: "" password: "" diff --git a/go.metrictest.sh b/go.metrictest.sh index c7759c22..0bb04b9f 100755 --- a/go.metrictest.sh +++ b/go.metrictest.sh @@ -1,11 +1,11 @@ #!/usr/bin/env bash echo "create database" -curl "localhost:8123" -d "CREATE DATABASE IF NOT EXISTS gauge ON CLUSTER abc" +curl "localhost:58123" -d "CREATE DATABASE IF NOT EXISTS gauge ON CLUSTER abc" echo "create metric test tables" -curl "localhost:8123" -d "DROP TABLE IF EXISTS test_prom_metric ON CLUSTER abc SYNC" -curl "localhost:8123" -d "CREATE TABLE test_prom_metric ON CLUSTER abc +curl "localhost:58123" -d "DROP TABLE IF EXISTS test_prom_metric ON CLUSTER abc SYNC" +curl "localhost:58123" -d "CREATE TABLE test_prom_metric ON CLUSTER abc ( __series_id__ Int64, timestamp DateTime CODEC(DoubleDelta, LZ4), @@ -14,11 +14,11 @@ curl "localhost:8123" -d "CREATE TABLE test_prom_metric ON CLUSTER abc PARTITION BY toYYYYMMDD(timestamp) ORDER BY (__series_id__, timestamp);" -curl "localhost:8123" -d "DROP TABLE IF EXISTS dist_test_prom_metric ON CLUSTER abc SYNC" -curl "localhost:8123" -d "CREATE TABLE dist_test_prom_metric ON CLUSTER abc AS test_prom_metric ENGINE = Distributed(abc, default, test_prom_metric);" +curl "localhost:58123" -d "DROP TABLE IF EXISTS dist_test_prom_metric ON CLUSTER abc SYNC" +curl "localhost:58123" -d "CREATE TABLE dist_test_prom_metric ON CLUSTER abc AS test_prom_metric ENGINE = Distributed(abc, default, test_prom_metric);" -curl "localhost:8123" -d "DROP TABLE IF EXISTS test_prom_series ON CLUSTER abc SYNC" -curl "localhost:8123" -d "CREATE TABLE test_prom_series ON CLUSTER abc +curl "localhost:58123" -d "DROP TABLE IF EXISTS test_prom_series ON CLUSTER abc SYNC" +curl "localhost:58123" -d "CREATE TABLE test_prom_series ON CLUSTER abc ( __series_id__ Int64, __mgmt_id__ Int64, @@ -27,8 +27,8 @@ curl "localhost:8123" -d "CREATE TABLE test_prom_series ON CLUSTER abc ) ENGINE=ReplicatedReplacingMergeTree() ORDER BY (__name__, __series_id__);" -curl "localhost:8123" -d "DROP TABLE IF EXISTS dist_test_prom_series ON CLUSTER abc SYNC" -curl "localhost:8123" -d "CREATE TABLE dist_test_prom_series ON CLUSTER abc AS test_prom_series ENGINE = Distributed(abc, default, test_prom_series);" +curl "localhost:58123" -d "DROP TABLE IF EXISTS dist_test_prom_series ON CLUSTER abc SYNC" +curl "localhost:58123" -d "CREATE TABLE dist_test_prom_series ON CLUSTER abc AS test_prom_series ENGINE = Distributed(abc, default, test_prom_series);" echo "send messages to kafka" echo "cat /tmp/test_prom_metric.data | kafka-console-producer --topic test_metric_topic --broker-list localhost:9092" > send.sh @@ -41,20 +41,20 @@ sudo docker exec kafka sh /tmp/send.sh echo "start clickhouse_sinker to consume" timeout 30 ./bin/clickhouse_sinker --local-cfg-file docker/test_prom_metric.hjson -schema=`curl "localhost:8123" -d 'DESC test_prom_metric' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` +schema=`curl "localhost:58123" -d 'DESC test_prom_metric' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` echo "Got test_prom_metric schema => $schema" [ $schema = "__series_id__Int64,timestampDateTimeDoubleDelta,LZ4,value1Nullable(Float64),value2Nullable(Int64),value3Nullable(Bool),valueFloat32ZSTD(15)," ] || exit 1 -schema=`curl "localhost:8123" -d 'DESC test_prom_series' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` +schema=`curl "localhost:58123" -d 'DESC test_prom_series' 2>/dev/null | sort | tr -d '\t' | tr -d ' '| tr '\n' ','` echo "Got test_prom_series schema => $schema" -[ $schema = "key_0Nullable(String),key_1Nullable(String),key_2Nullable(String),key_4Nullable(String),key_5Nullable(String),key_6Nullable(String),key_7Nullable(String),key_8Nullable(String),key_9Nullable(String),labelsString,__mgmt_idInt64,__name__String,__series_id__Int64," ] || exit 1 +[ $schema = "key_0Nullable(String),key_1Nullable(String),key_2Nullable(String),key_4Nullable(String),key_5Nullable(String),key_6Nullable(String),key_7Nullable(String),key_8Nullable(String),key_9Nullable(String),labelsString,__mgmt_id__Int64,__name__String,__series_id__Int64," ] || exit 1 echo "check result 1" -count=`curl "localhost:8123" -d 'select count() from dist_test_prom_metric'` +count=`curl "localhost:58123" -d 'select count() from dist_test_prom_metric'` echo "Got test_prom_metric count => $count" [ $count -le 10000 ] || exit 1 -count=`curl "localhost:8123" -d 'select count() from dist_test_prom_series'` +count=`curl "localhost:58123" -d 'select count() from dist_test_prom_series'` echo "Got test_prom_series count => $count" [ $count -eq 1000 ] || exit 1 \ No newline at end of file diff --git a/go.test.sh b/go.test.sh index 98a4ba98..7f60605a 100755 --- a/go.test.sh +++ b/go.test.sh @@ -1,8 +1,8 @@ #!/usr/bin/env bash echo "create tables" -curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_fixed_schema' -curl "localhost:8123" -d 'CREATE TABLE test_fixed_schema +curl "localhost:58123" -d 'DROP TABLE IF EXISTS test_fixed_schema' +curl "localhost:58123" -d 'CREATE TABLE test_fixed_schema ( time DateTime, name String, @@ -13,13 +13,13 @@ ENGINE = MergeTree PARTITION BY toYYYYMMDD(time) ORDER BY (time, name)' -curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_auto_schema' -curl "localhost:8123" -d 'CREATE TABLE test_auto_schema AS test_fixed_schema' +curl "localhost:58123" -d 'DROP TABLE IF EXISTS test_auto_schema' +curl "localhost:58123" -d 'CREATE TABLE test_auto_schema AS test_fixed_schema' -curl "localhost:8123" -d 'DROP TABLE IF EXISTS test_dynamic_schema' -curl "localhost:8123" -d 'CREATE TABLE test_dynamic_schema AS test_fixed_schema' +curl "localhost:58123" -d 'DROP TABLE IF EXISTS test_dynamic_schema' +curl "localhost:58123" -d 'CREATE TABLE test_dynamic_schema AS test_fixed_schema' -counts=`curl "localhost:8123" -d 'SELECT count() FROM test_fixed_schema UNION ALL SELECT count() FROM test_auto_schema UNION ALL SELECT count() FROM test_dynamic_schema' 2>/dev/null | tr '\n' ','` +counts=`curl "localhost:58123" -d 'SELECT count() FROM test_fixed_schema UNION ALL SELECT count() FROM test_auto_schema UNION ALL SELECT count() FROM test_dynamic_schema' 2>/dev/null | tr '\n' ','` echo "Got initial row counts => $counts" [ $counts = "0,0,0," ] || exit 1 @@ -54,50 +54,50 @@ timeout 30 ./bin/clickhouse_sinker --local-cfg-file docker/test_auto_schema.hjso timeout 60 ./bin/clickhouse_sinker --local-cfg-file docker/test_dynamic_schema.hjson echo "check result 1" -count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` +count=`curl "localhost:58123" -d 'select count() from test_fixed_schema'` echo "Got test_fixed_schema count => $count" [ $count -eq 100000 ] || exit 1 -count=`curl "localhost:8123" -d 'select count() from test_auto_schema'` +count=`curl "localhost:58123" -d 'select count() from test_auto_schema'` echo "Got test_auto_schema count => $count" [ $count -eq 100000 ] || exit 1 -schema=`curl "localhost:8123" -d 'DESC test_dynamic_schema' 2>/dev/null | grep newkey | sort | tr -d '\t' | tr '\n' ','` +schema=`curl "localhost:58123" -d 'DESC test_dynamic_schema' 2>/dev/null | grep newkey | sort | tr -d '\t' | tr '\n' ','` echo "Got test_dynamic_schema schema => $schema" [ $schema = "newkey00Nullable(Bool),newkey01Nullable(Int64),newkey02Nullable(Float64),newkey03Nullable(String),newkey04Nullable(DateTime64(3))," ] || exit 1 -count=`curl "localhost:8123" -d 'SELECT count() FROM test_dynamic_schema'` +count=`curl "localhost:58123" -d 'SELECT count() FROM test_dynamic_schema'` echo "Got test_dynamic_schema count => $count" [ $count -eq 100000 ] || exit 1 echo "truncate tables" -curl "localhost:8123" -d 'TRUNCATE TABLE test_fixed_schema' -curl "localhost:8123" -d 'TRUNCATE TABLE test_auto_schema' -curl "localhost:8123" -d 'TRUNCATE TABLE test_dynamic_schema' +curl "localhost:58123" -d 'TRUNCATE TABLE test_fixed_schema' +curl "localhost:58123" -d 'TRUNCATE TABLE test_auto_schema' +curl "localhost:58123" -d 'TRUNCATE TABLE test_dynamic_schema' echo "publish clickhouse_sinker config" -./bin/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema --local-cfg-file docker/test_fixed_schema.hjson -./bin/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema --local-cfg-file docker/test_auto_schema.hjson -./bin/nacos_publish_config --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.hjson +./bin/nacos_publish_config --nacos-addr 127.0.0.1:58848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema --local-cfg-file docker/test_fixed_schema.hjson +./bin/nacos_publish_config --nacos-addr 127.0.0.1:58848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema --local-cfg-file docker/test_auto_schema.hjson +./bin/nacos_publish_config --nacos-addr 127.0.0.1:58848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema --local-cfg-file docker/test_dynamic_schema.hjson echo "start clickhouse_sinker to consume" sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_fixed_schema --all-topics --to-earliest -timeout 30 ./bin/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema +timeout 30 ./bin/clickhouse_sinker --nacos-addr 127.0.0.1:58848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_fixed_schema sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_auto_schema --all-topics --to-earliest -timeout 30 ./bin/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema +timeout 30 ./bin/clickhouse_sinker --nacos-addr 127.0.0.1:58848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_auto_schema sudo docker exec kafka kafka-consumer-groups --bootstrap-server localhost:9093 --execute --reset-offsets --group test_dynamic_schema --all-topics --to-earliest -timeout 30 ./bin/clickhouse_sinker --nacos-addr 127.0.0.1:8848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema +timeout 30 ./bin/clickhouse_sinker --nacos-addr 127.0.0.1:58848 --nacos-username nacos --nacos-password nacos --nacos-dataid test_dynamic_schema echo "check result 2" -count=`curl "localhost:8123" -d 'select count() from test_fixed_schema'` +count=`curl "localhost:58123" -d 'select count() from test_fixed_schema'` echo "Got test_fixed_schema count => $count" [ $count -eq 100000 ] || exit 1 -count=`curl "localhost:8123" -d 'select count() from test_auto_schema'` +count=`curl "localhost:58123" -d 'select count() from test_auto_schema'` echo "Got test_auto_schema count => $count" [ $count -eq 100000 ] || exit 1 -count=`curl "localhost:8123" -d 'SELECT count() FROM test_dynamic_schema'` +count=`curl "localhost:58123" -d 'SELECT count() FROM test_dynamic_schema'` echo "Got test_dynamic_schema count => $count" [ $count -eq 100000 ] || exit 1 From 4d569a4055da5540ebb2a897a56b9ba61f3c219a Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Tue, 23 Jan 2024 10:16:44 +0800 Subject: [PATCH 382/404] chore: docker image package --- Dockerfile_goreleaser | 3 ++- Makefile | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/Dockerfile_goreleaser b/Dockerfile_goreleaser index d804a1b9..b66d70b4 100644 --- a/Dockerfile_goreleaser +++ b/Dockerfile_goreleaser @@ -1,5 +1,6 @@ FROM alpine:3.19 -RUN apk --no-cache add ca-certificates tzdata && \ +RUN sed -i 's/dl-cdn.alpinelinux.org/mirrors.aliyun.com/g' /etc/apk/repositories && \ + apk --no-cache add ca-certificates tzdata && \ echo "UTC" > /etc/timezone ADD dist/clickhouse_sinker_linux_amd64_v1/clickhouse_sinker /usr/local/bin/clickhouse_sinker ADD dist/nacos_publish_config_linux_amd64_v1/nacos_publish_config /usr/local/bin/nacos_publish_config diff --git a/Makefile b/Makefile index 27b7b1fb..4f1c629e 100644 --- a/Makefile +++ b/Makefile @@ -4,8 +4,8 @@ SINKER_LDFLAGS += -X "main.date=$(shell date --iso-8601=s)" SINKER_LDFLAGS += -X "main.commit=$(shell git rev-parse HEAD)" SINKER_LDFLAGS += -X "main.builtBy=$(shell echo `whoami`@`hostname`)" DEFAULT_CFG_PATH = /etc/clickhouse_sinker.hjson -IMG_TAGGED = hub.eoitek.net/storage/clickhouse_sinker:${VERSION} -IMG_LATEST = hub.eoitek.net/storage/clickhouse_sinker:latest +IMG_TAGGED = hub.eoitek.net/aimeter/clickhouse_sinker:${VERSION} +IMG_LATEST = hub.eoitek.net/aimeter/clickhouse_sinker:latest export GOPROXY=https://goproxy.cn,direct GO := CGO_ENABLED=0 go From 5e5176694e0694cdb57f92a60dbdf1feb0b068c7 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Wed, 28 Feb 2024 19:07:32 +0800 Subject: [PATCH 383/404] more debug logs and fixed bufLength incorrect --- input/kafka_franz.go | 1 + pool/ck_cli.go | 7 +++++-- task/consumer.go | 9 ++++++--- task/sharding.go | 1 + 4 files changed, 13 insertions(+), 5 deletions(-) diff --git a/input/kafka_franz.go b/input/kafka_franz.go index db2ef0a9..3527fcd4 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -175,6 +175,7 @@ func (k *KafkaFranz) Run() { defer k.wgRun.Done() LOOP: for { + util.Logger.Debug("Records fetching", zap.String("consumer group", k.grpConfig.Name), zap.Int("BufferSize", k.grpConfig.BufferSize)) fetches := k.cl.PollRecords(k.ctx, k.grpConfig.BufferSize) err := fetches.Err() if fetches == nil || fetches.IsClientClosed() || errors.Is(err, context.Canceled) { diff --git a/pool/ck_cli.go b/pool/ck_cli.go index d2d98670..651c6253 100644 --- a/pool/ck_cli.go +++ b/pool/ck_cli.go @@ -230,11 +230,14 @@ func (c *Conn) write_v2(prepareSQL string, rows model.Rows, idxBegin, idxEnd int } func (c *Conn) Write(prepareSQL string, rows model.Rows, idxBegin, idxEnd int) (numBad int, err error) { + util.Logger.Debug("start write to ck", zap.Int("begin", idxBegin), zap.Int("end", idxEnd)) if c.protocol == clickhouse.HTTP { - return c.write_v1(prepareSQL, rows, idxBegin, idxEnd) + numBad, err = c.write_v1(prepareSQL, rows, idxBegin, idxEnd) } else { - return c.write_v2(prepareSQL, rows, idxBegin, idxEnd) + numBad, err = c.write_v2(prepareSQL, rows, idxBegin, idxEnd) } + util.Logger.Debug("loop write completed", zap.Int("numbad", numBad)) + return numBad, err } func (c *Conn) Close() error { diff --git a/task/consumer.go b/task/consumer.go index fa63f770..debb387c 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -150,7 +150,7 @@ func (c *Consumer) processFetch() { c.processWg.Add(1) defer c.processWg.Done() recMap := make(model.RecordMap) - var bufLength int + var bufLength int64 flushFn := func() { if len(recMap) == 0 { @@ -186,6 +186,7 @@ func (c *Consumer) processFetch() { continue } + util.Logger.Debug("fetch records from fetchsCh, begin to sink to clickhouse") fetch := fetches.Records() items, done := int64(len(fetch)), int64(-1) var concurrency int @@ -225,7 +226,8 @@ func (c *Consumer) processFetch() { c.tasks.Range(func(key, value any) bool { tsk := value.(*Service) if (tablename != "" && tsk.clickhouse.TableName == tablename) || tsk.taskCfg.Topic == rec.Topic { - bufLength++ + //bufLength++ + atomic.AddInt64(&bufLength, 1) if e := tsk.Put(msg, flushFn); e != nil { atomic.StoreInt64(&done, items) err = e @@ -272,10 +274,11 @@ func (c *Consumer) processFetch() { } } - if bufLength > bufThreshold { + if bufLength > int64(bufThreshold) { flushFn() ticker.Reset(time.Duration(c.grpConfig.FlushInterval) * time.Second) } + util.Logger.Debug("finish flushing", zap.Int64("bufLength", bufLength)) case <-ticker.C: flushFn() case <-c.ctx.Done(): diff --git a/task/sharding.go b/task/sharding.go index b89fef4d..f0cb8f7d 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -161,6 +161,7 @@ func (sh *Sharder) Flush(c context.Context, wg *sync.WaitGroup, rmap map[int32]* return default: var msgCnt int + util.Logger.Debug("flush records to ck") taskCfg := sh.service.taskCfg batchId, _ := nanoid.New() for i, rows := range sh.msgBuf { From c9c364d2904dfefd608015952ba0706c45e8ad11 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Thu, 29 Feb 2024 14:20:23 +0800 Subject: [PATCH 384/404] perf: add kafka properties --- config/config.go | 29 ++++++++++++++++++++++++++--- docs/configuration/config.md | 11 +++++++++++ input/kafka_franz.go | 7 ++++--- 3 files changed, 41 insertions(+), 6 deletions(-) diff --git a/config/config.go b/config/config.go index de79af87..fba4000a 100644 --- a/config/config.go +++ b/config/config.go @@ -46,7 +46,13 @@ type Config struct { // KafkaConfig configuration parameters type KafkaConfig struct { - Brokers string + Brokers string + Properties struct { + HeartbeatInterval int `json:"heartbeat.interval.ms"` + SessionTimeout int `json:"session.timeout.ms"` + RebalanceTimeout int `json:"rebalance.timeout.ms"` + RequestTimeoutOverhead int `json:"request.timeout.ms"` + } ResetSaslRealm bool Security map[string]string TLS struct { @@ -187,8 +193,12 @@ const ( defaultLogLevel = "info" defaultKerberosConfigPath = "/etc/krb5.conf" defaultMaxOpenConns = 1 - defaultReloadSeriesMapInterval = 3600 // 1 hour - defaultActiveSeriesRange = 86400 // 1 day + defaultReloadSeriesMapInterval = 3600 // 1 hour + defaultActiveSeriesRange = 86400 // 1 day + defaultHeartbeatInterval = 3000 // 3 s + defaultSessionTimeout = 120000 // 2 min + defaultRebalanceTimeout = 120000 // 2 min + defaultRequestTimeoutOverhead = 60000 // 1 min ) func ParseLocalCfgFile(cfgPath string) (cfg *Config, err error) { @@ -259,6 +269,19 @@ func (cfg *Config) Normallize(constructGroup bool, httpAddr string, cred util.Cr os.Setenv("DOMAIN_REALM", net.JoinHostPort("hadoop."+strings.ToLower(cfg.Kafka.Sasl.GSSAPI.Realm), port)) } } + if cfg.Kafka.Properties.HeartbeatInterval == 0 { + cfg.Kafka.Properties.HeartbeatInterval = defaultHeartbeatInterval + } + if cfg.Kafka.Properties.RebalanceTimeout == 0 { + cfg.Kafka.Properties.RebalanceTimeout = defaultRebalanceTimeout + } + if cfg.Kafka.Properties.RequestTimeoutOverhead == 0 { + cfg.Kafka.Properties.RequestTimeoutOverhead = defaultRequestTimeoutOverhead + } + if cfg.Kafka.Properties.SessionTimeout == 0 { + cfg.Kafka.Properties.SessionTimeout = defaultSessionTimeout + } + if cfg.Clickhouse.RetryTimes < 0 { cfg.Clickhouse.RetryTimes = 0 } diff --git a/docs/configuration/config.md b/docs/configuration/config.md index 3a57e065..b624cc71 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -44,6 +44,17 @@ // Kafka config "kafka": { "brokers": "127.0.0.1:9093", + + "properties":{ + // This corresponds to Kafka's heartbeat.interval.ms. + "heartbeat.interval.ms": 3000, + // This option corresponds to Kafka's session.timeout.ms setting and must be within the broker's group.min.session.timeout.ms and group.max.session.timeout.ms. + "session.timeout.ms": 120000, + // This corresponds to Kafka's rebalance.timeout.ms. + "rebalance.timeout.ms": 120000, + // This option is roughly equivalent to request.timeout.ms, but grants additional time to requests that have timeout fields. + "request.timeout.ms": 60000 + } // jave client style security authentication "security":{ diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 3527fcd4..d765f9b8 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -90,9 +90,10 @@ func (k *KafkaFranz) Init(cfg *config.Config, gCfg *config.GroupConfig, f chan * kgo.FetchMaxBytes(maxPartBytes), kgo.FetchMaxPartitionBytes(maxPartBytes), kgo.OnPartitionsRevoked(k.onPartitionRevoked), - kgo.RebalanceTimeout(time.Minute*2), - kgo.SessionTimeout(time.Minute*2), - kgo.RequestTimeoutOverhead(time.Minute*1), + kgo.RebalanceTimeout(time.Millisecond*time.Duration(cfg.Kafka.Properties.RebalanceTimeout)), + kgo.SessionTimeout(time.Millisecond*time.Duration(cfg.Kafka.Properties.SessionTimeout)), + kgo.HeartbeatInterval(time.Millisecond*time.Duration(cfg.Kafka.Properties.HeartbeatInterval)), + kgo.RequestTimeoutOverhead(time.Millisecond*time.Duration(cfg.Kafka.Properties.RequestTimeoutOverhead)), ) if !k.grpConfig.Earliest { opts = append(opts, kgo.ConsumeResetOffset(kgo.NewOffset().AtEnd())) From e1c84249a7c110d829090faadf0ebf6ee35a05e4 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Thu, 29 Feb 2024 17:38:12 +0800 Subject: [PATCH 385/404] more debug message --- output/clickhouse.go | 6 ++++++ task/consumer.go | 7 ++++--- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index f459c0d7..3080153f 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -141,6 +141,7 @@ func (c *ClickHouse) Send(batch *model.Batch) { c.mux.Unlock() statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Dec() }); err != nil { + batch.Wg.Done() return } @@ -252,6 +253,11 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( func (c *ClickHouse) loopWrite(batch *model.Batch, sc *pool.ShardConn) { var retrycount int var dbVer int + + util.Logger.Debug("------- [write start] loopwrite to clickhouse", zap.Int("realsize", batch.RealSize)) + defer func() { + util.Logger.Debug("------ [write end] loopwrite completed") + }() times := c.cfg.Clickhouse.RetryTimes if times <= 0 { times = 0 diff --git a/task/consumer.go b/task/consumer.go index debb387c..23ef3cf7 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -156,6 +156,9 @@ func (c *Consumer) processFetch() { if len(recMap) == 0 { return } + if bufLength > 0 { + util.Logger.Debug("###[process end] flushFn invoked", zap.Int64("bufLength", bufLength)) + } var wg sync.WaitGroup c.tasks.Range(func(key, value any) bool { // flush to shard, ck @@ -185,8 +188,7 @@ func (c *Consumer) processFetch() { if c.state.Load() == util.StateStopped { continue } - - util.Logger.Debug("fetch records from fetchsCh, begin to sink to clickhouse") + util.Logger.Debug("### [process start]fetch records from fetchsCh, begin to sink to clickhouse") fetch := fetches.Records() items, done := int64(len(fetch)), int64(-1) var concurrency int @@ -278,7 +280,6 @@ func (c *Consumer) processFetch() { flushFn() ticker.Reset(time.Duration(c.grpConfig.FlushInterval) * time.Second) } - util.Logger.Debug("finish flushing", zap.Int64("bufLength", bufLength)) case <-ticker.C: flushFn() case <-c.ctx.Done(): From 98b0a66c238bc6dc6d3f57177dc0cbf9c6f8f83a Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Fri, 1 Mar 2024 14:30:07 +0800 Subject: [PATCH 386/404] add trace log to monitor perfermance --- config/config.go | 1 + input/kafka_franz.go | 21 ++++++++++++++------- output/clickhouse.go | 10 +++++----- task/consumer.go | 39 +++++++++++++++++++++++++++------------ task/sharding.go | 4 ++-- task/sinker.go | 1 + task/task.go | 4 ++-- util/common.go | 30 ++++++++++++++++++++++++++++++ 8 files changed, 82 insertions(+), 28 deletions(-) diff --git a/config/config.go b/config/config.go index fba4000a..d4d92ac5 100644 --- a/config/config.go +++ b/config/config.go @@ -38,6 +38,7 @@ type Config struct { Tasks []*TaskConfig Assignment Assignment LogLevel string + LogTrace bool ReloadSeriesMapInterval int ActiveSeriesRange int diff --git a/input/kafka_franz.go b/input/kafka_franz.go index d765f9b8..8841c683 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -48,6 +48,11 @@ const ( processTimeOut = 10 ) +type Fetches struct { + Fetch *kgo.Fetches + TraceId string +} + // KafkaFranz implements input.Inputer // refers to examples/group_consuming/main.go type KafkaFranz struct { @@ -57,7 +62,7 @@ type KafkaFranz struct { ctx context.Context cancel context.CancelFunc wgRun sync.WaitGroup - fetch chan *kgo.Fetches + fetch chan Fetches cleanupFn func() } @@ -67,7 +72,7 @@ func NewKafkaFranz() *KafkaFranz { } // Init Initialise the kafka instance with configuration -func (k *KafkaFranz) Init(cfg *config.Config, gCfg *config.GroupConfig, f chan *kgo.Fetches, cleanupFn func()) (err error) { +func (k *KafkaFranz) Init(cfg *config.Config, gCfg *config.GroupConfig, f chan Fetches, cleanupFn func()) (err error) { k.cfg = cfg k.grpConfig = gCfg k.ctx, k.cancel = context.WithCancel(context.Background()) @@ -176,7 +181,8 @@ func (k *KafkaFranz) Run() { defer k.wgRun.Done() LOOP: for { - util.Logger.Debug("Records fetching", zap.String("consumer group", k.grpConfig.Name), zap.Int("BufferSize", k.grpConfig.BufferSize)) + traceId := util.GenTraceId() + util.LogTrace(traceId, util.TraceKindFetchStart, zap.String("consumer group", k.grpConfig.Name), zap.Int("buffersize", k.grpConfig.BufferSize)) fetches := k.cl.PollRecords(k.ctx, k.grpConfig.BufferSize) err := fetches.Err() if fetches == nil || fetches.IsClientClosed() || errors.Is(err, context.Canceled) { @@ -186,13 +192,14 @@ LOOP: err = errors.Wrapf(err, "") util.Logger.Info("kgo.Client.PollFetchs() got an error", zap.Error(err)) } - - util.Logger.Debug("Records fetched", zap.String("records", strconv.Itoa(fetches.NumRecords())), zap.String("consumer group", k.grpConfig.Name)) - + util.LogTrace(traceId, util.TraceKindFetchEnd, zap.String("consumer group", k.grpConfig.Name), zap.String("records", strconv.Itoa(fetches.NumRecords()))) // Automatically end the program if it remains inactive for a specific duration of time. t := time.NewTimer(processTimeOut * time.Minute) select { - case k.fetch <- &fetches: + case k.fetch <- Fetches{ + TraceId: traceId, + Fetch: &fetches, + }: t.Stop() case <-k.ctx.Done(): t.Stop() diff --git a/output/clickhouse.go b/output/clickhouse.go index 3080153f..ade17265 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -128,10 +128,10 @@ func (c *ClickHouse) Drain() { } // Send a batch to clickhouse -func (c *ClickHouse) Send(batch *model.Batch) { +func (c *ClickHouse) Send(batch *model.Batch, traceId string) { sc := pool.GetShardConn(batch.BatchIdx) if err := sc.SubmitTask(func() { - c.loopWrite(batch, sc) + c.loopWrite(batch, sc, traceId) batch.Wg.Done() c.mux.Lock() c.numFlying-- @@ -250,13 +250,13 @@ func (c *ClickHouse) write(batch *model.Batch, sc *pool.ShardConn, dbVer *int) ( } // LoopWrite will dead loop to write the records -func (c *ClickHouse) loopWrite(batch *model.Batch, sc *pool.ShardConn) { +func (c *ClickHouse) loopWrite(batch *model.Batch, sc *pool.ShardConn, traceId string) { var retrycount int var dbVer int - util.Logger.Debug("------- [write start] loopwrite to clickhouse", zap.Int("realsize", batch.RealSize)) + util.LogTrace(traceId, util.TraceKindWriteStart, zap.Int("realsize", batch.RealSize)) defer func() { - util.Logger.Debug("------ [write end] loopwrite completed") + util.LogTrace(traceId, util.TraceKindWriteEnd, zap.Int("success", batch.RealSize)) }() times := c.cfg.Clickhouse.RetryTimes if times <= 0 { diff --git a/task/consumer.go b/task/consumer.go index 23ef3cf7..de798dfc 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -26,7 +26,6 @@ import ( "github.com/housepower/clickhouse_sinker/input" "github.com/housepower/clickhouse_sinker/model" "github.com/housepower/clickhouse_sinker/util" - "github.com/twmb/franz-go/pkg/kgo" "go.uber.org/zap" _ "github.com/ClickHouse/clickhouse-go/v2" @@ -44,7 +43,7 @@ type Consumer struct { inputer *input.KafkaFranz tasks sync.Map grpConfig *config.GroupConfig - fetchesCh chan *kgo.Fetches + fetchesCh chan input.Fetches processWg sync.WaitGroup ctx context.Context cancel context.CancelFunc @@ -67,7 +66,7 @@ func newConsumer(s *Sinker, gCfg *config.GroupConfig) *Consumer { numFlying: 0, errCommit: false, grpConfig: gCfg, - fetchesCh: make(chan *kgo.Fetches), + fetchesCh: make(chan input.Fetches), } c.state.Store(util.StateStopped) c.commitDone = sync.NewCond(&c.mux) @@ -152,18 +151,18 @@ func (c *Consumer) processFetch() { recMap := make(model.RecordMap) var bufLength int64 - flushFn := func() { + flushFn := func(traceId, with string) { if len(recMap) == 0 { return } if bufLength > 0 { - util.Logger.Debug("###[process end] flushFn invoked", zap.Int64("bufLength", bufLength)) + util.LogTrace(traceId, util.TraceKindProcessEnd, zap.String("with", with), zap.Int64("bufLength", bufLength)) } var wg sync.WaitGroup c.tasks.Range(func(key, value any) bool { // flush to shard, ck task := value.(*Service) - task.sharder.Flush(c.ctx, &wg, recMap[task.taskCfg.Topic]) + task.sharder.Flush(c.ctx, &wg, recMap[task.taskCfg.Topic], traceId) return true }) bufLength = 0 @@ -182,14 +181,27 @@ func (c *Consumer) processFetch() { ticker := time.NewTicker(time.Duration(c.grpConfig.FlushInterval) * time.Second) defer ticker.Stop() + traceId := "NO_RECORDS_FETCHED" + wait := false for { select { case fetches := <-c.fetchesCh: if c.state.Load() == util.StateStopped { continue } - util.Logger.Debug("### [process start]fetch records from fetchsCh, begin to sink to clickhouse") - fetch := fetches.Records() + fetch := fetches.Fetch.Records() + if wait { + util.LogTrace(fetches.TraceId, + util.TraceKindProcessing, + zap.String("message", "bufThreshold not reached, use old traceId"), + zap.String("old_trace_id", traceId), + zap.Int("records", len(fetch)), + zap.Int("bufThreshold", bufThreshold), + zap.Int64("totalLength", bufLength)) + } else { + traceId = fetches.TraceId + util.LogTrace(traceId, util.TraceKindProcessStart, zap.Int("records", len(fetch))) + } items, done := int64(len(fetch)), int64(-1) var concurrency int if concurrency = int(items/1000) + 1; concurrency > MaxParallelism { @@ -230,7 +242,7 @@ func (c *Consumer) processFetch() { if (tablename != "" && tsk.clickhouse.TableName == tablename) || tsk.taskCfg.Topic == rec.Topic { //bufLength++ atomic.AddInt64(&bufLength, 1) - if e := tsk.Put(msg, flushFn); e != nil { + if e := tsk.Put(msg, traceId, flushFn); e != nil { atomic.StoreInt64(&done, items) err = e return false @@ -246,7 +258,7 @@ func (c *Consumer) processFetch() { // record the latest offset in order // assume the c.state was reset to stopped when facing error, so that further fetch won't get processed if err == nil { - for _, f := range *fetches { + for _, f := range *fetches.Fetch { for i := range f.Topics { ft := &f.Topics[i] if recMap[ft.Topic] == nil { @@ -277,11 +289,14 @@ func (c *Consumer) processFetch() { } if bufLength > int64(bufThreshold) { - flushFn() + flushFn(traceId, "bufLength reached") ticker.Reset(time.Duration(c.grpConfig.FlushInterval) * time.Second) + wait = false + } else { + wait = true } case <-ticker.C: - flushFn() + flushFn(traceId, "ticker.C triggered") case <-c.ctx.Done(): util.Logger.Info("stopped processing loop", zap.String("group", c.grpConfig.Name)) return diff --git a/task/sharding.go b/task/sharding.go index f0cb8f7d..54fc14b0 100644 --- a/task/sharding.go +++ b/task/sharding.go @@ -152,7 +152,7 @@ func (sh *Sharder) PutElement(msgRow *model.MsgRow) { statistics.ShardMsgs.WithLabelValues(sh.service.taskCfg.Name).Inc() } -func (sh *Sharder) Flush(c context.Context, wg *sync.WaitGroup, rmap map[int32]*model.BatchRange) { +func (sh *Sharder) Flush(c context.Context, wg *sync.WaitGroup, rmap map[int32]*model.BatchRange, traceId string) { sh.mux.Lock() defer sh.mux.Unlock() select { @@ -176,7 +176,7 @@ func (sh *Sharder) Flush(c context.Context, wg *sync.WaitGroup, rmap map[int32]* Wg: wg, } batch.Wg.Add(1) - sh.service.clickhouse.Send(batch) + sh.service.clickhouse.Send(batch, traceId) rs := make(model.Rows, 0, realSize) sh.msgBuf[i] = &rs } diff --git a/task/sinker.go b/task/sinker.go index 578c5d34..0682b9b1 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -282,6 +282,7 @@ func (s *Sinker) stopAllTasks() { func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { util.SetLogLevel(newCfg.LogLevel) + util.SetLogTrace(newCfg.LogTrace) if s.curCfg == nil { // The first time invoking of applyConfig err = s.applyFirstConfig(newCfg) diff --git a/task/task.go b/task/task.go index 252af1a5..28fcd946 100644 --- a/task/task.go +++ b/task/task.go @@ -152,7 +152,7 @@ func (service *Service) Init() (err error) { return } -func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { +func (service *Service) Put(msg *model.InputMessage, traceId string, flushFn func(traceId, with string)) error { taskCfg := service.taskCfg statistics.ConsumeMsgsTotal.WithLabelValues(taskCfg.Name).Inc() var err error @@ -192,7 +192,7 @@ func (service *Service) Put(msg *model.InputMessage, flushFn func()) error { // 4) recreate the service util.Logger.Warn("new key detected, consumer is going to restart", zap.String("consumer group", service.taskCfg.ConsumerGroup), zap.Error(err)) go service.consumer.restart() - flushFn() + flushFn(traceId, "foundNewKeys and restart") if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { util.Logger.Fatal("clickhouse.ChangeSchema failed", zap.String("task", taskCfg.Name), zap.Error(err)) } diff --git a/util/common.go b/util/common.go index 616fdd04..902c9289 100644 --- a/util/common.go +++ b/util/common.go @@ -31,6 +31,7 @@ import ( "go.uber.org/zap/zapcore" "gopkg.in/natefinch/lumberjack.v2" + "github.com/google/uuid" "github.com/thanos-io/thanos/pkg/errors" ) @@ -38,6 +39,7 @@ var ( Logger *zap.Logger logAtomLevel zap.AtomicLevel logPaths []string + logTrace bool ) type CmdOptions struct { @@ -253,6 +255,34 @@ func SetLogLevel(newLogLevel string) { } } +func GenTraceId() string { + return uuid.NewString() +} + +const ( + TraceKindFetchStart string = "fetch start" + TraceKindFetchEnd string = "fetch end" + TraceKindProcessStart string = "process start" + TraceKindProcessEnd string = "process end" + TraceKindWriteStart string = "loopwrite start" + TraceKindWriteEnd string = "loopwrite end" + TraceKindProcessing string = "process continue" +) + +func SetLogTrace(enabled bool) { + logTrace = enabled +} + +func LogTrace(traceId string, kind string, fields ...zapcore.Field) { + if logTrace { + allFields := []zapcore.Field{ + zap.String("trace_id", traceId), + zap.String("trace_kind", kind)} + allFields = append(allFields, fields...) + Logger.Info("===trace===", allFields...) + } +} + // set v2 to v1, if v1 didn't bind any value // FIXME: how about v1 bind default value? func TrySetValue(v1, v2 interface{}) bool { From d74573713a6298cad14843fe72a958d73bb06575 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Mon, 4 Mar 2024 15:22:21 +0800 Subject: [PATCH 387/404] only check dist table but not logic table --- output/clickhouse.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index ade17265..e20d7afa 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -328,7 +328,7 @@ func (c *ClickHouse) initSeriesSchema(conn *pool.Conn) (err error) { } } if dimSerID == nil { - err = errors.Newf("Metric table %s.%s shall have column `%s UInt64`.", c.dbName, c.TableName, c.DimSerID) + err = errors.Newf("Metric table %s.%s shall have column `%s Int64`.", c.dbName, c.TableName, c.DimSerID) return } c.IdxSerID = len(c.Dims) @@ -620,8 +620,8 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []DistTblInfo, err erro return } query := fmt.Sprintf(`SELECT name, (extractAllGroups(engine_full, '(Distributed\\(\')(.*)\',\\s+\'(.*)\',\\s+\'(.*)\'(.*)')[1])[2] AS cluster - FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(engine_full, 'Distributed\(\'.*\', \'%s\', \'%s\'.*\)')`, - c.dbName, c.dbName, table) + FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(engine_full, 'Distributed\(\'%s\', \'%s\', \'%s\'.*\)')`, + c.dbName, c.cfg.Clickhouse.Cluster, c.dbName, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows *pool.Rows if rows, err = conn.Query(query); err != nil { From 3121b226f2ed9ad1e58196d4d28334bef4039235 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Mon, 4 Mar 2024 15:23:06 +0800 Subject: [PATCH 388/404] use lz4 compression in clickhouse client --- pool/conn.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pool/conn.go b/pool/conn.go index cad3023e..d58d4b81 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -106,11 +106,17 @@ func (sc *ShardConn) NextGoodReplica(failedVer int) (db *Conn, dbVer int, err er sc.opts.Addr = []string{replica} sc.nextRep = (sc.nextRep + 1) % len(sc.replicas) if sc.protocol == clickhouse.HTTP { + // refers to https://github.com/ClickHouse/clickhouse-go/issues/1150 + // An obscure error in the HTTP protocol when using compression + // disable compression in the HTTP protocol conn.db = clickhouse.OpenDB(&sc.opts) conn.db.SetMaxOpenConns(sc.chCfg.MaxOpenConns) conn.db.SetMaxIdleConns(sc.chCfg.MaxOpenConns) conn.db.SetConnMaxLifetime(time.Minute * 10) } else { + sc.opts.Compression = &clickhouse.Compression{ + Method: clickhouse.CompressionLZ4, + } conn.c, err = clickhouse.Open(&sc.opts) } if err != nil { From d9ab55743e194baed1f07a9e78f0cc0ba98c35f5 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Mon, 4 Mar 2024 17:18:56 +0800 Subject: [PATCH 389/404] feat: add record pool size to avoid using too much memory --- config/config.go | 4 ++++ docs/configuration/config.md | 5 ++++- input/kafka_franz.go | 8 ++++++-- output/clickhouse.go | 2 ++ task/consumer.go | 2 ++ task/sinker.go | 2 ++ util/recordpoolsize.go | 35 +++++++++++++++++++++++++++++++++++ 7 files changed, 55 insertions(+), 3 deletions(-) create mode 100644 util/recordpoolsize.go diff --git a/config/config.go b/config/config.go index d4d92ac5..d0443f75 100644 --- a/config/config.go +++ b/config/config.go @@ -39,6 +39,7 @@ type Config struct { Assignment Assignment LogLevel string LogTrace bool + RecordPoolSize int64 ReloadSeriesMapInterval int ActiveSeriesRange int @@ -340,6 +341,9 @@ func (cfg *Config) Normallize(constructGroup bool, httpAddr string, cred util.Cr } } } + if cfg.RecordPoolSize == 0 { + cfg.RecordPoolSize = MaxBufferSize + } switch strings.ToLower(cfg.LogLevel) { case "debug", "info", "warn", "error", "dpanic", "panic", "fatal": default: diff --git a/docs/configuration/config.md b/docs/configuration/config.md index b624cc71..22c1c735 100644 --- a/docs/configuration/config.md +++ b/docs/configuration/config.md @@ -199,6 +199,9 @@ // utilization, only active series from the last "activeSeriesRange" seconds will be cached, and the map in the cache will be updated every // "reloadSeriesMapInterval" seconds. By default, series from the last 24 hours will be cached, and the cache will be updated every hour. "reloadSeriesMapInterval": 3600, - "activeSeriesRange": 86400 + "activeSeriesRange": 86400, + "logTrace": false, + // It is recommended that recordPoolSize be 3 or 4 times the bufferSize, for the backpressure mechanism, to avoid using too much memory. + "recordPoolSize": 1048576 } ``` diff --git a/input/kafka_franz.go b/input/kafka_franz.go index 8841c683..cc5f6740 100644 --- a/input/kafka_franz.go +++ b/input/kafka_franz.go @@ -19,7 +19,6 @@ import ( "context" "crypto/tls" "fmt" - "strconv" "strings" "sync" "time" @@ -181,6 +180,9 @@ func (k *KafkaFranz) Run() { defer k.wgRun.Done() LOOP: for { + if !util.Rs.Allow() { + continue + } traceId := util.GenTraceId() util.LogTrace(traceId, util.TraceKindFetchStart, zap.String("consumer group", k.grpConfig.Name), zap.Int("buffersize", k.grpConfig.BufferSize)) fetches := k.cl.PollRecords(k.ctx, k.grpConfig.BufferSize) @@ -192,7 +194,9 @@ LOOP: err = errors.Wrapf(err, "") util.Logger.Info("kgo.Client.PollFetchs() got an error", zap.Error(err)) } - util.LogTrace(traceId, util.TraceKindFetchEnd, zap.String("consumer group", k.grpConfig.Name), zap.String("records", strconv.Itoa(fetches.NumRecords()))) + fetchRecords := fetches.NumRecords() + util.Rs.Inc(int64(fetchRecords)) + util.LogTrace(traceId, util.TraceKindFetchEnd, zap.String("consumer group", k.grpConfig.Name), zap.Int64("records", int64(fetchRecords))) // Automatically end the program if it remains inactive for a specific duration of time. t := time.NewTimer(processTimeOut * time.Minute) select { diff --git a/output/clickhouse.go b/output/clickhouse.go index e20d7afa..d246c8d3 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -142,6 +142,7 @@ func (c *ClickHouse) Send(batch *model.Batch, traceId string) { statistics.WritingPoolBacklog.WithLabelValues(c.taskCfg.Name).Dec() }); err != nil { batch.Wg.Done() + util.Rs.Dec(int64(batch.RealSize)) return } @@ -256,6 +257,7 @@ func (c *ClickHouse) loopWrite(batch *model.Batch, sc *pool.ShardConn, traceId s util.LogTrace(traceId, util.TraceKindWriteStart, zap.Int("realsize", batch.RealSize)) defer func() { + util.Rs.Dec(int64(batch.RealSize)) util.LogTrace(traceId, util.TraceKindWriteEnd, zap.Int("success", batch.RealSize)) }() times := c.cfg.Clickhouse.RetryTimes diff --git a/task/consumer.go b/task/consumer.go index de798dfc..259dd117 100644 --- a/task/consumer.go +++ b/task/consumer.go @@ -245,6 +245,8 @@ func (c *Consumer) processFetch() { if e := tsk.Put(msg, traceId, flushFn); e != nil { atomic.StoreInt64(&done, items) err = e + // decrise the error record + util.Rs.Dec(1) return false } } diff --git a/task/sinker.go b/task/sinker.go index 0682b9b1..e2b76db1 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -283,6 +283,8 @@ func (s *Sinker) stopAllTasks() { func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { util.SetLogLevel(newCfg.LogLevel) util.SetLogTrace(newCfg.LogTrace) + util.Rs.SetPoolSize(newCfg.RecordPoolSize) + util.Rs.Reset() if s.curCfg == nil { // The first time invoking of applyConfig err = s.applyFirstConfig(newCfg) diff --git a/util/recordpoolsize.go b/util/recordpoolsize.go new file mode 100644 index 00000000..487b70b5 --- /dev/null +++ b/util/recordpoolsize.go @@ -0,0 +1,35 @@ +package util + +import "sync/atomic" + +type RecordSize struct { + poolSize int64 + realSize int64 +} + +func (rs *RecordSize) SetPoolSize(size int64) { + rs.poolSize = size +} + +func (rs *RecordSize) Inc(size int64) { + atomic.AddInt64(&rs.realSize, size) +} + +func (rs *RecordSize) Reset() { + atomic.StoreInt64(&rs.realSize, 0) +} + +func (rs *RecordSize) Dec(size int64) { + atomic.AddInt64(&rs.realSize, size*(-1)) +} + +func (rs *RecordSize) Get() int64 { + return atomic.LoadInt64(&rs.realSize) +} + +func (rs *RecordSize) Allow() bool { + realSize := atomic.LoadInt64(&rs.realSize) + return realSize < rs.poolSize +} + +var Rs RecordSize From 9f6ff7cf5c329328eaf9026347884e7a98fda3a6 Mon Sep 17 00:00:00 2001 From: YenchangChan <43897067+YenchangChan@users.noreply.github.com> Date: Mon, 11 Mar 2024 17:08:21 +0800 Subject: [PATCH 390/404] bump clickhouse-go to 2.21.0 (#191) --- go.mod | 30 +++++++++++++-------------- go.sum | 64 +++++++++++++++++++++++++++++----------------------------- 2 files changed, 47 insertions(+), 47 deletions(-) diff --git a/go.mod b/go.mod index dcd71947..9c499e20 100644 --- a/go.mod +++ b/go.mod @@ -5,14 +5,14 @@ go 1.21 toolchain go1.21.4 require ( - github.com/ClickHouse/clickhouse-go/v2 v2.17.1 + github.com/ClickHouse/clickhouse-go/v2 v2.21.0 github.com/RoaringBitmap/roaring v1.7.0 github.com/YenchangChan/franz-go/pkg/sasl/kerberos v0.0.0-20231127011105-840a25342a2e github.com/avast/retry-go/v4 v4.5.1 github.com/bytedance/sonic v1.10.2 github.com/cespare/xxhash/v2 v2.2.0 github.com/google/gops v0.3.28 - github.com/google/uuid v1.5.0 + github.com/google/uuid v1.6.0 github.com/hjson/hjson-go/v4 v4.4.0 github.com/jcmturner/gokrb5/v8 v8.4.4 github.com/jinzhu/copier v0.4.0 @@ -21,7 +21,7 @@ require ( github.com/prometheus/client_golang v1.18.0 github.com/prometheus/common v0.45.0 github.com/shopspring/decimal v1.3.1 - github.com/stretchr/testify v1.8.4 + github.com/stretchr/testify v1.9.0 github.com/thanos-io/thanos v0.33.0 github.com/tidwall/gjson v1.17.0 github.com/troian/healthcheck v0.1.4-0.20200127040058-c373fb6a0dc1 @@ -29,16 +29,16 @@ require ( github.com/twmb/franz-go/pkg/kadm v1.10.0 github.com/twmb/franz-go/plugin/kzap v1.1.2 github.com/valyala/fastjson v1.6.4 - go.uber.org/zap v1.26.0 + go.uber.org/zap v1.27.0 golang.org/x/exp v0.0.0-20231226003508-02704c960a9b golang.org/x/time v0.5.0 gopkg.in/natefinch/lumberjack.v2 v2.2.1 ) require ( - github.com/ClickHouse/ch-go v0.61.0 // indirect + github.com/ClickHouse/ch-go v0.61.3 // indirect github.com/aliyun/alibaba-cloud-sdk-go v1.62.648 // indirect - github.com/andybalholm/brotli v1.0.6 // indirect + github.com/andybalholm/brotli v1.1.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.13.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect @@ -55,15 +55,15 @@ require ( github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.17.4 // indirect + github.com/klauspost/compress v1.17.7 // indirect github.com/klauspost/cpuid/v2 v2.2.6 // indirect github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mschoch/smat v0.2.0 // indirect github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b // indirect - github.com/paulmach/orb v0.10.0 // indirect - github.com/pierrec/lz4/v4 v4.1.19 // indirect + github.com/paulmach/orb v0.11.1 // indirect + github.com/pierrec/lz4/v4 v4.1.21 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_model v0.5.0 // indirect @@ -74,14 +74,14 @@ require ( github.com/tidwall/pretty v1.2.1 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect github.com/twmb/franz-go/pkg/kmsg v1.7.0 // indirect - go.opentelemetry.io/otel v1.21.0 // indirect - go.opentelemetry.io/otel/trace v1.21.0 // indirect + go.opentelemetry.io/otel v1.24.0 // indirect + go.opentelemetry.io/otel/trace v1.24.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.6.0 // indirect - golang.org/x/crypto v0.17.0 // indirect - golang.org/x/net v0.19.0 // indirect - golang.org/x/sync v0.5.0 // indirect - golang.org/x/sys v0.15.0 // indirect + golang.org/x/crypto v0.21.0 // indirect + golang.org/x/net v0.22.0 // indirect + golang.org/x/sync v0.6.0 // indirect + golang.org/x/sys v0.18.0 // indirect google.golang.org/protobuf v1.32.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 0f957bae..a3593075 100644 --- a/go.sum +++ b/go.sum @@ -1,10 +1,10 @@ dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/ClickHouse/ch-go v0.61.0 h1:22JYeFJoFNAU/Vod4etAeUEY28cYt7Ixnwqj1+EUfro= -github.com/ClickHouse/ch-go v0.61.0/go.mod h1:POJBl0MxEMS91Zd0uTgDDt05KfXEjf5KIwW6lNhje9Y= -github.com/ClickHouse/clickhouse-go/v2 v2.17.1 h1:ZCmAYWpu75IyEi7+Yrs/uaAjiCGY5wfW5kXo64exkX4= -github.com/ClickHouse/clickhouse-go/v2 v2.17.1/go.mod h1:rkGTvFDTLqLIm0ma+13xmcCfr/08Gvs7KmFt1tgiWHQ= +github.com/ClickHouse/ch-go v0.61.3 h1:MmBwUhXrAOBZK7n/sWBzq6FdIQ01cuF2SaaO8KlDRzI= +github.com/ClickHouse/ch-go v0.61.3/go.mod h1:1PqXjMz/7S1ZUaKvwPA3i35W2bz2mAMFeCi6DIXgGwQ= +github.com/ClickHouse/clickhouse-go/v2 v2.21.0 h1:q013XwDIrppqCQ0hEvyrZY7vDSTFgSgHhxgj/bwCUkA= +github.com/ClickHouse/clickhouse-go/v2 v2.21.0/go.mod h1:3smKzc0CmAGasFTCJ0BefGkUs0NIcPEseDYSx/gj8yA= github.com/DATA-DOG/go-sqlmock v1.3.0 h1:ljjRxlddjfChBJdFKJs5LuCwCWPLaC1UZLwAo3PBBMk= github.com/DATA-DOG/go-sqlmock v1.3.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/HdrHistogram/hdrhistogram-go v1.1.2/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= @@ -20,8 +20,8 @@ github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRF github.com/aliyun/alibaba-cloud-sdk-go v1.61.18/go.mod h1:v8ESoHo4SyHmuB4b1tJqDHxfTGEciD+yhvOU/5s1Rfk= github.com/aliyun/alibaba-cloud-sdk-go v1.62.648 h1:I63hduiA3oSEAtZcHD49b0BDe55urq3LeUowhOZSLms= github.com/aliyun/alibaba-cloud-sdk-go v1.62.648/go.mod h1:CJJYa1ZMxjlN/NbXEwmejEnBkhi0DV+Yb3B2lxf+74o= -github.com/andybalholm/brotli v1.0.6 h1:Yf9fFpf49Zrxb9NlQaluyE92/+X7UVHlhMNJN2sxfOI= -github.com/andybalholm/brotli v1.0.6/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= +github.com/andybalholm/brotli v1.1.0 h1:eLKJA0d02Lf0mVpIDgYnqXcUn0GqVmEFny3VuID1U3M= +github.com/andybalholm/brotli v1.1.0/go.mod h1:sms7XGricyQI9K10gOSf56VKKWS4oLer58Q+mhRPtnY= github.com/avast/retry-go/v4 v4.5.1 h1:AxIx0HGi4VZ3I02jr78j5lZ3M6x1E0Ivxa6b0pUUh7o= github.com/avast/retry-go/v4 v4.5.1/go.mod h1:/sipNsvNB3RRuT5iNcb6h73nw3IBmXJ/H3XrCQYSOpc= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -89,8 +89,8 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/ github.com/google/gops v0.3.28 h1:2Xr57tqKAmQYRAfG12E+yLcoa2Y42UJo2lOrUFL9ark= github.com/google/gops v0.3.28/go.mod h1:6f6+Nl8LcHrzJwi8+p0ii+vmBFSlB4f8cOOkTJ7sk4c= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.5.0 h1:1p67kYwdtXjb0gL0BPiP1Av9wiZPo5A8z2cWkTZ+eyU= -github.com/google/uuid v1.5.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= @@ -129,8 +129,8 @@ github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.17.4 h1:Ej5ixsIri7BrIjBkRZLTo6ghwrEtHFk7ijlczPW4fZ4= -github.com/klauspost/compress v1.17.4/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= +github.com/klauspost/compress v1.17.7 h1:ehO88t2UGzQK66LMdE8tibEd1ErmzZjNEqWkjLAKQQg= +github.com/klauspost/compress v1.17.7/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.6 h1:ndNyv040zDGIDh8thGkXYjnFtiN02M1PVVF+JE/48xc= github.com/klauspost/cpuid/v2 v2.2.6/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= @@ -166,11 +166,11 @@ github.com/nacos-group/nacos-sdk-go v1.1.4/go.mod h1:cBv9wy5iObs7khOqov1ERFQrCuT github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b h1:FfH+VrHHk6Lxt9HdVS0PXzSXFyS2NbZKXv33FYPol0A= github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b/go.mod h1:AC62GU6hc0BrNm+9RK9VSiwa/EUe1bkIeFORAMcHvJU= -github.com/paulmach/orb v0.10.0 h1:guVYVqzxHE/CQ1KpfGO077TR0ATHSNjp4s6XGLn3W9s= -github.com/paulmach/orb v0.10.0/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= +github.com/paulmach/orb v0.11.1 h1:3koVegMC4X/WeiXYz9iswopaTwMem53NzTJuTF20JzU= +github.com/paulmach/orb v0.11.1/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= -github.com/pierrec/lz4/v4 v4.1.19 h1:tYLzDnjDXh9qIxSTKHwXwOYmm9d887Y7Y1ZkyXYHAN4= -github.com/pierrec/lz4/v4 v4.1.19/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pierrec/lz4/v4 v4.1.21 h1:yOVMLb6qSIDP67pl/5F7RepeKYu/VmTyEXvuMI5d9mQ= +github.com/pierrec/lz4/v4 v4.1.21/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -224,8 +224,8 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= -github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/thanos-io/thanos v0.33.0 h1:TYDq9dS3HF7/p3aIGHeImowuQF7CMw0ZODwYoLGhyls= github.com/thanos-io/thanos v0.33.0/go.mod h1:qeDC74QOf5hWzTlvIrLT8WlNGg67nORFON0T2VF4qgg= github.com/tidwall/gjson v1.17.0 h1:/Jocvlh98kcTfpN2+JzGQWQcqrPQwDrVEMApx/M5ZwM= @@ -262,23 +262,23 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.mongodb.org/mongo-driver v1.11.4/go.mod h1:PTSz5yu21bkT/wXpkS7WR5f0ddqw5quethTUn9WM+2g= -go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= -go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= -go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= -go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= +go.opentelemetry.io/otel v1.24.0 h1:0LAOdjNmQeSTzGBzduGe/rU4tZhMwL5rWgtp9Ku5Jfo= +go.opentelemetry.io/otel v1.24.0/go.mod h1:W7b9Ozg4nkF5tWI5zsXkaKKDjdVjpD4oAt9Qi/MArHo= +go.opentelemetry.io/otel/trace v1.24.0 h1:CsKnnL4dUAr/0llH9FKuc698G04IrpWV0MQA/Y1YELI= +go.opentelemetry.io/otel/trace v1.24.0/go.mod h1:HPc3Xr/cOApsBI154IU0OI0HJexz+aw5uPdbs3UCjNU= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= -go.uber.org/goleak v1.2.1 h1:NBol2c7O1ZokfZ0LEU9K6Whx/KnwvepVetCUhtKja4A= -go.uber.org/goleak v1.2.1/go.mod h1:qlT2yGI9QafXHhZZLxlSuNsMw3FFLxBr+tBRlmO1xH4= +go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= +go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= -go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= +go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= +go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/arch v0.6.0 h1:S0JTfE48HbRj80+4tbvZDYsJ3tGv6BUU3XxyZ7CirAc= golang.org/x/arch v0.6.0/go.mod h1:FEVrYAQjsQXMVJ1nsMoVVXPZg6p2JE2mx8psSWTDQys= @@ -290,8 +290,8 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= -golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= +golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -322,8 +322,8 @@ golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.19.0 h1:zTwKpTd2XuCqf8huc7Fo2iSy+4RHPd10s4KzeTnVr1c= -golang.org/x/net v0.19.0/go.mod h1:CfAk/cbD4CthTvqiEl8NpboMuiuOYsAr/7NOjZJtv1U= +golang.org/x/net v0.22.0 h1:9sGLhx7iRIHEiX0oAJ3MRZMUCElJgy7Br1nO+AMN3Tc= +golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -331,8 +331,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.5.0 h1:60k92dhOjHxJkrqnwsfl8KuaHbn/5dl0lUPUklKo3qE= -golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.6.0 h1:5BMeUDZ7vkXGfEr1x9B4bRcTH4lpkTkpdh0T/J+qjbQ= +golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -347,8 +347,8 @@ golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= +golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= From ae35ab463a162a3f77e2f8602174a0d9e72d5e74 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Mar 2024 05:10:58 +0000 Subject: [PATCH 391/404] Bump google.golang.org/protobuf from 1.32.0 to 1.33.0 (#192) Bumps google.golang.org/protobuf from 1.32.0 to 1.33.0. --- updated-dependencies: - dependency-name: google.golang.org/protobuf dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 9c499e20..1907460d 100644 --- a/go.mod +++ b/go.mod @@ -82,7 +82,7 @@ require ( golang.org/x/net v0.22.0 // indirect golang.org/x/sync v0.6.0 // indirect golang.org/x/sys v0.18.0 // indirect - google.golang.org/protobuf v1.32.0 // indirect + google.golang.org/protobuf v1.33.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index a3593075..0e6ecc5c 100644 --- a/go.sum +++ b/go.sum @@ -383,8 +383,8 @@ gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6d gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= -google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= +google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= From 63d97f4c1353c23d665b3c8b13ca70e831ae529f Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Wed, 20 Mar 2024 16:53:01 +0800 Subject: [PATCH 392/404] fix: logic table not sync schema --- output/clickhouse.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index d246c8d3..76998994 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -622,8 +622,8 @@ func (c *ClickHouse) getDistTbls(table string) (distTbls []DistTblInfo, err erro return } query := fmt.Sprintf(`SELECT name, (extractAllGroups(engine_full, '(Distributed\\(\')(.*)\',\\s+\'(.*)\',\\s+\'(.*)\'(.*)')[1])[2] AS cluster - FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(engine_full, 'Distributed\(\'%s\', \'%s\', \'%s\'.*\)')`, - c.dbName, c.cfg.Clickhouse.Cluster, c.dbName, table) + FROM system.tables WHERE engine='Distributed' AND database='%s' AND match(engine_full, 'Distributed\(\'.*\', \'%s\', \'%s\'.*\)')`, + c.dbName, c.dbName, table) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) var rows *pool.Rows if rows, err = conn.Query(query); err != nil { From fd3c3324983cfa88fcccc1e74fe1a472260caf38 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Tue, 2 Apr 2024 18:36:21 +0800 Subject: [PATCH 393/404] fix: set alter_sync = 0 when add columns on cluster --- output/clickhouse.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index 76998994..d7cdb889 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -581,7 +581,7 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { } alterTable := func(tbl, col string) error { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s;", c.dbName, tbl, onCluster, col) + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s SETTINGS alter_sync = 0;", c.dbName, tbl, onCluster, col) util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) return conn.Exec(query) } From aca7950cb9fe8b2d4f13436a2b19d2a0e4f0d08b Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Wed, 17 Apr 2024 09:46:50 +0800 Subject: [PATCH 394/404] perf: http-port conflicts when more than 1 instance running on a machine --- cmd/clickhouse_sinker/main.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index e95e321e..fcc904e5 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -201,8 +201,9 @@ func main() { // cmdOps.HTTPPort=0: let OS choose the listen port, and record the exact metrics URL to log. httpPort := cmdOps.HTTPPort if httpPort == 0 { - httpPort = util.GetSpareTCPPort(HttpPortBase) + httpPort = HttpPortBase } + httpPort = util.GetSpareTCPPort(httpPort) httpHost := cmdOps.HTTPHost if httpHost == "" { From d3797134e149cb9940e5fedb774187eb141ba7eb Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Wed, 17 Apr 2024 10:37:48 +0800 Subject: [PATCH 395/404] change maxport from maxInt to 65535 --- cmd/clickhouse_sinker/main.go | 6 +----- util/net.go | 8 ++++++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index fcc904e5..456c1717 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -49,10 +49,6 @@ var ( runner *task.Sinker ) -const ( - HttpPortBase = 10000 -) - func initCmdOptions() { // 1. Set options to default value. cmdOps = util.CmdOptions{ @@ -201,7 +197,7 @@ func main() { // cmdOps.HTTPPort=0: let OS choose the listen port, and record the exact metrics URL to log. httpPort := cmdOps.HTTPPort if httpPort == 0 { - httpPort = HttpPortBase + httpPort = util.HttpPortBase } httpPort = util.GetSpareTCPPort(httpPort) diff --git a/util/net.go b/util/net.go index 2a8c5be1..cd305599 100644 --- a/util/net.go +++ b/util/net.go @@ -17,12 +17,16 @@ package util import ( "fmt" - "math" "net" "github.com/thanos-io/thanos/pkg/errors" ) +const ( + HttpPortBase = 10000 + MaxPort = 65535 +) + func GetIP4Byname(host string) (ips []string, err error) { addrs, err := net.LookupIP(host) if err != nil { @@ -53,7 +57,7 @@ func GetOutboundIP() (ip net.IP, err error) { // GetSpareTCPPort finds a spare TCP port. func GetSpareTCPPort(portBegin int) int { - for port := portBegin; port < math.MaxInt; port++ { + for port := portBegin; port <= MaxPort; port++ { if err := testListenOnPort(port); err == nil { return port } From 0c42c3816710eed6a0308003f6c807dd912538cd Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Mon, 6 May 2024 13:52:15 +0800 Subject: [PATCH 396/404] fix: rebalance issue - risk of int overflow - rebalance by offset and partition --- task/task.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/task/task.go b/task/task.go index 28fcd946..a0865046 100644 --- a/task/task.go +++ b/task/task.go @@ -209,7 +209,7 @@ func (service *Service) Put(msg *model.InputMessage, traceId string, flushFn fun util.Logger.Fatal("shard number calculation failed", zap.String("task", taskCfg.Name), zap.Error(err)) } } else { - msgRow.Shard = int(msgRow.Msg.Offset>>service.offShift) % service.sharder.shards + msgRow.Shard = int(msgRow.Msg.Offset * (int64(msgRow.Msg.Partition + 1)) >> service.offShift % int64(service.sharder.shards)) } service.sharder.PutElement(&msgRow) } From b6a1d53fe6a5b1e94b4333695bd09af932ef9925 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 20 May 2024 13:30:42 +0800 Subject: [PATCH 397/404] Bump golang.org/x/net from 0.22.0 to 0.23.0 (#194) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.22.0 to 0.23.0. - [Commits](https://github.com/golang/net/compare/v0.22.0...v0.23.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 1907460d..16fabec4 100644 --- a/go.mod +++ b/go.mod @@ -79,7 +79,7 @@ require ( go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.6.0 // indirect golang.org/x/crypto v0.21.0 // indirect - golang.org/x/net v0.22.0 // indirect + golang.org/x/net v0.23.0 // indirect golang.org/x/sync v0.6.0 // indirect golang.org/x/sys v0.18.0 // indirect google.golang.org/protobuf v1.33.0 // indirect diff --git a/go.sum b/go.sum index 0e6ecc5c..e7f5c4cc 100644 --- a/go.sum +++ b/go.sum @@ -322,8 +322,8 @@ golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.22.0 h1:9sGLhx7iRIHEiX0oAJ3MRZMUCElJgy7Br1nO+AMN3Tc= -golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= +golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= +golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= From 474034ea8dc81ff154c4aae761b1aae81c7e768b Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Tue, 21 May 2024 14:01:30 +0800 Subject: [PATCH 398/404] perf: log caller --- util/common.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/common.go b/util/common.go index 902c9289..713308ee 100644 --- a/util/common.go +++ b/util/common.go @@ -242,7 +242,7 @@ func InitLogger(newLogPaths []string) { zapcore.NewMultiWriteSyncer(syncers...), logAtomLevel, ) - Logger = zap.New(core, zap.AddStacktrace(zap.ErrorLevel)) + Logger = zap.New(core, zap.AddStacktrace(zap.ErrorLevel), zap.AddCaller()) } func SetLogLevel(newLogLevel string) { From 90b0b92e483091b42917b1df6e24bc51f4310e6c Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Thu, 23 May 2024 10:21:29 +0800 Subject: [PATCH 399/404] feat: If http-port is not specified, the http service is disabled to avoid security vulnerabilities --- cmd/clickhouse_sinker/main.go | 136 +++++++++++++++++----------------- 1 file changed, 69 insertions(+), 67 deletions(-) diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index 456c1717..d74d72c6 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -134,10 +134,27 @@ func init() { func main() { util.Run("clickhouse_sinker", func() error { - // Initialize http server for metrics and debug - mux := http.NewServeMux() - mux.HandleFunc("/", func(w http.ResponseWriter, r *http.Request) { - _, _ = w.Write([]byte(` + httpHost := cmdOps.HTTPHost + if httpHost == "" { + ip, err := util.GetOutboundIP() + if err != nil { + return fmt.Errorf("failed to determine outbound ip: %w", err) + } + httpHost = ip.String() + } + + httpPort := cmdOps.HTTPPort + if httpPort == 0 { + httpPort = util.HttpPortBase + } + httpPort = util.GetSpareTCPPort(httpPort) + + // cmdOps.HTTPPort=0: disable the http server + if cmdOps.HTTPPort > 0 { + // Initialize http server for metrics and debug + mux := http.NewServeMux() + mux.HandleFunc("/", func(w http.ResponseWriter, r *http.Request) { + _, _ = w.Write([]byte(` ClickHouse Sinker

ClickHouse Sinker

@@ -149,80 +166,65 @@ func main() {

Live Full

pprof

`)) - }) + }) - mux.HandleFunc("/state", func(w http.ResponseWriter, r *http.Request) { - w.Header().Set("Content-Type", "application/json") - if runner != nil && runner.GetCurrentConfig() != nil { - var stateLags map[string]cm.StateLag - var bs []byte - var err error - if stateLags, err = cm.GetTaskStateAndLags(runner.GetCurrentConfig()); err == nil { - if bs, err = json.Marshal(stateLags); err == nil { - _, _ = w.Write(bs) + mux.HandleFunc("/state", func(w http.ResponseWriter, r *http.Request) { + w.Header().Set("Content-Type", "application/json") + if runner != nil && runner.GetCurrentConfig() != nil { + var stateLags map[string]cm.StateLag + var bs []byte + var err error + if stateLags, err = cm.GetTaskStateAndLags(runner.GetCurrentConfig()); err == nil { + if bs, err = json.Marshal(stateLags); err == nil { + _, _ = w.Write(bs) + } } } - } - }) - health.Health.AddLivenessCheck("task", func() error { - var err error - if runner != nil && runner.GetCurrentConfig() != nil { - var stateLags map[string]cm.StateLag - var count int - if stateLags, err = cm.GetTaskStateAndLags(runner.GetCurrentConfig()); err == nil { - for _, value := range stateLags { - if value.State == "Dead" { - count++ + }) + health.Health.AddLivenessCheck("task", func() error { + var err error + if runner != nil && runner.GetCurrentConfig() != nil { + var stateLags map[string]cm.StateLag + var count int + if stateLags, err = cm.GetTaskStateAndLags(runner.GetCurrentConfig()); err == nil { + for _, value := range stateLags { + if value.State == "Dead" { + count++ + } } + if count == len(stateLags) { + return fmt.Errorf("All task is Dead.") + } + } else { + return err } - if count == len(stateLags) { - return fmt.Errorf("All task is Dead.") - } - } else { - return err } - } - return nil - }) - mux.Handle("/metrics", httpMetrics) - mux.HandleFunc("/ready", health.Health.ReadyEndpoint) // GET /ready?full=1 - mux.HandleFunc("/live", health.Health.LiveEndpoint) // GET /live?full=1 - mux.HandleFunc("/debug/pprof/", pprof.Index) - mux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) - mux.HandleFunc("/debug/pprof/profile", pprof.Profile) - mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) - mux.HandleFunc("/debug/pprof/trace", pprof.Trace) - mux.Handle("/debug/vars", http.DefaultServeMux) - - // cmdOps.HTTPPort=0: let OS choose the listen port, and record the exact metrics URL to log. - httpPort := cmdOps.HTTPPort - if httpPort == 0 { - httpPort = util.HttpPortBase - } - httpPort = util.GetSpareTCPPort(httpPort) + return nil + }) + mux.Handle("/metrics", httpMetrics) + mux.HandleFunc("/ready", health.Health.ReadyEndpoint) // GET /ready?full=1 + mux.HandleFunc("/live", health.Health.LiveEndpoint) // GET /live?full=1 + mux.HandleFunc("/debug/pprof/", pprof.Index) + mux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) + mux.HandleFunc("/debug/pprof/profile", pprof.Profile) + mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) + mux.HandleFunc("/debug/pprof/trace", pprof.Trace) + mux.Handle("/debug/vars", http.DefaultServeMux) - httpHost := cmdOps.HTTPHost - if httpHost == "" { - ip, err := util.GetOutboundIP() + httpAddr = fmt.Sprintf("%s:%d", httpHost, httpPort) + listener, err := net.Listen("tcp", httpAddr) if err != nil { - return fmt.Errorf("failed to determine outbound ip: %w", err) + return fmt.Errorf("failed to listen on %q: %w", httpAddr, err) } - httpHost = ip.String() - } - httpAddr = fmt.Sprintf("%s:%d", httpHost, httpPort) - listener, err := net.Listen("tcp", httpAddr) - if err != nil { - return fmt.Errorf("failed to listen on %q: %w", httpAddr, err) - } + util.Logger.Info(fmt.Sprintf("Run http server at http://%s/", httpAddr)) - util.Logger.Info(fmt.Sprintf("Run http server at http://%s/", httpAddr)) - - go func() { - if err := http.Serve(listener, mux); err != nil { - util.Logger.Error("http.ListenAndServe failed", zap.Error(err)) - } - }() + go func() { + if err := http.Serve(listener, mux); err != nil { + util.Logger.Error("http.ListenAndServe failed", zap.Error(err)) + } + }() + } var rcm cm.RemoteConfManager var properties map[string]interface{} From 7da534a7b4110d9d36c99a2a3f4d4d5149b5d56c Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Fri, 24 May 2024 14:12:59 +0800 Subject: [PATCH 400/404] fix: clickhoue not support alter_sync before v23.3 --- output/clickhouse.go | 9 ++++++++- util/common.go | 21 +++++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/output/clickhouse.go b/output/clickhouse.go index d7cdb889..a92bcd9d 100644 --- a/output/clickhouse.go +++ b/output/clickhouse.go @@ -580,8 +580,15 @@ func (c *ClickHouse) ChangeSchema(newKeys *sync.Map) (err error) { return } + var version string + if err = conn.QueryRow("SELECT Version()").Scan(&version); err != nil { + return + } alterTable := func(tbl, col string) error { - query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s SETTINGS alter_sync = 0;", c.dbName, tbl, onCluster, col) + query := fmt.Sprintf("ALTER TABLE `%s`.`%s` %s %s", c.dbName, tbl, onCluster, col) + if util.CompareClickHouseVersion(version, "23.3") >= 0 { + query += " SETTINGS alter_sync = 0" + } util.Logger.Info(fmt.Sprintf("executing sql=> %s", query), zap.String("task", taskCfg.Name)) return conn.Exec(query) } diff --git a/util/common.go b/util/common.go index 713308ee..c21be1d9 100644 --- a/util/common.go +++ b/util/common.go @@ -347,3 +347,24 @@ func TrySetValue(v1, v2 interface{}) bool { } return ok } + +func CompareClickHouseVersion(v1, v2 string) int { + s1 := strings.Split(v1, ".") + s2 := strings.Split(v2, ".") + for i := 0; i < len(s1); i++ { + if len(s2) <= i { + break + } + if s1[i] == "x" || s2[i] == "x" { + continue + } + f1, _ := strconv.Atoi(s1[i]) + f2, _ := strconv.Atoi(s2[i]) + if f1 > f2 { + return 1 + } else if f1 < f2 { + return -1 + } + } + return 0 +} From 756c3122365f41e0efe73c8f03f071267253a1a3 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Tue, 21 May 2024 10:48:22 +0800 Subject: [PATCH 401/404] feat: support UPv4/IPv6 data types --- model/metric.go | 2 ++ model/value.go | 12 +++++++- parser/csv.go | 15 +++++++++ parser/fastjson.go | 75 ++++++++++++++++++++++++++++++++++++++++++++ parser/gjson.go | 77 ++++++++++++++++++++++++++++++++++++++++++++++ 5 files changed, 180 insertions(+), 1 deletion(-) diff --git a/model/metric.go b/model/metric.go index cab37451..fdb5aa25 100644 --- a/model/metric.go +++ b/model/metric.go @@ -40,6 +40,8 @@ type Metric interface { GetObject(key string, nullable bool) (val interface{}) GetMap(key string, typeinfo *TypeInfo) (val interface{}) GetArray(key string, t int) (val interface{}) + GetIPv4(key string, nullable bool) (val interface{}) + GetIPv6(key string, nullable bool) (val interface{}) GetNewKeys(knownKeys, newKeys, warnKeys *sync.Map, white, black *regexp.Regexp, partition int, offset int64) bool } diff --git a/model/value.go b/model/value.go index 2eda1ab2..78480c85 100644 --- a/model/value.go +++ b/model/value.go @@ -41,6 +41,8 @@ const ( String Object Map + IPv4 + IPv6 ) type TypeInfo struct { @@ -91,6 +93,10 @@ func GetTypeName(typ int) (name string) { name = "Object('json')" case Map: name = "Map" + case IPv4: + name = "IPv4" + case IPv6: + name = "IPv6" default: name = "Unknown" } @@ -135,6 +141,10 @@ func GetValueByType(metric Metric, cwt *ColumnWithType) (val interface{}) { val = metric.GetMap(name, cwt.Type) case Object: val = metric.GetObject(name, cwt.Type.Nullable) + case IPv4: + val = metric.GetIPv4(name, cwt.Type.Nullable) + case IPv6: + val = metric.GetIPv6(name, cwt.Type.Nullable) default: util.Logger.Fatal("LOGIC ERROR: reached switch default condition") } @@ -190,7 +200,7 @@ func WhichType(typ string) (ti *TypeInfo) { func init() { typeInfo = make(map[string]*TypeInfo) - for _, t := range []int{Bool, Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32, Float64, DateTime, String, Object} { + for _, t := range []int{Bool, Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32, Float64, DateTime, String, Object, IPv4, IPv6} { tn := GetTypeName(t) typeInfo[tn] = &TypeInfo{Type: t} nullTn := fmt.Sprintf("Nullable(%s)", tn) diff --git a/parser/csv.go b/parser/csv.go index 9ec4249b..45853924 100644 --- a/parser/csv.go +++ b/parser/csv.go @@ -20,6 +20,7 @@ import ( "encoding/csv" "fmt" "math" + "net" "regexp" "strconv" "sync" @@ -154,6 +155,20 @@ func (c *CsvMetric) GetFloat64(key string, nullable bool) (val interface{}) { return CsvGetFloat[float64](c, key, nullable, math.MaxFloat64) } +func (c *CsvMetric) GetIPv4(key string, nullable bool) (val interface{}) { + return c.GetUint32(key, nullable) +} + +func (c *CsvMetric) GetIPv6(key string, nullable bool) (val interface{}) { + s := c.GetString(key, nullable).(string) + if net.ParseIP(s) != nil { + val = s + } else { + val = net.IPv6zero.String() + } + return val +} + func CsvGetInt[T constraints.Signed](c *CsvMetric, key string, nullable bool, min, max int64) (val interface{}) { var idx int var ok bool diff --git a/parser/fastjson.go b/parser/fastjson.go index d19516ae..0f97a1fb 100644 --- a/parser/fastjson.go +++ b/parser/fastjson.go @@ -18,6 +18,7 @@ package parser import ( "fmt" "math" + "net" "regexp" "strconv" "sync" @@ -117,6 +118,14 @@ func (c *FastjsonMetric) GetFloat64(key string, nullable bool) (val interface{}) return FastjsonGetFloat[float64](c.value.Get(key), nullable, math.MaxFloat64) } +func (c *FastjsonMetric) GetIPv4(key string, nullable bool) (val interface{}) { + return getIPv4(c.value.Get(key), nullable) +} + +func (c *FastjsonMetric) GetIPv6(key string, nullable bool) (val interface{}) { + return getIPv6(c.value.Get(key), nullable) +} + func FastjsonGetInt[T constraints.Signed](v *fastjson.Value, nullable bool, min, max int64) (val interface{}) { if !fjCompatibleInt(v) { val = getDefaultInt[T](nullable) @@ -271,6 +280,54 @@ func getBool(v *fastjson.Value, nullable bool) (val interface{}) { return } +func getIPv4(v *fastjson.Value, nullable bool) (val interface{}) { + if v == nil || v.Type() == fastjson.TypeNull { + if nullable { + return + } + val = "" + return + } + switch v.Type() { + case fastjson.TypeString: + b, _ := v.StringBytes() + s := string(b) + if net.ParseIP(s) != nil { + val = s + } else { + val = net.IPv4zero.String() + } + case fastjson.TypeNumber: + val = FastjsonGetUint[uint32](v, nullable, math.MaxUint32) + default: + val = net.IPv4zero.String() + } + return +} + +func getIPv6(v *fastjson.Value, nullable bool) (val interface{}) { + if v == nil || v.Type() == fastjson.TypeNull { + if nullable { + return + } + val = "" + return + } + switch v.Type() { + case fastjson.TypeString: + b, _ := v.StringBytes() + s := string(b) + if net.ParseIP(s) != nil { + val = s + } else { + val = net.IPv6zero.String() + } + default: + val = net.IPv6zero.String() + } + return +} + func getDecimal(v *fastjson.Value, nullable bool) (val interface{}) { if !fjCompatibleFloat(v) { val = getDefaultDecimal(nullable) @@ -404,6 +461,20 @@ func getArray(c *FastjsonMetric, sourcename string, v *fastjson.Value, typ int) } } val = arr + case model.IPv4: + arr := make([]interface{}, 0) + for _, e := range array { + v := getIPv4(e, false) + arr = append(arr, v) + } + val = arr + case model.IPv6: + arr := make([]interface{}, 0) + for _, e := range array { + v := getIPv6(e, false) + arr = append(arr, v) + } + val = arr default: util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) } @@ -515,6 +586,10 @@ func (c *FastjsonMetric) castMapValueByType(sourcename string, value *fastjson.V val = FastjsonGetUint[uint32](value, typeinfo.Nullable, math.MaxUint32) case model.UInt64: val = FastjsonGetUint[uint64](value, typeinfo.Nullable, math.MaxUint64) + case model.IPv4: + val = getIPv4(value, typeinfo.Nullable) + case model.IPv6: + val = getIPv6(value, typeinfo.Nullable) case model.Float32: val = FastjsonGetFloat[float32](value, typeinfo.Nullable, math.MaxFloat32) case model.Float64: diff --git a/parser/gjson.go b/parser/gjson.go index 93883d9f..732764d3 100644 --- a/parser/gjson.go +++ b/parser/gjson.go @@ -18,6 +18,7 @@ package parser import ( "fmt" "math" + "net" "regexp" "strconv" "sync" @@ -108,6 +109,14 @@ func (c *GjsonMetric) GetFloat64(key string, nullable bool) (val interface{}) { return GjsonGetFloat[float64](c, c.getField(key), nullable, math.MaxFloat64) } +func (c *GjsonMetric) GetIPv4(key string, nullable bool) (val interface{}) { + return getGJsonIPv4(c, c.getField(key), nullable) +} + +func (c *GjsonMetric) GetIPv6(key string, nullable bool) (val interface{}) { + return getGJsonIPv6(c, c.getField(key), nullable) +} + func GjsonGetInt[T constraints.Signed](c *GjsonMetric, r gjson.Result, nullable bool, min, max int64) (val interface{}) { if !gjCompatibleInt(r) { val = getDefaultInt[T](nullable) @@ -404,6 +413,10 @@ func (c *GjsonMetric) castResultByType(sourcename string, value gjson.Result, ty val = GjsonGetUint[uint32](c, value, typeinfo.Nullable, math.MaxUint32) case model.UInt64: val = GjsonGetUint[uint64](c, value, typeinfo.Nullable, math.MaxUint64) + case model.IPv6: + val = getGJsonIPv6(c, value, typeinfo.Nullable) + case model.IPv4: + val = getGJsonIPv4(c, value, typeinfo.Nullable) case model.Float32: val = GjsonGetFloat[float32](c, value, typeinfo.Nullable, math.MaxFloat32) case model.Float64: @@ -465,6 +478,56 @@ func getGJsonString(r gjson.Result, nullable bool) (val interface{}) { return } +func getGJsonIPv4(c *GjsonMetric, r gjson.Result, nullable bool) (val interface{}) { + if !r.Exists() || r.Type == gjson.Null { + if nullable { + return + } + val = "" + return + } + switch r.Type { + case gjson.Null: + val = "" + case gjson.String: + s := r.Str + if net.ParseIP(s) != nil { + val = s + } else { + val = net.IPv4zero.String() + } + case gjson.Number: + val = GjsonGetUint[uint32](c, r, nullable, math.MaxUint32) + default: + val = net.IPv4zero.String() + } + return +} + +func getGJsonIPv6(c *GjsonMetric, r gjson.Result, nullable bool) (val interface{}) { + if !r.Exists() || r.Type == gjson.Null { + if nullable { + return + } + val = "" + return + } + switch r.Type { + case gjson.Null: + val = "" + case gjson.String: + s := r.Str + if net.ParseIP(s) != nil { + val = s + } else { + val = net.IPv6zero.String() + } + default: + val = net.IPv6zero.String() + } + return +} + func getGJsonDateTime(c *GjsonMetric, key string, r gjson.Result, nullable bool) (val interface{}) { if !gjCompatibleDateTime(r) { val = getDefaultDateTime(nullable) @@ -563,6 +626,20 @@ func getGJsonArray(c *GjsonMetric, key string, r gjson.Result, typ int) (val int results = append(results, t) } val = results + case model.IPv4: + arr := make([]interface{}, 0) + for _, e := range array { + v := getGJsonIPv4(c, e, false) + arr = append(arr, v) + } + val = arr + case model.IPv6: + arr := make([]interface{}, 0) + for _, e := range array { + v := getGJsonIPv6(c, e, false) + arr = append(arr, v) + } + val = arr default: util.Logger.Fatal(fmt.Sprintf("LOGIC ERROR: unsupported array type %v", typ)) } From bff611bc4181187db83f90d6474607f8f3f15ffa Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Fri, 7 Jun 2024 10:01:37 +0800 Subject: [PATCH 402/404] feat: random idx of replicas when init cluster connection --- pool/conn.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/pool/conn.go b/pool/conn.go index d58d4b81..909a848c 100644 --- a/pool/conn.go +++ b/pool/conn.go @@ -22,6 +22,7 @@ import ( "context" "crypto/tls" "fmt" + "math/rand" "sync" "time" @@ -181,7 +182,10 @@ func InitClusterConn(chCfg *config.ClickHouseConfig) (err error) { sc.opts.ConnMaxLifetime = time.Minute * 10 } sc.protocol = proto - if _, _, err = sc.NextGoodReplica(0); err != nil { + r := rand.New(rand.NewSource(time.Now().UnixNano())) + idx := r.Intn(numReplicas) + sc.nextRep = idx + if _, _, err = sc.NextGoodReplica(idx); err != nil { return } clusterConn = append(clusterConn, sc) From 016f8039083998473b3b8c35a2feefa8222cd9c8 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Tue, 11 Jun 2024 13:48:28 +0800 Subject: [PATCH 403/404] feat: do not restart consumer when foundNewKeys to avoid rebalance --- task/task.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/task/task.go b/task/task.go index a0865046..1ff8f349 100644 --- a/task/task.go +++ b/task/task.go @@ -190,8 +190,10 @@ func (service *Service) Put(msg *model.InputMessage, traceId string, flushFn fun // 2) flush the shards // 3) apply the schema change. // 4) recreate the service - util.Logger.Warn("new key detected, consumer is going to restart", zap.String("consumer group", service.taskCfg.ConsumerGroup), zap.Error(err)) - go service.consumer.restart() + if len(service.consumer.grpConfig.Configs) > 1 { + util.Logger.Warn("new key detected, consumer is going to restart", zap.String("consumer group", service.taskCfg.ConsumerGroup), zap.Error(err)) + go service.consumer.restart() + } flushFn(traceId, "foundNewKeys and restart") if err = service.clickhouse.ChangeSchema(&service.newKeys); err != nil { util.Logger.Fatal("clickhouse.ChangeSchema failed", zap.String("task", taskCfg.Name), zap.Error(err)) From ab7f90b7cc33ddc5617c5663a7b7e82ac5910178 Mon Sep 17 00:00:00 2001 From: YenchangChan Date: Fri, 14 Jun 2024 08:23:10 +0800 Subject: [PATCH 404/404] feat: encrypt password --- cmd/clickhouse_sinker/main.go | 10 +++ go.mod | 2 +- task/sinker.go | 4 + util/aes.go | 71 ++++++++++++++++ util/common.go | 1 + util/gosypt.go | 149 ++++++++++++++++++++++++++++++++++ 6 files changed, 236 insertions(+), 1 deletion(-) create mode 100644 util/aes.go create mode 100644 util/gosypt.go diff --git a/cmd/clickhouse_sinker/main.go b/cmd/clickhouse_sinker/main.go index d74d72c6..6e471f50 100644 --- a/cmd/clickhouse_sinker/main.go +++ b/cmd/clickhouse_sinker/main.go @@ -60,10 +60,12 @@ func initCmdOptions() { NacosGroup: "DEFAULT_GROUP", NacosUsername: "nacos", NacosPassword: "nacos", + Encrypt: "", } // 2. Replace options with the corresponding env variable if present. util.EnvBoolVar(&cmdOps.ShowVer, "v") + util.EnvStringVar(&cmdOps.Encrypt, "e") util.EnvStringVar(&cmdOps.LogLevel, "log-level") util.EnvStringVar(&cmdOps.LogPaths, "log-paths") util.EnvIntVar(&cmdOps.HTTPPort, "http-port") @@ -89,6 +91,7 @@ func initCmdOptions() { // 3. Replace options with the corresponding CLI parameter if present. flag.BoolVar(&cmdOps.ShowVer, "v", cmdOps.ShowVer, "show build version and quit") + flag.StringVar(&cmdOps.Encrypt, "e", cmdOps.Encrypt, "encrypt password") flag.StringVar(&cmdOps.LogLevel, "log-level", cmdOps.LogLevel, "one of debug, info, warn, error, dpanic, panic, fatal") flag.StringVar(&cmdOps.LogPaths, "log-paths", cmdOps.LogPaths, "a list of comma-separated log file path. stdout means the console stdout") flag.IntVar(&cmdOps.HTTPPort, "http-port", cmdOps.HTTPPort, "http listen port") @@ -114,6 +117,9 @@ func initCmdOptions() { flag.StringVar(&cmdOps.KafkaGSSAPIPassword, "kafka-gssapi-password", cmdOps.KafkaGSSAPIPassword, "kafka GSSAPI password") flag.Parse() + if err := util.Gsypt.Unmarshal(&cmdOps); err != nil { + util.Logger.Fatal("failed to decrypt password", zap.Error(err)) + } } func getVersion() string { @@ -122,6 +128,10 @@ func getVersion() string { func init() { initCmdOptions() + if cmdOps.Encrypt != "" { + fmt.Println(util.AesEncryptECB(cmdOps.Encrypt)) + os.Exit(0) + } logPaths := strings.Split(cmdOps.LogPaths, ",") util.InitLogger(logPaths) util.SetLogLevel(cmdOps.LogLevel) diff --git a/go.mod b/go.mod index 16fabec4..62238dfb 100644 --- a/go.mod +++ b/go.mod @@ -18,6 +18,7 @@ require ( github.com/jinzhu/copier v0.4.0 github.com/matoous/go-nanoid/v2 v2.0.0 github.com/nacos-group/nacos-sdk-go v1.1.4 + github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.18.0 github.com/prometheus/common v0.45.0 github.com/shopspring/decimal v1.3.1 @@ -64,7 +65,6 @@ require ( github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b // indirect github.com/paulmach/orb v0.11.1 // indirect github.com/pierrec/lz4/v4 v4.1.21 // indirect - github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_model v0.5.0 // indirect github.com/prometheus/procfs v0.12.0 // indirect diff --git a/task/sinker.go b/task/sinker.go index e2b76db1..c5760d09 100644 --- a/task/sinker.go +++ b/task/sinker.go @@ -285,6 +285,10 @@ func (s *Sinker) applyConfig(newCfg *config.Config) (err error) { util.SetLogTrace(newCfg.LogTrace) util.Rs.SetPoolSize(newCfg.RecordPoolSize) util.Rs.Reset() + if err := util.Gsypt.Unmarshal(&newCfg.Clickhouse); err != nil { + util.Logger.Error("failed to decrypt config password", zap.Error(err)) + return err + } if s.curCfg == nil { // The first time invoking of applyConfig err = s.applyFirstConfig(newCfg) diff --git a/util/aes.go b/util/aes.go new file mode 100644 index 00000000..2bef763a --- /dev/null +++ b/util/aes.go @@ -0,0 +1,71 @@ +package util + +import ( + "crypto/aes" + "encoding/hex" + "strings" +) + +var salt = "656f6974656b" + +// select hex(aes_encrypt("123456", unhex("656f6974656b"))); => E310E892E56801CED9ED98AA177F18E6 +func AesEncryptECB(origData string) string { + if origData == "" { + return origData + } + var encrypted []byte + var o = []byte(origData) + s, _ := hex.DecodeString(salt) + cipher, _ := aes.NewCipher(generateKey(s)) + length := (len(o) + aes.BlockSize) / aes.BlockSize + plain := make([]byte, length*aes.BlockSize) + copy(plain, o) + pad := byte(len(plain) - len(o)) + for i := len(o); i < len(plain); i++ { + plain[i] = pad + } + encrypted = make([]byte, len(plain)) + for bs, be := 0, cipher.BlockSize(); bs <= len(o); bs, be = bs+cipher.BlockSize(), be+cipher.BlockSize() { + cipher.Encrypt(encrypted[bs:be], plain[bs:be]) + } + return strings.ToUpper(hex.EncodeToString(encrypted)) +} + +// select aes_decrypt(unhex("E310E892E56801CED9ED98AA177F18E6"), unhex("656f6974656b")); => 123456 +func AesDecryptECB(encrypted string) string { + if encrypted == "" { + return encrypted + } + var decrypted []byte + h, _ := hex.DecodeString(encrypted) + s, _ := hex.DecodeString(salt) + cipher, _ := aes.NewCipher(generateKey(s)) + decrypted = make([]byte, len(h)) + + for bs, be := 0, cipher.BlockSize(); bs < len(h); bs, be = bs+cipher.BlockSize(), be+cipher.BlockSize() { + cipher.Decrypt(decrypted[bs:be], h[bs:be]) + } + + bEnd := searchByteSliceIndex(decrypted, 32) + return string(decrypted[:bEnd]) +} +func generateKey(key []byte) (genKey []byte) { + genKey = make([]byte, 16) + copy(genKey, key) + for i := 16; i < len(key); { + for j := 0; j < 16 && i < len(key); j, i = j+1, i+1 { + genKey[j] ^= key[i] + } + } + return genKey +} + +func searchByteSliceIndex(bSrc []byte, b byte) int { + for i := 0; i < len(bSrc); i++ { + if bSrc[i] < b { + return i + } + } + + return len(bSrc) +} diff --git a/util/common.go b/util/common.go index c21be1d9..9ca5a34e 100644 --- a/util/common.go +++ b/util/common.go @@ -62,6 +62,7 @@ type CmdOptions struct { NacosPassword string NacosDataID string NacosServiceName string // participate in assignment management if not empty + Encrypt string Credentials } diff --git a/util/gosypt.go b/util/gosypt.go new file mode 100644 index 00000000..9d9ef829 --- /dev/null +++ b/util/gosypt.go @@ -0,0 +1,149 @@ +package util + +import ( + "reflect" + "strings" + + "github.com/pkg/errors" +) + +const ( + GosyptPrefixDefault = "ENC(" + GosyptSuffxiDefault = ")" + GosyptAlgorithm = "AESWITHHEXANDBASE64" +) + +/* Golang Simple Encrypt, simulate jasypt */ +type Gosypt struct { + prefix string + suffix string + algorithm string +} + +var Gsypt = &Gosypt{ + prefix: GosyptPrefixDefault, + suffix: GosyptSuffxiDefault, + algorithm: GosyptAlgorithm, +} + +func (gsypt *Gosypt) ensurePassword(password string) string { + if !strings.HasPrefix(password, gsypt.prefix) || !strings.HasSuffix(password, gsypt.suffix) { + return password + } + passwd := strings.TrimSuffix(strings.TrimPrefix(password, gsypt.prefix), gsypt.suffix) + if gsypt.algorithm == GosyptAlgorithm { + return AesDecryptECB(passwd) + } + return password +} + +func (gsypt *Gosypt) SetAttribution(prefix, suffix, algorithm string) { + gsypt.prefix = prefix + gsypt.suffix = suffix + gsypt.algorithm = algorithm +} + +func (gsypt *Gosypt) Unmarshal(v interface{}) error { + rt := reflect.TypeOf(v) + rv := reflect.ValueOf(v) + + if rt.Kind() != reflect.Ptr { + return errors.Wrap(nil, "invalid args, expect ptr") + } + + for rt.Kind() == reflect.Ptr { + rt = rt.Elem() + rv = rv.Elem() + } + + if rt.Kind() == reflect.Struct { + v, err := gsypt.structHandle(rt, rv) + if err != nil { + return err + } + rv.Set(v) + } else if rt.Kind() == reflect.Slice || rt.Kind() == reflect.Array { + v, err := gsypt.sliceHandle(rt, rv) + if err != nil { + return err + } + rv.Set(v) + } else if rt.Kind() == reflect.Map { + v, err := gsypt.mapHandle(rt, rv) + if err != nil { + return err + } + rv.Set(v) + } else if rt.Kind() == reflect.Interface { + v, err := gsypt.interfaceHandle(rt, rv) + if err != nil { + return err + } + rv.Set(v) + } else if rt.Kind() == reflect.String { + rv.Set(gsypt.stringHandle(rv)) + } + + return nil +} + +func (gsypt *Gosypt) sliceHandle(rt reflect.Type, rv reflect.Value) (reflect.Value, error) { + for j := 0; j < rv.Len(); j++ { + if rt.Elem().Kind() == reflect.String { + rv.Index(j).Set(gsypt.stringHandle(rv.Index(j))) + } else { + if err := gsypt.Unmarshal(rv.Index(j).Addr().Interface()); err != nil { + return rv, err + } + } + } + return rv, nil +} + +func (gsypt *Gosypt) mapHandle(rt reflect.Type, rv reflect.Value) (reflect.Value, error) { + for _, k := range rv.MapKeys() { + key := k.Convert(rv.Type().Key()) + if rt.Elem().Kind() == reflect.String { + v := gsypt.ensurePassword(rv.MapIndex(key).String()) + rv.SetMapIndex(key, reflect.ValueOf(v)) + } else { + v := rv.MapIndex(key).Interface() + if err := gsypt.Unmarshal(&v); err != nil { + return rv, err + } + rv.SetMapIndex(key, reflect.ValueOf(v)) + } + } + return rv, nil +} + +func (gsypt *Gosypt) interfaceHandle(rt reflect.Type, rv reflect.Value) (reflect.Value, error) { + //todo + return rv, nil +} + +func (gsypt *Gosypt) structHandle(rt reflect.Type, rv reflect.Value) (reflect.Value, error) { + for i := 0; i < rt.NumField(); i++ { + rtf := rt.Field(i) + rvf := rv.Field(i) + + rtt := rtf.Type + for rtt.Kind() == reflect.Ptr { + rtt = rtt.Elem() + } + + if rtt.Kind() == reflect.String { + rv.Field(i).Set(gsypt.stringHandle(rvf)) + } else { + if err := gsypt.Unmarshal(rvf.Addr().Interface()); err != nil { + return rv, err + } + } + } + return rv, nil +} + +func (gsypt *Gosypt) stringHandle(rv reflect.Value) reflect.Value { + rv.SetString(gsypt.ensurePassword(rv.String())) + return rv +}